From 97188b4c75536db2b1e3b3e7845cf6e546cc90da Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 5 Nov 2024 10:42:52 +0000 Subject: [PATCH 01/25] [refactoring] Move explicit sync wrapper to aptos-types --- aptos-move/block-executor/src/code_cache_global.rs | 3 +-- aptos-move/block-executor/src/executor.rs | 2 +- aptos-move/block-executor/src/lib.rs | 1 - aptos-move/block-executor/src/scheduler.rs | 6 ++++-- aptos-move/block-executor/src/txn_last_input_output.rs | 2 +- .../block-executor => types}/src/explicit_sync_wrapper.rs | 2 ++ types/src/lib.rs | 3 ++- 7 files changed, 11 insertions(+), 8 deletions(-) rename {aptos-move/block-executor => types}/src/explicit_sync_wrapper.rs (98%) diff --git a/aptos-move/block-executor/src/code_cache_global.rs b/aptos-move/block-executor/src/code_cache_global.rs index 6ff31da656e79..e5f89f9eb1e81 100644 --- a/aptos-move/block-executor/src/code_cache_global.rs +++ b/aptos-move/block-executor/src/code_cache_global.rs @@ -1,9 +1,8 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use crate::explicit_sync_wrapper::ExplicitSyncWrapper; use aptos_mvhashmap::types::TxnIndex; -use aptos_types::error::PanicError; +use aptos_types::{error::PanicError, explicit_sync_wrapper::ExplicitSyncWrapper}; use crossbeam::utils::CachePadded; use hashbrown::HashMap; use move_vm_types::code::ModuleCode; diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 182ff626059f0..67d6787f7a250 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -11,7 +11,6 @@ use crate::{ }, errors::*, executor_utilities::*, - explicit_sync_wrapper::ExplicitSyncWrapper, limit_processor::BlockGasLimitProcessor, scheduler::{DependencyStatus, ExecutionTaskType, Scheduler, SchedulerTask, Wave}, task::{ExecutionStatus, ExecutorTask, TransactionOutput}, @@ -36,6 +35,7 @@ use aptos_types::{ block_executor::config::BlockExecutorConfig, error::{code_invariant_error, expect_ok, PanicError, PanicOr}, executable::Executable, + explicit_sync_wrapper::ExplicitSyncWrapper, on_chain_config::BlockGasLimitType, state_store::{state_value::StateValue, TStateView}, transaction::{ diff --git a/aptos-move/block-executor/src/lib.rs b/aptos-move/block-executor/src/lib.rs index d7bfdc4a0c5cf..c3f0f4d5b767c 100644 --- a/aptos-move/block-executor/src/lib.rs +++ b/aptos-move/block-executor/src/lib.rs @@ -146,7 +146,6 @@ pub mod counters; pub mod errors; pub mod executor; mod executor_utilities; -pub mod explicit_sync_wrapper; mod limit_processor; #[cfg(any(test, feature = "fuzzing"))] pub mod proptest_types; diff --git a/aptos-move/block-executor/src/scheduler.rs b/aptos-move/block-executor/src/scheduler.rs index 0d27eb94ba8f5..796d024619075 100644 --- a/aptos-move/block-executor/src/scheduler.rs +++ b/aptos-move/block-executor/src/scheduler.rs @@ -2,10 +2,12 @@ // Parts of the project are originally copyright © Meta Platforms, Inc. // SPDX-License-Identifier: Apache-2.0 -use crate::explicit_sync_wrapper::ExplicitSyncWrapper; use aptos_infallible::Mutex; use aptos_mvhashmap::types::{Incarnation, TxnIndex}; -use aptos_types::error::{code_invariant_error, PanicError}; +use aptos_types::{ + error::{code_invariant_error, PanicError}, + explicit_sync_wrapper::ExplicitSyncWrapper, +}; use concurrent_queue::{ConcurrentQueue, PopError}; use crossbeam::utils::CachePadded; use parking_lot::{RwLock, RwLockUpgradableReadGuard}; diff --git a/aptos-move/block-executor/src/txn_last_input_output.rs b/aptos-move/block-executor/src/txn_last_input_output.rs index fcfbde7452aa7..e8391b3ea6986 100644 --- a/aptos-move/block-executor/src/txn_last_input_output.rs +++ b/aptos-move/block-executor/src/txn_last_input_output.rs @@ -4,7 +4,6 @@ use crate::{ captured_reads::CapturedReads, errors::ParallelBlockExecutionError, - explicit_sync_wrapper::ExplicitSyncWrapper, task::{ExecutionStatus, TransactionOutput}, types::{InputOutputKey, ReadWriteSummary}, }; @@ -12,6 +11,7 @@ use aptos_logger::error; use aptos_mvhashmap::types::TxnIndex; use aptos_types::{ error::{code_invariant_error, PanicError}, + explicit_sync_wrapper::ExplicitSyncWrapper, fee_statement::FeeStatement, state_store::state_value::StateValueMetadata, transaction::BlockExecutableTransaction as Transaction, diff --git a/aptos-move/block-executor/src/explicit_sync_wrapper.rs b/types/src/explicit_sync_wrapper.rs similarity index 98% rename from aptos-move/block-executor/src/explicit_sync_wrapper.rs rename to types/src/explicit_sync_wrapper.rs index c088c71f88958..7f57b2ae81c00 100644 --- a/aptos-move/block-executor/src/explicit_sync_wrapper.rs +++ b/types/src/explicit_sync_wrapper.rs @@ -1,6 +1,8 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 +#![allow(unsafe_code)] + use std::{ cell::UnsafeCell, ops::{Deref, DerefMut}, diff --git a/types/src/lib.rs b/types/src/lib.rs index 9081b6c0f0a4d..69600de4dddfb 100644 --- a/types/src/lib.rs +++ b/types/src/lib.rs @@ -2,7 +2,7 @@ // Parts of the project are originally copyright © Meta Platforms, Inc. // SPDX-License-Identifier: Apache-2.0 -#![forbid(unsafe_code)] +#![deny(unsafe_code)] pub mod access_path; pub mod account_address; @@ -18,6 +18,7 @@ pub mod epoch_state; pub mod error; pub mod event; pub mod executable; +pub mod explicit_sync_wrapper; pub mod fee_statement; pub mod governance; pub mod indexer; From d958da337b7a7d393796e0c92f1ddfd5d57b540c Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 5 Nov 2024 11:18:32 +0000 Subject: [PATCH 02/25] [refactoring] Move & rename immutabke module cache to aptos-types --- Cargo.lock | 1 + aptos-move/aptos-vm/src/block_executor/mod.rs | 16 ++--- .../block-executor/src/captured_reads.rs | 16 ++--- aptos-move/block-executor/src/executor.rs | 16 ++--- aptos-move/block-executor/src/lib.rs | 1 - .../src/proptest_types/bencher.rs | 5 +- .../src/proptest_types/tests.rs | 17 +++-- .../block-executor/src/unit_tests/mod.rs | 12 ++-- aptos-move/block-executor/src/view.rs | 12 ++-- types/Cargo.toml | 1 + types/src/lib.rs | 1 + .../src/read_only_module_cache.rs | 71 +++++++++---------- 12 files changed, 81 insertions(+), 88 deletions(-) rename aptos-move/block-executor/src/code_cache_global.rs => types/src/read_only_module_cache.rs (78%) diff --git a/Cargo.lock b/Cargo.lock index 3ef97fa355e0b..f27aa4a990d04 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4346,6 +4346,7 @@ dependencies = [ "claims", "coset", "criterion", + "crossbeam", "dashmap", "derivative", "fixed", diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index de16c676676b6..29093463d0de8 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -12,7 +12,7 @@ use aptos_aggregator::{ delayed_change::DelayedChange, delta_change_set::DeltaOp, resolver::TAggregatorV1View, }; use aptos_block_executor::{ - code_cache_global::ImmutableModuleCache, errors::BlockExecutionError, executor::BlockExecutor, + errors::BlockExecutionError, executor::BlockExecutor, task::TransactionOutput as BlockExecutorTransactionOutput, txn_commit_hook::TransactionCommitHook, types::InputOutputKey, }; @@ -23,6 +23,7 @@ use aptos_types::{ error::PanicError, executable::ExecutableTestType, fee_statement::FeeStatement, + read_only_module_cache::ReadOnlyModuleCache, state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView, StateViewId}, transaction::{ signature_verified_transaction::SignatureVerifiedTransaction, BlockOutput, @@ -67,8 +68,8 @@ static RAYON_EXEC_POOL: Lazy> = Lazy::new(|| { /// the cache is fixed within a single block (modules are not inserted or removed) and it is only /// mutated at the block boundaries. Do not use if multiple blocks are executed concurrently. static GLOBAL_MODULE_CACHE: Lazy< - Arc>, -> = Lazy::new(|| Arc::new(ImmutableModuleCache::empty())); + Arc>, +> = Lazy::new(|| Arc::new(ReadOnlyModuleCache::empty())); /// The maximum size of struct name index map in runtime environment. Checked at block boundaries /// only. @@ -82,7 +83,7 @@ static GLOBAL_ENVIRONMENT: Lazy>> = Lazy::new(|| /// called at the block boundaries. fn get_environment_with_delayed_field_optimization_enabled( state_view: &impl StateView, - global_module_cache: &ImmutableModuleCache, + global_module_cache: &ReadOnlyModuleCache, ) -> Result where K: Hash + Eq + Clone, @@ -456,7 +457,7 @@ impl BlockAptosVM { signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, global_module_cache: Arc< - ImmutableModuleCache, + ReadOnlyModuleCache, >, config: BlockExecutorConfig, transaction_commit_listener: Option, @@ -534,7 +535,7 @@ impl BlockAptosVM { executor_thread_pool, signature_verified_block, state_view, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), config, transaction_commit_listener, ) @@ -564,7 +565,6 @@ impl BlockAptosVM { #[cfg(test)] mod test { use super::*; - use aptos_block_executor::code_cache_global::ImmutableModuleCache; use aptos_language_e2e_tests::data_store::FakeDataStore; use aptos_types::on_chain_config::{FeatureFlag, Features}; use aptos_vm_environment::environment::AptosEnvironment; @@ -573,7 +573,7 @@ mod test { #[test] fn test_cross_block_module_cache_flush() { - let global_module_cache = ImmutableModuleCache::empty(); + let global_module_cache = ReadOnlyModuleCache::empty(); global_module_cache.insert(0, mock_verified_code(0, None)); assert_eq!(global_module_cache.size(), 1); diff --git a/aptos-move/block-executor/src/captured_reads.rs b/aptos-move/block-executor/src/captured_reads.rs index 007ae211b9d2c..dc82e33c6b2ef 100644 --- a/aptos-move/block-executor/src/captured_reads.rs +++ b/aptos-move/block-executor/src/captured_reads.rs @@ -1,10 +1,7 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use crate::{ - code_cache_global::ImmutableModuleCache, types::InputOutputKey, - value_exchange::filter_value_for_exchange, -}; +use crate::{types::InputOutputKey, value_exchange::filter_value_for_exchange}; use anyhow::bail; use aptos_aggregator::{ delta_math::DeltaHistory, @@ -22,6 +19,7 @@ use aptos_mvhashmap::{ use aptos_types::{ error::{code_invariant_error, PanicError, PanicOr}, executable::ModulePath, + read_only_module_cache::ReadOnlyModuleCache, state_store::state_value::StateValueMetadata, transaction::BlockExecutableTransaction as Transaction, write_set::TransactionWrite, @@ -652,7 +650,7 @@ where /// 3. Entries that were in per-block cache have the same commit index. pub(crate) fn validate_module_reads( &self, - global_module_cache: &ImmutableModuleCache, + global_module_cache: &ReadOnlyModuleCache, per_block_module_cache: &SyncModuleCache>, ) -> bool { if self.non_delayed_field_speculative_failure { @@ -1497,7 +1495,7 @@ mod test { MockVerifiedCode, (), >::new(); - let global_module_cache = ImmutableModuleCache::empty(); + let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); @@ -1532,7 +1530,7 @@ mod test { MockVerifiedCode, (), >::new(); - let global_module_cache = ImmutableModuleCache::empty(); + let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); global_module_cache.insert(0, mock_verified_code(0, None)); @@ -1609,7 +1607,7 @@ mod test { MockVerifiedCode, (), >::new(); - let global_module_cache = ImmutableModuleCache::empty(); + let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); let a = mock_deserialized_code(0, Some(10)); @@ -1669,7 +1667,7 @@ mod test { MockVerifiedCode, (), >::new(); - let global_module_cache = ImmutableModuleCache::empty(); + let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); // Module exists in global cache. diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 67d6787f7a250..661cb08570e5b 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -3,7 +3,6 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{ - code_cache_global::ImmutableModuleCache, counters::{ self, BLOCK_EXECUTOR_INNER_EXECUTE_BLOCK, PARALLEL_EXECUTION_SECONDS, RAYON_EXECUTION_SECONDS, TASK_EXECUTE_SECONDS, TASK_VALIDATE_SECONDS, VM_INIT_SECONDS, @@ -37,6 +36,7 @@ use aptos_types::{ executable::Executable, explicit_sync_wrapper::ExplicitSyncWrapper, on_chain_config::BlockGasLimitType, + read_only_module_cache::ReadOnlyModuleCache, state_store::{state_value::StateValue, TStateView}, transaction::{ block_epilogue::BlockEndInfo, BlockExecutableTransaction as Transaction, BlockOutput, @@ -75,7 +75,7 @@ pub struct BlockExecutor { config: BlockExecutorConfig, executor_thread_pool: Arc, global_module_cache: - Arc>, + Arc>, transaction_commit_hook: Option, phantom: PhantomData<(T, E, S, L, X)>, } @@ -94,7 +94,7 @@ where config: BlockExecutorConfig, executor_thread_pool: Arc, global_module_cache: Arc< - ImmutableModuleCache, + ReadOnlyModuleCache, >, transaction_commit_hook: Option, ) -> Self { @@ -120,7 +120,7 @@ where versioned_cache: &MVHashMap, executor: &E, base_view: &S, - global_module_cache: &ImmutableModuleCache< + global_module_cache: &ReadOnlyModuleCache< ModuleId, CompiledModule, Module, @@ -402,7 +402,7 @@ where fn validate( idx_to_validate: TxnIndex, last_input_output: &TxnLastInputOutput, - global_module_cache: &ImmutableModuleCache< + global_module_cache: &ReadOnlyModuleCache< ModuleId, CompiledModule, Module, @@ -755,7 +755,7 @@ where fn publish_module_writes( txn_idx: TxnIndex, module_write_set: BTreeMap>, - global_module_cache: &ImmutableModuleCache< + global_module_cache: &ReadOnlyModuleCache< ModuleId, CompiledModule, Module, @@ -1198,7 +1198,7 @@ where write: ModuleWrite, txn_idx: TxnIndex, runtime_environment: &RuntimeEnvironment, - global_module_cache: &ImmutableModuleCache< + global_module_cache: &ReadOnlyModuleCache< ModuleId, CompiledModule, Module, @@ -1252,7 +1252,7 @@ where fn apply_output_sequential( txn_idx: TxnIndex, runtime_environment: &RuntimeEnvironment, - global_module_cache: &ImmutableModuleCache< + global_module_cache: &ReadOnlyModuleCache< ModuleId, CompiledModule, Module, diff --git a/aptos-move/block-executor/src/lib.rs b/aptos-move/block-executor/src/lib.rs index c3f0f4d5b767c..902fe9caa0876 100644 --- a/aptos-move/block-executor/src/lib.rs +++ b/aptos-move/block-executor/src/lib.rs @@ -141,7 +141,6 @@ extern crate scopeguard; mod captured_reads; mod code_cache; -pub mod code_cache_global; pub mod counters; pub mod errors; pub mod executor; diff --git a/aptos-move/block-executor/src/proptest_types/bencher.rs b/aptos-move/block-executor/src/proptest_types/bencher.rs index b34768150a126..a764df3f373cf 100644 --- a/aptos-move/block-executor/src/proptest_types/bencher.rs +++ b/aptos-move/block-executor/src/proptest_types/bencher.rs @@ -3,7 +3,6 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{ - code_cache_global::ImmutableModuleCache, executor::BlockExecutor, proptest_types::{ baseline::BaselineOutput, @@ -16,7 +15,7 @@ use crate::{ }; use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, - executable::ExecutableTestType, + executable::ExecutableTestType, read_only_module_cache::ReadOnlyModuleCache, }; use criterion::{BatchSize, Bencher as CBencher}; use num_cpus; @@ -128,7 +127,7 @@ where .build() .unwrap(), ); - let global_module_cache = Arc::new(ImmutableModuleCache::empty()); + let global_module_cache = Arc::new(ReadOnlyModuleCache::empty()); let config = BlockExecutorConfig::new_no_block_limit(num_cpus::get()); let env = MockEnvironment::new(); diff --git a/aptos-move/block-executor/src/proptest_types/tests.rs b/aptos-move/block-executor/src/proptest_types/tests.rs index 7ed0894bec414..df113a2663d59 100644 --- a/aptos-move/block-executor/src/proptest_types/tests.rs +++ b/aptos-move/block-executor/src/proptest_types/tests.rs @@ -3,7 +3,6 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{ - code_cache_global::ImmutableModuleCache, errors::SequentialBlockExecutionError, executor::BlockExecutor, proptest_types::{ @@ -18,7 +17,7 @@ use crate::{ }; use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, - executable::ExecutableTestType, + executable::ExecutableTestType, read_only_module_cache::ReadOnlyModuleCache, }; use claims::{assert_matches, assert_ok}; use num_cpus; @@ -82,7 +81,7 @@ fn run_transactions( >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -219,7 +218,7 @@ fn deltas_writes_mixed_with_block_gas_limit(num_txns: usize, maybe_block_gas_lim >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -272,7 +271,7 @@ fn deltas_resolver_with_block_gas_limit(num_txns: usize, maybe_block_gas_limit: >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -430,7 +429,7 @@ fn publishing_fixed_params_with_block_gas_limit( >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -477,7 +476,7 @@ fn publishing_fixed_params_with_block_gas_limit( Some(max(w_index, r_index) as u64 * MAX_GAS_PER_TXN + 1), ), executor_thread_pool.clone(), - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) // Ensure enough gas limit to commit the module txns (4 is maximum gas per txn) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -558,7 +557,7 @@ fn non_empty_group( >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool.clone(), - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -577,7 +576,7 @@ fn non_empty_group( >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool.clone(), - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_sequential(&env, &transactions, &data_view, false); diff --git a/aptos-move/block-executor/src/unit_tests/mod.rs b/aptos-move/block-executor/src/unit_tests/mod.rs index 4f7fb54a41155..39dca409555c5 100644 --- a/aptos-move/block-executor/src/unit_tests/mod.rs +++ b/aptos-move/block-executor/src/unit_tests/mod.rs @@ -5,7 +5,6 @@ mod code_cache_tests; use crate::{ - code_cache_global::ImmutableModuleCache, errors::SequentialBlockExecutionError, executor::BlockExecutor, proptest_types::{ @@ -30,6 +29,7 @@ use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, executable::{ExecutableTestType, ModulePath}, + read_only_module_cache::ReadOnlyModuleCache, state_store::state_value::StateValueMetadata, write_set::WriteOpKind, }; @@ -87,7 +87,7 @@ fn test_resource_group_deletion() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ); @@ -154,7 +154,7 @@ fn resource_group_bcs_fallback() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ); @@ -254,7 +254,7 @@ fn block_output_err_precedence() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ); @@ -294,7 +294,7 @@ fn skip_rest_gas_limit() { >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), Some(5)), executor_thread_pool, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ); @@ -330,7 +330,7 @@ where >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ImmutableModuleCache::empty()), + Arc::new(ReadOnlyModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); diff --git a/aptos-move/block-executor/src/view.rs b/aptos-move/block-executor/src/view.rs index b70f67a544d5c..8e1a48fca897f 100644 --- a/aptos-move/block-executor/src/view.rs +++ b/aptos-move/block-executor/src/view.rs @@ -8,7 +8,6 @@ use crate::{ CapturedReads, DataRead, DelayedFieldRead, DelayedFieldReadKind, GroupRead, ReadKind, UnsyncReadSet, }, - code_cache_global::ImmutableModuleCache, counters, scheduler::{DependencyResult, DependencyStatus, Scheduler, TWaitForDependency}, value_exchange::{ @@ -36,6 +35,7 @@ use aptos_mvhashmap::{ use aptos_types::{ error::{code_invariant_error, expect_ok, PanicError, PanicOr}, executable::{Executable, ModulePath}, + read_only_module_cache::ReadOnlyModuleCache, state_store::{ errors::StateviewError, state_storage_usage::StateStorageUsage, @@ -991,7 +991,7 @@ impl<'a, T: Transaction, X: Executable> ViewState<'a, T, X> { pub(crate) struct LatestView<'a, T: Transaction, S: TStateView, X: Executable> { base_view: &'a S, pub(crate) global_module_cache: - &'a ImmutableModuleCache, + &'a ReadOnlyModuleCache, pub(crate) runtime_environment: &'a RuntimeEnvironment, pub(crate) latest_view: ViewState<'a, T, X>, pub(crate) txn_idx: TxnIndex, @@ -1000,7 +1000,7 @@ pub(crate) struct LatestView<'a, T: Transaction, S: TStateView, X: impl<'a, T: Transaction, S: TStateView, X: Executable> LatestView<'a, T, S, X> { pub(crate) fn new( base_view: &'a S, - global_module_cache: &'a ImmutableModuleCache< + global_module_cache: &'a ReadOnlyModuleCache< ModuleId, CompiledModule, Module, @@ -2521,7 +2521,7 @@ mod test { let base_view = MockStateView::new(HashMap::new()); let start_counter = 5; let runtime_environment = RuntimeEnvironment::new(vec![]); - let global_module_cache = ImmutableModuleCache::empty(); + let global_module_cache = ReadOnlyModuleCache::empty(); let latest_view = LatestView::::new( &base_view, @@ -2790,7 +2790,7 @@ mod test { counter: RefCell, base_view: MockStateView, empty_global_module_cache: - ImmutableModuleCache, + ReadOnlyModuleCache, runtime_environment: RuntimeEnvironment, } @@ -2804,7 +2804,7 @@ mod test { unsync_map, counter, base_view, - empty_global_module_cache: ImmutableModuleCache::empty(), + empty_global_module_cache: ReadOnlyModuleCache::empty(), runtime_environment, } } diff --git a/types/Cargo.toml b/types/Cargo.toml index 088b34699ac06..79cb83ee9e9cc 100644 --- a/types/Cargo.toml +++ b/types/Cargo.toml @@ -32,6 +32,7 @@ arr_macro = { workspace = true } base64 = { workspace = true } bcs = { workspace = true } bytes = { workspace = true } +crossbeam = { workspace = true } dashmap = { workspace = true } derivative = { workspace = true } fixed = { workspace = true } diff --git a/types/src/lib.rs b/types/src/lib.rs index 69600de4dddfb..bec1414ad4104 100644 --- a/types/src/lib.rs +++ b/types/src/lib.rs @@ -36,6 +36,7 @@ pub mod proof; #[cfg(any(test, feature = "fuzzing"))] pub mod proptest_types; pub mod randomness; +pub mod read_only_module_cache; pub mod serde_helper; pub mod stake_pool; pub mod staking_contract; diff --git a/aptos-move/block-executor/src/code_cache_global.rs b/types/src/read_only_module_cache.rs similarity index 78% rename from aptos-move/block-executor/src/code_cache_global.rs rename to types/src/read_only_module_cache.rs index e5f89f9eb1e81..c6b18615e945b 100644 --- a/aptos-move/block-executor/src/code_cache_global.rs +++ b/types/src/read_only_module_cache.rs @@ -1,8 +1,7 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use aptos_mvhashmap::types::TxnIndex; -use aptos_types::{error::PanicError, explicit_sync_wrapper::ExplicitSyncWrapper}; +use crate::{error::PanicError, explicit_sync_wrapper::ExplicitSyncWrapper}; use crossbeam::utils::CachePadded; use hashbrown::HashMap; use move_vm_types::code::ModuleCode; @@ -15,11 +14,8 @@ use std::{ }, }; -/// Module code stored in cross-block module cache. -// TODO(loader_v2): -// We can move this to move-vm-types, but then we also need to have version generic or expose -// transaction index there, and define PanicError in Move (or convert from VMError). -struct ImmutableModuleCode { +/// Entry stored in [ReadOnlyModuleCache]. +struct Entry { /// True if this code is "valid" within the block execution context (i.e, there has been no /// republishing of this module so far). If false, executor needs to read the module from the /// sync/unsync module caches. @@ -28,16 +24,16 @@ struct ImmutableModuleCode { /// hold: /// 1. Module's version is [None] (storage version). /// 2. Module's code is always verified. - module: CachePadded>>>, + module: CachePadded>>>, } -impl ImmutableModuleCode +impl Entry where VC: Deref>, { /// Returns a new valid module. Returns a (panic) error if the module is not verified or has /// non-storage version. - fn new(module: Arc>>) -> Result { + fn new(module: Arc>>) -> Result { if !module.code().is_verified() || module.version().is_some() { let msg = format!( "Invariant violated for immutable module code : verified ({}), version({:?})", @@ -63,23 +59,23 @@ where self.valid.load(Ordering::Acquire) } - /// Returns the module code stored is this [ImmutableModuleCode]. - fn inner(&self) -> &Arc>> { + /// Returns the module code stored is this [Entry]. + fn inner(&self) -> &Arc>> { self.module.deref() } } -/// An immutable cache for verified code, that can be accessed concurrently thought the block, and -/// only modified at block boundaries. -pub struct ImmutableModuleCache { +/// A read-only module cache for verified code, that can be accessed concurrently within the block. +/// It can only be modified at block boundaries. +pub struct ReadOnlyModuleCache { /// Module cache containing the verified code. - module_cache: ExplicitSyncWrapper>>, + module_cache: ExplicitSyncWrapper>>, /// Maximum cache size. If the size is greater than this limit, the cache is flushed. Note that /// this can only be done at block boundaries. capacity: usize, } -impl ImmutableModuleCache +impl ReadOnlyModuleCache where K: Hash + Eq + Clone, VC: Deref>, @@ -99,7 +95,7 @@ where } /// Returns true if the key exists in immutable cache and the corresponding module is valid. - pub(crate) fn contains_valid(&self, key: &K) -> bool { + pub fn contains_valid(&self, key: &K) -> bool { self.module_cache .acquire() .get(key) @@ -109,7 +105,7 @@ where /// Marks the cached module (if it exists) as invalid. As a result, all subsequent calls to the /// cache for the associated key will result in a cache miss. Note that it is fine for an /// entry not to exist, in which case this is a no-op. - pub(crate) fn mark_invalid(&self, key: &K) { + pub fn mark_invalid(&self, key: &K) { if let Some(module) = self.module_cache.acquire().get(key) { module.mark_invalid(); } @@ -117,7 +113,7 @@ where /// Returns the module stored in cache. If the module has not been cached, or it exists but is /// not valid, [None] is returned. - pub(crate) fn get(&self, key: &K) -> Option>>> { + pub fn get(&self, key: &K) -> Option>>> { self.module_cache.acquire().get(key).and_then(|module| { if module.is_valid() { Some(module.inner().clone()) @@ -142,9 +138,9 @@ where /// these constraints are violated, a panic error is returned. /// 4. If the cache size exceeds its capacity after all verified modules have been inserted, /// the cache is flushed. - pub(crate) fn insert_verified_unchecked( + pub fn insert_verified_unchecked( &self, - modules: impl Iterator>>)>, + modules: impl Iterator>>)>, ) -> Result<(), PanicError> { use hashbrown::hash_map::Entry::*; @@ -166,8 +162,7 @@ where if module.code().is_verified() { let mut module = module.as_ref().clone(); module.set_version(None); - let prev = - module_cache.insert(key.clone(), ImmutableModuleCode::new(Arc::new(module))?); + let prev = module_cache.insert(key.clone(), Entry::new(Arc::new(module))?); // At this point, we must have removed the entry, or returned a panic error. assert!(prev.is_none()) @@ -183,10 +178,10 @@ where /// Insert the module to cache. Used for tests only. #[cfg(any(test, feature = "testing"))] - pub fn insert(&self, key: K, module: Arc>>) { + pub fn insert(&self, key: K, module: Arc>>) { self.module_cache .acquire() - .insert(key, ImmutableModuleCode::new(module).unwrap()); + .insert(key, Entry::new(module).unwrap()); } /// Removes the module from cache. Used for tests only. @@ -209,16 +204,16 @@ mod test { use move_vm_types::code::{mock_deserialized_code, mock_verified_code}; #[test] - fn test_immutable_module_code() { - assert!(ImmutableModuleCode::new(mock_deserialized_code(0, None)).is_err()); - assert!(ImmutableModuleCode::new(mock_deserialized_code(0, Some(22))).is_err()); - assert!(ImmutableModuleCode::new(mock_verified_code(0, Some(22))).is_err()); - assert!(ImmutableModuleCode::new(mock_verified_code(0, None)).is_ok()); + fn test_new_entry() { + assert!(Entry::new(mock_deserialized_code(0, None)).is_err()); + assert!(Entry::new(mock_deserialized_code(0, Some(22))).is_err()); + assert!(Entry::new(mock_verified_code(0, Some(22))).is_err()); + assert!(Entry::new(mock_verified_code(0, None)).is_ok()); } #[test] - fn test_immutable_module_code_validity() { - let module_code = assert_ok!(ImmutableModuleCode::new(mock_verified_code(0, None))); + fn test_mark_entry_invalid() { + let module_code = assert_ok!(Entry::new(mock_verified_code(0, None))); assert!(module_code.is_valid()); module_code.mark_invalid(); @@ -226,8 +221,8 @@ mod test { } #[test] - fn test_global_module_cache() { - let global_cache = ImmutableModuleCache::empty(); + fn test_get_entry() { + let global_cache = ReadOnlyModuleCache::empty(); global_cache.insert(0, mock_verified_code(0, None)); global_cache.insert(1, mock_verified_code(1, None)); @@ -245,13 +240,13 @@ mod test { } #[test] - fn test_insert_verified_for_global_module_cache() { + fn test_insert_verified_for_read_only_module_cache() { let capacity = 10; - let global_cache = ImmutableModuleCache::with_capacity(capacity); + let global_cache = ReadOnlyModuleCache::with_capacity(capacity); let mut new_modules = vec![]; for i in 0..capacity { - new_modules.push((i, mock_verified_code(i, Some(i as TxnIndex)))); + new_modules.push((i, mock_verified_code(i, Some(i as u32)))); } let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); assert!(result.is_ok()); From 9f0f9767e577fbac4d5cc9d2d228b01c9e5d5240 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 5 Nov 2024 17:45:12 +0000 Subject: [PATCH 03/25] [global cache] Draft e2e implementation --- Cargo.lock | 24 ++ Cargo.toml | 2 + aptos-move/aptos-debugger/Cargo.toml | 1 + .../aptos-debugger/src/aptos_debugger.rs | 13 +- .../aptos-e2e-comparison-testing/Cargo.toml | 1 + .../aptos-global-cache-manager/Cargo.toml | 22 ++ .../aptos-global-cache-manager/src/config.rs | 24 ++ .../aptos-global-cache-manager/src/lib.rs | 7 + .../aptos-global-cache-manager/src/manager.rs | 293 ++++++++++++++++++ .../aptos-transaction-benchmarks/Cargo.toml | 1 + .../src/transaction_bench_state.rs | 16 + .../Cargo.toml | 1 + .../src/aptos_test_harness.rs | 12 +- aptos-move/aptos-vm-profiling/Cargo.toml | 1 + .../src/bins/run_aptos_p2p.rs | 6 +- aptos-move/aptos-vm/Cargo.toml | 1 + aptos-move/aptos-vm/src/aptos_vm.rs | 3 + aptos-move/aptos-vm/src/block_executor/mod.rs | 137 +------- aptos-move/aptos-vm/src/lib.rs | 5 +- .../sharded_executor_service.rs | 2 +- .../aptos-vm/tests/sharded_block_executor.rs | 1 + aptos-move/e2e-tests/src/executor.rs | 2 +- execution/executor-benchmark/Cargo.toml | 1 + .../executor-benchmark/src/native_executor.rs | 1 + execution/executor-service/Cargo.toml | 1 + execution/executor/Cargo.toml | 1 + execution/executor/src/chunk_executor/mod.rs | 1 + .../src/chunk_executor/transaction_chunk.rs | 1 + execution/executor/src/db_bootstrapper/mod.rs | 1 + execution/executor/src/fuzzing.rs | 1 + execution/executor/src/tests/mock_vm/mod.rs | 1 + execution/executor/src/tests/mod.rs | 11 +- .../src/workflow/do_get_execution_output.rs | 5 + .../execution/ptx-executor/Cargo.toml | 1 + .../execution/ptx-executor/src/lib.rs | 2 + storage/db-tool/Cargo.toml | 1 + storage/db-tool/src/replay_on_archive.rs | 5 +- types/src/read_only_module_cache.rs | 44 +-- 38 files changed, 481 insertions(+), 172 deletions(-) create mode 100644 aptos-move/aptos-global-cache-manager/Cargo.toml create mode 100644 aptos-move/aptos-global-cache-manager/src/config.rs create mode 100644 aptos-move/aptos-global-cache-manager/src/lib.rs create mode 100644 aptos-move/aptos-global-cache-manager/src/manager.rs diff --git a/Cargo.lock b/Cargo.lock index f27aa4a990d04..d45a83ed7b7a1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -811,6 +811,7 @@ version = "0.1.0" dependencies = [ "anyhow", "aptos-framework", + "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-rest-client", "aptos-types", @@ -1263,6 +1264,7 @@ dependencies = [ "aptos-executor", "aptos-executor-test-helpers", "aptos-executor-types", + "aptos-global-cache-manager", "aptos-logger", "aptos-storage-interface", "aptos-temppath", @@ -1461,6 +1463,7 @@ dependencies = [ "aptos-executor-types", "aptos-experimental-runtimes", "aptos-genesis", + "aptos-global-cache-manager", "aptos-indexer-grpc-table-info", "aptos-infallible", "aptos-logger", @@ -1502,6 +1505,7 @@ dependencies = [ "aptos-experimental-ptx-executor", "aptos-experimental-runtimes", "aptos-genesis", + "aptos-global-cache-manager", "aptos-jellyfish-merkle", "aptos-logger", "aptos-metrics-core", @@ -1540,6 +1544,7 @@ version = "0.1.0" dependencies = [ "aptos-block-partitioner", "aptos-config", + "aptos-global-cache-manager", "aptos-infallible", "aptos-language-e2e-tests", "aptos-logger", @@ -1646,6 +1651,7 @@ name = "aptos-experimental-ptx-executor" version = "0.1.0" dependencies = [ "aptos-experimental-runtimes", + "aptos-global-cache-manager", "aptos-infallible", "aptos-logger", "aptos-metrics-core", @@ -2049,6 +2055,19 @@ dependencies = [ "ureq", ] +[[package]] +name = "aptos-global-cache-manager" +version = "0.0.1" +dependencies = [ + "aptos-crypto", + "aptos-types", + "aptos-vm-environment", + "move-binary-format", + "move-core-types", + "move-vm-runtime", + "parking_lot 0.12.1", +] + [[package]] name = "aptos-global-constants" version = "0.1.0" @@ -2833,6 +2852,7 @@ dependencies = [ "aptos-consensus", "aptos-crypto", "aptos-gas-profiling", + "aptos-global-cache-manager", "aptos-logger", "aptos-rest-client", "aptos-types", @@ -4184,6 +4204,7 @@ dependencies = [ "aptos-block-executor", "aptos-block-partitioner", "aptos-crypto", + "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-logger", "aptos-metrics-core", @@ -4288,6 +4309,7 @@ dependencies = [ "aptos-crypto", "aptos-framework", "aptos-gas-schedule", + "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-resource-viewer", "aptos-storage-interface", @@ -4461,6 +4483,7 @@ dependencies = [ "aptos-gas-algebra", "aptos-gas-meter", "aptos-gas-schedule", + "aptos-global-cache-manager", "aptos-infallible", "aptos-language-e2e-tests", "aptos-logger", @@ -4584,6 +4607,7 @@ dependencies = [ "anyhow", "aptos-cached-packages", "aptos-gas-schedule", + "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-move-stdlib", "aptos-native-interface", diff --git a/Cargo.toml b/Cargo.toml index 64384fd8c1c8f..0c6a40ff966b9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,6 +16,7 @@ members = [ "aptos-move/aptos-gas-profiling", "aptos-move/aptos-gas-schedule", "aptos-move/aptos-gas-schedule-updator", + "aptos-move/aptos-global-cache-manager", "aptos-move/aptos-memory-usage-tracker", "aptos-move/aptos-native-interface", "aptos-move/aptos-release-builder", @@ -357,6 +358,7 @@ aptos-gas-schedule = { path = "aptos-move/aptos-gas-schedule" } aptos-gas-schedule-updator = { path = "aptos-move/aptos-gas-schedule-updator" } aptos-genesis = { path = "crates/aptos-genesis" } aptos-github-client = { path = "crates/aptos-github-client" } +aptos-global-cache-manager = { path = "aptos-move/aptos-global-cache-manager" } aptos-global-constants = { path = "config/global-constants" } aptos-id-generator = { path = "crates/aptos-id-generator" } aptos-indexer = { path = "crates/indexer" } diff --git a/aptos-move/aptos-debugger/Cargo.toml b/aptos-move/aptos-debugger/Cargo.toml index 8e83673767603..d8bfc76cd3e60 100644 --- a/aptos-move/aptos-debugger/Cargo.toml +++ b/aptos-move/aptos-debugger/Cargo.toml @@ -18,6 +18,7 @@ aptos-block-executor = { workspace = true } aptos-consensus = { workspace = true } aptos-crypto = { workspace = true } aptos-gas-profiling = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-logger = { workspace = true } aptos-rest-client = { workspace = true } aptos-types = { workspace = true } diff --git a/aptos-move/aptos-debugger/src/aptos_debugger.rs b/aptos-move/aptos-debugger/src/aptos_debugger.rs index 743bea9f09616..81e95438fb8d1 100644 --- a/aptos-move/aptos-debugger/src/aptos_debugger.rs +++ b/aptos-move/aptos-debugger/src/aptos_debugger.rs @@ -4,6 +4,7 @@ use anyhow::{bail, format_err, Result}; use aptos_block_executor::txn_commit_hook::NoOpTransactionCommitHook; use aptos_gas_profiling::{GasProfiler, TransactionGasLog}; +use aptos_global_cache_manager::GlobalCacheManager; use aptos_rest_client::Client; use aptos_types::{ account_address::AccountAddress, @@ -428,9 +429,15 @@ fn execute_block_no_limit( state_view: &DebuggerStateView, concurrency_level: usize, ) -> Result, VMStatus> { - BlockAptosVM::execute_block::<_, NoOpTransactionCommitHook>( + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + global_cache_manager.mark_block_execution_start(state_view, None)?; + let result = BlockAptosVM::execute_block::< + _, + NoOpTransactionCommitHook, + >( sig_verified_txns, state_view, + &global_cache_manager, BlockExecutorConfig { local: BlockExecutorLocalConfig { concurrency_level, @@ -441,5 +448,7 @@ fn execute_block_no_limit( }, None, ) - .map(BlockOutput::into_transaction_outputs_forced) + .map(BlockOutput::into_transaction_outputs_forced); + global_cache_manager.mark_block_execution_end(None)?; + result } diff --git a/aptos-move/aptos-e2e-comparison-testing/Cargo.toml b/aptos-move/aptos-e2e-comparison-testing/Cargo.toml index 30c851670b4df..71af9ffdc387d 100644 --- a/aptos-move/aptos-e2e-comparison-testing/Cargo.toml +++ b/aptos-move/aptos-e2e-comparison-testing/Cargo.toml @@ -13,6 +13,7 @@ default-run = "aptos-comparison-testing" [dependencies] anyhow = { workspace = true } aptos-framework = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-rest-client = { workspace = true } aptos-types = { workspace = true } diff --git a/aptos-move/aptos-global-cache-manager/Cargo.toml b/aptos-move/aptos-global-cache-manager/Cargo.toml new file mode 100644 index 0000000000000..71fcaae3ce1e5 --- /dev/null +++ b/aptos-move/aptos-global-cache-manager/Cargo.toml @@ -0,0 +1,22 @@ +[package] +name = "aptos-global-cache-manager" +description = "Aptos global module and environement cache manager" +version = "0.0.1" + +# Workspace inherited keys +authors = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +license = { workspace = true } +publish = { workspace = true } +repository = { workspace = true } +rust-version = { workspace = true } + +[dependencies] +aptos-crypto = { workspace = true } +aptos-types = { workspace = true } +aptos-vm-environment = { workspace = true } +parking_lot = { workspace = true } +move-binary-format = { workspace = true } +move-core-types = { workspace = true } +move-vm-runtime = { workspace = true } diff --git a/aptos-move/aptos-global-cache-manager/src/config.rs b/aptos-move/aptos-global-cache-manager/src/config.rs new file mode 100644 index 0000000000000..0f5cdd14d215a --- /dev/null +++ b/aptos-move/aptos-global-cache-manager/src/config.rs @@ -0,0 +1,24 @@ +// Copyright © Aptos Foundation +// SPDX-License-Identifier: Apache-2.0 + +/// Configuration used for global caches. +pub struct GlobalCacheConfig { + /// The maximum size of module cache. If module cache exceeds this capacity, it should be + /// flushed. + pub module_cache_capacity: usize, + /// The maximum size of struct name re-indexing map stored in runtime environment. + pub struct_name_index_map_capacity: usize, +} + +impl Default for GlobalCacheConfig { + fn default() -> Self { + // TODO(loader_v2): + // Right now these are just some numbers, we can set them based on the upper bounds of + // module or identifier sizes, or keep track in read-only module cache how many bytes we + // are using. + Self { + module_cache_capacity: 100_000, + struct_name_index_map_capacity: 100_000, + } + } +} diff --git a/aptos-move/aptos-global-cache-manager/src/lib.rs b/aptos-move/aptos-global-cache-manager/src/lib.rs new file mode 100644 index 0000000000000..a5844418599cc --- /dev/null +++ b/aptos-move/aptos-global-cache-manager/src/lib.rs @@ -0,0 +1,7 @@ +// Copyright © Aptos Foundation +// SPDX-License-Identifier: Apache-2.0 + +pub(crate) mod config; +mod manager; + +pub use manager::GlobalCacheManager; diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs new file mode 100644 index 0000000000000..1e632ba644eda --- /dev/null +++ b/aptos-move/aptos-global-cache-manager/src/manager.rs @@ -0,0 +1,293 @@ +// Copyright © Aptos Foundation +// SPDX-License-Identifier: Apache-2.0 + +use crate::config::GlobalCacheConfig; +use aptos_crypto::HashValue; +use aptos_types::{ + read_only_module_cache::ReadOnlyModuleCache, state_store::StateView, + vm::modules::AptosModuleExtension, +}; +use aptos_vm_environment::environment::AptosEnvironment; +use move_binary_format::{errors::Location, CompiledModule}; +use move_core_types::{ + language_storage::ModuleId, + vm_status::{StatusCode, VMStatus}, +}; +use move_vm_runtime::{Module, WithRuntimeEnvironment}; +use parking_lot::Mutex; +use std::{ + hash::Hash, + ops::Deref, + sync::{ + atomic::{AtomicBool, Ordering}, + Arc, + }, +}; + +/// Returns an invariant violation [VMStatus]. +fn invariant_violation(msg: &str) -> VMStatus { + VMStatus::error( + StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some(msg.to_string()), + ) +} + +/// Represents previously executed block, recorded by [GlobalCacheManager]. +#[derive(Clone, Copy, Eq, Hash, PartialEq, PartialOrd, Ord)] +enum BlockId { + /// No block has been executed yet. + Unset, + /// Block of transactions has been executed, with known or unknown hash. Usually, the hash is + /// [None] in tests, replay, etc. + Set(Option), +} + +impl BlockId { + /// Returns true if the ID corresponds to no executed blocks. + fn is_unset(&self) -> bool { + matches!(self, Self::Unset) + } +} + +/// Manages global caches, e.g., modules or execution environment. Should not be used concurrently. +pub struct GlobalCacheManagerInner { + /// Different configurations used for handling global caches. + config: GlobalCacheConfig, + /// Cache for modules. It is read-only for any concurrent execution, and can only be mutated + /// when it is known that there are no concurrent accesses, e.g., at blok boundaries. + /// [GlobalCacheManager] tries to ensure that these invariants always hold. + module_cache: Arc>, + + /// Identifies previously executed block, initially [BlockId::Unset]. + previous_block_id: Mutex, + /// Identifies the previously used execution environment, initially [None]. The environment, as + /// long as it does not change, it maintained across multiple block executions. + previous_environment: Mutex>, + + /// A marker that indicates that the state of global caches is ready for block execution. Used + /// to prevent concurrent block executions. + ready_for_next_block: AtomicBool, +} + +impl GlobalCacheManagerInner +where + K: Hash + Eq + Clone, + VC: Deref>, +{ + /// Returns a new instance of [GlobalCacheManagerInner] with default [GlobalCacheConfig]. + pub fn new_with_default_config() -> Self { + Self::new_with_config(GlobalCacheConfig::default()) + } + + /// Returns a new instance of [GlobalCacheManagerInner] with the provided [GlobalCacheConfig]. + pub fn new_with_config(config: GlobalCacheConfig) -> Self { + Self { + config, + module_cache: Arc::new(ReadOnlyModuleCache::empty()), + previous_block_id: Mutex::new(BlockId::Unset), + previous_environment: Mutex::new(None), + ready_for_next_block: AtomicBool::new(true), + } + } + + /// Sets the state of global caches prior to block execution on top of the provided state (with + /// the block ID). Should always sbe called prior to block execution. + /// + /// The caches stored globally (modules, struct name re-indexing map and type caches) are all + /// flushed if: + /// 1. Previously executed block ID does not match the provided value. + /// 2. The environment has changed for this state. + /// 3. The size of the struct name re-indexing map is too large. + /// 4. The size of the module cache is too large. + /// + /// Marks [GlobalCacheManagerInner] as not ready for next block execution. If called + /// concurrently, only a single invocation ever succeeds and other calls return an error. + pub fn mark_block_execution_start( + &self, + state_view: &impl StateView, + previous_block_id: Option, + ) -> Result<(), VMStatus> { + let recorded_previous_block_id = { + // Acquire a lock, and check if we are ready to execute the next block. + let previous_block_id = self.previous_block_id.lock(); + if !self.ready_for_next_block.load(Ordering::SeqCst) { + let msg = "Trying to execute blocks concurrently over shared global state"; + return Err(invariant_violation(msg)); + } + + // Prepare for execution. Set the flag as not ready to ensure that blocks are not + // executed concurrently using the same cache. + self.ready_for_next_block.store(false, Ordering::SeqCst); + *previous_block_id + }; + + // From here, we perform checks if we need to flush the global caches. If so, this variable + // is set to true. + let mut flush_all_caches = false; + + // Check 1: We must be executing on top of the state we have seen just before. + use BlockId::*; + match (recorded_previous_block_id, previous_block_id) { + // We execute on top of empty state, everything is ok. + (Unset, None) | (Unset, Some(_)) => {}, + + // We execute on top of different (maybe also unspecified) state. In this case, caches + // need to be reset. + (Set(None), None) | (Set(None), Some(_)) | (Set(Some(_)), None) => { + flush_all_caches = true; + }, + + // Otherwise, just check if block hashes do not match. + (Set(Some(recorded_hash)), Some(hash)) => { + if recorded_hash != hash { + flush_all_caches = true; + }; + }, + }; + + // Check 2: Reset global environment if it has changed. If so, caches needs to be flushed. + let new_environment = + AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); + let mut previous_environment = self.previous_environment.lock(); + match previous_environment.as_ref() { + Some(environment) => { + if environment != &new_environment { + *previous_environment = Some(new_environment); + flush_all_caches = true; + } + }, + None => { + // If the environment is not yet set, set it. + debug_assert!(self.previous_block_id.lock().is_unset()); + *previous_environment = Some(new_environment); + }, + } + + // Check 3: At this point, environment is set to the most-up-to-date value. Check the size + // of caches is within bounds. + let runtime_environment = previous_environment + .as_ref() + .expect("Environment has to be set") + .runtime_environment(); + let struct_name_index_map_size = runtime_environment + .struct_name_index_map_size() + .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; + if struct_name_index_map_size > self.config.struct_name_index_map_capacity { + flush_all_caches = true; + } + if self.module_cache.size() > self.config.module_cache_capacity { + flush_all_caches = true; + } + + // Finally, if flag is set, flush the caches. + if flush_all_caches { + runtime_environment.flush_struct_name_and_info_caches(); + self.module_cache.flush_unchecked(); + } + + Ok(()) + } + + /// Should always be called after block execution. Sets the [GlobalCacheManagerInner] to be + /// execution-ready (and if it is already execution-ready, returns an error). Sets the ID for + /// the executed block so that the next execution can check it. + pub fn mark_block_execution_end( + &self, + executed_block_id: Option, + ) -> Result<(), VMStatus> { + // We are done executing a block, reset the previous block id. Do everything under lock to + // ensure it is not possible to execute blocks concurrently. + let mut previous_block_id = self.previous_block_id.lock(); + if self.ready_for_next_block.load(Ordering::SeqCst) { + let msg = "Should not be possible to mark block execution end for execution-ready \ + global cache, check if blocks are executed concurrently"; + return Err(invariant_violation(msg)); + } + *previous_block_id = BlockId::Set(executed_block_id); + + // Set the flag that the global cache is ready for next execution. + self.ready_for_next_block.store(true, Ordering::SeqCst); + + Ok(()) + } + + /// Returns the cached environment that [GlobalCacheManagerInner::mark_block_execution_start] + /// must set. If it has not been set, an invariant violation error is returned. + pub fn environment(&self) -> Result { + self.previous_environment.lock().clone().ok_or_else(|| { + invariant_violation("Environment must always be set at block execution start") + }) + } + + /// Returns the global module cache. + pub fn module_cache(&self) -> Arc> { + self.module_cache.clone() + } +} + +/// Same as [GlobalCacheManagerInner], but uses concrete types used by execution on Aptos instead +/// of generics. Allows us not to propagate generic type parameters everywhere (for now), but be +/// able to mock and test. +pub struct GlobalCacheManager { + inner: GlobalCacheManagerInner, +} + +impl GlobalCacheManager { + /// Returns a new instance of [GlobalCacheManager] with default [GlobalCacheConfig]. + pub fn new_with_default_config() -> Self { + Self { + inner: GlobalCacheManagerInner::new_with_default_config(), + } + } +} + +impl Deref for GlobalCacheManager { + type Target = GlobalCacheManagerInner; + + fn deref(&self) -> &Self::Target { + &self.inner + } +} + +#[cfg(test)] +mod test { + // use super::*; + // use aptos_language_e2e_tests::data_store::FakeDataStore; + // use aptos_types::on_chain_config::{FeatureFlag, Features}; + // use aptos_vm_environment::environment::AptosEnvironment; + // use claims::assert_ok; + // use move_vm_types::code::mock_verified_code; + // + // #[test] + // fn test_cross_block_module_cache_flush() { + // let global_module_cache = ReadOnlyModuleCache::empty(); + // + // global_module_cache.insert(0, mock_verified_code(0, None)); + // assert_eq!(global_module_cache.size(), 1); + // + // global_module_cache.flush_unchecked(); + // assert_eq!(global_module_cache.size(), 0); + // + // // Now check that cache is flushed when the environment is flushed. + // let mut state_view = FakeDataStore::default(); + // let env_old = AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); + // + // for i in 0..10 { + // global_module_cache.insert(i, mock_verified_code(i, None)); + // } + // assert_eq!(global_module_cache.size(), 10); + // + // let mut features = Features::default(); + // features.disable(FeatureFlag::KEYLESS_ACCOUNTS); + // state_view.set_features(features); + // + // // New environment means we need to also flush global caches - to invalidate struct name + // // indices. + // let env_new = assert_ok!(get_environment_with_delayed_field_optimization_enabled( + // &state_view, + // &global_module_cache, + // )); + // assert!(env_old != env_new); + // assert_eq!(global_module_cache.size(), 0); + // } +} diff --git a/aptos-move/aptos-transaction-benchmarks/Cargo.toml b/aptos-move/aptos-transaction-benchmarks/Cargo.toml index 3fe147d12d47a..43674345b0732 100644 --- a/aptos-move/aptos-transaction-benchmarks/Cargo.toml +++ b/aptos-move/aptos-transaction-benchmarks/Cargo.toml @@ -17,6 +17,7 @@ aptos-bitvec = { workspace = true } aptos-block-executor = { workspace = true } aptos-block-partitioner = { workspace = true } aptos-crypto = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } diff --git a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs index d3a71a1d22862..1e177095ef345 100644 --- a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs +++ b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs @@ -8,6 +8,7 @@ use aptos_block_partitioner::{ v2::config::PartitionerV2Config, BlockPartitioner, PartitionerConfig, }; use aptos_crypto::HashValue; +use aptos_global_cache_manager::GlobalCacheManager; use aptos_language_e2e_tests::{ account_universe::{AUTransactionGen, AccountPickStyle, AccountUniverse, AccountUniverseGen}, data_store::FakeDataStore, @@ -211,6 +212,11 @@ where maybe_block_gas_limit: Option, ) -> (Vec, usize) { let block_size = transactions.len(); + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + global_cache_manager + .mark_block_execution_start(self.state_view.as_ref(), None) + .unwrap(); + let timer = Instant::now(); let output = BlockAptosVM::execute_block::< _, @@ -218,12 +224,14 @@ where >( transactions, self.state_view.as_ref(), + &global_cache_manager, BlockExecutorConfig::new_maybe_block_limit(1, maybe_block_gas_limit), None, ) .expect("VM should not fail to start") .into_transaction_outputs_forced(); let exec_time = timer.elapsed().as_millis(); + global_cache_manager.mark_block_execution_end(None).unwrap(); (output, block_size * 1000 / exec_time as usize) } @@ -259,6 +267,11 @@ where maybe_block_gas_limit: Option, ) -> (Vec, usize) { let block_size = transactions.len(); + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + global_cache_manager + .mark_block_execution_start(self.state_view.as_ref(), None) + .unwrap(); + let timer = Instant::now(); let output = BlockAptosVM::execute_block::< _, @@ -266,6 +279,7 @@ where >( transactions, self.state_view.as_ref(), + &global_cache_manager, BlockExecutorConfig::new_maybe_block_limit( concurrency_level_per_shard, maybe_block_gas_limit, @@ -276,6 +290,8 @@ where .into_transaction_outputs_forced(); let exec_time = timer.elapsed().as_millis(); + global_cache_manager.mark_block_execution_end(None).unwrap(); + (output, block_size * 1000 / exec_time as usize) } diff --git a/aptos-move/aptos-transactional-test-harness/Cargo.toml b/aptos-move/aptos-transactional-test-harness/Cargo.toml index c0e44b746718d..af96620411278 100644 --- a/aptos-move/aptos-transactional-test-harness/Cargo.toml +++ b/aptos-move/aptos-transactional-test-harness/Cargo.toml @@ -19,6 +19,7 @@ aptos-cached-packages = { workspace = true } aptos-crypto = { workspace = true } aptos-framework = { workspace = true } aptos-gas-schedule = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-resource-viewer = { workspace = true } aptos-storage-interface = { workspace = true } diff --git a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs index 534608bfc2fac..e07aec25a6fc5 100644 --- a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs +++ b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs @@ -14,11 +14,9 @@ use aptos_language_e2e_tests::data_store::{FakeDataStore, GENESIS_CHANGE_SET_HEA use aptos_resource_viewer::{AnnotatedMoveValue, AptosValueAnnotator}; use aptos_types::{ account_config::{aptos_test_root_address, AccountResource, CoinStoreResource}, - block_executor::config::BlockExecutorConfigFromOnchain, block_metadata::BlockMetadata, chain_id::ChainId, contract_event::ContractEvent, - on_chain_config::BlockGasLimitType, state_store::{state_key::StateKey, table::TableHandle, TStateView}, transaction::{ signature_verified_transaction::into_signature_verified_block, @@ -526,6 +524,16 @@ impl<'a> AptosTestAdapter<'a> { .execute_block(&sig_verified_block, &self.storage.clone(), onchain_config)? .into_inner(); + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + global_cache_manager.mark_block_execution_start(&state_view, None)?; + let result = AptosVM::execute_block_no_limit( + &sig_verified_block, + &state_view, + &global_cache_manager, + ); + global_cache_manager.mark_block_execution_end(None)?; + + let mut outputs = result?; assert_eq!(outputs.len(), 1); let output = outputs.pop().unwrap(); diff --git a/aptos-move/aptos-vm-profiling/Cargo.toml b/aptos-move/aptos-vm-profiling/Cargo.toml index 68bfadfcd39ea..dcb4c5701e4c5 100644 --- a/aptos-move/aptos-vm-profiling/Cargo.toml +++ b/aptos-move/aptos-vm-profiling/Cargo.toml @@ -19,6 +19,7 @@ smallvec = { workspace = true } aptos-cached-packages = { workspace = true } aptos-gas-schedule = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-move-stdlib = { workspace = true } aptos-native-interface = { workspace = true } diff --git a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs index c0972ced8a47a..3ad2a722eab4a 100644 --- a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs +++ b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs @@ -50,7 +50,11 @@ fn main() -> Result<()> { let res = AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store)?; for i in 0..NUM_TXNS { - assert!(res[i as usize].status().status().unwrap().is_success()); + assert!(result.as_ref().unwrap()[i as usize] + .status() + .status() + .unwrap() + .is_success()); } Ok(()) diff --git a/aptos-move/aptos-vm/Cargo.toml b/aptos-move/aptos-vm/Cargo.toml index a3458288716d2..abae28d9434ef 100644 --- a/aptos-move/aptos-vm/Cargo.toml +++ b/aptos-move/aptos-vm/Cargo.toml @@ -24,6 +24,7 @@ aptos-framework = { workspace = true } aptos-gas-algebra = { workspace = true } aptos-gas-meter = { workspace = true } aptos-gas-schedule = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } aptos-memory-usage-tracker = { workspace = true } diff --git a/aptos-move/aptos-vm/src/aptos_vm.rs b/aptos-move/aptos-vm/src/aptos_vm.rs index 5b65381445967..8f5f411416101 100644 --- a/aptos-move/aptos-vm/src/aptos_vm.rs +++ b/aptos-move/aptos-vm/src/aptos_vm.rs @@ -36,6 +36,7 @@ use aptos_framework::{ use aptos_gas_algebra::{Gas, GasQuantity, NumBytes, Octa}; use aptos_gas_meter::{AptosGasMeter, GasAlgebra}; use aptos_gas_schedule::{AptosGasParameters, VMGasParameters}; +use aptos_global_cache_manager::GlobalCacheManager; use aptos_logger::{enabled, prelude::*, Level}; use aptos_metrics_core::TimerHelper; #[cfg(any(test, feature = "testing"))] @@ -2795,6 +2796,7 @@ impl VMBlockExecutor for AptosVMBlockExecutor { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), + global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { fail_point!("move_adapter::execute_block", |_| { @@ -2817,6 +2819,7 @@ impl VMBlockExecutor for AptosVMBlockExecutor { >( transactions, state_view, + global_cache_manager, BlockExecutorConfig { local: BlockExecutorLocalConfig { concurrency_level: AptosVM::get_concurrency_level(), diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index 29093463d0de8..5fae5f7faec66 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -16,6 +16,7 @@ use aptos_block_executor::{ task::TransactionOutput as BlockExecutorTransactionOutput, txn_commit_hook::TransactionCommitHook, types::InputOutputKey, }; +use aptos_global_cache_manager::GlobalCacheManager; use aptos_infallible::Mutex; use aptos_types::{ block_executor::config::BlockExecutorConfig, @@ -23,34 +24,27 @@ use aptos_types::{ error::PanicError, executable::ExecutableTestType, fee_statement::FeeStatement, - read_only_module_cache::ReadOnlyModuleCache, state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView, StateViewId}, transaction::{ signature_verified_transaction::SignatureVerifiedTransaction, BlockOutput, TransactionOutput, TransactionStatus, }, - vm::modules::AptosModuleExtension, write_set::WriteOp, }; -use aptos_vm_environment::environment::AptosEnvironment; use aptos_vm_logging::{flush_speculative_logs, init_speculative_logs}; use aptos_vm_types::{ abstract_write_op::AbstractResourceWriteOp, module_write_set::ModuleWrite, output::VMOutput, resolver::ResourceGroupSize, }; -use move_binary_format::{errors::Location, CompiledModule}; use move_core_types::{ - language_storage::{ModuleId, StructTag}, + language_storage::StructTag, value::MoveTypeLayout, vm_status::{StatusCode, VMStatus}, }; -use move_vm_runtime::{Module, WithRuntimeEnvironment}; use move_vm_types::delayed_values::delayed_field_id::DelayedFieldID; use once_cell::sync::{Lazy, OnceCell}; use std::{ collections::{BTreeMap, HashSet}, - hash::Hash, - ops::Deref, sync::Arc, }; @@ -64,60 +58,6 @@ static RAYON_EXEC_POOL: Lazy> = Lazy::new(|| { ) }); -/// Immutable global module cache that can be shared across multiple block executions. The size of -/// the cache is fixed within a single block (modules are not inserted or removed) and it is only -/// mutated at the block boundaries. Do not use if multiple blocks are executed concurrently. -static GLOBAL_MODULE_CACHE: Lazy< - Arc>, -> = Lazy::new(|| Arc::new(ReadOnlyModuleCache::empty())); - -/// The maximum size of struct name index map in runtime environment. Checked at block boundaries -/// only. -const MAX_STRUCT_NAME_INDEX_MAP_SIZE: usize = 100_000; - -/// A cached environment that can be persisted globally across blocks. -static GLOBAL_ENVIRONMENT: Lazy>> = Lazy::new(|| Mutex::new(None)); - -/// Returns the cached environment if it exists and has the same configuration as if it was -/// created based on the current state, or creates a new one and caches it. Should only be -/// called at the block boundaries. -fn get_environment_with_delayed_field_optimization_enabled( - state_view: &impl StateView, - global_module_cache: &ReadOnlyModuleCache, -) -> Result -where - K: Hash + Eq + Clone, - VC: Deref>, -{ - // Create a new environment. - let current_env = AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); - - // Lock the cache, and check if the environment is the same. - let mut global_environment = GLOBAL_ENVIRONMENT.lock(); - if let Some(previous_env) = global_environment.as_ref() { - if ¤t_env == previous_env { - let runtime_env = previous_env.runtime_environment(); - let struct_name_index_map_size = runtime_env - .struct_name_index_map_size() - .map_err(|e| e.finish(Location::Undefined).into_vm_status())?; - if struct_name_index_map_size > MAX_STRUCT_NAME_INDEX_MAP_SIZE { - // Cache is too large, flush it. Also flush the module cache. - runtime_env.flush_struct_name_and_info_caches(); - global_module_cache.flush_unchecked(); - } - return Ok(previous_env.clone()); - } - } - - // It is not cached or has changed, so we have to reset it. As a result, we need to flush - // the cross-block cache because we need to reload all modules with new configs. - *global_environment = Some(current_env.clone()); - drop(global_environment); - global_module_cache.flush_unchecked(); - - Ok(current_env) -} - /// Output type wrapper used by block executor. VM output is stored first, then /// transformed into TransactionOutput type that is returned. #[derive(Debug)] @@ -456,9 +396,7 @@ impl BlockAptosVM { executor_thread_pool: Arc, signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, - global_module_cache: Arc< - ReadOnlyModuleCache, - >, + global_cache_manager: &GlobalCacheManager, config: BlockExecutorConfig, transaction_commit_listener: Option, ) -> Result, VMStatus> { @@ -472,11 +410,6 @@ impl BlockAptosVM { BLOCK_EXECUTOR_CONCURRENCY.set(config.local.concurrency_level as i64); - let environment = get_environment_with_delayed_field_optimization_enabled( - state_view, - global_module_cache.as_ref(), - )?; - let executor = BlockExecutor::< SignatureVerifiedTransaction, AptosExecutorTask, @@ -486,10 +419,11 @@ impl BlockAptosVM { >::new( config, executor_thread_pool, - global_module_cache, + global_cache_manager.module_cache(), transaction_commit_listener, ); + let environment = global_cache_manager.environment()?; let ret = executor.execute_block(environment, signature_verified_block, state_view); match ret { Ok(block_output) => { @@ -521,7 +455,7 @@ impl BlockAptosVM { } } - pub fn execute_block_on_thread_pool_without_global_module_cache< + pub fn execute_block_on_thread_pool_without_global_caches< S: StateView + Sync, L: TransactionCommitHook, >( @@ -531,23 +465,29 @@ impl BlockAptosVM { config: BlockExecutorConfig, transaction_commit_listener: Option, ) -> Result, VMStatus> { - Self::execute_block_on_thread_pool::( + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + global_cache_manager.mark_block_execution_start(state_view, None)?; + + let result = Self::execute_block_on_thread_pool::( executor_thread_pool, signature_verified_block, state_view, - Arc::new(ReadOnlyModuleCache::empty()), + &global_cache_manager, config, transaction_commit_listener, - ) + ); + global_cache_manager.mark_block_execution_end(None)?; + result } - /// Uses shared thread pool and shared global module cache to execute blocks. + /// Uses shared thread pool to execute blocks. pub fn execute_block< S: StateView + Sync, L: TransactionCommitHook, >( signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, + global_cache_manager: &GlobalCacheManager, config: BlockExecutorConfig, transaction_commit_listener: Option, ) -> Result, VMStatus> { @@ -555,52 +495,9 @@ impl BlockAptosVM { Arc::clone(&RAYON_EXEC_POOL), signature_verified_block, state_view, - Arc::clone(&GLOBAL_MODULE_CACHE), + global_cache_manager, config, transaction_commit_listener, ) } } - -#[cfg(test)] -mod test { - use super::*; - use aptos_language_e2e_tests::data_store::FakeDataStore; - use aptos_types::on_chain_config::{FeatureFlag, Features}; - use aptos_vm_environment::environment::AptosEnvironment; - use claims::assert_ok; - use move_vm_types::code::mock_verified_code; - - #[test] - fn test_cross_block_module_cache_flush() { - let global_module_cache = ReadOnlyModuleCache::empty(); - - global_module_cache.insert(0, mock_verified_code(0, None)); - assert_eq!(global_module_cache.size(), 1); - - global_module_cache.flush_unchecked(); - assert_eq!(global_module_cache.size(), 0); - - // Now check that cache is flushed when the environment is flushed. - let mut state_view = FakeDataStore::default(); - let env_old = AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); - - for i in 0..10 { - global_module_cache.insert(i, mock_verified_code(i, None)); - } - assert_eq!(global_module_cache.size(), 10); - - let mut features = Features::default(); - features.disable(FeatureFlag::KEYLESS_ACCOUNTS); - state_view.set_features(features); - - // New environment means we need to also flush global caches - to invalidate struct name - // indices. - let env_new = assert_ok!(get_environment_with_delayed_field_optimization_enabled( - &state_view, - &global_module_cache, - )); - assert!(env_old != env_new); - assert_eq!(global_module_cache.size(), 0); - } -} diff --git a/aptos-move/aptos-vm/src/lib.rs b/aptos-move/aptos-vm/src/lib.rs index 56e876ee4b887..e99784753a552 100644 --- a/aptos-move/aptos-vm/src/lib.rs +++ b/aptos-move/aptos-vm/src/lib.rs @@ -167,12 +167,13 @@ pub trait VMBlockExecutor: Send + Sync { onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus>; - /// Executes a block of transactions and returns output for each one of them, - /// Without applying any block limit + /// Executes a block of transactions and returns output for each one of them, without applying + /// any block limit. fn execute_block_no_limit( &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), + global_cache_manager: &GlobalCacheManager, ) -> Result, VMStatus> { self.execute_block( transactions, diff --git a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs index efe860c37103e..5ad1dc5e602f6 100644 --- a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs +++ b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs @@ -135,7 +135,7 @@ impl ShardedExecutorService { ); }); s.spawn(move |_| { - let ret = BlockAptosVM::execute_block_on_thread_pool_without_global_module_cache( + let ret = BlockAptosVM::execute_block_on_thread_pool_without_global_caches( executor_thread_pool, &signature_verified_transactions, aggr_overridden_state_view.as_ref(), diff --git a/aptos-move/aptos-vm/tests/sharded_block_executor.rs b/aptos-move/aptos-vm/tests/sharded_block_executor.rs index 5968d0a495ab9..2f5c08eaa86a1 100644 --- a/aptos-move/aptos-vm/tests/sharded_block_executor.rs +++ b/aptos-move/aptos-vm/tests/sharded_block_executor.rs @@ -187,6 +187,7 @@ fn test_partitioner_v2_connected_component_sharded_block_executor_with_random_tr mod test_utils { use aptos_block_partitioner::BlockPartitioner; + use aptos_global_cache_manager::GlobalCacheManager; use aptos_language_e2e_tests::{ account::AccountData, common_transactions::peer_to_peer_txn, data_store::FakeDataStore, executor::FakeExecutor, diff --git a/aptos-move/e2e-tests/src/executor.rs b/aptos-move/e2e-tests/src/executor.rs index b97961dad4efc..6c64a10fffa30 100644 --- a/aptos-move/e2e-tests/src/executor.rs +++ b/aptos-move/e2e-tests/src/executor.rs @@ -636,7 +636,7 @@ impl FakeExecutor { }, onchain: onchain_config, }; - BlockAptosVM::execute_block_on_thread_pool_without_global_module_cache::< + BlockAptosVM::execute_block_on_thread_pool_without_global_caches::< _, NoOpTransactionCommitHook, >( diff --git a/execution/executor-benchmark/Cargo.toml b/execution/executor-benchmark/Cargo.toml index 4f99fe0a77268..6c572c0b7d329 100644 --- a/execution/executor-benchmark/Cargo.toml +++ b/execution/executor-benchmark/Cargo.toml @@ -25,6 +25,7 @@ aptos-executor-types = { workspace = true } aptos-experimental-ptx-executor = { workspace = true } aptos-experimental-runtimes = { workspace = true } aptos-genesis = { workspace = true, features = ["testing"] } +aptos-global-cache-manager = { workspace = true } aptos-jellyfish-merkle = { workspace = true } aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } diff --git a/execution/executor-benchmark/src/native_executor.rs b/execution/executor-benchmark/src/native_executor.rs index f90eb1498f79e..e25bb063d2fbc 100644 --- a/execution/executor-benchmark/src/native_executor.rs +++ b/execution/executor-benchmark/src/native_executor.rs @@ -358,6 +358,7 @@ impl VMBlockExecutor for NativeExecutor { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), + _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { let transaction_outputs = NATIVE_EXECUTOR_POOL diff --git a/execution/executor-service/Cargo.toml b/execution/executor-service/Cargo.toml index e590b54912c69..5d631700f1af2 100644 --- a/execution/executor-service/Cargo.toml +++ b/execution/executor-service/Cargo.toml @@ -15,6 +15,7 @@ rust-version = { workspace = true } [dependencies] aptos-block-partitioner = { workspace = true } aptos-config = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-infallible = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-logger = { workspace = true } diff --git a/execution/executor/Cargo.toml b/execution/executor/Cargo.toml index fcc3213c594bb..3c433b3d58201 100644 --- a/execution/executor/Cargo.toml +++ b/execution/executor/Cargo.toml @@ -20,6 +20,7 @@ aptos-drop-helper = { workspace = true } aptos-executor-service = { workspace = true } aptos-executor-types = { workspace = true } aptos-experimental-runtimes = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-indexer-grpc-table-info = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } diff --git a/execution/executor/src/chunk_executor/mod.rs b/execution/executor/src/chunk_executor/mod.rs index 26391ae28acd6..db8c2052dc288 100644 --- a/execution/executor/src/chunk_executor/mod.rs +++ b/execution/executor/src/chunk_executor/mod.rs @@ -605,6 +605,7 @@ impl ChunkExecutorInner { BlockExecutorConfigFromOnchain::new_no_block_limit(), None, )?; + // not `zip_eq`, deliberately for (version, txn_out, txn_info, write_set, events) in multizip(( begin_version..end_version, diff --git a/execution/executor/src/chunk_executor/transaction_chunk.rs b/execution/executor/src/chunk_executor/transaction_chunk.rs index 26f873162eb61..aca1d387e2ddc 100644 --- a/execution/executor/src/chunk_executor/transaction_chunk.rs +++ b/execution/executor/src/chunk_executor/transaction_chunk.rs @@ -87,6 +87,7 @@ impl TransactionChunk for ChunkToExecute { &V::new(), sig_verified_txns.into(), state_view, + &global_cache_manager, BlockExecutorConfigFromOnchain::new_no_block_limit(), None, ) diff --git a/execution/executor/src/db_bootstrapper/mod.rs b/execution/executor/src/db_bootstrapper/mod.rs index 1d9a3c2742cc8..0445db5a6a2ff 100644 --- a/execution/executor/src/db_bootstrapper/mod.rs +++ b/execution/executor/src/db_bootstrapper/mod.rs @@ -138,6 +138,7 @@ pub fn calculate_genesis( BlockExecutorConfigFromOnchain::new_no_block_limit(), None, )?; + ensure!( execution_output.num_transactions_to_commit() != 0, "Genesis txn execution failed." diff --git a/execution/executor/src/fuzzing.rs b/execution/executor/src/fuzzing.rs index 67639b5831107..2ea96225c51ed 100644 --- a/execution/executor/src/fuzzing.rs +++ b/execution/executor/src/fuzzing.rs @@ -78,6 +78,7 @@ impl VMBlockExecutor for FakeVM { &self, _transactions: &[SignatureVerifiedTransaction], _state_view: &impl StateView, + _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { Ok(BlockOutput::new(vec![], None)) diff --git a/execution/executor/src/tests/mock_vm/mod.rs b/execution/executor/src/tests/mock_vm/mod.rs index b72f1d7270e62..608c45076976e 100644 --- a/execution/executor/src/tests/mock_vm/mod.rs +++ b/execution/executor/src/tests/mock_vm/mod.rs @@ -67,6 +67,7 @@ impl VMBlockExecutor for MockVM { &self, transactions: &[SignatureVerifiedTransaction], state_view: &impl StateView, + _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { // output_cache is used to store the output of transactions so they are visible to later diff --git a/execution/executor/src/tests/mod.rs b/execution/executor/src/tests/mod.rs index 8ca28af1a83ff..ba76568c38f7e 100644 --- a/execution/executor/src/tests/mod.rs +++ b/execution/executor/src/tests/mod.rs @@ -675,23 +675,20 @@ fn run_transactions_naive( ) -> HashValue { let executor = TestExecutor::new(); let db = &executor.db; + let global_cache_manager = GlobalCacheManager::new_with_default_config(); for txn in transactions { let ledger_view: ExecutedTrees = db.reader.get_latest_executed_trees().unwrap(); let out = DoGetExecutionOutput::by_transaction_execution( &MockVM::new(), vec![txn].into(), - ledger_view - .verified_state_view( - StateViewId::Miscellaneous, - Arc::clone(&db.reader), - Arc::new(AsyncProofFetcher::new(db.reader.clone())), - ) - .unwrap(), + state_view, + &global_cache_manager, block_executor_onchain_config.clone(), None, ) .unwrap(); + let output = ApplyExecutionOutput::run(out, &ledger_view).unwrap(); db.writer .save_transactions( diff --git a/execution/executor/src/workflow/do_get_execution_output.rs b/execution/executor/src/workflow/do_get_execution_output.rs index 4fad63d2e8261..cb1c76c96b284 100644 --- a/execution/executor/src/workflow/do_get_execution_output.rs +++ b/execution/executor/src/workflow/do_get_execution_output.rs @@ -48,6 +48,7 @@ impl DoGetExecutionOutput { executor: &V, transactions: ExecutableTransactions, state_view: CachedStateView, + global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, append_state_checkpoint_to_block: Option, ) -> Result { @@ -57,6 +58,7 @@ impl DoGetExecutionOutput { executor, txns, state_view, + global_cache_manager, onchain_config, append_state_checkpoint_to_block, )? @@ -88,6 +90,7 @@ impl DoGetExecutionOutput { executor: &V, transactions: Vec, state_view: CachedStateView, + global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, append_state_checkpoint_to_block: Option, ) -> Result { @@ -201,6 +204,7 @@ impl DoGetExecutionOutput { executor: &V, transactions: &[SignatureVerifiedTransaction], state_view: &CachedStateView, + global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, ) -> Result> { let _timer = OTHER_TIMERS.timer_with(&["vm_execute_block"]); @@ -216,6 +220,7 @@ impl DoGetExecutionOutput { executor: &V, transactions: &[SignatureVerifiedTransaction], state_view: &CachedStateView, + global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, ) -> Result> { use aptos_types::{ diff --git a/experimental/execution/ptx-executor/Cargo.toml b/experimental/execution/ptx-executor/Cargo.toml index 0da896d31500a..20b9e6a3a28e3 100644 --- a/experimental/execution/ptx-executor/Cargo.toml +++ b/experimental/execution/ptx-executor/Cargo.toml @@ -14,6 +14,7 @@ rust-version = { workspace = true } [dependencies] aptos-experimental-runtimes = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } diff --git a/experimental/execution/ptx-executor/src/lib.rs b/experimental/execution/ptx-executor/src/lib.rs index 9ed83b7d7a3f6..a080223eb7159 100644 --- a/experimental/execution/ptx-executor/src/lib.rs +++ b/experimental/execution/ptx-executor/src/lib.rs @@ -22,6 +22,7 @@ use crate::{ scheduler::PtxScheduler, sorter::PtxSorter, state_reader::PtxStateReader, }; use aptos_experimental_runtimes::thread_manager::THREAD_MANAGER; +use aptos_global_cache_manager::GlobalCacheManager; use aptos_infallible::Mutex; use aptos_metrics_core::TimerHelper; use aptos_types::{ @@ -52,6 +53,7 @@ impl VMBlockExecutor for PtxBlockExecutor { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), + _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { let _timer = TIMER.timer_with(&["block_total"]); diff --git a/storage/db-tool/Cargo.toml b/storage/db-tool/Cargo.toml index b2feeb952750d..0f9f772776d00 100644 --- a/storage/db-tool/Cargo.toml +++ b/storage/db-tool/Cargo.toml @@ -18,6 +18,7 @@ aptos-config = { workspace = true } aptos-db = { workspace = true, features = ["db-debugger"] } aptos-executor = { workspace = true } aptos-executor-types = { workspace = true } +aptos-global-cache-manager = { workspace = true } aptos-logger = { workspace = true } aptos-storage-interface = { workspace = true } aptos-temppath = { workspace = true } diff --git a/storage/db-tool/src/replay_on_archive.rs b/storage/db-tool/src/replay_on_archive.rs index 5bc13308bee98..09f873ef2c434 100644 --- a/storage/db-tool/src/replay_on_archive.rs +++ b/storage/db-tool/src/replay_on_archive.rs @@ -29,6 +29,7 @@ use std::{ sync::{atomic::AtomicU64, Arc}, time::Instant, }; + // Replay Verify controller is responsible for providing legit range with start and end versions. #[derive(Parser)] pub struct Opt { @@ -297,9 +298,7 @@ impl Verifier { .map(|txn| SignatureVerifiedTransaction::from(txn.clone())) .collect::>() .as_slice(), - &self - .arc_db - .state_view_at_version(start_version.checked_sub(1))?, + &state_view, )?; let mut failed_txns = Vec::new(); diff --git a/types/src/read_only_module_cache.rs b/types/src/read_only_module_cache.rs index c6b18615e945b..55b3cbeef599a 100644 --- a/types/src/read_only_module_cache.rs +++ b/types/src/read_only_module_cache.rs @@ -70,9 +70,6 @@ where pub struct ReadOnlyModuleCache { /// Module cache containing the verified code. module_cache: ExplicitSyncWrapper>>, - /// Maximum cache size. If the size is greater than this limit, the cache is flushed. Note that - /// this can only be done at block boundaries. - capacity: usize, } impl ReadOnlyModuleCache @@ -80,17 +77,10 @@ where K: Hash + Eq + Clone, VC: Deref>, { - /// Returns new empty module cache with default capacity. + /// Returns new empty module cache. pub fn empty() -> Self { - let default_capacity = 100_000; - Self::with_capacity(default_capacity) - } - - /// Returns new empty module cache with specified capacity. - fn with_capacity(capacity: usize) -> Self { Self { module_cache: ExplicitSyncWrapper::new(HashMap::new()), - capacity, } } @@ -136,8 +126,6 @@ where /// 2. Versions of inserted modules are set to [None] (storage version). /// 3. Valid modules should not be removed, and new modules should have unique ownership. If /// these constraints are violated, a panic error is returned. - /// 4. If the cache size exceeds its capacity after all verified modules have been inserted, - /// the cache is flushed. pub fn insert_verified_unchecked( &self, modules: impl Iterator>>)>, @@ -168,14 +156,14 @@ where assert!(prev.is_none()) } } - - if module_cache.len() > self.capacity { - module_cache.clear(); - } - Ok(()) } + /// Returns the size of the cache. + pub fn size(&self) -> usize { + self.module_cache.acquire().len() + } + /// Insert the module to cache. Used for tests only. #[cfg(any(test, feature = "testing"))] pub fn insert(&self, key: K, module: Arc>>) { @@ -189,12 +177,6 @@ where pub fn remove(&self, key: &K) { self.module_cache.acquire().remove(key); } - - /// Returns the size of the cache. Used for tests only. - #[cfg(any(test, feature = "testing"))] - pub fn size(&self) -> usize { - self.module_cache.acquire().len() - } } #[cfg(test)] @@ -241,27 +223,23 @@ mod test { #[test] fn test_insert_verified_for_read_only_module_cache() { - let capacity = 10; - let global_cache = ReadOnlyModuleCache::with_capacity(capacity); + let global_cache = ReadOnlyModuleCache::empty(); let mut new_modules = vec![]; - for i in 0..capacity { + for i in 0..10 { new_modules.push((i, mock_verified_code(i, Some(i as u32)))); } let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); assert!(result.is_ok()); - assert_eq!(global_cache.size(), capacity); + assert_eq!(global_cache.size(), 10); // Versions should be set to storage. - for key in 0..capacity { + for key in 0..10 { let code = assert_some!(global_cache.get(&key)); assert!(code.version().is_none()) } - // Too many modules added, the cache should be flushed. - let new_modules = vec![(11, mock_verified_code(11, None))]; - let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); - assert!(result.is_ok()); + global_cache.flush_unchecked(); assert_eq!(global_cache.size(), 0); // Should not add deserialized code. From 32ebdd3bf51d976a01bb986ff8876a529233c3dc Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 6 Nov 2024 13:18:22 +0000 Subject: [PATCH 04/25] [testing] Added different test cases for global caches --- Cargo.lock | 4 + .../aptos-global-cache-manager/Cargo.toml | 10 +- .../aptos-global-cache-manager/src/manager.rs | 316 +++++++++++++++--- .../src/move_vm_ext/write_op_converter.rs | 41 +-- .../src/proptest_types/bencher.rs | 13 +- .../src/proptest_types/tests.rs | 13 +- .../src/proptest_types/types.rs | 24 -- aptos-move/block-executor/src/view.rs | 92 ++--- execution/executor/Cargo.toml | 2 +- .../src/tests/mock_vm/mock_vm_test.rs | 25 +- types/src/state_store/mod.rs | 39 ++- 11 files changed, 375 insertions(+), 204 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index d45a83ed7b7a1..e84ea6644a56d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2062,10 +2062,14 @@ dependencies = [ "aptos-crypto", "aptos-types", "aptos-vm-environment", + "bcs 0.1.4", + "claims", "move-binary-format", "move-core-types", "move-vm-runtime", + "move-vm-types", "parking_lot 0.12.1", + "test-case", ] [[package]] diff --git a/aptos-move/aptos-global-cache-manager/Cargo.toml b/aptos-move/aptos-global-cache-manager/Cargo.toml index 71fcaae3ce1e5..05e196a13f7f0 100644 --- a/aptos-move/aptos-global-cache-manager/Cargo.toml +++ b/aptos-move/aptos-global-cache-manager/Cargo.toml @@ -16,7 +16,15 @@ rust-version = { workspace = true } aptos-crypto = { workspace = true } aptos-types = { workspace = true } aptos-vm-environment = { workspace = true } -parking_lot = { workspace = true } move-binary-format = { workspace = true } move-core-types = { workspace = true } move-vm-runtime = { workspace = true } +parking_lot = { workspace = true } + +[dev-dependencies] +aptos-crypto = { workspace = true, features = ["fuzzing"] } +aptos-types = { workspace = true, features = ["testing"] } +bcs = { workspace = true } +claims = { workspace = true } +move-vm-types = { workspace = true, features = ["testing"] } +test-case = { workspace = true } diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs index 1e632ba644eda..30a4b3dd7222a 100644 --- a/aptos-move/aptos-global-cache-manager/src/manager.rs +++ b/aptos-move/aptos-global-cache-manager/src/manager.rs @@ -33,7 +33,7 @@ fn invariant_violation(msg: &str) -> VMStatus { } /// Represents previously executed block, recorded by [GlobalCacheManager]. -#[derive(Clone, Copy, Eq, Hash, PartialEq, PartialOrd, Ord)] +#[derive(Clone, Copy, Debug, Hash, Eq, PartialEq, PartialOrd, Ord)] enum BlockId { /// No block has been executed yet. Unset, @@ -54,8 +54,8 @@ pub struct GlobalCacheManagerInner { /// Different configurations used for handling global caches. config: GlobalCacheConfig, /// Cache for modules. It is read-only for any concurrent execution, and can only be mutated - /// when it is known that there are no concurrent accesses, e.g., at blok boundaries. - /// [GlobalCacheManager] tries to ensure that these invariants always hold. + /// when it is known that there are no concurrent accesses, e.g., at block boundaries. + /// [GlobalCacheManagerInner] tries to ensure that these invariants always hold. module_cache: Arc>, /// Identifies previously executed block, initially [BlockId::Unset]. @@ -75,12 +75,12 @@ where VC: Deref>, { /// Returns a new instance of [GlobalCacheManagerInner] with default [GlobalCacheConfig]. - pub fn new_with_default_config() -> Self { + fn new_with_default_config() -> Self { Self::new_with_config(GlobalCacheConfig::default()) } /// Returns a new instance of [GlobalCacheManagerInner] with the provided [GlobalCacheConfig]. - pub fn new_with_config(config: GlobalCacheConfig) -> Self { + fn new_with_config(config: GlobalCacheConfig) -> Self { Self { config, module_cache: Arc::new(ReadOnlyModuleCache::empty()), @@ -110,14 +110,14 @@ where let recorded_previous_block_id = { // Acquire a lock, and check if we are ready to execute the next block. let previous_block_id = self.previous_block_id.lock(); - if !self.ready_for_next_block.load(Ordering::SeqCst) { + if !self.ready_for_next_block() { let msg = "Trying to execute blocks concurrently over shared global state"; return Err(invariant_violation(msg)); } // Prepare for execution. Set the flag as not ready to ensure that blocks are not // executed concurrently using the same cache. - self.ready_for_next_block.store(false, Ordering::SeqCst); + self.mark_not_ready_for_next_block(); *previous_block_id }; @@ -176,6 +176,8 @@ where flush_all_caches = true; } if self.module_cache.size() > self.config.module_cache_capacity { + // Technically, if we flush modules we do not need to flush type caches, but we unify + // flushing logic for easier reasoning. flush_all_caches = true; } @@ -198,7 +200,7 @@ where // We are done executing a block, reset the previous block id. Do everything under lock to // ensure it is not possible to execute blocks concurrently. let mut previous_block_id = self.previous_block_id.lock(); - if self.ready_for_next_block.load(Ordering::SeqCst) { + if self.ready_for_next_block() { let msg = "Should not be possible to mark block execution end for execution-ready \ global cache, check if blocks are executed concurrently"; return Err(invariant_violation(msg)); @@ -206,7 +208,7 @@ where *previous_block_id = BlockId::Set(executed_block_id); // Set the flag that the global cache is ready for next execution. - self.ready_for_next_block.store(true, Ordering::SeqCst); + self.mark_ready_for_next_block(); Ok(()) } @@ -223,6 +225,24 @@ where pub fn module_cache(&self) -> Arc> { self.module_cache.clone() } + + /// Returns true of a next block is ready be executed. This is the case only when: + /// 1. the global caches have just been created, or + /// 2. [GlobalCacheManagerInner::mark_block_execution_end] was called indicating that + /// previous block execution has finished. + fn ready_for_next_block(&self) -> bool { + self.ready_for_next_block.load(Ordering::SeqCst) + } + + /// Marks caches as ready for next block execution. + fn mark_ready_for_next_block(&self) { + self.ready_for_next_block.store(true, Ordering::SeqCst); + } + + /// Marks caches as not ready for next block execution. + fn mark_not_ready_for_next_block(&self) { + self.ready_for_next_block.store(false, Ordering::SeqCst); + } } /// Same as [GlobalCacheManagerInner], but uses concrete types used by execution on Aptos instead @@ -251,43 +271,243 @@ impl Deref for GlobalCacheManager { #[cfg(test)] mod test { - // use super::*; - // use aptos_language_e2e_tests::data_store::FakeDataStore; - // use aptos_types::on_chain_config::{FeatureFlag, Features}; - // use aptos_vm_environment::environment::AptosEnvironment; - // use claims::assert_ok; - // use move_vm_types::code::mock_verified_code; - // - // #[test] - // fn test_cross_block_module_cache_flush() { - // let global_module_cache = ReadOnlyModuleCache::empty(); - // - // global_module_cache.insert(0, mock_verified_code(0, None)); - // assert_eq!(global_module_cache.size(), 1); - // - // global_module_cache.flush_unchecked(); - // assert_eq!(global_module_cache.size(), 0); - // - // // Now check that cache is flushed when the environment is flushed. - // let mut state_view = FakeDataStore::default(); - // let env_old = AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); - // - // for i in 0..10 { - // global_module_cache.insert(i, mock_verified_code(i, None)); - // } - // assert_eq!(global_module_cache.size(), 10); - // - // let mut features = Features::default(); - // features.disable(FeatureFlag::KEYLESS_ACCOUNTS); - // state_view.set_features(features); - // - // // New environment means we need to also flush global caches - to invalidate struct name - // // indices. - // let env_new = assert_ok!(get_environment_with_delayed_field_optimization_enabled( - // &state_view, - // &global_module_cache, - // )); - // assert!(env_old != env_new); - // assert_eq!(global_module_cache.size(), 0); - // } + use super::*; + use aptos_types::{ + on_chain_config::{FeatureFlag, Features, OnChainConfig}, + state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, + }; + use claims::{assert_err, assert_ok}; + use move_vm_types::code::mock_verified_code; + use std::{collections::HashMap, thread, thread::JoinHandle}; + use test_case::test_case; + + /// Joins threads. Succeeds only if a single handle evaluates to [Ok] and the rest are [Err]s. + fn join_and_assert_single_ok(handles: Vec>>) { + let mut num_oks = 0; + let mut num_errs = 0; + + let num_handles = handles.len(); + for handle in handles { + let result = handle.join().unwrap(); + if result.is_ok() { + num_oks += 1; + } else { + num_errs += 1; + } + } + assert_eq!(num_oks, 1); + assert_eq!(num_errs, num_handles - 1); + } + + #[test] + fn mark_ready() { + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + assert!(global_cache_manager.ready_for_next_block()); + + global_cache_manager.mark_not_ready_for_next_block(); + assert!(!global_cache_manager.ready_for_next_block()); + + global_cache_manager.mark_ready_for_next_block(); + assert!(global_cache_manager.ready_for_next_block()); + } + + #[test] + fn environment_should_always_be_set() { + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + assert!(global_cache_manager.environment().is_err()); + + let state_view = MockStateView::empty(); + assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); + assert_ok!(global_cache_manager.environment()); + } + + #[test] + fn mark_execution_start_when_different_environment() { + let state_view = MockStateView::empty(); + let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); + + global_cache_manager + .module_cache() + .insert(0, mock_verified_code(0, None)); + global_cache_manager + .module_cache() + .insert(1, mock_verified_code(1, None)); + assert_eq!(global_cache_manager.module_cache().size(), 2); + + assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); + let old_environment = assert_ok!(global_cache_manager.environment()); + assert_ok!(global_cache_manager.mark_block_execution_end(Some(HashValue::zero()))); + assert_eq!(global_cache_manager.module_cache().size(), 2); + + // Tweak feature flags to force a different config. + let mut features = global_cache_manager + .environment() + .unwrap() + .features() + .clone(); + assert!(features.is_enabled(FeatureFlag::LIMIT_VM_TYPE_SIZE)); + features.disable(FeatureFlag::LIMIT_VM_TYPE_SIZE); + let bytes = bcs::to_bytes(&features).unwrap(); + let state_key = StateKey::resource(Features::address(), &Features::struct_tag()).unwrap(); + + let state_view = MockStateView::new(HashMap::from([( + state_key, + StateValue::new_legacy(bytes.into()), + )])); + + // We use the same previous ID, but the cache is still flushed: the environment changed. + assert_ok!( + global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::zero())) + ); + assert_eq!(global_cache_manager.module_cache().size(), 0); + + let new_environment = assert_ok!(global_cache_manager.environment()); + assert!(old_environment != new_environment); + } + + #[test] + fn mark_execution_start_when_too_many_types() { + // TODO(loader_v2): + // Propagate type caches/struct name index map APIs to here so we can mock & test. + } + + #[test] + fn mark_execution_start_when_too_many_modules() { + let state_view = MockStateView::empty(); + + let config = GlobalCacheConfig { + module_cache_capacity: 1, + ..Default::default() + }; + let global_cache_manager = GlobalCacheManagerInner::new_with_config(config); + + global_cache_manager + .module_cache() + .insert(0, mock_verified_code(0, None)); + global_cache_manager + .module_cache() + .insert(1, mock_verified_code(1, None)); + assert_eq!(global_cache_manager.module_cache().size(), 2); + + // Cache is too large, should be flushed for next block. + assert_ok!( + global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) + ); + assert_eq!(global_cache_manager.module_cache().size(), 0); + } + + #[test_case(None)] + #[test_case(Some(HashValue::zero()))] + fn mark_execution_start_when_unset(previous_block_id: Option) { + let state_view = MockStateView::empty(); + let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); + + global_cache_manager + .module_cache() + .insert(0, mock_verified_code(0, None)); + assert_eq!(global_cache_manager.module_cache().size(), 1); + + // If executed on top of unset state, or the state with matching previous hash, the cache + // is not flushed. + assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); + assert_eq!(global_cache_manager.module_cache().size(), 1); + assert!(!global_cache_manager.ready_for_next_block()); + } + + #[test_case(None, None)] + #[test_case(None, Some(HashValue::zero()))] + #[test_case(Some(HashValue::zero()), None)] + #[test_case(Some(HashValue::zero()), Some(HashValue::zero()))] + #[test_case(Some(HashValue::from_u64(0)), Some(HashValue::from_u64(1)))] + fn mark_execution_start_when_set( + recorded_previous_block_id: Option, + previous_block_id: Option, + ) { + let state_view = MockStateView::empty(); + let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); + + assert_ok!( + global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) + ); + assert_ok!(global_cache_manager.mark_block_execution_end(recorded_previous_block_id)); + + global_cache_manager + .module_cache() + .insert(0, mock_verified_code(0, None)); + assert_eq!(global_cache_manager.module_cache().size(), 1); + + assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); + assert!(!global_cache_manager.ready_for_next_block()); + + if recorded_previous_block_id.is_some() && recorded_previous_block_id == previous_block_id { + // In this case both IDs match, no cache flushing. + assert_eq!(global_cache_manager.module_cache().size(), 1); + } else { + // If previous block IDs do not match, or are unknown, caches must be flushed! + assert_eq!(global_cache_manager.module_cache().size(), 0); + } + } + + #[test] + fn mark_execution_start_concurrent() { + let state_view = Box::new(MockStateView::empty()); + let state_view: &'static _ = Box::leak(state_view); + + let global_cache_manager = Arc::new(GlobalCacheManager::new_with_default_config()); + assert!(global_cache_manager.ready_for_next_block()); + + let mut handles = vec![]; + for _ in 0..32 { + let handle = thread::spawn({ + let global_cache_manager = global_cache_manager.clone(); + move || global_cache_manager.mark_block_execution_start(state_view, None) + }); + handles.push(handle); + } + join_and_assert_single_ok(handles); + } + + #[test_case(None)] + #[test_case(Some(HashValue::from_u64(0)))] + fn mark_block_execution_end(block_id: Option) { + let global_cache_manager = GlobalCacheManager::new_with_default_config(); + assert!(global_cache_manager.previous_block_id.lock().is_unset()); + + // The global cache is ready, so we cannot mark execution end. + assert_err!(global_cache_manager.mark_block_execution_end(block_id)); + + global_cache_manager.mark_not_ready_for_next_block(); + let previous_block_id = *global_cache_manager.previous_block_id.lock(); + assert!(previous_block_id.is_unset()); + assert_ok!(global_cache_manager.mark_block_execution_end(block_id)); + + // The previous block ID should be set now, and the state is ready. + let new_block_id = *global_cache_manager.previous_block_id.lock(); + assert_eq!(new_block_id, BlockId::Set(block_id)); + assert!(global_cache_manager.ready_for_next_block()); + + global_cache_manager.mark_not_ready_for_next_block(); + let next_block_id = Some(HashValue::from_u64(1)); + assert_ok!(global_cache_manager.mark_block_execution_end(next_block_id)); + + // Previous block ID is again reset. + let new_block_id = *global_cache_manager.previous_block_id.lock(); + assert_eq!(new_block_id, BlockId::Set(next_block_id)); + } + + #[test] + fn mark_block_execution_end_concurrent() { + let global_cache_manager = Arc::new(GlobalCacheManager::new_with_default_config()); + global_cache_manager.mark_not_ready_for_next_block(); + + let mut handles = vec![]; + for _ in 0..32 { + let handle = thread::spawn({ + let global_cache_manager = global_cache_manager.clone(); + move || global_cache_manager.mark_block_execution_end(None) + }); + handles.push(handle); + } + join_and_assert_single_ok(handles); + } } diff --git a/aptos-move/aptos-vm/src/move_vm_ext/write_op_converter.rs b/aptos-move/aptos-vm/src/move_vm_ext/write_op_converter.rs index 1d54be49fe16f..56b1a33161d62 100644 --- a/aptos-move/aptos-vm/src/move_vm_ext/write_op_converter.rs +++ b/aptos-move/aptos-vm/src/move_vm_ext/write_op_converter.rs @@ -282,10 +282,7 @@ mod tests { }; use aptos_types::{ account_address::AccountAddress, - state_store::{ - errors::StateviewError, state_storage_usage::StateStorageUsage, - state_value::StateValue, TStateView, - }, + state_store::{state_value::StateValue, MockStateView}, write_set::TransactionWrite, }; use aptos_vm_environment::environment::AptosEnvironment; @@ -301,6 +298,7 @@ mod tests { identifier::Identifier, language_storage::{StructTag, TypeTag}, }; + use std::collections::HashMap; fn raw_metadata(v: u64) -> StateValueMetadata { StateValueMetadata::legacy(v, &CurrentTimeMicroseconds { microseconds: v }) @@ -334,31 +332,6 @@ mod tests { } } - struct MockStateView { - data: BTreeMap, - } - - impl MockStateView { - fn new(data: BTreeMap) -> Self { - Self { data } - } - } - - impl TStateView for MockStateView { - type Key = StateKey; - - fn get_state_value( - &self, - state_key: &Self::Key, - ) -> Result, StateviewError> { - Ok(self.data.get(state_key).cloned()) - } - - fn get_usage(&self) -> Result { - unimplemented!(); - } - } - fn module(name: &str) -> (StateKey, Bytes, CompiledModule) { let module = empty_module_with_dependencies_and_friends(name, vec![], vec![]); let state_key = StateKey::module(module.self_addr(), module.self_name()); @@ -400,7 +373,7 @@ mod tests { let state_value = StateValue::new_legacy(bytes.into()); // Setting up the state. - let state_view = MockStateView::new(BTreeMap::from([ + let state_view = MockStateView::new(HashMap::from([ (state_key, state_value), (a_state_key.clone(), a_state_value.clone()), (b_state_key.clone(), b_state_value.clone()), @@ -472,7 +445,7 @@ mod tests { let metadata = raw_metadata(100); let key = StateKey::raw(&[0]); - let data = BTreeMap::from([( + let data = HashMap::from([( key.clone(), StateValue::new_with_metadata(bcs::to_bytes(&group).unwrap().into(), metadata.clone()), )]); @@ -528,7 +501,7 @@ mod tests { let metadata = raw_metadata(100); let key = StateKey::raw(&[0]); - let data = BTreeMap::from([( + let data = HashMap::from([( key.clone(), StateValue::new_with_metadata(bcs::to_bytes(&group).unwrap().into(), metadata.clone()), )]); @@ -562,7 +535,7 @@ mod tests { // #[test] #[allow(unused)] fn size_computation_new_group() { - let s = MockStateView::new(BTreeMap::new()); + let s = MockStateView::empty(); let resolver = as_resolver_with_group_size_kind(&s, GroupSizeKind::AsSum); // TODO[agg_v2](test): Layout hardcoded to None. Test with layout = Some(..) @@ -595,7 +568,7 @@ mod tests { let metadata = raw_metadata(100); let key = StateKey::raw(&[0]); - let data = BTreeMap::from([( + let data = HashMap::from([( key.clone(), StateValue::new_with_metadata(bcs::to_bytes(&group).unwrap().into(), metadata.clone()), )]); diff --git a/aptos-move/block-executor/src/proptest_types/bencher.rs b/aptos-move/block-executor/src/proptest_types/bencher.rs index a764df3f373cf..d4b989c041c98 100644 --- a/aptos-move/block-executor/src/proptest_types/bencher.rs +++ b/aptos-move/block-executor/src/proptest_types/bencher.rs @@ -7,8 +7,8 @@ use crate::{ proptest_types::{ baseline::BaselineOutput, types::{ - EmptyDataView, KeyType, MockEnvironment, MockOutput, MockTask, MockTransaction, - TransactionGen, TransactionGenParams, + KeyType, MockEnvironment, MockOutput, MockTask, MockTransaction, TransactionGen, + TransactionGenParams, }, }, txn_commit_hook::NoOpTransactionCommitHook, @@ -16,6 +16,7 @@ use crate::{ use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, executable::ExecutableTestType, read_only_module_cache::ReadOnlyModuleCache, + state_store::MockStateView, }; use criterion::{BatchSize, Bencher as CBencher}; use num_cpus; @@ -117,9 +118,7 @@ where } pub(crate) fn run(self) { - let data_view = EmptyDataView::> { - phantom: PhantomData, - }; + let state_view = MockStateView::empty(); let executor_thread_pool = Arc::new( rayon::ThreadPoolBuilder::new() @@ -134,11 +133,11 @@ where let output = BlockExecutor::< MockTransaction, E>, MockTask, E>, - EmptyDataView>, + MockStateView>, NoOpTransactionCommitHook, E>, usize>, ExecutableTestType, >::new(config, executor_thread_pool, global_module_cache, None) - .execute_transactions_parallel(&env, &self.transactions, &data_view); + .execute_transactions_parallel(&env, &self.transactions, &state_view); self.baseline_output.assert_parallel_output(&output); } diff --git a/aptos-move/block-executor/src/proptest_types/tests.rs b/aptos-move/block-executor/src/proptest_types/tests.rs index df113a2663d59..f065ae3e307b1 100644 --- a/aptos-move/block-executor/src/proptest_types/tests.rs +++ b/aptos-move/block-executor/src/proptest_types/tests.rs @@ -8,8 +8,8 @@ use crate::{ proptest_types::{ baseline::BaselineOutput, types::{ - DeltaDataView, EmptyDataView, KeyType, MockEnvironment, MockEvent, MockOutput, - MockTask, MockTransaction, NonEmptyGroupDataView, TransactionGen, TransactionGenParams, + DeltaDataView, KeyType, MockEnvironment, MockEvent, MockOutput, MockTask, + MockTransaction, NonEmptyGroupDataView, TransactionGen, TransactionGenParams, MAX_GAS_PER_TXN, }, }, @@ -18,6 +18,7 @@ use crate::{ use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, executable::ExecutableTestType, read_only_module_cache::ReadOnlyModuleCache, + state_store::MockStateView, }; use claims::{assert_matches, assert_ok}; use num_cpus; @@ -59,9 +60,7 @@ fn run_transactions( *transactions.get_mut(i.index(length)).unwrap() = MockTransaction::SkipRest(0); } - let data_view = EmptyDataView::> { - phantom: PhantomData, - }; + let state_view = MockStateView::empty(); let executor_thread_pool = Arc::new( rayon::ThreadPoolBuilder::new() @@ -75,7 +74,7 @@ fn run_transactions( let output = BlockExecutor::< MockTransaction, E>, MockTask, E>, - EmptyDataView>, + MockStateView>, NoOpTransactionCommitHook, E>, usize>, ExecutableTestType, >::new( @@ -84,7 +83,7 @@ fn run_transactions( Arc::new(ReadOnlyModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel(&env, &transactions, &state_view); if module_access.0 && module_access.1 { assert_matches!(output, Err(())); diff --git a/aptos-move/block-executor/src/proptest_types/types.rs b/aptos-move/block-executor/src/proptest_types/types.rs index 507e27fa7999a..27e6bdfb53653 100644 --- a/aptos-move/block-executor/src/proptest_types/types.rs +++ b/aptos-move/block-executor/src/proptest_types/types.rs @@ -122,30 +122,6 @@ where } } -pub(crate) struct EmptyDataView { - pub(crate) phantom: PhantomData, -} - -impl TStateView for EmptyDataView -where - K: PartialOrd + Ord + Send + Sync + Clone + Hash + Eq + ModulePath + 'static, -{ - type Key = K; - - /// Gets the state value for a given state key. - fn get_state_value(&self, _: &K) -> Result, StateviewError> { - Ok(None) - } - - fn id(&self) -> StateViewId { - StateViewId::Miscellaneous - } - - fn get_usage(&self) -> Result { - unreachable!("Not used in tests"); - } -} - /////////////////////////////////////////////////////////////////////////// // Generation of transactions /////////////////////////////////////////////////////////////////////////// diff --git a/aptos-move/block-executor/src/view.rs b/aptos-move/block-executor/src/view.rs index 8e1a48fca897f..96956b91d6886 100644 --- a/aptos-move/block-executor/src/view.rs +++ b/aptos-move/block-executor/src/view.rs @@ -1824,10 +1824,7 @@ mod test { use aptos_types::{ error::PanicOr, executable::Executable, - state_store::{ - errors::StateviewError, state_storage_usage::StateStorageUsage, - state_value::StateValue, TStateView, - }, + state_store::{state_value::StateValue, MockStateView}, transaction::BlockExecutableTransaction, write_set::TransactionWrite, }; @@ -2478,33 +2475,6 @@ mod test { StateValue::new_legacy(value.simple_serialize(layout).unwrap().into()) } - // TODO: Check how to import MockStateView from other tests - // rather than rewriting it here again - struct MockStateView { - data: HashMap, StateValue>, - } - - impl MockStateView { - fn new(data: HashMap, StateValue>) -> Self { - Self { data } - } - } - - impl TStateView for MockStateView { - type Key = KeyType; - - fn get_state_value( - &self, - state_key: &Self::Key, - ) -> Result, StateviewError> { - Ok(self.data.get(state_key).cloned()) - } - - fn get_usage(&self) -> Result { - unimplemented!(); - } - } - #[derive(Clone)] struct MockExecutable {} @@ -2518,18 +2488,19 @@ mod test { fn test_id_value_exchange() { let unsync_map = UnsyncMap::new(); let counter = RefCell::new(5); - let base_view = MockStateView::new(HashMap::new()); + let base_view = MockStateView::empty(); let start_counter = 5; let runtime_environment = RuntimeEnvironment::new(vec![]); let global_module_cache = ReadOnlyModuleCache::empty(); - let latest_view = LatestView::::new( - &base_view, - &global_module_cache, - &runtime_environment, - ViewState::Unsync(SequentialState::new(&unsync_map, start_counter, &counter)), - 1, - ); + let latest_view = + LatestView::>, MockExecutable>::new( + &base_view, + &global_module_cache, + &runtime_environment, + ViewState::Unsync(SequentialState::new(&unsync_map, start_counter, &counter)), + 1, + ); // Test id -- value exchange for a value that does not contain delayed fields let layout = MoveTypeLayout::Struct(MoveStructLayout::new(vec![ @@ -2788,7 +2759,7 @@ mod test { struct Holder { unsync_map: UnsyncMap, u32, ValueType, DelayedFieldID>, counter: RefCell, - base_view: MockStateView, + base_view: MockStateView>, empty_global_module_cache: ReadOnlyModuleCache, runtime_environment: RuntimeEnvironment, @@ -2812,11 +2783,11 @@ mod test { fn create_sequential_latest_view<'a>( h: &'a Holder, - ) -> LatestView<'a, TestTransactionType, MockStateView, MockExecutable> { + ) -> LatestView<'a, TestTransactionType, MockStateView>, MockExecutable> { let sequential_state: SequentialState<'a, TestTransactionType> = SequentialState::new(&h.unsync_map, *h.counter.borrow(), &h.counter); - LatestView::<'a, TestTransactionType, MockStateView, MockExecutable>::new( + LatestView::<'a, TestTransactionType, MockStateView>, MockExecutable>::new( &h.base_view, &h.empty_global_module_cache, &h.runtime_environment, @@ -2829,7 +2800,7 @@ mod test { start_counter: u32, holder: Holder, counter: AtomicU32, - base_view: MockStateView, + base_view: MockStateView>, runtime_environment: RuntimeEnvironment, versioned_map: MVHashMap, u32, ValueType, MockExecutable, DelayedFieldID>, scheduler: Scheduler, @@ -2857,19 +2828,22 @@ mod test { fn new_view(&self) -> ViewsComparison<'_> { let latest_view_seq = create_sequential_latest_view(&self.holder); - let latest_view_par = - LatestView::::new( - &self.base_view, - &self.holder.empty_global_module_cache, - &self.runtime_environment, - ViewState::Sync(ParallelState::new( - &self.versioned_map, - &self.scheduler, - self.start_counter, - &self.counter, - )), - 1, - ); + let latest_view_par = LatestView::< + TestTransactionType, + MockStateView>, + MockExecutable, + >::new( + &self.base_view, + &self.holder.empty_global_module_cache, + &self.runtime_environment, + ViewState::Sync(ParallelState::new( + &self.versioned_map, + &self.scheduler, + self.start_counter, + &self.counter, + )), + 1, + ); ViewsComparison { latest_view_seq, @@ -2879,8 +2853,10 @@ mod test { } struct ViewsComparison<'a> { - latest_view_seq: LatestView<'a, TestTransactionType, MockStateView, MockExecutable>, - latest_view_par: LatestView<'a, TestTransactionType, MockStateView, MockExecutable>, + latest_view_seq: + LatestView<'a, TestTransactionType, MockStateView>, MockExecutable>, + latest_view_par: + LatestView<'a, TestTransactionType, MockStateView>, MockExecutable>, } impl<'a> ViewsComparison<'a> { diff --git a/execution/executor/Cargo.toml b/execution/executor/Cargo.toml index 3c433b3d58201..39ab296d4f35e 100644 --- a/execution/executor/Cargo.toml +++ b/execution/executor/Cargo.toml @@ -51,7 +51,7 @@ aptos-executor-test-helpers = { workspace = true } aptos-genesis = { workspace = true } aptos-storage-interface = { workspace = true } aptos-temppath = { workspace = true } -aptos-types = { workspace = true } +aptos-types = { workspace = true, features = ["testing"] } aptos-vm-genesis = { workspace = true } proptest = { workspace = true } rand = { workspace = true } diff --git a/execution/executor/src/tests/mock_vm/mock_vm_test.rs b/execution/executor/src/tests/mock_vm/mock_vm_test.rs index 9481ac37aeba5..fcf97bb230bad 100644 --- a/execution/executor/src/tests/mock_vm/mock_vm_test.rs +++ b/execution/executor/src/tests/mock_vm/mock_vm_test.rs @@ -6,10 +6,7 @@ use super::{balance_ap, encode_mint_transaction, encode_transfer_transaction, se use aptos_types::{ account_address::AccountAddress, bytes::NumToBytes, - state_store::{ - state_key::StateKey, state_storage_usage::StateStorageUsage, state_value::StateValue, - Result, TStateView, - }, + state_store::{state_key::StateKey, MockStateView}, transaction::signature_verified_transaction::into_signature_verified_block, write_set::WriteOp, }; @@ -20,20 +17,6 @@ fn gen_address(index: u8) -> AccountAddress { AccountAddress::new([index; AccountAddress::LENGTH]) } -struct MockStateView; - -impl TStateView for MockStateView { - type Key = StateKey; - - fn get_state_value(&self, _state_key: &StateKey) -> Result> { - Ok(None) - } - - fn get_usage(&self) -> Result { - Ok(StateStorageUsage::new_untracked()) - } -} - #[test] fn test_mock_vm_different_senders() { let amount = 100; @@ -43,7 +26,7 @@ fn test_mock_vm_different_senders() { } let outputs = MockVM - .execute_block_no_limit(&into_signature_verified_block(txns.clone()), &MockStateView) + .execute_block_no_limit(&into_signature_verified_block(txns.clone()), &MockStateView::empty()) .expect("MockVM should not fail to start"); for (output, txn) in itertools::zip_eq(outputs.iter(), txns.iter()) { @@ -80,7 +63,7 @@ fn test_mock_vm_same_sender() { } let outputs = MockVM - .execute_block_no_limit(&into_signature_verified_block(txns), &MockStateView) + .execute_block_no_limit(&into_signature_verified_block(txns), &MockStateView::empty()) .expect("MockVM should not fail to start"); for (i, output) in outputs.iter().enumerate() { @@ -115,7 +98,7 @@ fn test_mock_vm_payment() { ]; let output = MockVM - .execute_block_no_limit(&into_signature_verified_block(txns), &MockStateView) + .execute_block_no_limit(&into_signature_verified_block(txns), &MockStateView::empty()) .expect("MockVM should not fail to start"); let mut output_iter = output.iter(); diff --git a/types/src/state_store/mod.rs b/types/src/state_store/mod.rs index 3b16a4f75d6fe..49450ffd7a4df 100644 --- a/types/src/state_store/mod.rs +++ b/types/src/state_store/mod.rs @@ -14,6 +14,8 @@ use aptos_experimental_runtimes::thread_manager::THREAD_MANAGER; use arr_macro::arr; use bytes::Bytes; use move_core_types::move_resource::MoveResource; +#[cfg(any(test, feature = "testing"))] +use std::hash::Hash; use std::{collections::HashMap, ops::Deref}; pub mod errors; @@ -25,9 +27,8 @@ pub mod table; pub type Result = std::result::Result; -/// `StateView` is a trait that defines a read-only snapshot of the global state. It is passed to -/// the VM for transaction execution, during which the VM is guaranteed to read anything at the -/// given state. +/// A trait that defines a read-only snapshot of the global state. It is passed to the VM for +/// transaction execution, during which the VM is guaranteed to read anything at the given state. pub trait TStateView { type Key; @@ -96,6 +97,38 @@ where } } +/// Test-only basic [StateView] implementation with generic keys. +#[cfg(any(test, feature = "testing"))] +pub struct MockStateView { + data: HashMap, +} + +#[cfg(any(test, feature = "testing"))] +impl MockStateView { + pub fn empty() -> Self { + Self { + data: HashMap::new(), + } + } + + pub fn new(data: HashMap) -> Self { + Self { data } + } +} + +#[cfg(any(test, feature = "testing"))] +impl TStateView for MockStateView { + type Key = K; + + fn get_state_value(&self, state_key: &Self::Key) -> Result, StateviewError> { + Ok(self.data.get(state_key).cloned()) + } + + fn get_usage(&self) -> std::result::Result { + unimplemented!("Irrelevant for tests"); + } +} + pub type ShardedStateUpdates = [HashMap>; 16]; pub fn create_empty_sharded_state_updates() -> ShardedStateUpdates { From 2a65957bc0791fc078cd022a2f15d80e318343cf Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 6 Nov 2024 16:00:16 +0000 Subject: [PATCH 05/25] [perf] Framework prefetch --- Cargo.lock | 1 + .../aptos-global-cache-manager/Cargo.toml | 1 + .../aptos-global-cache-manager/src/config.rs | 3 + .../aptos-global-cache-manager/src/manager.rs | 224 ++++++++++++------ .../state_view_adapter.rs | 44 +++- .../implementations/unsync_code_storage.rs | 4 + .../implementations/unsync_module_storage.rs | 12 + .../types/src/code/cache/module_cache.rs | 9 + .../move-vm/types/src/code/cache/types.rs | 5 + 9 files changed, 222 insertions(+), 81 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e84ea6644a56d..bd80614bb7da1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2062,6 +2062,7 @@ dependencies = [ "aptos-crypto", "aptos-types", "aptos-vm-environment", + "aptos-vm-types", "bcs 0.1.4", "claims", "move-binary-format", diff --git a/aptos-move/aptos-global-cache-manager/Cargo.toml b/aptos-move/aptos-global-cache-manager/Cargo.toml index 05e196a13f7f0..272333f72bda7 100644 --- a/aptos-move/aptos-global-cache-manager/Cargo.toml +++ b/aptos-move/aptos-global-cache-manager/Cargo.toml @@ -16,6 +16,7 @@ rust-version = { workspace = true } aptos-crypto = { workspace = true } aptos-types = { workspace = true } aptos-vm-environment = { workspace = true } +aptos-vm-types = { workspace = true } move-binary-format = { workspace = true } move-core-types = { workspace = true } move-vm-runtime = { workspace = true } diff --git a/aptos-move/aptos-global-cache-manager/src/config.rs b/aptos-move/aptos-global-cache-manager/src/config.rs index 0f5cdd14d215a..67eaabf212257 100644 --- a/aptos-move/aptos-global-cache-manager/src/config.rs +++ b/aptos-move/aptos-global-cache-manager/src/config.rs @@ -3,6 +3,8 @@ /// Configuration used for global caches. pub struct GlobalCacheConfig { + /// If true, when global caches are empty, Aptos framework is prefetched into module cache. + pub prefetch_framework_code: bool, /// The maximum size of module cache. If module cache exceeds this capacity, it should be /// flushed. pub module_cache_capacity: usize, @@ -17,6 +19,7 @@ impl Default for GlobalCacheConfig { // module or identifier sizes, or keep track in read-only module cache how many bytes we // are using. Self { + prefetch_framework_code: true, module_cache_capacity: 100_000, struct_name_index_map_capacity: 100_000, } diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs index 30a4b3dd7222a..dcd3f50e1981e 100644 --- a/aptos-move/aptos-global-cache-manager/src/manager.rs +++ b/aptos-move/aptos-global-cache-manager/src/manager.rs @@ -8,12 +8,15 @@ use aptos_types::{ vm::modules::AptosModuleExtension, }; use aptos_vm_environment::environment::AptosEnvironment; +use aptos_vm_types::module_and_script_storage::AsAptosCodeStorage; use move_binary_format::{errors::Location, CompiledModule}; use move_core_types::{ + account_address::AccountAddress, + ident_str, language_storage::ModuleId, vm_status::{StatusCode, VMStatus}, }; -use move_vm_runtime::{Module, WithRuntimeEnvironment}; +use move_vm_runtime::{Module, ModuleStorage, WithRuntimeEnvironment}; use parking_lot::Mutex; use std::{ hash::Hash, @@ -50,7 +53,7 @@ impl BlockId { } /// Manages global caches, e.g., modules or execution environment. Should not be used concurrently. -pub struct GlobalCacheManagerInner { +struct GlobalCacheManagerInner { /// Different configurations used for handling global caches. config: GlobalCacheConfig, /// Cache for modules. It is read-only for any concurrent execution, and can only be mutated @@ -90,19 +93,9 @@ where } } - /// Sets the state of global caches prior to block execution on top of the provided state (with - /// the block ID). Should always sbe called prior to block execution. - /// - /// The caches stored globally (modules, struct name re-indexing map and type caches) are all - /// flushed if: - /// 1. Previously executed block ID does not match the provided value. - /// 2. The environment has changed for this state. - /// 3. The size of the struct name re-indexing map is too large. - /// 4. The size of the module cache is too large. - /// - /// Marks [GlobalCacheManagerInner] as not ready for next block execution. If called - /// concurrently, only a single invocation ever succeeds and other calls return an error. - pub fn mark_block_execution_start( + /// See the documentation for [GlobalCacheManager::mark_block_execution_start]. The only + /// difference here is that there is no framework prefetching. + fn mark_block_execution_start( &self, state_view: &impl StateView, previous_block_id: Option, @@ -169,9 +162,15 @@ where .as_ref() .expect("Environment has to be set") .runtime_environment(); - let struct_name_index_map_size = runtime_environment - .struct_name_index_map_size() - .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; + let struct_name_index_map_size = + runtime_environment + .struct_name_index_map_size() + .map_err(|err| { + // TODO(loader_v2): + // Is this fine to fail here? We leave the state as not ready forever? + // Seems like it is better to reset the state and reset everything. + err.finish(Location::Undefined).into_vm_status() + })?; if struct_name_index_map_size > self.config.struct_name_index_map_capacity { flush_all_caches = true; } @@ -190,10 +189,8 @@ where Ok(()) } - /// Should always be called after block execution. Sets the [GlobalCacheManagerInner] to be - /// execution-ready (and if it is already execution-ready, returns an error). Sets the ID for - /// the executed block so that the next execution can check it. - pub fn mark_block_execution_end( + /// See the documentation for [GlobalCacheManager::mark_block_execution_end]. + fn mark_block_execution_end( &self, executed_block_id: Option, ) -> Result<(), VMStatus> { @@ -213,19 +210,6 @@ where Ok(()) } - /// Returns the cached environment that [GlobalCacheManagerInner::mark_block_execution_start] - /// must set. If it has not been set, an invariant violation error is returned. - pub fn environment(&self) -> Result { - self.previous_environment.lock().clone().ok_or_else(|| { - invariant_violation("Environment must always be set at block execution start") - }) - } - - /// Returns the global module cache. - pub fn module_cache(&self) -> Arc> { - self.module_cache.clone() - } - /// Returns true of a next block is ready be executed. This is the case only when: /// 1. the global caches have just been created, or /// 2. [GlobalCacheManagerInner::mark_block_execution_end] was called indicating that @@ -259,13 +243,73 @@ impl GlobalCacheManager { inner: GlobalCacheManagerInner::new_with_default_config(), } } -} -impl Deref for GlobalCacheManager { - type Target = GlobalCacheManagerInner; + /// Sets the state of global caches prior to block execution on top of the provided state (with + /// the block ID). Should always sbe called prior to block execution. + /// + /// The caches stored globally (modules, struct name re-indexing map and type caches) are all + /// flushed if: + /// 1. Previously executed block ID does not match the provided value. + /// 2. The environment has changed for this state. + /// 3. The size of the struct name re-indexing map is too large. + /// 4. The size of the module cache is too large. + /// + /// Marks [GlobalCacheManager] as not ready for next block execution. If called concurrently, + /// only a single invocation ever succeeds and other calls return an error. + pub fn mark_block_execution_start( + &self, + state_view: &impl StateView, + previous_block_id: Option, + ) -> Result<(), VMStatus> { + self.inner + .mark_block_execution_start(state_view, previous_block_id)?; + + if self.inner.config.prefetch_framework_code && self.module_cache().size() == 0 { + let runtime_environment = self.environment()?; + + let code_storage = state_view.as_aptos_code_storage(runtime_environment); + let result = code_storage + .fetch_verified_module(&AccountAddress::ONE, ident_str!("transaction_validation")); + // Framework must exist, prefetch. + if let Ok(Some(_)) = result { + // TODO(loader_v2): Replace with invariant violations! + self.inner + .module_cache + .insert_verified_unchecked(code_storage.into_verified_module_code_iter()) + .unwrap(); + } + } + + Ok(()) + } + + /// Should always be called after block execution. Sets the [GlobalCacheManager] to be ready + /// for execution (and if it is already execution-ready, returns an error). Sets the ID for the + /// executed block so that the next execution can check it. + pub fn mark_block_execution_end( + &self, + executed_block_id: Option, + ) -> Result<(), VMStatus> { + self.inner.mark_block_execution_end(executed_block_id) + } + + /// Returns the cached environment set by [GlobalCacheManager::mark_block_execution_start]. If + /// it has not been set, an invariant violation error is returned. + pub fn environment(&self) -> Result { + self.inner + .previous_environment + .lock() + .clone() + .ok_or_else(|| { + invariant_violation("Environment must always be set at block execution start") + }) + } - fn deref(&self) -> &Self::Target { - &self.inner + /// Returns the global module cache. + pub fn module_cache( + &self, + ) -> Arc> { + self.inner.module_cache.clone() } } @@ -277,7 +321,7 @@ mod test { state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, }; use claims::{assert_err, assert_ok}; - use move_vm_types::code::mock_verified_code; + use move_vm_types::code::{mock_verified_code, MockDeserializedCode, MockVerifiedCode}; use std::{collections::HashMap, thread, thread::JoinHandle}; use test_case::test_case; @@ -300,8 +344,23 @@ mod test { } #[test] - fn mark_ready() { + fn environment_should_always_be_set() { let global_cache_manager = GlobalCacheManager::new_with_default_config(); + assert!(global_cache_manager.environment().is_err()); + + let state_view = MockStateView::empty(); + assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); + assert_ok!(global_cache_manager.environment()); + } + + #[test] + fn mark_ready() { + let global_cache_manager = GlobalCacheManagerInner::< + u32, + MockDeserializedCode, + MockVerifiedCode, + (), + >::new_with_default_config(); assert!(global_cache_manager.ready_for_next_block()); global_cache_manager.mark_not_ready_for_next_block(); @@ -311,40 +370,30 @@ mod test { assert!(global_cache_manager.ready_for_next_block()); } - #[test] - fn environment_should_always_be_set() { - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - assert!(global_cache_manager.environment().is_err()); - - let state_view = MockStateView::empty(); - assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); - assert_ok!(global_cache_manager.environment()); - } - #[test] fn mark_execution_start_when_different_environment() { let state_view = MockStateView::empty(); let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); global_cache_manager - .module_cache() + .module_cache .insert(0, mock_verified_code(0, None)); global_cache_manager - .module_cache() + .module_cache .insert(1, mock_verified_code(1, None)); - assert_eq!(global_cache_manager.module_cache().size(), 2); + assert_eq!(global_cache_manager.module_cache.size(), 2); assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); - let old_environment = assert_ok!(global_cache_manager.environment()); + let old_environment = global_cache_manager + .previous_environment + .lock() + .clone() + .unwrap(); assert_ok!(global_cache_manager.mark_block_execution_end(Some(HashValue::zero()))); - assert_eq!(global_cache_manager.module_cache().size(), 2); + assert_eq!(global_cache_manager.module_cache.size(), 2); // Tweak feature flags to force a different config. - let mut features = global_cache_manager - .environment() - .unwrap() - .features() - .clone(); + let mut features = old_environment.features().clone(); assert!(features.is_enabled(FeatureFlag::LIMIT_VM_TYPE_SIZE)); features.disable(FeatureFlag::LIMIT_VM_TYPE_SIZE); let bytes = bcs::to_bytes(&features).unwrap(); @@ -359,9 +408,13 @@ mod test { assert_ok!( global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::zero())) ); - assert_eq!(global_cache_manager.module_cache().size(), 0); + assert_eq!(global_cache_manager.module_cache.size(), 0); - let new_environment = assert_ok!(global_cache_manager.environment()); + let new_environment = global_cache_manager + .previous_environment + .lock() + .clone() + .unwrap(); assert!(old_environment != new_environment); } @@ -382,18 +435,18 @@ mod test { let global_cache_manager = GlobalCacheManagerInner::new_with_config(config); global_cache_manager - .module_cache() + .module_cache .insert(0, mock_verified_code(0, None)); global_cache_manager - .module_cache() + .module_cache .insert(1, mock_verified_code(1, None)); - assert_eq!(global_cache_manager.module_cache().size(), 2); + assert_eq!(global_cache_manager.module_cache.size(), 2); // Cache is too large, should be flushed for next block. assert_ok!( global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) ); - assert_eq!(global_cache_manager.module_cache().size(), 0); + assert_eq!(global_cache_manager.module_cache.size(), 0); } #[test_case(None)] @@ -403,14 +456,14 @@ mod test { let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); global_cache_manager - .module_cache() + .module_cache .insert(0, mock_verified_code(0, None)); - assert_eq!(global_cache_manager.module_cache().size(), 1); + assert_eq!(global_cache_manager.module_cache.size(), 1); // If executed on top of unset state, or the state with matching previous hash, the cache // is not flushed. assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); - assert_eq!(global_cache_manager.module_cache().size(), 1); + assert_eq!(global_cache_manager.module_cache.size(), 1); assert!(!global_cache_manager.ready_for_next_block()); } @@ -432,19 +485,19 @@ mod test { assert_ok!(global_cache_manager.mark_block_execution_end(recorded_previous_block_id)); global_cache_manager - .module_cache() + .module_cache .insert(0, mock_verified_code(0, None)); - assert_eq!(global_cache_manager.module_cache().size(), 1); + assert_eq!(global_cache_manager.module_cache.size(), 1); assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); assert!(!global_cache_manager.ready_for_next_block()); if recorded_previous_block_id.is_some() && recorded_previous_block_id == previous_block_id { // In this case both IDs match, no cache flushing. - assert_eq!(global_cache_manager.module_cache().size(), 1); + assert_eq!(global_cache_manager.module_cache.size(), 1); } else { // If previous block IDs do not match, or are unknown, caches must be flushed! - assert_eq!(global_cache_manager.module_cache().size(), 0); + assert_eq!(global_cache_manager.module_cache.size(), 0); } } @@ -453,7 +506,12 @@ mod test { let state_view = Box::new(MockStateView::empty()); let state_view: &'static _ = Box::leak(state_view); - let global_cache_manager = Arc::new(GlobalCacheManager::new_with_default_config()); + let global_cache_manager = Arc::new(GlobalCacheManagerInner::< + u32, + MockDeserializedCode, + MockVerifiedCode, + (), + >::new_with_default_config()); assert!(global_cache_manager.ready_for_next_block()); let mut handles = vec![]; @@ -470,7 +528,12 @@ mod test { #[test_case(None)] #[test_case(Some(HashValue::from_u64(0)))] fn mark_block_execution_end(block_id: Option) { - let global_cache_manager = GlobalCacheManager::new_with_default_config(); + let global_cache_manager = GlobalCacheManagerInner::< + u32, + MockDeserializedCode, + MockVerifiedCode, + (), + >::new_with_default_config(); assert!(global_cache_manager.previous_block_id.lock().is_unset()); // The global cache is ready, so we cannot mark execution end. @@ -497,7 +560,12 @@ mod test { #[test] fn mark_block_execution_end_concurrent() { - let global_cache_manager = Arc::new(GlobalCacheManager::new_with_default_config()); + let global_cache_manager = Arc::new(GlobalCacheManagerInner::< + u32, + MockDeserializedCode, + MockVerifiedCode, + (), + >::new_with_default_config()); global_cache_manager.mark_not_ready_for_next_block(); let mut handles = vec![]; diff --git a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs index c4aa089e28730..39670da6165e1 100644 --- a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs +++ b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs @@ -3,21 +3,30 @@ use crate::module_and_script_storage::module_storage::AptosModuleStorage; use ambassador::Delegate; -use aptos_types::state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView}; +use aptos_types::{ + state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView}, + vm::modules::AptosModuleExtension, +}; use bytes::Bytes; use move_binary_format::{ errors::{PartialVMResult, VMResult}, file_format::CompiledScript, CompiledModule, }; -use move_core_types::{account_address::AccountAddress, identifier::IdentStr, metadata::Metadata}; +use move_core_types::{ + account_address::AccountAddress, identifier::IdentStr, language_storage::ModuleId, + metadata::Metadata, +}; use move_vm_runtime::{ ambassador_impl_CodeStorage, ambassador_impl_ModuleStorage, ambassador_impl_WithRuntimeEnvironment, AsUnsyncCodeStorage, BorrowedOrOwned, CodeStorage, Module, ModuleStorage, RuntimeEnvironment, Script, UnsyncCodeStorage, UnsyncModuleStorage, WithRuntimeEnvironment, }; -use move_vm_types::{code::ModuleBytesStorage, module_storage_error}; +use move_vm_types::{ + code::{ModuleBytesStorage, ModuleCode}, + module_storage_error, +}; use std::sync::Arc; /// Avoids orphan rule to implement [ModuleBytesStorage] for [StateView]. @@ -72,6 +81,35 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, let storage = adapter.into_unsync_code_storage(runtime_environment); Self { storage } } + + pub fn into_verified_module_code_iter( + self, + ) -> impl Iterator< + Item = ( + ModuleId, + Arc>>, + ), + > { + let state_view = match self.storage.module_storage().byte_storage().state_view { + BorrowedOrOwned::Borrowed(state_view) => state_view, + BorrowedOrOwned::Owned(_) => unreachable!(), + }; + + let mut modules_to_add = vec![]; + for (key, verified_code) in self + .storage + .into_module_storage() + .into_verified_modules_iter() + { + let state_key = StateKey::module_id(&key); + // TODO(loader_v2): Replace with invariant violations! + let state_value = state_view.get_state_value(&state_key).unwrap().unwrap(); + let extension = AptosModuleExtension::new(state_value); + let module = ModuleCode::from_verified_ref(verified_code, Arc::new(extension), None); + modules_to_add.push((key, Arc::new(module))) + } + modules_to_add.into_iter() + } } impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosModuleStorage diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs index 51539b35ffcd0..6ce4f94ea71b8 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs @@ -34,6 +34,10 @@ impl UnsyncCodeStorage { &self.0.module_storage } + pub fn into_module_storage(self) -> M { + self.0.module_storage + } + /// Test-only method that checks the state of the script cache. #[cfg(test)] pub(crate) fn assert_cached_state<'b>( diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs index c423c410b0ea3..e6e0e33f86868 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs @@ -167,6 +167,18 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> UnsyncModuleStorage<' &self.0.base_storage } + pub fn into_verified_modules_iter(self) -> impl Iterator)> { + self.0 + .module_cache + .into_modules_iter() + .flat_map(|(key, module)| { + module + .code() + .is_verified() + .then(|| (key, module.code().verified().clone())) + }) + } + /// Test-only method that checks the state of the module cache. #[cfg(test)] pub(crate) fn assert_cached_state<'b>( diff --git a/third_party/move/move-vm/types/src/code/cache/module_cache.rs b/third_party/move/move-vm/types/src/code/cache/module_cache.rs index 5d9bf1772c633..74d5f683c9852 100644 --- a/third_party/move/move-vm/types/src/code/cache/module_cache.rs +++ b/third_party/move/move-vm/types/src/code/cache/module_cache.rs @@ -42,6 +42,15 @@ where } } + /// Creates new [ModuleCode] from [Arc]ed verified code. + pub fn from_verified_ref(verified_code: Arc, extension: Arc, version: V) -> Self { + Self { + code: Code::from_verified_ref(verified_code), + extension, + version, + } + } + /// Returns module's code. pub fn code(&self) -> &Code { &self.code diff --git a/third_party/move/move-vm/types/src/code/cache/types.rs b/third_party/move/move-vm/types/src/code/cache/types.rs index b9cb0154b65a4..82477e5a6a795 100644 --- a/third_party/move/move-vm/types/src/code/cache/types.rs +++ b/third_party/move/move-vm/types/src/code/cache/types.rs @@ -61,6 +61,11 @@ where Self::Verified(Arc::new(verified_code)) } + /// Returns new verified code from [Arc]ed instance. + pub fn from_verified_ref(verified_code: Arc) -> Self { + Self::Verified(verified_code) + } + /// Returns true if the code is verified. pub fn is_verified(&self) -> bool { match self { From 0fda56999efee42fe6c422084317e5034ad6bdc5 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 6 Nov 2024 17:05:44 +0000 Subject: [PATCH 06/25] [exp] Avoid repeated arcswap load --- aptos-move/block-executor/src/executor.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 661cb08570e5b..87e89d8a704d0 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -618,9 +618,9 @@ where // Publish modules before we decrease validation index so that validations observe // the new module writes as well. if runtime_environment.vm_config().use_loader_v2 { + executed_at_commit = true; let module_write_set = last_input_output.module_write_set(txn_idx); if !module_write_set.is_empty() { - executed_at_commit = true; Self::publish_module_writes( txn_idx, module_write_set, From 60ceafb45e65ac8f1c5a452329e4511f61aefbfe Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 6 Nov 2024 18:12:50 +0000 Subject: [PATCH 07/25] [cleanup] Add errors, documentation, reset state on panics --- .../aptos-global-cache-manager/src/manager.rs | 94 +++++++++++++++---- .../state_view_adapter.rs | 52 +++++++--- .../implementations/unsync_code_storage.rs | 3 +- .../implementations/unsync_module_storage.rs | 1 + 4 files changed, 118 insertions(+), 32 deletions(-) diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs index dcd3f50e1981e..055879840c18d 100644 --- a/aptos-move/aptos-global-cache-manager/src/manager.rs +++ b/aptos-move/aptos-global-cache-manager/src/manager.rs @@ -9,7 +9,7 @@ use aptos_types::{ }; use aptos_vm_environment::environment::AptosEnvironment; use aptos_vm_types::module_and_script_storage::AsAptosCodeStorage; -use move_binary_format::{errors::Location, CompiledModule}; +use move_binary_format::CompiledModule; use move_core_types::{ account_address::AccountAddress, ident_str, @@ -162,15 +162,19 @@ where .as_ref() .expect("Environment has to be set") .runtime_environment(); - let struct_name_index_map_size = - runtime_environment - .struct_name_index_map_size() - .map_err(|err| { - // TODO(loader_v2): - // Is this fine to fail here? We leave the state as not ready forever? - // Seems like it is better to reset the state and reset everything. - err.finish(Location::Undefined).into_vm_status() - })?; + let struct_name_index_map_size = match runtime_environment.struct_name_index_map_size() { + Err(err) => { + // Unlock the cache, reset all states, and return. + drop(previous_environment); + let err = self.reset_and_return_invariant_violation(&format!( + "Error when getting struct name index map size: {:?}", + err + )); + return Err(err); + }, + Ok(size) => size, + }; + if struct_name_index_map_size > self.config.struct_name_index_map_capacity { flush_all_caches = true; } @@ -198,6 +202,8 @@ where // ensure it is not possible to execute blocks concurrently. let mut previous_block_id = self.previous_block_id.lock(); if self.ready_for_next_block() { + // This means we are executing concurrently. If so, all-but-one thread will return an + // error. Note that the caches are still consistent for that one thread. let msg = "Should not be possible to mark block execution end for execution-ready \ global cache, check if blocks are executed concurrently"; return Err(invariant_violation(msg)); @@ -227,6 +233,27 @@ where fn mark_not_ready_for_next_block(&self) { self.ready_for_next_block.store(false, Ordering::SeqCst); } + + /// Resets all states (under a lock) as if global caches are empty and no blocks have been + /// executed so far. Reruns an invariant violation error. + fn reset_and_return_invariant_violation(&self, msg: &str) -> VMStatus { + // Lock to reset the state under lock. + let mut previous_block_id = self.previous_block_id.lock(); + + // 1. Should be ready for next execution. + self.mark_not_ready_for_next_block(); + // 2. Should contain no environment. + *self.previous_environment.lock() = None; + // 3. Module cache is empty. + self.module_cache.flush_unchecked(); + // 4. Block ID is unset. + *previous_block_id = BlockId::Unset; + + // State reset, unlock. + drop(previous_block_id); + + invariant_violation(msg) + } } /// Same as [GlobalCacheManagerInner], but uses concrete types used by execution on Aptos instead @@ -254,6 +281,8 @@ impl GlobalCacheManager { /// 3. The size of the struct name re-indexing map is too large. /// 4. The size of the module cache is too large. /// + /// Additionally, if cache is empty, prefetches the framework code into it. + /// /// Marks [GlobalCacheManager] as not ready for next block execution. If called concurrently, /// only a single invocation ever succeeds and other calls return an error. pub fn mark_block_execution_start( @@ -265,21 +294,45 @@ impl GlobalCacheManager { .mark_block_execution_start(state_view, previous_block_id)?; if self.inner.config.prefetch_framework_code && self.module_cache().size() == 0 { - let runtime_environment = self.environment()?; + let code_storage = state_view.as_aptos_code_storage(self.environment()?); - let code_storage = state_view.as_aptos_code_storage(runtime_environment); + // If framework code exists in storage, the transitive closure will be verified and + // cached. let result = code_storage .fetch_verified_module(&AccountAddress::ONE, ident_str!("transaction_validation")); - // Framework must exist, prefetch. - if let Ok(Some(_)) = result { - // TODO(loader_v2): Replace with invariant violations! - self.inner - .module_cache - .insert_verified_unchecked(code_storage.into_verified_module_code_iter()) - .unwrap(); + + match result { + Ok(Some(_)) => { + // Framework must have been loaded. Drain verified modules from local cache + // into global cache. + let verified_module_code_iter = code_storage + .into_verified_module_code_iter() + .map_err(|err| { + let msg = format!( + "Unable to convert cached modules into verified code: {:?}", + err + ); + self.inner.reset_and_return_invariant_violation(&msg) + })?; + self.inner + .module_cache + .insert_verified_unchecked(verified_module_code_iter) + .map_err(|err| { + let msg = format!("Unable to cache verified framework: {:?}", err); + self.inner.reset_and_return_invariant_violation(&msg) + })?; + }, + Ok(None) => { + // No framework in the state, do nothing. + }, + Err(err) => { + // There should be no errors when pre-fetching the framework, if there are, we + // better return an error here. + let msg = format!("Error when pre-fetching the framework: {:?}", err); + return Err(self.inner.reset_and_return_invariant_violation(&msg)); + }, } } - Ok(()) } @@ -301,6 +354,7 @@ impl GlobalCacheManager { .lock() .clone() .ok_or_else(|| { + // Note: we do not expect this to happen (this is really more of an unreachable). invariant_violation("Environment must always be set at block execution start") }) } diff --git a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs index 39670da6165e1..82a7015c30dff 100644 --- a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs +++ b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs @@ -4,6 +4,7 @@ use crate::module_and_script_storage::module_storage::AptosModuleStorage; use ambassador::Delegate; use aptos_types::{ + error::PanicError, state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView}, vm::modules::AptosModuleExtension, }; @@ -82,17 +83,27 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, Self { storage } } + /// Drains cached verified modules from the code storage, transforming them into format used by + /// global caches (i.e., with extension and no versioning). Should only be called when the code + /// storage borrows [StateView]. pub fn into_verified_module_code_iter( self, - ) -> impl Iterator< - Item = ( - ModuleId, - Arc>>, - ), + ) -> Result< + impl Iterator< + Item = ( + ModuleId, + Arc>>, + ), + >, + PanicError, > { let state_view = match self.storage.module_storage().byte_storage().state_view { BorrowedOrOwned::Borrowed(state_view) => state_view, - BorrowedOrOwned::Owned(_) => unreachable!(), + BorrowedOrOwned::Owned(_) => { + return Err(PanicError::CodeInvariantError( + "Verified modules should only be extracted from borrowed state".to_string(), + )) + }, }; let mut modules_to_add = vec![]; @@ -101,14 +112,33 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, .into_module_storage() .into_verified_modules_iter() { - let state_key = StateKey::module_id(&key); - // TODO(loader_v2): Replace with invariant violations! - let state_value = state_view.get_state_value(&state_key).unwrap().unwrap(); - let extension = AptosModuleExtension::new(state_value); + // We have cached the module previously, so we must be able to find it in storage. + let extension = state_view + .get_state_value(&StateKey::module_id(&key)) + .map_err(|err| { + let msg = format!( + "Failed to retrieve module {}::{} from storage {:?}", + key.address(), + key.name(), + err + ); + PanicError::CodeInvariantError(msg) + })? + .map(AptosModuleExtension::new) + .ok_or_else(|| { + let msg = format!( + "Module {}::{} should exist, but it does not anymore", + key.address(), + key.name() + ); + PanicError::CodeInvariantError(msg) + })?; + + // We are using storage version here. let module = ModuleCode::from_verified_ref(verified_code, Arc::new(extension), None); modules_to_add.push((key, Arc::new(module))) } - modules_to_add.into_iter() + Ok(modules_to_add.into_iter()) } } diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs index 6ce4f94ea71b8..19f6f1ae55fee 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_code_storage.rs @@ -29,11 +29,12 @@ use std::sync::Arc; pub struct UnsyncCodeStorage(UnsyncCodeStorageImpl); impl UnsyncCodeStorage { - /// Returns the underlying module storage used by this code storage. + /// Returns the reference to the underlying module storage used by this code storage. pub fn module_storage(&self) -> &M { &self.0.module_storage } + /// Returns the underlying module storage used by this code storage. pub fn into_module_storage(self) -> M { self.0.module_storage } diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs index e6e0e33f86868..c85d698d5c704 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs @@ -167,6 +167,7 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> UnsyncModuleStorage<' &self.0.base_storage } + /// Returns an iterator of all modules that have been cached and verified. pub fn into_verified_modules_iter(self) -> impl Iterator)> { self.0 .module_cache From 583ccfa156588b0f735a443d90e51f42be68fc28 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Thu, 7 Nov 2024 14:48:09 +0000 Subject: [PATCH 08/25] [refactoring] Remove versions from module code --- .../aptos-global-cache-manager/src/config.rs | 8 +- .../aptos-global-cache-manager/src/manager.rs | 43 +- .../state_view_adapter.rs | 4 +- .../block-executor/src/captured_reads.rs | 106 +++-- aptos-move/block-executor/src/code_cache.rs | 29 +- aptos-move/block-executor/src/executor.rs | 11 +- aptos-move/mvhashmap/src/lib.rs | 2 +- aptos-move/mvhashmap/src/unsync_map.rs | 2 +- .../implementations/unsync_module_storage.rs | 13 +- .../runtime/src/storage/module_storage.rs | 27 +- .../types/src/code/cache/module_cache.rs | 412 ++++++++++-------- .../types/src/code/cache/test_types.rs | 37 +- .../move/move-vm/types/src/code/mod.rs | 3 +- types/src/read_only_module_cache.rs | 178 ++++---- 14 files changed, 470 insertions(+), 405 deletions(-) diff --git a/aptos-move/aptos-global-cache-manager/src/config.rs b/aptos-move/aptos-global-cache-manager/src/config.rs index 67eaabf212257..c0c31120d105e 100644 --- a/aptos-move/aptos-global-cache-manager/src/config.rs +++ b/aptos-move/aptos-global-cache-manager/src/config.rs @@ -5,8 +5,8 @@ pub struct GlobalCacheConfig { /// If true, when global caches are empty, Aptos framework is prefetched into module cache. pub prefetch_framework_code: bool, - /// The maximum size of module cache. If module cache exceeds this capacity, it should be - /// flushed. + /// The maximum number of entries stored in module cache. If module cache exceeds this value, + /// all its entries should be flushed. pub module_cache_capacity: usize, /// The maximum size of struct name re-indexing map stored in runtime environment. pub struct_name_index_map_capacity: usize, @@ -20,8 +20,8 @@ impl Default for GlobalCacheConfig { // are using. Self { prefetch_framework_code: true, - module_cache_capacity: 100_000, - struct_name_index_map_capacity: 100_000, + module_cache_capacity: 10_000, + struct_name_index_map_capacity: 10_000, } } } diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs index 055879840c18d..72ea645e2d65b 100644 --- a/aptos-move/aptos-global-cache-manager/src/manager.rs +++ b/aptos-move/aptos-global-cache-manager/src/manager.rs @@ -60,7 +60,6 @@ struct GlobalCacheManagerInner { /// when it is known that there are no concurrent accesses, e.g., at block boundaries. /// [GlobalCacheManagerInner] tries to ensure that these invariants always hold. module_cache: Arc>, - /// Identifies previously executed block, initially [BlockId::Unset]. previous_block_id: Mutex, /// Identifies the previously used execution environment, initially [None]. The environment, as @@ -178,7 +177,7 @@ where if struct_name_index_map_size > self.config.struct_name_index_map_capacity { flush_all_caches = true; } - if self.module_cache.size() > self.config.module_cache_capacity { + if self.module_cache.num_modules() > self.config.module_cache_capacity { // Technically, if we flush modules we do not need to flush type caches, but we unify // flushing logic for easier reasoning. flush_all_caches = true; @@ -293,7 +292,7 @@ impl GlobalCacheManager { self.inner .mark_block_execution_start(state_view, previous_block_id)?; - if self.inner.config.prefetch_framework_code && self.module_cache().size() == 0 { + if self.inner.config.prefetch_framework_code && self.module_cache().num_modules() == 0 { let code_storage = state_view.as_aptos_code_storage(self.environment()?); // If framework code exists in storage, the transitive closure will be verified and @@ -375,7 +374,9 @@ mod test { state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, }; use claims::{assert_err, assert_ok}; - use move_vm_types::code::{mock_verified_code, MockDeserializedCode, MockVerifiedCode}; + use move_vm_types::code::{ + mock_verified_code, MockDeserializedCode, MockExtension, MockVerifiedCode, + }; use std::{collections::HashMap, thread, thread::JoinHandle}; use test_case::test_case; @@ -413,7 +414,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new_with_default_config(); assert!(global_cache_manager.ready_for_next_block()); @@ -431,11 +432,11 @@ mod test { global_cache_manager .module_cache - .insert(0, mock_verified_code(0, None)); + .insert(0, mock_verified_code(0, MockExtension::new(8))); global_cache_manager .module_cache - .insert(1, mock_verified_code(1, None)); - assert_eq!(global_cache_manager.module_cache.size(), 2); + .insert(1, mock_verified_code(1, MockExtension::new(8))); + assert_eq!(global_cache_manager.module_cache.num_modules(), 2); assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); let old_environment = global_cache_manager @@ -444,7 +445,7 @@ mod test { .clone() .unwrap(); assert_ok!(global_cache_manager.mark_block_execution_end(Some(HashValue::zero()))); - assert_eq!(global_cache_manager.module_cache.size(), 2); + assert_eq!(global_cache_manager.module_cache.num_modules(), 2); // Tweak feature flags to force a different config. let mut features = old_environment.features().clone(); @@ -462,7 +463,7 @@ mod test { assert_ok!( global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::zero())) ); - assert_eq!(global_cache_manager.module_cache.size(), 0); + assert_eq!(global_cache_manager.module_cache.num_modules(), 0); let new_environment = global_cache_manager .previous_environment @@ -490,17 +491,17 @@ mod test { global_cache_manager .module_cache - .insert(0, mock_verified_code(0, None)); + .insert(0, mock_verified_code(0, MockExtension::new(8))); global_cache_manager .module_cache - .insert(1, mock_verified_code(1, None)); - assert_eq!(global_cache_manager.module_cache.size(), 2); + .insert(1, mock_verified_code(1, MockExtension::new(8))); + assert_eq!(global_cache_manager.module_cache.num_modules(), 2); // Cache is too large, should be flushed for next block. assert_ok!( global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) ); - assert_eq!(global_cache_manager.module_cache.size(), 0); + assert_eq!(global_cache_manager.module_cache.num_modules(), 0); } #[test_case(None)] @@ -511,13 +512,13 @@ mod test { global_cache_manager .module_cache - .insert(0, mock_verified_code(0, None)); - assert_eq!(global_cache_manager.module_cache.size(), 1); + .insert(0, mock_verified_code(0, MockExtension::new(8))); + assert_eq!(global_cache_manager.module_cache.num_modules(), 1); // If executed on top of unset state, or the state with matching previous hash, the cache // is not flushed. assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); - assert_eq!(global_cache_manager.module_cache.size(), 1); + assert_eq!(global_cache_manager.module_cache.num_modules(), 1); assert!(!global_cache_manager.ready_for_next_block()); } @@ -540,18 +541,18 @@ mod test { global_cache_manager .module_cache - .insert(0, mock_verified_code(0, None)); - assert_eq!(global_cache_manager.module_cache.size(), 1); + .insert(0, mock_verified_code(0, MockExtension::new(8))); + assert_eq!(global_cache_manager.module_cache.num_modules(), 1); assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); assert!(!global_cache_manager.ready_for_next_block()); if recorded_previous_block_id.is_some() && recorded_previous_block_id == previous_block_id { // In this case both IDs match, no cache flushing. - assert_eq!(global_cache_manager.module_cache.size(), 1); + assert_eq!(global_cache_manager.module_cache.num_modules(), 1); } else { // If previous block IDs do not match, or are unknown, caches must be flushed! - assert_eq!(global_cache_manager.module_cache.size(), 0); + assert_eq!(global_cache_manager.module_cache.num_modules(), 0); } } diff --git a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs index 82a7015c30dff..0e78bfce3d1b9 100644 --- a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs +++ b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs @@ -92,7 +92,7 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, impl Iterator< Item = ( ModuleId, - Arc>>, + Arc>, ), >, PanicError, @@ -135,7 +135,7 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, })?; // We are using storage version here. - let module = ModuleCode::from_verified_ref(verified_code, Arc::new(extension), None); + let module = ModuleCode::from_verified_ref(verified_code, Arc::new(extension)); modules_to_add.push((key, Arc::new(module))) } Ok(modules_to_add.into_iter()) diff --git a/aptos-move/block-executor/src/captured_reads.rs b/aptos-move/block-executor/src/captured_reads.rs index dc82e33c6b2ef..9c0ba6c2ea4cf 100644 --- a/aptos-move/block-executor/src/captured_reads.rs +++ b/aptos-move/block-executor/src/captured_reads.rs @@ -293,11 +293,11 @@ impl DelayedFieldRead { /// Represents a module read, either from immutable cross-block cache, or from code [SyncCodeCache] /// used by block executor (per-block cache). This way, when transaction needs to read a module /// from [SyncCodeCache] it can first check the read-set here. -enum ModuleRead { +enum ModuleRead { /// Read from the cross-block module cache. GlobalCache, /// Read from per-block cache ([SyncCodeCache]) used by parallel execution. - PerBlockCache(Option>>>), + PerBlockCache(Option<(Arc>, V)>), } /// Represents a result of a read from [CapturedReads] when they are used as the transaction-level @@ -324,7 +324,7 @@ pub(crate) struct CapturedReads { #[deprecated] pub(crate) deprecated_module_reads: Vec, - module_reads: hashbrown::HashMap>, + module_reads: hashbrown::HashMap>>, /// If there is a speculative failure (e.g. delta application failure, or an observed /// inconsistency), the transaction output is irrelevant (must be discarded and transaction @@ -620,7 +620,7 @@ where pub(crate) fn capture_per_block_cache_read( &mut self, key: K, - read: Option>>>, + read: Option<(Arc>, Option)>, ) { self.module_reads .insert(key, ModuleRead::PerBlockCache(read)); @@ -632,7 +632,7 @@ where pub(crate) fn get_module_read( &self, key: &K, - ) -> Result>>>>, PanicError> { + ) -> Result>, Option)>>, PanicError> { Ok(match self.module_reads.get(key) { Some(ModuleRead::PerBlockCache(read)) => CacheRead::Hit(read.clone()), Some(ModuleRead::GlobalCache) => { @@ -661,7 +661,7 @@ where ModuleRead::GlobalCache => global_module_cache.contains_valid(key), ModuleRead::PerBlockCache(previous) => { let current_version = per_block_module_cache.get_module_version(key); - let previous_version = previous.as_ref().map(|module| module.version()); + let previous_version = previous.as_ref().map(|(_, version)| *version); current_version == previous_version }, }) @@ -879,8 +879,8 @@ mod test { }; use move_vm_types::{ code::{ - mock_deserialized_code, mock_verified_code, MockDeserializedCode, MockVerifiedCode, - ModuleCache, + mock_deserialized_code, mock_verified_code, MockDeserializedCode, MockExtension, + MockVerifiedCode, ModuleCache, }, delayed_values::delayed_field_id::DelayedFieldID, }; @@ -1075,7 +1075,13 @@ mod test { ($m:expr, $x:expr, $y:expr) => {{ let original = $m.get(&$x).cloned().unwrap(); assert_matches!( - CapturedReads::::update_entry($m.entry($x), $y.clone()), + CapturedReads::< + TestTransactionType, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::update_entry($m.entry($x), $y.clone()), UpdateResult::IncorrectUse(_) ); assert_some_eq!($m.get(&$x), &original); @@ -1086,7 +1092,13 @@ mod test { ($m:expr, $x:expr, $y:expr) => {{ let original = $m.get(&$x).cloned().unwrap(); assert_matches!( - CapturedReads::::update_entry($m.entry($x), $y.clone()), + CapturedReads::< + TestTransactionType, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::update_entry($m.entry($x), $y.clone()), UpdateResult::Inconsistency(_) ); assert_some_eq!($m.get(&$x), &original); @@ -1096,7 +1108,13 @@ mod test { macro_rules! assert_update { ($m:expr, $x:expr, $y:expr) => {{ assert_matches!( - CapturedReads::::update_entry($m.entry($x), $y.clone()), + CapturedReads::< + TestTransactionType, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::update_entry($m.entry($x), $y.clone()), UpdateResult::Updated ); assert_some_eq!($m.get(&$x), &$y); @@ -1106,7 +1124,13 @@ mod test { macro_rules! assert_insert { ($m:expr, $x:expr, $y:expr) => {{ assert_matches!( - CapturedReads::::update_entry($m.entry($x), $y.clone()), + CapturedReads::< + TestTransactionType, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::update_entry($m.entry($x), $y.clone()), UpdateResult::Inserted ); assert_some_eq!($m.get(&$x), &$y); @@ -1278,7 +1302,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - u32, + MockExtension, >::new(); let legacy_reads = legacy_reads_by_kind(); let deletion_reads = deletion_reads_by_kind(); @@ -1312,7 +1336,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - u32, + MockExtension, >::new(); captured_reads.get_by_kind(&KeyType::(21, false), Some(&10), ReadKind::Metadata); } @@ -1340,7 +1364,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let legacy_reads = legacy_reads_by_kind(); let deletion_reads = deletion_reads_by_kind(); @@ -1402,7 +1426,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let versioned_legacy = DataRead::Versioned( Err(StorageVersion), @@ -1459,7 +1483,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); captured_reads.non_delayed_field_speculative_failure = false; captured_reads.delayed_field_speculative_failure = false; @@ -1493,7 +1517,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); @@ -1514,7 +1538,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); captured_reads.capture_global_cache_read(0); @@ -1528,21 +1552,21 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); - global_module_cache.insert(0, mock_verified_code(0, None)); + global_module_cache.insert(0, mock_verified_code(0, MockExtension::new(8))); captured_reads.capture_global_cache_read(0); - global_module_cache.insert(1, mock_verified_code(1, None)); + global_module_cache.insert(1, mock_verified_code(1, MockExtension::new(8))); captured_reads.capture_global_cache_read(1); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); // Now, mark one of the entries in invalid. Validations should fail! - global_module_cache.mark_invalid(&1); + global_module_cache.mark_invalid_if_contains(&1); let valid = captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache); assert!(!valid); @@ -1566,21 +1590,21 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let per_block_module_cache: SyncModuleCache = SyncModuleCache::empty(); - let a = mock_deserialized_code(0, Some(2)); + let a = mock_deserialized_code(0, MockExtension::new(8)); per_block_module_cache .insert_deserialized_module( 0, a.code().deserialized().as_ref().clone(), a.extension().clone(), - a.version(), + Some(2), ) .unwrap(); - captured_reads.capture_per_block_cache_read(0, Some(a)); + captured_reads.capture_per_block_cache_read(0, Some((a, Some(2)))); assert!(matches!( captured_reads.get_module_read(&0), Ok(CacheRead::Hit(Some(_))) @@ -1605,32 +1629,32 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); - let a = mock_deserialized_code(0, Some(10)); + let a = mock_deserialized_code(0, MockExtension::new(8)); per_block_module_cache .insert_deserialized_module( 0, a.code().deserialized().as_ref().clone(), a.extension().clone(), - a.version(), + Some(10), ) .unwrap(); - captured_reads.capture_per_block_cache_read(0, Some(a)); + captured_reads.capture_per_block_cache_read(0, Some((a, Some(10)))); captured_reads.capture_per_block_cache_read(1, None); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); - let b = mock_deserialized_code(1, Some(12)); + let b = mock_deserialized_code(1, MockExtension::new(8)); per_block_module_cache .insert_deserialized_module( 1, b.code().deserialized().as_ref().clone(), b.extension().clone(), - b.version(), + Some(12), ) .unwrap(); @@ -1643,13 +1667,13 @@ mod test { assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); // Version has been republished, with a higher transaction index. Should fail validation. - let a = mock_deserialized_code(0, Some(20)); + let a = mock_deserialized_code(0, MockExtension::new(8)); per_block_module_cache .insert_deserialized_module( 0, a.code().deserialized().as_ref().clone(), a.extension().clone(), - a.version(), + Some(20), ) .unwrap(); @@ -1665,25 +1689,25 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new(); let global_module_cache = ReadOnlyModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); // Module exists in global cache. - global_module_cache.insert(0, mock_verified_code(0, None)); + global_module_cache.insert(0, mock_verified_code(0, MockExtension::new(8))); captured_reads.capture_global_cache_read(0); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); // Assume we republish this module: validation must fail. - let a = mock_deserialized_code(100, Some(10)); - global_module_cache.mark_invalid(&0); + let a = mock_deserialized_code(100, MockExtension::new(8)); + global_module_cache.mark_invalid_if_contains(&0); per_block_module_cache .insert_deserialized_module( 0, a.code().deserialized().as_ref().clone(), a.extension().clone(), - a.version(), + Some(10), ) .unwrap(); @@ -1692,7 +1716,7 @@ mod test { assert!(!valid); // Assume we re-read the new correct version. Then validation should pass again. - captured_reads.capture_per_block_cache_read(0, Some(a)); + captured_reads.capture_per_block_cache_read(0, Some((a, Some(10)))); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); assert!(!global_module_cache.contains_valid(&0)); } diff --git a/aptos-move/block-executor/src/code_cache.rs b/aptos-move/block-executor/src/code_cache.rs index 5f4864621b1f1..7ffe0d78b2ef4 100644 --- a/aptos-move/block-executor/src/code_cache.rs +++ b/aptos-move/block-executor/src/code_cache.rs @@ -44,28 +44,20 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCodeB type Extension = AptosModuleExtension; type Key = ModuleId; type Verified = Module; - type Version = Option; fn build( &self, key: &Self::Key, - ) -> VMResult< - Option>, - > { + ) -> VMResult>> { let key = T::Key::from_address_and_module_name(key.address(), key.name()); self.get_raw_base_value(&key) .map_err(|err| err.finish(Location::Undefined))? .map(|state_value| { - let extension = AptosModuleExtension::new(state_value); + let extension = Arc::new(AptosModuleExtension::new(state_value)); let (compiled_module, _, _) = self .runtime_environment() .deserialize_into_compiled_module(extension.bytes())?; - let version = None; - Ok(ModuleCode::from_deserialized( - compiled_module, - Arc::new(extension), - version, - )) + Ok(ModuleCode::from_deserialized(compiled_module, extension)) }) .transpose() } @@ -101,8 +93,7 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCache verified_code: Self::Verified, extension: Arc, version: Self::Version, - ) -> VMResult>> - { + ) -> VMResult>> { match &self.latest_view { ViewState::Sync(state) => { // For parallel execution, if we insert a verified module, we might need to also @@ -118,7 +109,7 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCache state .captured_reads .borrow_mut() - .capture_per_block_cache_read(key, Some(module.clone())); + .capture_per_block_cache_read(key, Some((module.clone(), version))); Ok(module) }, ViewState::Unsync(state) => state.unsync_map.module_cache().insert_verified_module( @@ -138,10 +129,12 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCache Deserialized = Self::Deserialized, Verified = Self::Verified, Extension = Self::Extension, - Version = Self::Version, >, ) -> VMResult< - Option>>, + Option<( + Arc>, + Self::Version, + )>, > { // First, look up the module in the cross-block global module cache. Record the read for // later validation in case the read module is republished. @@ -155,7 +148,7 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCache state.read_set.borrow_mut().capture_module_read(key.clone()) }, } - return Ok(Some(module.clone())); + return Ok(Some((module, Self::Version::default()))); } // Global cache miss: check module cache in versioned/unsync maps. @@ -208,7 +201,7 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> AptosModule let state_value_metadata = self .get_module_or_build_with(&id, self) .map_err(|err| err.to_partial())? - .map(|module| module.extension().state_value_metadata().clone()); + .map(|(module, _)| module.extension().state_value_metadata().clone()); Ok(state_value_metadata) } } diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 87e89d8a704d0..05521aea84c5f 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -1226,16 +1226,11 @@ where let msg = format!("Failed to construct the module from state value: {:?}", err); PanicError::CodeInvariantError(msg) })?; - let extension = AptosModuleExtension::new(state_value); + let extension = Arc::new(AptosModuleExtension::new(state_value)); - global_module_cache.mark_invalid(&id); + global_module_cache.mark_invalid_if_contains(&id); per_block_module_cache - .insert_deserialized_module( - id.clone(), - compiled_module, - Arc::new(extension), - Some(txn_idx), - ) + .insert_deserialized_module(id.clone(), compiled_module, extension, Some(txn_idx)) .map_err(|err| { let msg = format!( "Failed to insert code for module {}::{} at version {} to module cache: {:?}", diff --git a/aptos-move/mvhashmap/src/lib.rs b/aptos-move/mvhashmap/src/lib.rs index cd6fa6134826f..3241a99ff1a0e 100644 --- a/aptos-move/mvhashmap/src/lib.rs +++ b/aptos-move/mvhashmap/src/lib.rs @@ -125,7 +125,7 @@ impl< ) -> impl Iterator< Item = ( ModuleId, - Arc>>, + Arc>, ), > { self.module_cache.take_modules_iter() diff --git a/aptos-move/mvhashmap/src/unsync_map.rs b/aptos-move/mvhashmap/src/unsync_map.rs index e909c60b2ef22..c4c67948ed5b3 100644 --- a/aptos-move/mvhashmap/src/unsync_map.rs +++ b/aptos-move/mvhashmap/src/unsync_map.rs @@ -110,7 +110,7 @@ impl< ) -> impl Iterator< Item = ( ModuleId, - Arc>>, + Arc>, ), > { self.module_cache.into_modules_iter() diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs index c85d698d5c704..578b3690d3cb6 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs @@ -71,7 +71,7 @@ impl WithHash for BytesWithHash { } /// Placeholder for module versioning since we do not allow to mutate [UnsyncModuleStorage]. -#[derive(Clone, Eq, PartialEq, Ord, PartialOrd)] +#[derive(Clone, Default, Eq, PartialEq, Ord, PartialOrd)] struct NoVersion; /// Private implementation of module storage based on non-[Sync] module cache and the baseline @@ -125,14 +125,11 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> ModuleCodeBuilder type Extension = BytesWithHash; type Key = ModuleId; type Verified = Module; - type Version = NoVersion; fn build( &self, key: &Self::Key, - ) -> VMResult< - Option>, - > { + ) -> VMResult>> { let bytes = match self .base_storage .fetch_module_bytes(key.address(), key.name())? @@ -144,7 +141,7 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> ModuleCodeBuilder .runtime_environment() .deserialize_into_compiled_module(&bytes)?; let extension = Arc::new(BytesWithHash::new(bytes, hash)); - let module = ModuleCode::from_deserialized(compiled_module, extension, NoVersion); + let module = ModuleCode::from_deserialized(compiled_module, extension); Ok(Some(module)) } } @@ -190,12 +187,12 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> UnsyncModuleStorage<' assert_eq!(self.0.num_modules(), deserialized.len() + verified.len()); for id in deserialized { let result = self.0.get_module_or_build_with(id, &self.0); - let module = claims::assert_some!(claims::assert_ok!(result)); + let module = claims::assert_some!(claims::assert_ok!(result)).0; assert!(!module.code().is_verified()) } for id in verified { let result = self.0.get_module_or_build_with(id, &self.0); - let module = claims::assert_some!(claims::assert_ok!(result)); + let module = claims::assert_some!(claims::assert_ok!(result)).0; assert!(module.code().is_verified()) } } diff --git a/third_party/move/move-vm/runtime/src/storage/module_storage.rs b/third_party/move/move-vm/runtime/src/storage/module_storage.rs index 041c4eb92f775..5191c270422a3 100644 --- a/third_party/move/move-vm/runtime/src/storage/module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/module_storage.rs @@ -116,10 +116,9 @@ where Deserialized = CompiledModule, Verified = Module, Extension = E, - Version = V, >, E: WithBytes + WithHash, - V: Clone + Ord, + V: Clone + Default + Ord, { fn check_module_exists( &self, @@ -138,7 +137,7 @@ where let id = ModuleId::new(*address, module_name.to_owned()); Ok(self .get_module_or_build_with(&id, self)? - .map(|module| module.extension().bytes().clone())) + .map(|(module, _)| module.extension().bytes().clone())) } fn fetch_module_size_in_bytes( @@ -149,7 +148,7 @@ where let id = ModuleId::new(*address, module_name.to_owned()); Ok(self .get_module_or_build_with(&id, self)? - .map(|module| module.extension().bytes().len())) + .map(|(module, _)| module.extension().bytes().len())) } fn fetch_module_metadata( @@ -160,7 +159,7 @@ where let id = ModuleId::new(*address, module_name.to_owned()); Ok(self .get_module_or_build_with(&id, self)? - .map(|module| module.code().deserialized().metadata.clone())) + .map(|(module, _)| module.code().deserialized().metadata.clone())) } fn fetch_deserialized_module( @@ -171,7 +170,7 @@ where let id = ModuleId::new(*address, module_name.to_owned()); Ok(self .get_module_or_build_with(&id, self)? - .map(|module| module.code().deserialized().clone())) + .map(|(module, _)| module.code().deserialized().clone())) } fn fetch_verified_module( @@ -183,8 +182,8 @@ where // Look up the verified module in cache, if it is not there, or if the module is not yet // verified, we need to load & verify its transitive dependencies. - let module = match self.get_module_or_build_with(&id, self)? { - Some(module) => module, + let (module, version) = match self.get_module_or_build_with(&id, self)? { + Some(module_and_version) => module_and_version, None => return Ok(None), }; @@ -197,6 +196,7 @@ where Ok(Some(visit_dependencies_and_verify( id, module, + version, &mut visited, self, )?)) @@ -218,7 +218,8 @@ where /// is clearly infeasible. fn visit_dependencies_and_verify( module_id: ModuleId, - module: Arc>, + module: Arc>, + version: V, visited: &mut HashSet, module_cache_with_context: &T, ) -> VMResult> @@ -235,10 +236,9 @@ where Deserialized = CompiledModule, Verified = Module, Extension = E, - Version = V, >, E: WithBytes + WithHash, - V: Clone + Ord, + V: Clone + Default + Ord, { let runtime_environment = module_cache_with_context.runtime_environment(); @@ -260,7 +260,7 @@ where for (addr, name) in locally_verified_code.immediate_dependencies_iter() { let dependency_id = ModuleId::new(*addr, name.to_owned()); - let dependency = module_cache_with_context + let (dependency, dependency_version) = module_cache_with_context .get_module_or_build_with(&dependency_id, module_cache_with_context)? .ok_or_else(|| module_linker_error!(addr, name))?; @@ -275,6 +275,7 @@ where let verified_dependency = visit_dependencies_and_verify( dependency_id.clone(), dependency, + dependency_version, visited, module_cache_with_context, )?; @@ -294,7 +295,7 @@ where module_id, verified_code, module.extension().clone(), - module.version(), + version, )?; Ok(module.code().verified().clone()) } diff --git a/third_party/move/move-vm/types/src/code/cache/module_cache.rs b/third_party/move/move-vm/types/src/code/cache/module_cache.rs index 74d5f683c9852..10a0f201acbc0 100644 --- a/third_party/move/move-vm/types/src/code/cache/module_cache.rs +++ b/third_party/move/move-vm/types/src/code/cache/module_cache.rs @@ -10,44 +10,36 @@ use move_binary_format::errors::VMResult; use std::{cell::RefCell, cmp::Ordering, hash::Hash, mem, ops::Deref, sync::Arc}; /// Represents module code stored in [ModuleCode]. -pub struct ModuleCode { +pub struct ModuleCode { /// Module's code, either deserialized or verified. code: Code, - /// Module's extension - any additional metadata associated with this module. + /// Module's extension - any additional metadata associated with this module. It can be module + /// bytes, its size, etc. We use an arc here to avoid expensive clones. extension: Arc, - /// Version of the code (e.g., which transaction within the block published this module). - version: V, } -impl ModuleCode +impl ModuleCode where VC: Deref>, - V: Clone + Ord, { /// Creates new [ModuleCode] from deserialized code. - pub fn from_deserialized(deserialized_code: DC, extension: Arc, version: V) -> Self { + pub fn from_deserialized(deserialized_code: DC, extension: Arc) -> Self { Self { code: Code::from_deserialized(deserialized_code), extension, - version, } } /// Creates new [ModuleCode] from verified code. - pub fn from_verified(verified_code: VC, extension: Arc, version: V) -> Self { - Self { - code: Code::from_verified(verified_code), - extension, - version, - } + pub fn from_verified(verified_code: VC, extension: Arc) -> Self { + Self::from_verified_ref(Arc::new(verified_code), extension) } /// Creates new [ModuleCode] from [Arc]ed verified code. - pub fn from_verified_ref(verified_code: Arc, extension: Arc, version: V) -> Self { + pub fn from_verified_ref(verified_code: Arc, extension: Arc) -> Self { Self { code: Code::from_verified_ref(verified_code), extension, - version, } } @@ -60,27 +52,13 @@ where pub fn extension(&self) -> &Arc { &self.extension } - - /// Returns module's version. - pub fn version(&self) -> V { - self.version.clone() - } - - /// Sets the module version. - pub fn set_version(&mut self, version: V) { - self.version = version; - } } -impl Clone for ModuleCode -where - V: Clone, -{ +impl Clone for ModuleCode { fn clone(&self) -> Self { Self { code: self.code.clone(), extension: self.extension.clone(), - version: self.version.clone(), } } } @@ -91,16 +69,13 @@ pub trait ModuleCodeBuilder { type Deserialized; type Verified; type Extension; - type Version: Clone + Ord; /// For the given key, returns [ModuleCode] if it exists, and [None] otherwise. In case /// initialization fails, returns an error. fn build( &self, key: &Self::Key, - ) -> VMResult< - Option>, - >; + ) -> VMResult>>; } /// Interface used by any module cache implementation. @@ -110,7 +85,7 @@ pub trait ModuleCache { type Deserialized; type Verified; type Extension; - type Version: Clone + Ord; + type Version: Clone + Default + Ord; /// Stores deserialized code at specified version to the module cache if there was no entry /// associated with this key before. If module cache already contains an entry, then: @@ -138,12 +113,11 @@ pub trait ModuleCache { verified_code: Self::Verified, extension: Arc, version: Self::Version, - ) -> VMResult>>; + ) -> VMResult>>; /// Ensures that the entry in the module cache is initialized using the provided initializer, /// if it was not stored before. Returns the stored module, or [None] if it does not exist. If - /// initialization fails, returns the error. The caller can also provide the kind of read they - /// intend to do. + /// initialization fails, returns the error. fn get_module_or_build_with( &self, key: &Self::Key, @@ -152,16 +126,67 @@ pub trait ModuleCache { Deserialized = Self::Deserialized, Verified = Self::Verified, Extension = Self::Extension, - Version = Self::Version, >, ) -> VMResult< - Option>>, + Option<( + Arc>, + Self::Version, + )>, >; /// Returns the number of modules in cache. fn num_modules(&self) -> usize; } +struct VersionedModuleCode { + module_code: Arc>, + version: V, +} + +impl VersionedModuleCode +where + V: Default + Clone + Ord, +{ + fn new(module_code: ModuleCode, version: V) -> Self { + Self { + module_code: Arc::new(module_code), + version, + } + } + + fn new_with_default_version(module_code: ModuleCode) -> Self { + Self::new(module_code, V::default()) + } + + fn module_code(&self) -> &Arc> { + &self.module_code + } + + fn into_module_code(self) -> Arc> { + self.module_code + } + + fn version(&self) -> V { + self.version.clone() + } + + fn as_module_code_and_version(&self) -> (Arc>, V) { + (self.module_code.clone(), self.version.clone()) + } +} + +impl Clone for VersionedModuleCode +where + V: Default + Clone + Ord, +{ + fn clone(&self) -> Self { + Self { + module_code: self.module_code.clone(), + version: self.version.clone(), + } + } +} + /// An error when inserting a module with a smaller version to module cache containing the higher /// version. macro_rules! version_too_small_error { @@ -176,14 +201,14 @@ macro_rules! version_too_small_error { /// Non-[Sync] version of module cache suitable for sequential execution. pub struct UnsyncModuleCache { - module_cache: RefCell>>>, + module_cache: RefCell>>, } impl UnsyncModuleCache where K: Eq + Hash + Clone, VC: Deref>, - V: Clone + Ord, + V: Clone + Default + Ord, { /// Returns an empty module cache. pub fn empty() -> Self { @@ -193,8 +218,11 @@ where } /// Returns the iterator to all keys and modules stored in the cache. - pub fn into_modules_iter(self) -> impl Iterator>)> { - self.module_cache.into_inner().into_iter() + pub fn into_modules_iter(self) -> impl Iterator>)> { + self.module_cache + .into_inner() + .into_iter() + .map(|(k, m)| (k, m.into_module_code())) } } @@ -202,7 +230,7 @@ impl ModuleCache for UnsyncModuleCache where K: Eq + Hash + Clone, VC: Deref>, - V: Clone + Ord, + V: Clone + Default + Ord, { type Deserialized = DC; type Extension = E; @@ -224,18 +252,14 @@ where Ordering::Less => Err(version_too_small_error!()), Ordering::Equal => Ok(()), Ordering::Greater => { - let module = Arc::new(ModuleCode::from_deserialized( - deserialized_code, - extension, - version, - )); - entry.insert(module); + let module = ModuleCode::from_deserialized(deserialized_code, extension); + entry.insert(VersionedModuleCode::new(module, version)); Ok(()) }, }, Vacant(entry) => { - let module = ModuleCode::from_deserialized(deserialized_code, extension, version); - entry.insert(Arc::new(module)); + let module = ModuleCode::from_deserialized(deserialized_code, extension); + entry.insert(VersionedModuleCode::new(module, version)); Ok(()) }, } @@ -247,34 +271,42 @@ where verified_code: Self::Verified, extension: Arc, version: Self::Version, - ) -> VMResult>> - { + ) -> VMResult>> { use hashbrown::hash_map::Entry::*; match self.module_cache.borrow_mut().entry(key) { Occupied(mut entry) => match version.cmp(&entry.get().version()) { Ordering::Less => Err(version_too_small_error!()), Ordering::Equal => { - if entry.get().code().is_verified() { - Ok(entry.get().clone()) + if entry.get().module_code().code().is_verified() { + Ok(entry.get().module_code().clone()) } else { - let module = - Arc::new(ModuleCode::from_verified(verified_code, extension, version)); - entry.insert(module.clone()); + let versioned_module = VersionedModuleCode::new( + ModuleCode::from_verified(verified_code, extension), + version, + ); + let module = versioned_module.module_code().clone(); + entry.insert(versioned_module); Ok(module) } }, Ordering::Greater => { - let module = - Arc::new(ModuleCode::from_verified(verified_code, extension, version)); - entry.insert(module.clone()); + let versioned_module = VersionedModuleCode::new( + ModuleCode::from_verified(verified_code, extension), + version, + ); + let module = versioned_module.module_code().clone(); + entry.insert(versioned_module); Ok(module) }, }, - Vacant(entry) => { - let module = ModuleCode::from_verified(verified_code, extension, version); - Ok(entry.insert(Arc::new(module)).clone()) - }, + Vacant(entry) => Ok(entry + .insert(VersionedModuleCode::new( + ModuleCode::from_verified(verified_code, extension), + version, + )) + .module_code() + .clone()), } } @@ -286,18 +318,22 @@ where Deserialized = Self::Deserialized, Verified = Self::Verified, Extension = Self::Extension, - Version = Self::Version, >, ) -> VMResult< - Option>>, + Option<( + Arc>, + Self::Version, + )>, > { use hashbrown::hash_map::Entry::*; Ok(match self.module_cache.borrow_mut().entry(key.clone()) { - Occupied(entry) => Some(entry.get().clone()), - Vacant(entry) => builder - .build(key)? - .map(|module| entry.insert(Arc::new(module)).clone()), + Occupied(entry) => Some(entry.get().as_module_code_and_version()), + Vacant(entry) => builder.build(key)?.map(|module| { + entry + .insert(VersionedModuleCode::new_with_default_version(module)) + .as_module_code_and_version() + }), }) } @@ -308,14 +344,14 @@ where /// [Sync] version of module cache, suitable for parallel execution. pub struct SyncModuleCache { - module_cache: DashMap>>>, + module_cache: DashMap>>, } impl SyncModuleCache where K: Eq + Hash + Clone, VC: Deref>, - V: Clone + Ord, + V: Clone + Default + Ord, { /// Returns a new empty module cache. pub fn empty() -> Self { @@ -331,12 +367,10 @@ where } /// Takes the modules stored in the module cache, and returns an iterator of keys and modules. - pub fn take_modules_iter( - &mut self, - ) -> impl Iterator>)> { + pub fn take_modules_iter(&mut self) -> impl Iterator>)> { mem::take(&mut self.module_cache) .into_iter() - .map(|(key, module)| (key, module.into_inner())) + .map(|(key, module)| (key, module.into_inner().into_module_code())) } } @@ -344,7 +378,7 @@ impl ModuleCache for SyncModuleCache where K: Eq + Hash + Clone, VC: Deref>, - V: Clone + Ord, + V: Clone + Default + Ord, { type Deserialized = DC; type Extension = E; @@ -366,18 +400,14 @@ where Ordering::Less => Err(version_too_small_error!()), Ordering::Equal => Ok(()), Ordering::Greater => { - let module = Arc::new(ModuleCode::from_deserialized( - deserialized_code, - extension, - version, - )); - entry.insert(CachePadded::new(module)); + let module = ModuleCode::from_deserialized(deserialized_code, extension); + entry.insert(CachePadded::new(VersionedModuleCode::new(module, version))); Ok(()) }, }, Vacant(entry) => { - let module = ModuleCode::from_deserialized(deserialized_code, extension, version); - entry.insert(CachePadded::new(Arc::new(module))); + let module = ModuleCode::from_deserialized(deserialized_code, extension); + entry.insert(CachePadded::new(VersionedModuleCode::new(module, version))); Ok(()) }, } @@ -389,37 +419,39 @@ where verified_code: Self::Verified, extension: Arc, version: Self::Version, - ) -> VMResult>> - { + ) -> VMResult>> { use dashmap::mapref::entry::Entry::*; match self.module_cache.entry(key) { Occupied(mut entry) => match version.cmp(&entry.get().version()) { Ordering::Less => Err(version_too_small_error!()), Ordering::Equal => { - if entry.get().code().is_verified() { - Ok(entry.get().deref().clone()) + if entry.get().module_code().code().is_verified() { + Ok(entry.get().module_code().clone()) } else { - let module = - Arc::new(ModuleCode::from_verified(verified_code, extension, version)); - entry.insert(CachePadded::new(module.clone())); + let versioned_module = VersionedModuleCode::new( + ModuleCode::from_verified(verified_code, extension), + version, + ); + let module = versioned_module.module_code().clone(); + entry.insert(CachePadded::new(versioned_module)); Ok(module) } }, Ordering::Greater => { - let module = - Arc::new(ModuleCode::from_verified(verified_code, extension, version)); - entry.insert(CachePadded::new(module.clone())); + let versioned_module = VersionedModuleCode::new( + ModuleCode::from_verified(verified_code, extension), + version, + ); + let module = versioned_module.module_code().clone(); + entry.insert(CachePadded::new(versioned_module)); Ok(module) }, }, Vacant(entry) => { - let module = ModuleCode::from_verified(verified_code, extension, version); - Ok(entry - .insert(CachePadded::new(Arc::new(module))) - .deref() - .deref() - .clone()) + let module = ModuleCode::from_verified(verified_code, extension); + let v = entry.insert(CachePadded::new(VersionedModuleCode::new(module, version))); + Ok(v.module_code().clone()) }, } } @@ -432,27 +464,28 @@ where Deserialized = Self::Deserialized, Verified = Self::Verified, Extension = Self::Extension, - Version = Self::Version, >, ) -> VMResult< - Option>>, + Option<( + Arc>, + Self::Version, + )>, > { use dashmap::mapref::entry::Entry::*; - if let Some(module) = self.module_cache.get(key) { - return Ok(Some(Arc::clone(module.deref()))); + if let Some(v) = self.module_cache.get(key).as_deref() { + return Ok(Some(v.as_module_code_and_version())); } Ok(match self.module_cache.entry(key.clone()) { - Occupied(entry) => Some(Arc::clone(entry.get())), - Vacant(entry) => match builder.build(key)? { - Some(module) => { - let module = Arc::new(module); - entry.insert(CachePadded::new(module.clone())); - Some(module) - }, - None => None, - }, + Occupied(entry) => Some(entry.get().as_module_code_and_version()), + Vacant(entry) => builder.build(key)?.map(|module| { + entry + .insert(CachePadded::new( + VersionedModuleCode::new_with_default_version(module), + )) + .as_module_code_and_version() + }), }) } @@ -464,7 +497,7 @@ where #[cfg(test)] mod test { use super::*; - use crate::code::{MockDeserializedCode, MockVerifiedCode}; + use crate::code::{mock_extension, MockDeserializedCode, MockExtension, MockVerifiedCode}; use claims::{assert_ok, assert_some}; use move_binary_format::errors::{Location, PartialVMError}; use move_core_types::vm_status::StatusCode; @@ -473,17 +506,15 @@ mod test { impl ModuleCodeBuilder for Unreachable { type Deserialized = MockDeserializedCode; - type Extension = (); + type Extension = MockExtension; type Key = usize; type Verified = MockVerifiedCode; - type Version = u32; fn build( &self, _key: &Self::Key, - ) -> VMResult< - Option>, - > { + ) -> VMResult>> + { unreachable!("Should never be called!") } } @@ -492,19 +523,17 @@ mod test { impl ModuleCodeBuilder for WithSomeValue { type Deserialized = MockDeserializedCode; - type Extension = (); + type Extension = MockExtension; type Key = usize; type Verified = MockVerifiedCode; - type Version = u32; fn build( &self, _key: &Self::Key, - ) -> VMResult< - Option>, - > { + ) -> VMResult>> + { let code = MockDeserializedCode::new(self.0); - Ok(Some(ModuleCode::from_deserialized(code, Arc::new(()), 0))) + Ok(Some(ModuleCode::from_deserialized(code, mock_extension(8)))) } } @@ -512,17 +541,15 @@ mod test { impl ModuleCodeBuilder for WithNone { type Deserialized = MockDeserializedCode; - type Extension = (); + type Extension = MockExtension; type Key = usize; type Verified = MockVerifiedCode; - type Version = u32; fn build( &self, _key: &Self::Key, - ) -> VMResult< - Option>, - > { + ) -> VMResult>> + { Ok(None) } } @@ -531,17 +558,15 @@ mod test { impl ModuleCodeBuilder for WithError { type Deserialized = MockDeserializedCode; - type Extension = (); + type Extension = MockExtension; type Key = usize; type Verified = MockVerifiedCode; - type Version = u32; fn build( &self, _key: &Self::Key, - ) -> VMResult< - Option>, - > { + ) -> VMResult>> + { Err(PartialVMError::new(StatusCode::STORAGE_ERROR).finish(Location::Undefined)) } } @@ -551,7 +576,7 @@ mod test { Key = usize, Deserialized = MockDeserializedCode, Verified = MockVerifiedCode, - Extension = (), + Extension = MockExtension, Version = u32, >, ) { @@ -559,57 +584,61 @@ mod test { assert_ok!(module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(1), - Arc::new(()), - 0, + mock_extension(8), + 0 )); assert_ok!(module_cache.insert_deserialized_module( 2, MockDeserializedCode::new(2), - Arc::new(()), - 0, + mock_extension(8), + 0 )); assert_eq!(module_cache.num_modules(), 2); let deserialized_module_1 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&1, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module_1.code().deserialized().value(), 1); let deserialized_module_2 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&2, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module_2.code().deserialized().value(), 2); // Module cache already stores deserialized code at the same version. assert_ok!(module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(10), - Arc::new(()), + mock_extension(8), 0 )); assert_eq!(module_cache.num_modules(), 2); let deserialized_module = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&1, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module.code().deserialized().value(), 1); // Module cache stores deserialized code at lower version, so it should be replaced. assert_ok!(module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(100), - Arc::new(()), + mock_extension(8), 10 )); assert_eq!(module_cache.num_modules(), 2); let deserialized_module = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&1, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module.code().deserialized().value(), 100); // We already have higher-versioned deserialized code stored. let result = module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(100), - Arc::new(()), + mock_extension(8), 5, ); assert!(result.is_err()); @@ -618,7 +647,7 @@ mod test { assert_ok!(module_cache.insert_verified_module( 3, MockVerifiedCode::new(3), - Arc::new(()), + mock_extension(8), 10 )); assert_eq!(module_cache.num_modules(), 3); @@ -627,7 +656,7 @@ mod test { let result = module_cache.insert_deserialized_module( 3, MockDeserializedCode::new(30), - Arc::new(()), + mock_extension(8), 0, ); assert!(result.is_err()); @@ -636,38 +665,43 @@ mod test { assert_ok!(module_cache.insert_deserialized_module( 3, MockDeserializedCode::new(300), - Arc::new(()), + mock_extension(8), 10 )); assert_eq!(module_cache.num_modules(), 3); let deserialized_module = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&3, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module.code().deserialized().value(), 3); // If the version is higher, we replace the module even though it was verified before. assert_ok!(module_cache.insert_deserialized_module( 3, MockDeserializedCode::new(3000), - Arc::new(()), + mock_extension(8), 20 )); assert_eq!(module_cache.num_modules(), 3); let deserialized_module = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&3, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module.code().deserialized().value(), 3000); // Check states. let module_1 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&1, &Unreachable) - )); + )) + .0; let module_2 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&2, &Unreachable) - )); + )) + .0; let module_3 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&3, &Unreachable) - )); + )) + .0; assert!(matches!(module_1.code(), Code::Deserialized(s) if s.value() == 100)); assert!(matches!(module_2.code(), Code::Deserialized(s) if s.value() == 2)); assert!(matches!(module_3.code(), Code::Deserialized(s) if s.value() == 3000)); @@ -678,7 +712,7 @@ mod test { Key = usize, Deserialized = MockDeserializedCode, Verified = MockVerifiedCode, - Extension = (), + Extension = MockExtension, Version = u32, >, ) { @@ -686,14 +720,14 @@ mod test { let verified_module_1 = assert_ok!(module_cache.insert_verified_module( 1, MockVerifiedCode::new(1), - Arc::new(()), + mock_extension(8), 10, )); let verified_module_2 = assert_ok!(module_cache.insert_verified_module( 2, MockVerifiedCode::new(2), - Arc::new(()), - 10, + mock_extension(8), + 10 )); assert_eq!(module_cache.num_modules(), 2); @@ -706,18 +740,19 @@ mod test { assert_ok!(module_cache.insert_deserialized_module( 2, MockDeserializedCode::new(20), - Arc::new(()), + mock_extension(8), 10 )); assert_eq!(module_cache.num_modules(), 2); let deserialized_module = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&2, &Unreachable) - )); + )) + .0; assert_eq!(deserialized_module.code().deserialized().value(), 2); let verified_module = assert_ok!(module_cache.insert_verified_module( 2, MockVerifiedCode::new(200), - Arc::new(()), + mock_extension(8), 10 )); assert_eq!(module_cache.num_modules(), 2); @@ -727,32 +762,37 @@ mod test { let result = module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(10), - Arc::new(()), + mock_extension(8), 0, ); assert!(result.is_err()); - let result = - module_cache.insert_verified_module(1, MockVerifiedCode::new(100), Arc::new(()), 0); + let result = module_cache.insert_verified_module( + 1, + MockVerifiedCode::new(100), + mock_extension(8), + 0, + ); assert!(result.is_err()); // Higher versions should be inserted, whether they are verified or deserialized. assert_ok!(module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(1000), - Arc::new(()), + mock_extension(8), 100 )); assert_eq!(module_cache.num_modules(), 2); let deserialized_module = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&1, &Unreachable) - )); + )) + .0; assert!(!deserialized_module.code().is_verified()); assert_eq!(deserialized_module.code().deserialized().value(), 1000); let verified_module = assert_ok!(module_cache.insert_verified_module( 1, MockVerifiedCode::new(10_000), - Arc::new(()), + mock_extension(8), 1000 )); assert_eq!(module_cache.num_modules(), 2); @@ -762,10 +802,12 @@ mod test { // Check states. let module_1 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&1, &Unreachable) - )); + )) + .0; let module_2 = assert_some!(assert_ok!( module_cache.get_module_or_build_with(&2, &Unreachable) - )); + )) + .0; assert!(matches!(module_1.code(), Code::Verified(s) if s.value() == 10_000)); assert!(matches!(module_2.code(), Code::Verified(s) if s.value() == 2)); } @@ -775,32 +817,32 @@ mod test { Key = usize, Deserialized = MockDeserializedCode, Verified = MockVerifiedCode, - Extension = (), + Extension = MockExtension, Version = u32, >, ) { assert_ok!(module_cache.insert_deserialized_module( 1, MockDeserializedCode::new(1), - Arc::new(()), - 0, + mock_extension(8), + 0 )); assert_ok!(module_cache.insert_verified_module( 2, MockVerifiedCode::new(2), - Arc::new(()), - 0, + mock_extension(8), + 0 )); // Get existing deserialized module. let result = module_cache.get_module_or_build_with(&1, &WithSomeValue(100)); - let module_1 = assert_some!(assert_ok!(result)); + let module_1 = assert_some!(assert_ok!(result)).0; assert!(!module_1.code().is_verified()); assert_eq!(module_1.code().deserialized().value(), 1); // Get existing verified module. let result = module_cache.get_module_or_build_with(&2, &WithError); - let module_2 = assert_some!(assert_ok!(result)); + let module_2 = assert_some!(assert_ok!(result)).0; assert!(module_2.code().is_verified()); assert_eq!(module_2.code().deserialized().value(), 2); @@ -817,13 +859,13 @@ mod test { // Successful initialization. let result = module_cache.get_module_or_build_with(&3, &WithSomeValue(300)); - let module_3 = assert_some!(assert_ok!(result)); + let module_3 = assert_some!(assert_ok!(result)).0; assert!(!module_3.code().is_verified()); assert_eq!(module_3.code().deserialized().value(), 300); assert_eq!(module_cache.num_modules(), 3); let result = module_cache.get_module_or_build_with(&3, &WithSomeValue(1000)); - let module_3 = assert_some!(assert_ok!(result)); + let module_3 = assert_some!(assert_ok!(result)).0; assert!(!module_3.code().is_verified()); assert_eq!(module_3.code().deserialized().value(), 300); assert_eq!(module_cache.num_modules(), 3); diff --git a/third_party/move/move-vm/types/src/code/cache/test_types.rs b/third_party/move/move-vm/types/src/code/cache/test_types.rs index 2b79b5d607045..35f7cf7ae56ed 100644 --- a/third_party/move/move-vm/types/src/code/cache/test_types.rs +++ b/third_party/move/move-vm/types/src/code/cache/test_types.rs @@ -4,7 +4,7 @@ use crate::code::ModuleCode; use std::{ops::Deref, sync::Arc}; -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct MockDeserializedCode(usize); impl MockDeserializedCode { @@ -17,6 +17,7 @@ impl MockDeserializedCode { } } +#[derive(Debug)] pub struct MockVerifiedCode(Arc); impl MockVerifiedCode { @@ -33,24 +34,38 @@ impl Deref for MockVerifiedCode { } } -pub fn mock_deserialized_code( +pub fn mock_deserialized_code( value: usize, - version: V, -) -> Arc> { + extension: E, +) -> Arc> { Arc::new(ModuleCode::from_deserialized( MockDeserializedCode::new(value), - Arc::new(()), - version, + Arc::new(extension), )) } -pub fn mock_verified_code( +pub fn mock_verified_code( value: usize, - version: V, -) -> Arc> { + extension: E, +) -> Arc> { Arc::new(ModuleCode::from_verified( MockVerifiedCode::new(value), - Arc::new(()), - version, + Arc::new(extension), )) } + +#[derive(Clone, Debug)] +pub struct MockExtension { + #[allow(dead_code)] + size: usize, +} + +impl MockExtension { + pub fn new(size: usize) -> Self { + Self { size } + } +} + +pub fn mock_extension(size: usize) -> Arc { + Arc::new(MockExtension::new(size)) +} diff --git a/third_party/move/move-vm/types/src/code/mod.rs b/third_party/move/move-vm/types/src/code/mod.rs index 399519dca919b..880394caab392 100644 --- a/third_party/move/move-vm/types/src/code/mod.rs +++ b/third_party/move/move-vm/types/src/code/mod.rs @@ -7,7 +7,8 @@ mod storage; #[cfg(any(test, feature = "testing"))] pub use cache::test_types::{ - mock_deserialized_code, mock_verified_code, MockDeserializedCode, MockVerifiedCode, + mock_deserialized_code, mock_extension, mock_verified_code, MockDeserializedCode, + MockExtension, MockVerifiedCode, }; pub use cache::{ module_cache::{ diff --git a/types/src/read_only_module_cache.rs b/types/src/read_only_module_cache.rs index 55b3cbeef599a..ae7c07335f133 100644 --- a/types/src/read_only_module_cache.rs +++ b/types/src/read_only_module_cache.rs @@ -16,31 +16,24 @@ use std::{ /// Entry stored in [ReadOnlyModuleCache]. struct Entry { - /// True if this code is "valid" within the block execution context (i.e, there has been no + /// True if this code is "valid" within the block execution context (i.e., there has been no /// republishing of this module so far). If false, executor needs to read the module from the /// sync/unsync module caches. valid: CachePadded, - /// Cached verified module. While [ModuleCode] type is used, the following invariants always - /// hold: - /// 1. Module's version is [None] (storage version). - /// 2. Module's code is always verified. - module: CachePadded>>>, + /// Cached verified module. Must always be verified. + module: CachePadded>>, } impl Entry where VC: Deref>, { - /// Returns a new valid module. Returns a (panic) error if the module is not verified or has - /// non-storage version. - fn new(module: Arc>>) -> Result { - if !module.code().is_verified() || module.version().is_some() { - let msg = format!( - "Invariant violated for immutable module code : verified ({}), version({:?})", - module.code().is_verified(), - module.version() - ); - return Err(PanicError::CodeInvariantError(msg)); + /// Returns a new valid module. Returns a (panic) error if the module is not verified. + fn new(module: Arc>) -> Result { + if !module.code().is_verified() { + return Err(PanicError::CodeInvariantError( + "Module code is not verified".to_string(), + )); } Ok(Self { @@ -60,13 +53,13 @@ where } /// Returns the module code stored is this [Entry]. - fn inner(&self) -> &Arc>> { + fn module_code(&self) -> &Arc> { self.module.deref() } } /// A read-only module cache for verified code, that can be accessed concurrently within the block. -/// It can only be modified at block boundaries. +/// Can only be modified safely at block boundaries. pub struct ReadOnlyModuleCache { /// Module cache containing the verified code. module_cache: ExplicitSyncWrapper>>, @@ -84,33 +77,30 @@ where } } - /// Returns true if the key exists in immutable cache and the corresponding module is valid. + /// Returns true if the key exists in cache and the corresponding module is valid. pub fn contains_valid(&self, key: &K) -> bool { self.module_cache .acquire() .get(key) - .is_some_and(|module| module.is_valid()) + .is_some_and(|entry| entry.is_valid()) } /// Marks the cached module (if it exists) as invalid. As a result, all subsequent calls to the - /// cache for the associated key will result in a cache miss. Note that it is fine for an - /// entry not to exist, in which case this is a no-op. - pub fn mark_invalid(&self, key: &K) { - if let Some(module) = self.module_cache.acquire().get(key) { - module.mark_invalid(); + /// cache for the associated key will result in a cache miss. If an entry does not to exist, is + /// a no-op. + pub fn mark_invalid_if_contains(&self, key: &K) { + if let Some(entry) = self.module_cache.acquire().get(key) { + entry.mark_invalid(); } } - /// Returns the module stored in cache. If the module has not been cached, or it exists but is - /// not valid, [None] is returned. - pub fn get(&self, key: &K) -> Option>>> { - self.module_cache.acquire().get(key).and_then(|module| { - if module.is_valid() { - Some(module.inner().clone()) - } else { - None - } - }) + /// Returns the module stored in cache. If the module has not been cached, or it exists but it + /// is not valid, [None] is returned. + pub fn get(&self, key: &K) -> Option>> { + self.module_cache + .acquire() + .get(key) + .and_then(|entry| entry.is_valid().then(|| entry.module_code().clone())) } /// Flushes the cache. Should never be called throughout block-execution. Use with caution. @@ -118,17 +108,21 @@ where self.module_cache.acquire().clear(); } + /// Returns the number of entries in the cache. + pub fn num_modules(&self) -> usize { + self.module_cache.acquire().len() + } + /// Inserts modules into the cache. Should never be called throughout block-execution. Use with /// caution. /// /// Notes: /// 1. Only verified modules are inserted. - /// 2. Versions of inserted modules are set to [None] (storage version). - /// 3. Valid modules should not be removed, and new modules should have unique ownership. If + /// 2. Valid modules should not be removed, and new modules should have unique ownership. If /// these constraints are violated, a panic error is returned. pub fn insert_verified_unchecked( &self, - modules: impl Iterator>>)>, + modules: impl Iterator>)>, ) -> Result<(), PanicError> { use hashbrown::hash_map::Entry::*; @@ -148,9 +142,9 @@ where } if module.code().is_verified() { - let mut module = module.as_ref().clone(); - module.set_version(None); - let prev = module_cache.insert(key.clone(), Entry::new(Arc::new(module))?); + let entry = + Entry::new(module).expect("Module has been checked and must be verified"); + let prev = module_cache.insert(key.clone(), entry); // At this point, we must have removed the entry, or returned a panic error. assert!(prev.is_none()) @@ -159,17 +153,13 @@ where Ok(()) } - /// Returns the size of the cache. - pub fn size(&self) -> usize { - self.module_cache.acquire().len() - } - /// Insert the module to cache. Used for tests only. #[cfg(any(test, feature = "testing"))] - pub fn insert(&self, key: K, module: Arc>>) { - self.module_cache - .acquire() - .insert(key, Entry::new(module).unwrap()); + pub fn insert(&self, key: K, module: Arc>) { + self.module_cache.acquire().insert( + key, + Entry::new(module).expect("Module code should be verified"), + ); } /// Removes the module from cache. Used for tests only. @@ -182,81 +172,87 @@ where #[cfg(test)] mod test { use super::*; - use claims::{assert_err, assert_ok, assert_some}; - use move_vm_types::code::{mock_deserialized_code, mock_verified_code}; + use claims::{assert_err, assert_ok}; + use move_vm_types::code::{mock_deserialized_code, mock_extension, mock_verified_code}; #[test] - fn test_new_entry() { - assert!(Entry::new(mock_deserialized_code(0, None)).is_err()); - assert!(Entry::new(mock_deserialized_code(0, Some(22))).is_err()); - assert!(Entry::new(mock_verified_code(0, Some(22))).is_err()); - assert!(Entry::new(mock_verified_code(0, None)).is_ok()); + fn test_entry_new() { + assert!(Entry::new(mock_deserialized_code(0, mock_extension(8))).is_err()); + assert!(Entry::new(mock_verified_code(0, mock_extension(8))).is_ok()); } #[test] - fn test_mark_entry_invalid() { - let module_code = assert_ok!(Entry::new(mock_verified_code(0, None))); - assert!(module_code.is_valid()); + fn test_entry_mark_invalid() { + let entry = assert_ok!(Entry::new(mock_verified_code(0, mock_extension(8)))); + assert!(entry.is_valid()); - module_code.mark_invalid(); - assert!(!module_code.is_valid()); + entry.mark_invalid(); + assert!(!entry.is_valid()); } #[test] - fn test_get_entry() { - let global_cache = ReadOnlyModuleCache::empty(); + fn test_cache_contains_valid_and_get() { + let cache = ReadOnlyModuleCache::empty(); + + // Set the state. + cache.insert(0, mock_verified_code(0, mock_extension(8))); + cache.insert(1, mock_verified_code(1, mock_extension(8))); + cache.mark_invalid_if_contains(&1); + + assert_eq!(cache.num_modules(), 2); - global_cache.insert(0, mock_verified_code(0, None)); - global_cache.insert(1, mock_verified_code(1, None)); - global_cache.mark_invalid(&1); + assert!(cache.contains_valid(&0)); + assert!(!cache.contains_valid(&1)); + assert!(!cache.contains_valid(&3)); - assert_eq!(global_cache.size(), 2); + assert!(cache.get(&0).is_some()); + assert!(cache.get(&1).is_none()); + assert!(cache.get(&3).is_none()); + } + + #[test] + fn test_num_modules_and_flush_unchecked() { + let cache = ReadOnlyModuleCache::empty(); + assert_eq!(cache.num_modules(), 0); - assert!(global_cache.contains_valid(&0)); - assert!(!global_cache.contains_valid(&1)); - assert!(!global_cache.contains_valid(&3)); + cache.insert(0, mock_verified_code(0, mock_extension(8))); + cache.insert(1, mock_verified_code(1, mock_extension(8))); + assert_eq!(cache.num_modules(), 2); - assert!(global_cache.get(&0).is_some()); - assert!(global_cache.get(&1).is_none()); - assert!(global_cache.get(&3).is_none()); + cache.flush_unchecked(); + assert_eq!(cache.num_modules(), 0); } #[test] - fn test_insert_verified_for_read_only_module_cache() { + fn test_cache_insert_verified_unchecked() { let global_cache = ReadOnlyModuleCache::empty(); let mut new_modules = vec![]; for i in 0..10 { - new_modules.push((i, mock_verified_code(i, Some(i as u32)))); + new_modules.push((i, mock_verified_code(i, mock_extension(8)))); } let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); assert!(result.is_ok()); - assert_eq!(global_cache.size(), 10); - - // Versions should be set to storage. - for key in 0..10 { - let code = assert_some!(global_cache.get(&key)); - assert!(code.version().is_none()) - } + assert_eq!(global_cache.num_modules(), 10); global_cache.flush_unchecked(); - assert_eq!(global_cache.size(), 0); + assert_eq!(global_cache.num_modules(), 0); // Should not add deserialized code. - let deserialized_modules = vec![(0, mock_deserialized_code(0, None))]; + let deserialized_modules = vec![(0, mock_deserialized_code(0, mock_extension(8)))]; assert_ok!(global_cache.insert_verified_unchecked(deserialized_modules.into_iter())); - assert_eq!(global_cache.size(), 0); + assert_eq!(global_cache.num_modules(), 0); // Should not override valid modules. - global_cache.insert(0, mock_verified_code(0, None)); - let new_modules = vec![(0, mock_verified_code(100, None))]; + global_cache.insert(0, mock_verified_code(0, mock_extension(8))); + let new_modules = vec![(0, mock_verified_code(100, mock_extension(8)))]; assert_err!(global_cache.insert_verified_unchecked(new_modules.into_iter())); // Can override invalid modules. - global_cache.mark_invalid(&0); - let new_modules = vec![(0, mock_verified_code(100, None))]; + global_cache.mark_invalid_if_contains(&0); + let new_modules = vec![(0, mock_verified_code(100, mock_extension(8)))]; let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); assert!(result.is_ok()); - assert_eq!(global_cache.size(), 1); + assert_eq!(global_cache.num_modules(), 1); } } From 8bee4e8845f398c099dd90d0536e79b03118f8bc Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Thu, 7 Nov 2024 16:45:19 +0000 Subject: [PATCH 09/25] [comments] Addressing Rati's comments + adding size in bytes to module cache --- .../aptos-global-cache-manager/Cargo.toml | 1 + .../aptos-global-cache-manager/src/config.rs | 20 +-- .../aptos-global-cache-manager/src/manager.rs | 28 ++-- .../block-executor/src/captured_reads.rs | 3 +- .../runtime/src/storage/module_storage.rs | 6 +- .../types/src/code/cache/module_cache.rs | 7 + .../types/src/code/cache/test_types.rs | 9 +- .../move-vm/types/src/code/cache/types.rs | 6 + .../move/move-vm/types/src/code/mod.rs | 2 +- types/src/read_only_module_cache.rs | 124 +++++++++++++----- 10 files changed, 141 insertions(+), 65 deletions(-) diff --git a/aptos-move/aptos-global-cache-manager/Cargo.toml b/aptos-move/aptos-global-cache-manager/Cargo.toml index 272333f72bda7..203c76ef0927e 100644 --- a/aptos-move/aptos-global-cache-manager/Cargo.toml +++ b/aptos-move/aptos-global-cache-manager/Cargo.toml @@ -20,6 +20,7 @@ aptos-vm-types = { workspace = true } move-binary-format = { workspace = true } move-core-types = { workspace = true } move-vm-runtime = { workspace = true } +move-vm-types = { workspace = true } parking_lot = { workspace = true } [dev-dependencies] diff --git a/aptos-move/aptos-global-cache-manager/src/config.rs b/aptos-move/aptos-global-cache-manager/src/config.rs index c0c31120d105e..8427f087ffa66 100644 --- a/aptos-move/aptos-global-cache-manager/src/config.rs +++ b/aptos-move/aptos-global-cache-manager/src/config.rs @@ -5,23 +5,23 @@ pub struct GlobalCacheConfig { /// If true, when global caches are empty, Aptos framework is prefetched into module cache. pub prefetch_framework_code: bool, - /// The maximum number of entries stored in module cache. If module cache exceeds this value, - /// all its entries should be flushed. - pub module_cache_capacity: usize, - /// The maximum size of struct name re-indexing map stored in runtime environment. - pub struct_name_index_map_capacity: usize, + /// The maximum size serialized modules can take in module cache. + pub max_module_cache_size_in_bytes: usize, + /// The maximum size (in terms of entries) of struct name re-indexing map stored in runtime + /// environment. + pub max_struct_name_index_map_size: usize, } impl Default for GlobalCacheConfig { fn default() -> Self { // TODO(loader_v2): - // Right now these are just some numbers, we can set them based on the upper bounds of - // module or identifier sizes, or keep track in read-only module cache how many bytes we - // are using. + // Right now these are hardcoded here, we probably want to add them to gas schedule or + // some on-chain config. Self { prefetch_framework_code: true, - module_cache_capacity: 10_000, - struct_name_index_map_capacity: 10_000, + // Use 50 Mb for now, should be large enough to cache many modules. + max_module_cache_size_in_bytes: 50 * 1024 * 1024, + max_struct_name_index_map_size: 100_000, } } } diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs index 72ea645e2d65b..5f0b96d4aa665 100644 --- a/aptos-move/aptos-global-cache-manager/src/manager.rs +++ b/aptos-move/aptos-global-cache-manager/src/manager.rs @@ -17,6 +17,7 @@ use move_core_types::{ vm_status::{StatusCode, VMStatus}, }; use move_vm_runtime::{Module, ModuleStorage, WithRuntimeEnvironment}; +use move_vm_types::code::WithSize; use parking_lot::Mutex; use std::{ hash::Hash, @@ -54,11 +55,11 @@ impl BlockId { /// Manages global caches, e.g., modules or execution environment. Should not be used concurrently. struct GlobalCacheManagerInner { - /// Different configurations used for handling global caches. config: GlobalCacheConfig, + /// Cache for modules. It is read-only for any concurrent execution, and can only be mutated /// when it is known that there are no concurrent accesses, e.g., at block boundaries. - /// [GlobalCacheManagerInner] tries to ensure that these invariants always hold. + /// [GlobalCacheManagerInner] must ensure that these invariants always hold. module_cache: Arc>, /// Identifies previously executed block, initially [BlockId::Unset]. previous_block_id: Mutex, @@ -75,6 +76,7 @@ impl GlobalCacheManagerInner where K: Hash + Eq + Clone, VC: Deref>, + E: WithSize, { /// Returns a new instance of [GlobalCacheManagerInner] with default [GlobalCacheConfig]. fn new_with_default_config() -> Self { @@ -174,10 +176,10 @@ where Ok(size) => size, }; - if struct_name_index_map_size > self.config.struct_name_index_map_capacity { + if struct_name_index_map_size > self.config.max_struct_name_index_map_size { flush_all_caches = true; } - if self.module_cache.num_modules() > self.config.module_cache_capacity { + if self.module_cache.size_in_bytes() > self.config.max_module_cache_size_in_bytes { // Technically, if we flush modules we do not need to flush type caches, but we unify // flushing logic for easier reasoning. flush_all_caches = true; @@ -234,7 +236,7 @@ where } /// Resets all states (under a lock) as if global caches are empty and no blocks have been - /// executed so far. Reruns an invariant violation error. + /// executed so far. Returns an invariant violation error. fn reset_and_return_invariant_violation(&self, msg: &str) -> VMStatus { // Lock to reset the state under lock. let mut previous_block_id = self.previous_block_id.lock(); @@ -278,7 +280,7 @@ impl GlobalCacheManager { /// 1. Previously executed block ID does not match the provided value. /// 2. The environment has changed for this state. /// 3. The size of the struct name re-indexing map is too large. - /// 4. The size of the module cache is too large. + /// 4. The size (in bytes) of the module cache is too large. /// /// Additionally, if cache is empty, prefetches the framework code into it. /// @@ -480,11 +482,11 @@ mod test { } #[test] - fn mark_execution_start_when_too_many_modules() { + fn mark_execution_start_when_module_cache_is_too_large() { let state_view = MockStateView::empty(); let config = GlobalCacheConfig { - module_cache_capacity: 1, + max_module_cache_size_in_bytes: 8, ..Default::default() }; let global_cache_manager = GlobalCacheManagerInner::new_with_config(config); @@ -494,14 +496,16 @@ mod test { .insert(0, mock_verified_code(0, MockExtension::new(8))); global_cache_manager .module_cache - .insert(1, mock_verified_code(1, MockExtension::new(8))); + .insert(1, mock_verified_code(1, MockExtension::new(24))); assert_eq!(global_cache_manager.module_cache.num_modules(), 2); + assert_eq!(global_cache_manager.module_cache.size_in_bytes(), 32); // Cache is too large, should be flushed for next block. assert_ok!( global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) ); assert_eq!(global_cache_manager.module_cache.num_modules(), 0); + assert_eq!(global_cache_manager.module_cache.size_in_bytes(), 0); } #[test_case(None)] @@ -565,7 +569,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new_with_default_config()); assert!(global_cache_manager.ready_for_next_block()); @@ -587,7 +591,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new_with_default_config(); assert!(global_cache_manager.previous_block_id.lock().is_unset()); @@ -619,7 +623,7 @@ mod test { u32, MockDeserializedCode, MockVerifiedCode, - (), + MockExtension, >::new_with_default_config()); global_cache_manager.mark_not_ready_for_next_block(); diff --git a/aptos-move/block-executor/src/captured_reads.rs b/aptos-move/block-executor/src/captured_reads.rs index 9c0ba6c2ea4cf..7abe1da817a9a 100644 --- a/aptos-move/block-executor/src/captured_reads.rs +++ b/aptos-move/block-executor/src/captured_reads.rs @@ -27,7 +27,7 @@ use aptos_types::{ use aptos_vm_types::resolver::ResourceGroupSize; use derivative::Derivative; use move_core_types::value::MoveTypeLayout; -use move_vm_types::code::{ModuleCode, SyncModuleCache, WithAddress, WithName}; +use move_vm_types::code::{ModuleCode, SyncModuleCache, WithAddress, WithName, WithSize}; use std::{ collections::{ hash_map::{ @@ -351,6 +351,7 @@ where T: Transaction, K: Hash + Eq + Ord + Clone, VC: Deref>, + S: WithSize, { // Return an iterator over the captured reads. pub(crate) fn get_read_values_with_delayed_fields( diff --git a/third_party/move/move-vm/runtime/src/storage/module_storage.rs b/third_party/move/move-vm/runtime/src/storage/module_storage.rs index 5191c270422a3..714871a70cc77 100644 --- a/third_party/move/move-vm/runtime/src/storage/module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/module_storage.rs @@ -11,7 +11,7 @@ use move_core_types::{ metadata::Metadata, }; use move_vm_types::{ - code::{ModuleCache, ModuleCode, ModuleCodeBuilder, WithBytes, WithHash}, + code::{ModuleCache, ModuleCode, ModuleCodeBuilder, WithBytes, WithHash, WithSize}, module_cyclic_dependency_error, module_linker_error, }; use std::sync::Arc; @@ -117,7 +117,7 @@ where Verified = Module, Extension = E, >, - E: WithBytes + WithHash, + E: WithBytes + WithSize + WithHash, V: Clone + Default + Ord, { fn check_module_exists( @@ -237,7 +237,7 @@ where Verified = Module, Extension = E, >, - E: WithBytes + WithHash, + E: WithBytes + WithSize + WithHash, V: Clone + Default + Ord, { let runtime_environment = module_cache_with_context.runtime_environment(); diff --git a/third_party/move/move-vm/types/src/code/cache/module_cache.rs b/third_party/move/move-vm/types/src/code/cache/module_cache.rs index 10a0f201acbc0..e3663c4f388da 100644 --- a/third_party/move/move-vm/types/src/code/cache/module_cache.rs +++ b/third_party/move/move-vm/types/src/code/cache/module_cache.rs @@ -138,6 +138,7 @@ pub trait ModuleCache { fn num_modules(&self) -> usize; } +/// Same as [ModuleCode], additionally storing a version. struct VersionedModuleCode { module_code: Arc>, version: V, @@ -147,6 +148,7 @@ impl VersionedModuleCode where V: Default + Clone + Ord, { + /// Returns new [ModuleCode] with the specified version. fn new(module_code: ModuleCode, version: V) -> Self { Self { module_code: Arc::new(module_code), @@ -154,22 +156,27 @@ where } } + /// Returns new [ModuleCode] with the default (storage) version. fn new_with_default_version(module_code: ModuleCode) -> Self { Self::new(module_code, V::default()) } + /// Returns the reference to the stored module. fn module_code(&self) -> &Arc> { &self.module_code } + /// Returns the stored module. fn into_module_code(self) -> Arc> { self.module_code } + /// Returns the version associated with this module. fn version(&self) -> V { self.version.clone() } + /// Returns the clone of the module along with its version. fn as_module_code_and_version(&self) -> (Arc>, V) { (self.module_code.clone(), self.version.clone()) } diff --git a/third_party/move/move-vm/types/src/code/cache/test_types.rs b/third_party/move/move-vm/types/src/code/cache/test_types.rs index 35f7cf7ae56ed..793d5f11951ef 100644 --- a/third_party/move/move-vm/types/src/code/cache/test_types.rs +++ b/third_party/move/move-vm/types/src/code/cache/test_types.rs @@ -1,7 +1,7 @@ // Copyright (c) The Move Contributors // SPDX-License-Identifier: Apache-2.0 -use crate::code::ModuleCode; +use crate::code::{ModuleCode, WithSize}; use std::{ops::Deref, sync::Arc}; #[derive(Clone, Debug)] @@ -56,7 +56,6 @@ pub fn mock_verified_code( #[derive(Clone, Debug)] pub struct MockExtension { - #[allow(dead_code)] size: usize, } @@ -66,6 +65,12 @@ impl MockExtension { } } +impl WithSize for MockExtension { + fn size_in_bytes(&self) -> usize { + self.size + } +} + pub fn mock_extension(size: usize) -> Arc { Arc::new(MockExtension::new(size)) } diff --git a/third_party/move/move-vm/types/src/code/cache/types.rs b/third_party/move/move-vm/types/src/code/cache/types.rs index 82477e5a6a795..09f51096b9cb9 100644 --- a/third_party/move/move-vm/types/src/code/cache/types.rs +++ b/third_party/move/move-vm/types/src/code/cache/types.rs @@ -9,7 +9,13 @@ use std::{ops::Deref, sync::Arc}; pub trait WithBytes { fn bytes(&self) -> &Bytes; +} + +pub trait WithSize { + fn size_in_bytes(&self) -> usize; +} +impl WithSize for T { fn size_in_bytes(&self) -> usize { self.bytes().len() } diff --git a/third_party/move/move-vm/types/src/code/mod.rs b/third_party/move/move-vm/types/src/code/mod.rs index 880394caab392..eadb57123c3bf 100644 --- a/third_party/move/move-vm/types/src/code/mod.rs +++ b/third_party/move/move-vm/types/src/code/mod.rs @@ -16,6 +16,6 @@ pub use cache::{ UnsyncModuleCache, }, script_cache::{ambassador_impl_ScriptCache, ScriptCache, SyncScriptCache, UnsyncScriptCache}, - types::{Code, WithAddress, WithBytes, WithHash, WithName}, + types::{Code, WithAddress, WithBytes, WithHash, WithName, WithSize}, }; pub use storage::ModuleBytesStorage; diff --git a/types/src/read_only_module_cache.rs b/types/src/read_only_module_cache.rs index ae7c07335f133..a7d70138822aa 100644 --- a/types/src/read_only_module_cache.rs +++ b/types/src/read_only_module_cache.rs @@ -4,12 +4,12 @@ use crate::{error::PanicError, explicit_sync_wrapper::ExplicitSyncWrapper}; use crossbeam::utils::CachePadded; use hashbrown::HashMap; -use move_vm_types::code::ModuleCode; +use move_vm_types::code::{ModuleCode, WithSize}; use std::{ hash::Hash, ops::Deref, sync::{ - atomic::{AtomicBool, Ordering}, + atomic::{AtomicBool, AtomicUsize, Ordering}, Arc, }, }; @@ -27,6 +27,7 @@ struct Entry { impl Entry where VC: Deref>, + E: WithSize, { /// Returns a new valid module. Returns a (panic) error if the module is not verified. fn new(module: Arc>) -> Result { @@ -63,17 +64,21 @@ where pub struct ReadOnlyModuleCache { /// Module cache containing the verified code. module_cache: ExplicitSyncWrapper>>, + /// Sum of serialized sizes (in bytes) of all cached modules. + size: AtomicUsize, } impl ReadOnlyModuleCache where K: Hash + Eq + Clone, VC: Deref>, + E: WithSize, { /// Returns new empty module cache. pub fn empty() -> Self { Self { module_cache: ExplicitSyncWrapper::new(HashMap::new()), + size: AtomicUsize::new(0), } } @@ -106,6 +111,7 @@ where /// Flushes the cache. Should never be called throughout block-execution. Use with caution. pub fn flush_unchecked(&self) { self.module_cache.acquire().clear(); + self.size.store(0, Ordering::Relaxed); } /// Returns the number of entries in the cache. @@ -113,6 +119,11 @@ where self.module_cache.acquire().len() } + /// Returns the sum of serialized sizes of modules stored in cache. + pub fn size_in_bytes(&self) -> usize { + self.size.load(Ordering::Relaxed) + } + /// Inserts modules into the cache. Should never be called throughout block-execution. Use with /// caution. /// @@ -137,11 +148,15 @@ where )); } else { // Otherwise, remove the invalid entry. + let size = entry.get().module_code().extension().size_in_bytes(); + self.size.fetch_sub(size, Ordering::Relaxed); entry.remove(); } } if module.code().is_verified() { + self.size + .fetch_add(module.extension().size_in_bytes(), Ordering::Relaxed); let entry = Entry::new(module).expect("Module has been checked and must be verified"); let prev = module_cache.insert(key.clone(), entry); @@ -156,6 +171,8 @@ where /// Insert the module to cache. Used for tests only. #[cfg(any(test, feature = "testing"))] pub fn insert(&self, key: K, module: Arc>) { + self.size + .fetch_add(module.extension().size_in_bytes(), Ordering::Relaxed); self.module_cache.acquire().insert( key, Entry::new(module).expect("Module code should be verified"), @@ -165,7 +182,12 @@ where /// Removes the module from cache. Used for tests only. #[cfg(any(test, feature = "testing"))] pub fn remove(&self, key: &K) { - self.module_cache.acquire().remove(key); + if let Some(entry) = self.module_cache.acquire().remove(key) { + self.size.fetch_sub( + entry.module_code().extension().size_in_bytes(), + Ordering::Relaxed, + ); + } } } @@ -173,17 +195,17 @@ where mod test { use super::*; use claims::{assert_err, assert_ok}; - use move_vm_types::code::{mock_deserialized_code, mock_extension, mock_verified_code}; + use move_vm_types::code::{mock_deserialized_code, mock_verified_code, MockExtension}; #[test] fn test_entry_new() { - assert!(Entry::new(mock_deserialized_code(0, mock_extension(8))).is_err()); - assert!(Entry::new(mock_verified_code(0, mock_extension(8))).is_ok()); + assert!(Entry::new(mock_deserialized_code(0, MockExtension::new(8))).is_err()); + assert!(Entry::new(mock_verified_code(0, MockExtension::new(8))).is_ok()); } #[test] fn test_entry_mark_invalid() { - let entry = assert_ok!(Entry::new(mock_verified_code(0, mock_extension(8)))); + let entry = assert_ok!(Entry::new(mock_verified_code(0, MockExtension::new(8)))); assert!(entry.is_valid()); entry.mark_invalid(); @@ -195,8 +217,8 @@ mod test { let cache = ReadOnlyModuleCache::empty(); // Set the state. - cache.insert(0, mock_verified_code(0, mock_extension(8))); - cache.insert(1, mock_verified_code(1, mock_extension(8))); + cache.insert(0, mock_verified_code(0, MockExtension::new(8))); + cache.insert(1, mock_verified_code(1, MockExtension::new(8))); cache.mark_invalid_if_contains(&1); assert_eq!(cache.num_modules(), 2); @@ -214,45 +236,75 @@ mod test { fn test_num_modules_and_flush_unchecked() { let cache = ReadOnlyModuleCache::empty(); assert_eq!(cache.num_modules(), 0); + assert_eq!(cache.size_in_bytes(), 0); + + cache.insert(0, mock_verified_code(0, MockExtension::new(8))); + cache.insert(1, mock_verified_code(1, MockExtension::new(16))); + cache.insert(2, mock_verified_code(2, MockExtension::new(8))); + assert_eq!(cache.num_modules(), 3); + assert_eq!(cache.size_in_bytes(), 32); - cache.insert(0, mock_verified_code(0, mock_extension(8))); - cache.insert(1, mock_verified_code(1, mock_extension(8))); + cache.remove(&2); assert_eq!(cache.num_modules(), 2); + assert_eq!(cache.size_in_bytes(), 24); cache.flush_unchecked(); assert_eq!(cache.num_modules(), 0); + assert_eq!(cache.size_in_bytes(), 0); } #[test] fn test_cache_insert_verified_unchecked() { - let global_cache = ReadOnlyModuleCache::empty(); + let cache = ReadOnlyModuleCache::empty(); let mut new_modules = vec![]; for i in 0..10 { - new_modules.push((i, mock_verified_code(i, mock_extension(8)))); + new_modules.push((i, mock_verified_code(i, MockExtension::new(8)))); } - let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); - assert!(result.is_ok()); - assert_eq!(global_cache.num_modules(), 10); - - global_cache.flush_unchecked(); - assert_eq!(global_cache.num_modules(), 0); - - // Should not add deserialized code. - let deserialized_modules = vec![(0, mock_deserialized_code(0, mock_extension(8)))]; - assert_ok!(global_cache.insert_verified_unchecked(deserialized_modules.into_iter())); - assert_eq!(global_cache.num_modules(), 0); - - // Should not override valid modules. - global_cache.insert(0, mock_verified_code(0, mock_extension(8))); - let new_modules = vec![(0, mock_verified_code(100, mock_extension(8)))]; - assert_err!(global_cache.insert_verified_unchecked(new_modules.into_iter())); - - // Can override invalid modules. - global_cache.mark_invalid_if_contains(&0); - let new_modules = vec![(0, mock_verified_code(100, mock_extension(8)))]; - let result = global_cache.insert_verified_unchecked(new_modules.into_iter()); - assert!(result.is_ok()); - assert_eq!(global_cache.num_modules(), 1); + assert!(cache + .insert_verified_unchecked(new_modules.into_iter()) + .is_ok()); + + assert_eq!(cache.num_modules(), 10); + assert_eq!(cache.size_in_bytes(), 80); + } + + #[test] + fn test_cache_insert_verified_unchecked_does_not_add_deserialized_code() { + let cache = ReadOnlyModuleCache::empty(); + + let deserialized_modules = vec![(0, mock_deserialized_code(0, MockExtension::new(8)))]; + assert_ok!(cache.insert_verified_unchecked(deserialized_modules.into_iter())); + + assert_eq!(cache.num_modules(), 0); + assert_eq!(cache.size_in_bytes(), 0); + } + + #[test] + fn test_cache_insert_verified_unchecked_does_not_override_valid_modules() { + let cache = ReadOnlyModuleCache::empty(); + + cache.insert(0, mock_verified_code(0, MockExtension::new(8))); + assert_eq!(cache.num_modules(), 1); + assert_eq!(cache.size_in_bytes(), 8); + + let new_modules = vec![(0, mock_verified_code(100, MockExtension::new(32)))]; + assert_err!(cache.insert_verified_unchecked(new_modules.into_iter())); + } + + #[test] + fn test_cache_insert_verified_unchecked_overrides_invalid_modules() { + let cache = ReadOnlyModuleCache::empty(); + + cache.insert(0, mock_verified_code(0, MockExtension::new(8))); + cache.mark_invalid_if_contains(&0); + assert_eq!(cache.num_modules(), 1); + assert_eq!(cache.size_in_bytes(), 8); + + let new_modules = vec![(0, mock_verified_code(100, MockExtension::new(32)))]; + assert_ok!(cache.insert_verified_unchecked(new_modules.into_iter())); + + assert_eq!(cache.num_modules(), 1); + assert_eq!(cache.size_in_bytes(), 32); } } From 6c4060343017423a9ed0e9258cbebf00a8720a49 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Fri, 8 Nov 2024 09:55:12 +0000 Subject: [PATCH 10/25] [draft] Better states and encapsulation --- Cargo.lock | 31 +- Cargo.toml | 2 - aptos-move/aptos-debugger/Cargo.toml | 1 - .../aptos-debugger/src/aptos_debugger.rs | 20 +- .../aptos-e2e-comparison-testing/Cargo.toml | 1 - .../aptos-global-cache-manager/Cargo.toml | 32 - .../aptos-global-cache-manager/src/config.rs | 27 - .../aptos-global-cache-manager/src/lib.rs | 7 - .../aptos-global-cache-manager/src/manager.rs | 640 ------------------ .../aptos-transaction-benchmarks/Cargo.toml | 1 - .../src/transaction_bench_state.rs | 24 +- .../Cargo.toml | 1 - .../src/aptos_test_harness.rs | 20 +- aptos-move/aptos-vm-profiling/Cargo.toml | 1 - .../src/bins/run_aptos_p2p.rs | 8 +- .../state_view_adapter.rs | 4 +- aptos-move/aptos-vm/Cargo.toml | 3 +- aptos-move/aptos-vm/src/aptos_vm.rs | 47 +- aptos-move/aptos-vm/src/block_executor/mod.rs | 189 +++++- aptos-move/aptos-vm/src/lib.rs | 26 +- .../sharded_block_executor/global_executor.rs | 8 +- .../sharded_executor_service.rs | 12 +- .../aptos-vm/tests/sharded_block_executor.rs | 1 - aptos-move/block-executor/Cargo.toml | 1 + .../block-executor/src/captured_reads.rs | 21 +- aptos-move/block-executor/src/code_cache.rs | 2 + .../block-executor/src/code_cache_global.rs | 33 +- .../src/code_cache_global_manager.rs | 530 +++++++++++++++ aptos-move/block-executor/src/executor.rs | 18 +- .../src/explicit_sync_wrapper.rs | 2 - aptos-move/block-executor/src/lib.rs | 3 + .../src/proptest_types/bencher.rs | 6 +- .../src/proptest_types/tests.rs | 18 +- aptos-move/block-executor/src/scheduler.rs | 6 +- .../src/txn_last_input_output.rs | 2 +- .../block-executor/src/unit_tests/mod.rs | 12 +- aptos-move/block-executor/src/view.rs | 12 +- aptos-move/e2e-tests/src/executor.rs | 6 +- execution/executor-benchmark/Cargo.toml | 1 - .../executor-benchmark/src/native_executor.rs | 1 - execution/executor-service/Cargo.toml | 1 - execution/executor/Cargo.toml | 1 - execution/executor/src/block_executor/mod.rs | 10 + execution/executor/src/chunk_executor/mod.rs | 1 - .../src/chunk_executor/transaction_chunk.rs | 1 - execution/executor/src/db_bootstrapper/mod.rs | 1 - execution/executor/src/fuzzing.rs | 1 - .../src/tests/mock_vm/mock_vm_test.rs | 21 +- execution/executor/src/tests/mock_vm/mod.rs | 2 - execution/executor/src/tests/mod.rs | 11 +- .../src/workflow/do_get_execution_output.rs | 5 - .../execution/ptx-executor/Cargo.toml | 1 - .../execution/ptx-executor/src/lib.rs | 2 - storage/db-tool/Cargo.toml | 1 - storage/db-tool/src/replay_on_archive.rs | 5 +- .../implementations/unsync_module_storage.rs | 9 +- types/Cargo.toml | 1 - types/src/block_executor/config.rs | 57 +- types/src/lib.rs | 4 +- 59 files changed, 926 insertions(+), 989 deletions(-) delete mode 100644 aptos-move/aptos-global-cache-manager/Cargo.toml delete mode 100644 aptos-move/aptos-global-cache-manager/src/config.rs delete mode 100644 aptos-move/aptos-global-cache-manager/src/lib.rs delete mode 100644 aptos-move/aptos-global-cache-manager/src/manager.rs rename types/src/read_only_module_cache.rs => aptos-move/block-executor/src/code_cache_global.rs (91%) create mode 100644 aptos-move/block-executor/src/code_cache_global_manager.rs rename {types => aptos-move/block-executor}/src/explicit_sync_wrapper.rs (98%) diff --git a/Cargo.lock b/Cargo.lock index bd80614bb7da1..a61a62bab4508 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -682,6 +682,7 @@ dependencies = [ "aptos-metrics-core", "aptos-mvhashmap", "aptos-types", + "aptos-vm-environment", "aptos-vm-logging", "aptos-vm-types", "arc-swap", @@ -811,7 +812,6 @@ version = "0.1.0" dependencies = [ "anyhow", "aptos-framework", - "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-rest-client", "aptos-types", @@ -1264,7 +1264,6 @@ dependencies = [ "aptos-executor", "aptos-executor-test-helpers", "aptos-executor-types", - "aptos-global-cache-manager", "aptos-logger", "aptos-storage-interface", "aptos-temppath", @@ -1463,7 +1462,6 @@ dependencies = [ "aptos-executor-types", "aptos-experimental-runtimes", "aptos-genesis", - "aptos-global-cache-manager", "aptos-indexer-grpc-table-info", "aptos-infallible", "aptos-logger", @@ -1505,7 +1503,6 @@ dependencies = [ "aptos-experimental-ptx-executor", "aptos-experimental-runtimes", "aptos-genesis", - "aptos-global-cache-manager", "aptos-jellyfish-merkle", "aptos-logger", "aptos-metrics-core", @@ -1544,7 +1541,6 @@ version = "0.1.0" dependencies = [ "aptos-block-partitioner", "aptos-config", - "aptos-global-cache-manager", "aptos-infallible", "aptos-language-e2e-tests", "aptos-logger", @@ -1651,7 +1647,6 @@ name = "aptos-experimental-ptx-executor" version = "0.1.0" dependencies = [ "aptos-experimental-runtimes", - "aptos-global-cache-manager", "aptos-infallible", "aptos-logger", "aptos-metrics-core", @@ -2055,24 +2050,6 @@ dependencies = [ "ureq", ] -[[package]] -name = "aptos-global-cache-manager" -version = "0.0.1" -dependencies = [ - "aptos-crypto", - "aptos-types", - "aptos-vm-environment", - "aptos-vm-types", - "bcs 0.1.4", - "claims", - "move-binary-format", - "move-core-types", - "move-vm-runtime", - "move-vm-types", - "parking_lot 0.12.1", - "test-case", -] - [[package]] name = "aptos-global-constants" version = "0.1.0" @@ -2857,7 +2834,6 @@ dependencies = [ "aptos-consensus", "aptos-crypto", "aptos-gas-profiling", - "aptos-global-cache-manager", "aptos-logger", "aptos-rest-client", "aptos-types", @@ -4209,7 +4185,6 @@ dependencies = [ "aptos-block-executor", "aptos-block-partitioner", "aptos-crypto", - "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-logger", "aptos-metrics-core", @@ -4314,7 +4289,6 @@ dependencies = [ "aptos-crypto", "aptos-framework", "aptos-gas-schedule", - "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-resource-viewer", "aptos-storage-interface", @@ -4373,7 +4347,6 @@ dependencies = [ "claims", "coset", "criterion", - "crossbeam", "dashmap", "derivative", "fixed", @@ -4488,7 +4461,6 @@ dependencies = [ "aptos-gas-algebra", "aptos-gas-meter", "aptos-gas-schedule", - "aptos-global-cache-manager", "aptos-infallible", "aptos-language-e2e-tests", "aptos-logger", @@ -4612,7 +4584,6 @@ dependencies = [ "anyhow", "aptos-cached-packages", "aptos-gas-schedule", - "aptos-global-cache-manager", "aptos-language-e2e-tests", "aptos-move-stdlib", "aptos-native-interface", diff --git a/Cargo.toml b/Cargo.toml index 0c6a40ff966b9..64384fd8c1c8f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,7 +16,6 @@ members = [ "aptos-move/aptos-gas-profiling", "aptos-move/aptos-gas-schedule", "aptos-move/aptos-gas-schedule-updator", - "aptos-move/aptos-global-cache-manager", "aptos-move/aptos-memory-usage-tracker", "aptos-move/aptos-native-interface", "aptos-move/aptos-release-builder", @@ -358,7 +357,6 @@ aptos-gas-schedule = { path = "aptos-move/aptos-gas-schedule" } aptos-gas-schedule-updator = { path = "aptos-move/aptos-gas-schedule-updator" } aptos-genesis = { path = "crates/aptos-genesis" } aptos-github-client = { path = "crates/aptos-github-client" } -aptos-global-cache-manager = { path = "aptos-move/aptos-global-cache-manager" } aptos-global-constants = { path = "config/global-constants" } aptos-id-generator = { path = "crates/aptos-id-generator" } aptos-indexer = { path = "crates/indexer" } diff --git a/aptos-move/aptos-debugger/Cargo.toml b/aptos-move/aptos-debugger/Cargo.toml index d8bfc76cd3e60..8e83673767603 100644 --- a/aptos-move/aptos-debugger/Cargo.toml +++ b/aptos-move/aptos-debugger/Cargo.toml @@ -18,7 +18,6 @@ aptos-block-executor = { workspace = true } aptos-consensus = { workspace = true } aptos-crypto = { workspace = true } aptos-gas-profiling = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-logger = { workspace = true } aptos-rest-client = { workspace = true } aptos-types = { workspace = true } diff --git a/aptos-move/aptos-debugger/src/aptos_debugger.rs b/aptos-move/aptos-debugger/src/aptos_debugger.rs index 81e95438fb8d1..0b4946fce95dc 100644 --- a/aptos-move/aptos-debugger/src/aptos_debugger.rs +++ b/aptos-move/aptos-debugger/src/aptos_debugger.rs @@ -4,7 +4,6 @@ use anyhow::{bail, format_err, Result}; use aptos_block_executor::txn_commit_hook::NoOpTransactionCommitHook; use aptos_gas_profiling::{GasProfiler, TransactionGasLog}; -use aptos_global_cache_manager::GlobalCacheManager; use aptos_rest_client::Client; use aptos_types::{ account_address::AccountAddress, @@ -429,26 +428,15 @@ fn execute_block_no_limit( state_view: &DebuggerStateView, concurrency_level: usize, ) -> Result, VMStatus> { - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - global_cache_manager.mark_block_execution_start(state_view, None)?; - let result = BlockAptosVM::execute_block::< - _, - NoOpTransactionCommitHook, - >( + BlockAptosVM::execute_block::<_, NoOpTransactionCommitHook>( sig_verified_txns, state_view, - &global_cache_manager, + None, BlockExecutorConfig { - local: BlockExecutorLocalConfig { - concurrency_level, - allow_fallback: true, - discard_failed_blocks: false, - }, + local: BlockExecutorLocalConfig::default_with_concurrency_level(concurrency_level), onchain: BlockExecutorConfigFromOnchain::new_no_block_limit(), }, None, ) - .map(BlockOutput::into_transaction_outputs_forced); - global_cache_manager.mark_block_execution_end(None)?; - result + .map(BlockOutput::into_transaction_outputs_forced) } diff --git a/aptos-move/aptos-e2e-comparison-testing/Cargo.toml b/aptos-move/aptos-e2e-comparison-testing/Cargo.toml index 71af9ffdc387d..30c851670b4df 100644 --- a/aptos-move/aptos-e2e-comparison-testing/Cargo.toml +++ b/aptos-move/aptos-e2e-comparison-testing/Cargo.toml @@ -13,7 +13,6 @@ default-run = "aptos-comparison-testing" [dependencies] anyhow = { workspace = true } aptos-framework = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-rest-client = { workspace = true } aptos-types = { workspace = true } diff --git a/aptos-move/aptos-global-cache-manager/Cargo.toml b/aptos-move/aptos-global-cache-manager/Cargo.toml deleted file mode 100644 index 203c76ef0927e..0000000000000 --- a/aptos-move/aptos-global-cache-manager/Cargo.toml +++ /dev/null @@ -1,32 +0,0 @@ -[package] -name = "aptos-global-cache-manager" -description = "Aptos global module and environement cache manager" -version = "0.0.1" - -# Workspace inherited keys -authors = { workspace = true } -edition = { workspace = true } -homepage = { workspace = true } -license = { workspace = true } -publish = { workspace = true } -repository = { workspace = true } -rust-version = { workspace = true } - -[dependencies] -aptos-crypto = { workspace = true } -aptos-types = { workspace = true } -aptos-vm-environment = { workspace = true } -aptos-vm-types = { workspace = true } -move-binary-format = { workspace = true } -move-core-types = { workspace = true } -move-vm-runtime = { workspace = true } -move-vm-types = { workspace = true } -parking_lot = { workspace = true } - -[dev-dependencies] -aptos-crypto = { workspace = true, features = ["fuzzing"] } -aptos-types = { workspace = true, features = ["testing"] } -bcs = { workspace = true } -claims = { workspace = true } -move-vm-types = { workspace = true, features = ["testing"] } -test-case = { workspace = true } diff --git a/aptos-move/aptos-global-cache-manager/src/config.rs b/aptos-move/aptos-global-cache-manager/src/config.rs deleted file mode 100644 index 8427f087ffa66..0000000000000 --- a/aptos-move/aptos-global-cache-manager/src/config.rs +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright © Aptos Foundation -// SPDX-License-Identifier: Apache-2.0 - -/// Configuration used for global caches. -pub struct GlobalCacheConfig { - /// If true, when global caches are empty, Aptos framework is prefetched into module cache. - pub prefetch_framework_code: bool, - /// The maximum size serialized modules can take in module cache. - pub max_module_cache_size_in_bytes: usize, - /// The maximum size (in terms of entries) of struct name re-indexing map stored in runtime - /// environment. - pub max_struct_name_index_map_size: usize, -} - -impl Default for GlobalCacheConfig { - fn default() -> Self { - // TODO(loader_v2): - // Right now these are hardcoded here, we probably want to add them to gas schedule or - // some on-chain config. - Self { - prefetch_framework_code: true, - // Use 50 Mb for now, should be large enough to cache many modules. - max_module_cache_size_in_bytes: 50 * 1024 * 1024, - max_struct_name_index_map_size: 100_000, - } - } -} diff --git a/aptos-move/aptos-global-cache-manager/src/lib.rs b/aptos-move/aptos-global-cache-manager/src/lib.rs deleted file mode 100644 index a5844418599cc..0000000000000 --- a/aptos-move/aptos-global-cache-manager/src/lib.rs +++ /dev/null @@ -1,7 +0,0 @@ -// Copyright © Aptos Foundation -// SPDX-License-Identifier: Apache-2.0 - -pub(crate) mod config; -mod manager; - -pub use manager::GlobalCacheManager; diff --git a/aptos-move/aptos-global-cache-manager/src/manager.rs b/aptos-move/aptos-global-cache-manager/src/manager.rs deleted file mode 100644 index 5f0b96d4aa665..0000000000000 --- a/aptos-move/aptos-global-cache-manager/src/manager.rs +++ /dev/null @@ -1,640 +0,0 @@ -// Copyright © Aptos Foundation -// SPDX-License-Identifier: Apache-2.0 - -use crate::config::GlobalCacheConfig; -use aptos_crypto::HashValue; -use aptos_types::{ - read_only_module_cache::ReadOnlyModuleCache, state_store::StateView, - vm::modules::AptosModuleExtension, -}; -use aptos_vm_environment::environment::AptosEnvironment; -use aptos_vm_types::module_and_script_storage::AsAptosCodeStorage; -use move_binary_format::CompiledModule; -use move_core_types::{ - account_address::AccountAddress, - ident_str, - language_storage::ModuleId, - vm_status::{StatusCode, VMStatus}, -}; -use move_vm_runtime::{Module, ModuleStorage, WithRuntimeEnvironment}; -use move_vm_types::code::WithSize; -use parking_lot::Mutex; -use std::{ - hash::Hash, - ops::Deref, - sync::{ - atomic::{AtomicBool, Ordering}, - Arc, - }, -}; - -/// Returns an invariant violation [VMStatus]. -fn invariant_violation(msg: &str) -> VMStatus { - VMStatus::error( - StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, - Some(msg.to_string()), - ) -} - -/// Represents previously executed block, recorded by [GlobalCacheManager]. -#[derive(Clone, Copy, Debug, Hash, Eq, PartialEq, PartialOrd, Ord)] -enum BlockId { - /// No block has been executed yet. - Unset, - /// Block of transactions has been executed, with known or unknown hash. Usually, the hash is - /// [None] in tests, replay, etc. - Set(Option), -} - -impl BlockId { - /// Returns true if the ID corresponds to no executed blocks. - fn is_unset(&self) -> bool { - matches!(self, Self::Unset) - } -} - -/// Manages global caches, e.g., modules or execution environment. Should not be used concurrently. -struct GlobalCacheManagerInner { - config: GlobalCacheConfig, - - /// Cache for modules. It is read-only for any concurrent execution, and can only be mutated - /// when it is known that there are no concurrent accesses, e.g., at block boundaries. - /// [GlobalCacheManagerInner] must ensure that these invariants always hold. - module_cache: Arc>, - /// Identifies previously executed block, initially [BlockId::Unset]. - previous_block_id: Mutex, - /// Identifies the previously used execution environment, initially [None]. The environment, as - /// long as it does not change, it maintained across multiple block executions. - previous_environment: Mutex>, - - /// A marker that indicates that the state of global caches is ready for block execution. Used - /// to prevent concurrent block executions. - ready_for_next_block: AtomicBool, -} - -impl GlobalCacheManagerInner -where - K: Hash + Eq + Clone, - VC: Deref>, - E: WithSize, -{ - /// Returns a new instance of [GlobalCacheManagerInner] with default [GlobalCacheConfig]. - fn new_with_default_config() -> Self { - Self::new_with_config(GlobalCacheConfig::default()) - } - - /// Returns a new instance of [GlobalCacheManagerInner] with the provided [GlobalCacheConfig]. - fn new_with_config(config: GlobalCacheConfig) -> Self { - Self { - config, - module_cache: Arc::new(ReadOnlyModuleCache::empty()), - previous_block_id: Mutex::new(BlockId::Unset), - previous_environment: Mutex::new(None), - ready_for_next_block: AtomicBool::new(true), - } - } - - /// See the documentation for [GlobalCacheManager::mark_block_execution_start]. The only - /// difference here is that there is no framework prefetching. - fn mark_block_execution_start( - &self, - state_view: &impl StateView, - previous_block_id: Option, - ) -> Result<(), VMStatus> { - let recorded_previous_block_id = { - // Acquire a lock, and check if we are ready to execute the next block. - let previous_block_id = self.previous_block_id.lock(); - if !self.ready_for_next_block() { - let msg = "Trying to execute blocks concurrently over shared global state"; - return Err(invariant_violation(msg)); - } - - // Prepare for execution. Set the flag as not ready to ensure that blocks are not - // executed concurrently using the same cache. - self.mark_not_ready_for_next_block(); - *previous_block_id - }; - - // From here, we perform checks if we need to flush the global caches. If so, this variable - // is set to true. - let mut flush_all_caches = false; - - // Check 1: We must be executing on top of the state we have seen just before. - use BlockId::*; - match (recorded_previous_block_id, previous_block_id) { - // We execute on top of empty state, everything is ok. - (Unset, None) | (Unset, Some(_)) => {}, - - // We execute on top of different (maybe also unspecified) state. In this case, caches - // need to be reset. - (Set(None), None) | (Set(None), Some(_)) | (Set(Some(_)), None) => { - flush_all_caches = true; - }, - - // Otherwise, just check if block hashes do not match. - (Set(Some(recorded_hash)), Some(hash)) => { - if recorded_hash != hash { - flush_all_caches = true; - }; - }, - }; - - // Check 2: Reset global environment if it has changed. If so, caches needs to be flushed. - let new_environment = - AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); - let mut previous_environment = self.previous_environment.lock(); - match previous_environment.as_ref() { - Some(environment) => { - if environment != &new_environment { - *previous_environment = Some(new_environment); - flush_all_caches = true; - } - }, - None => { - // If the environment is not yet set, set it. - debug_assert!(self.previous_block_id.lock().is_unset()); - *previous_environment = Some(new_environment); - }, - } - - // Check 3: At this point, environment is set to the most-up-to-date value. Check the size - // of caches is within bounds. - let runtime_environment = previous_environment - .as_ref() - .expect("Environment has to be set") - .runtime_environment(); - let struct_name_index_map_size = match runtime_environment.struct_name_index_map_size() { - Err(err) => { - // Unlock the cache, reset all states, and return. - drop(previous_environment); - let err = self.reset_and_return_invariant_violation(&format!( - "Error when getting struct name index map size: {:?}", - err - )); - return Err(err); - }, - Ok(size) => size, - }; - - if struct_name_index_map_size > self.config.max_struct_name_index_map_size { - flush_all_caches = true; - } - if self.module_cache.size_in_bytes() > self.config.max_module_cache_size_in_bytes { - // Technically, if we flush modules we do not need to flush type caches, but we unify - // flushing logic for easier reasoning. - flush_all_caches = true; - } - - // Finally, if flag is set, flush the caches. - if flush_all_caches { - runtime_environment.flush_struct_name_and_info_caches(); - self.module_cache.flush_unchecked(); - } - - Ok(()) - } - - /// See the documentation for [GlobalCacheManager::mark_block_execution_end]. - fn mark_block_execution_end( - &self, - executed_block_id: Option, - ) -> Result<(), VMStatus> { - // We are done executing a block, reset the previous block id. Do everything under lock to - // ensure it is not possible to execute blocks concurrently. - let mut previous_block_id = self.previous_block_id.lock(); - if self.ready_for_next_block() { - // This means we are executing concurrently. If so, all-but-one thread will return an - // error. Note that the caches are still consistent for that one thread. - let msg = "Should not be possible to mark block execution end for execution-ready \ - global cache, check if blocks are executed concurrently"; - return Err(invariant_violation(msg)); - } - *previous_block_id = BlockId::Set(executed_block_id); - - // Set the flag that the global cache is ready for next execution. - self.mark_ready_for_next_block(); - - Ok(()) - } - - /// Returns true of a next block is ready be executed. This is the case only when: - /// 1. the global caches have just been created, or - /// 2. [GlobalCacheManagerInner::mark_block_execution_end] was called indicating that - /// previous block execution has finished. - fn ready_for_next_block(&self) -> bool { - self.ready_for_next_block.load(Ordering::SeqCst) - } - - /// Marks caches as ready for next block execution. - fn mark_ready_for_next_block(&self) { - self.ready_for_next_block.store(true, Ordering::SeqCst); - } - - /// Marks caches as not ready for next block execution. - fn mark_not_ready_for_next_block(&self) { - self.ready_for_next_block.store(false, Ordering::SeqCst); - } - - /// Resets all states (under a lock) as if global caches are empty and no blocks have been - /// executed so far. Returns an invariant violation error. - fn reset_and_return_invariant_violation(&self, msg: &str) -> VMStatus { - // Lock to reset the state under lock. - let mut previous_block_id = self.previous_block_id.lock(); - - // 1. Should be ready for next execution. - self.mark_not_ready_for_next_block(); - // 2. Should contain no environment. - *self.previous_environment.lock() = None; - // 3. Module cache is empty. - self.module_cache.flush_unchecked(); - // 4. Block ID is unset. - *previous_block_id = BlockId::Unset; - - // State reset, unlock. - drop(previous_block_id); - - invariant_violation(msg) - } -} - -/// Same as [GlobalCacheManagerInner], but uses concrete types used by execution on Aptos instead -/// of generics. Allows us not to propagate generic type parameters everywhere (for now), but be -/// able to mock and test. -pub struct GlobalCacheManager { - inner: GlobalCacheManagerInner, -} - -impl GlobalCacheManager { - /// Returns a new instance of [GlobalCacheManager] with default [GlobalCacheConfig]. - pub fn new_with_default_config() -> Self { - Self { - inner: GlobalCacheManagerInner::new_with_default_config(), - } - } - - /// Sets the state of global caches prior to block execution on top of the provided state (with - /// the block ID). Should always sbe called prior to block execution. - /// - /// The caches stored globally (modules, struct name re-indexing map and type caches) are all - /// flushed if: - /// 1. Previously executed block ID does not match the provided value. - /// 2. The environment has changed for this state. - /// 3. The size of the struct name re-indexing map is too large. - /// 4. The size (in bytes) of the module cache is too large. - /// - /// Additionally, if cache is empty, prefetches the framework code into it. - /// - /// Marks [GlobalCacheManager] as not ready for next block execution. If called concurrently, - /// only a single invocation ever succeeds and other calls return an error. - pub fn mark_block_execution_start( - &self, - state_view: &impl StateView, - previous_block_id: Option, - ) -> Result<(), VMStatus> { - self.inner - .mark_block_execution_start(state_view, previous_block_id)?; - - if self.inner.config.prefetch_framework_code && self.module_cache().num_modules() == 0 { - let code_storage = state_view.as_aptos_code_storage(self.environment()?); - - // If framework code exists in storage, the transitive closure will be verified and - // cached. - let result = code_storage - .fetch_verified_module(&AccountAddress::ONE, ident_str!("transaction_validation")); - - match result { - Ok(Some(_)) => { - // Framework must have been loaded. Drain verified modules from local cache - // into global cache. - let verified_module_code_iter = code_storage - .into_verified_module_code_iter() - .map_err(|err| { - let msg = format!( - "Unable to convert cached modules into verified code: {:?}", - err - ); - self.inner.reset_and_return_invariant_violation(&msg) - })?; - self.inner - .module_cache - .insert_verified_unchecked(verified_module_code_iter) - .map_err(|err| { - let msg = format!("Unable to cache verified framework: {:?}", err); - self.inner.reset_and_return_invariant_violation(&msg) - })?; - }, - Ok(None) => { - // No framework in the state, do nothing. - }, - Err(err) => { - // There should be no errors when pre-fetching the framework, if there are, we - // better return an error here. - let msg = format!("Error when pre-fetching the framework: {:?}", err); - return Err(self.inner.reset_and_return_invariant_violation(&msg)); - }, - } - } - Ok(()) - } - - /// Should always be called after block execution. Sets the [GlobalCacheManager] to be ready - /// for execution (and if it is already execution-ready, returns an error). Sets the ID for the - /// executed block so that the next execution can check it. - pub fn mark_block_execution_end( - &self, - executed_block_id: Option, - ) -> Result<(), VMStatus> { - self.inner.mark_block_execution_end(executed_block_id) - } - - /// Returns the cached environment set by [GlobalCacheManager::mark_block_execution_start]. If - /// it has not been set, an invariant violation error is returned. - pub fn environment(&self) -> Result { - self.inner - .previous_environment - .lock() - .clone() - .ok_or_else(|| { - // Note: we do not expect this to happen (this is really more of an unreachable). - invariant_violation("Environment must always be set at block execution start") - }) - } - - /// Returns the global module cache. - pub fn module_cache( - &self, - ) -> Arc> { - self.inner.module_cache.clone() - } -} - -#[cfg(test)] -mod test { - use super::*; - use aptos_types::{ - on_chain_config::{FeatureFlag, Features, OnChainConfig}, - state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, - }; - use claims::{assert_err, assert_ok}; - use move_vm_types::code::{ - mock_verified_code, MockDeserializedCode, MockExtension, MockVerifiedCode, - }; - use std::{collections::HashMap, thread, thread::JoinHandle}; - use test_case::test_case; - - /// Joins threads. Succeeds only if a single handle evaluates to [Ok] and the rest are [Err]s. - fn join_and_assert_single_ok(handles: Vec>>) { - let mut num_oks = 0; - let mut num_errs = 0; - - let num_handles = handles.len(); - for handle in handles { - let result = handle.join().unwrap(); - if result.is_ok() { - num_oks += 1; - } else { - num_errs += 1; - } - } - assert_eq!(num_oks, 1); - assert_eq!(num_errs, num_handles - 1); - } - - #[test] - fn environment_should_always_be_set() { - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - assert!(global_cache_manager.environment().is_err()); - - let state_view = MockStateView::empty(); - assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); - assert_ok!(global_cache_manager.environment()); - } - - #[test] - fn mark_ready() { - let global_cache_manager = GlobalCacheManagerInner::< - u32, - MockDeserializedCode, - MockVerifiedCode, - MockExtension, - >::new_with_default_config(); - assert!(global_cache_manager.ready_for_next_block()); - - global_cache_manager.mark_not_ready_for_next_block(); - assert!(!global_cache_manager.ready_for_next_block()); - - global_cache_manager.mark_ready_for_next_block(); - assert!(global_cache_manager.ready_for_next_block()); - } - - #[test] - fn mark_execution_start_when_different_environment() { - let state_view = MockStateView::empty(); - let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); - - global_cache_manager - .module_cache - .insert(0, mock_verified_code(0, MockExtension::new(8))); - global_cache_manager - .module_cache - .insert(1, mock_verified_code(1, MockExtension::new(8))); - assert_eq!(global_cache_manager.module_cache.num_modules(), 2); - - assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, None)); - let old_environment = global_cache_manager - .previous_environment - .lock() - .clone() - .unwrap(); - assert_ok!(global_cache_manager.mark_block_execution_end(Some(HashValue::zero()))); - assert_eq!(global_cache_manager.module_cache.num_modules(), 2); - - // Tweak feature flags to force a different config. - let mut features = old_environment.features().clone(); - assert!(features.is_enabled(FeatureFlag::LIMIT_VM_TYPE_SIZE)); - features.disable(FeatureFlag::LIMIT_VM_TYPE_SIZE); - let bytes = bcs::to_bytes(&features).unwrap(); - let state_key = StateKey::resource(Features::address(), &Features::struct_tag()).unwrap(); - - let state_view = MockStateView::new(HashMap::from([( - state_key, - StateValue::new_legacy(bytes.into()), - )])); - - // We use the same previous ID, but the cache is still flushed: the environment changed. - assert_ok!( - global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::zero())) - ); - assert_eq!(global_cache_manager.module_cache.num_modules(), 0); - - let new_environment = global_cache_manager - .previous_environment - .lock() - .clone() - .unwrap(); - assert!(old_environment != new_environment); - } - - #[test] - fn mark_execution_start_when_too_many_types() { - // TODO(loader_v2): - // Propagate type caches/struct name index map APIs to here so we can mock & test. - } - - #[test] - fn mark_execution_start_when_module_cache_is_too_large() { - let state_view = MockStateView::empty(); - - let config = GlobalCacheConfig { - max_module_cache_size_in_bytes: 8, - ..Default::default() - }; - let global_cache_manager = GlobalCacheManagerInner::new_with_config(config); - - global_cache_manager - .module_cache - .insert(0, mock_verified_code(0, MockExtension::new(8))); - global_cache_manager - .module_cache - .insert(1, mock_verified_code(1, MockExtension::new(24))); - assert_eq!(global_cache_manager.module_cache.num_modules(), 2); - assert_eq!(global_cache_manager.module_cache.size_in_bytes(), 32); - - // Cache is too large, should be flushed for next block. - assert_ok!( - global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) - ); - assert_eq!(global_cache_manager.module_cache.num_modules(), 0); - assert_eq!(global_cache_manager.module_cache.size_in_bytes(), 0); - } - - #[test_case(None)] - #[test_case(Some(HashValue::zero()))] - fn mark_execution_start_when_unset(previous_block_id: Option) { - let state_view = MockStateView::empty(); - let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); - - global_cache_manager - .module_cache - .insert(0, mock_verified_code(0, MockExtension::new(8))); - assert_eq!(global_cache_manager.module_cache.num_modules(), 1); - - // If executed on top of unset state, or the state with matching previous hash, the cache - // is not flushed. - assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); - assert_eq!(global_cache_manager.module_cache.num_modules(), 1); - assert!(!global_cache_manager.ready_for_next_block()); - } - - #[test_case(None, None)] - #[test_case(None, Some(HashValue::zero()))] - #[test_case(Some(HashValue::zero()), None)] - #[test_case(Some(HashValue::zero()), Some(HashValue::zero()))] - #[test_case(Some(HashValue::from_u64(0)), Some(HashValue::from_u64(1)))] - fn mark_execution_start_when_set( - recorded_previous_block_id: Option, - previous_block_id: Option, - ) { - let state_view = MockStateView::empty(); - let global_cache_manager = GlobalCacheManagerInner::new_with_default_config(); - - assert_ok!( - global_cache_manager.mark_block_execution_start(&state_view, Some(HashValue::random())) - ); - assert_ok!(global_cache_manager.mark_block_execution_end(recorded_previous_block_id)); - - global_cache_manager - .module_cache - .insert(0, mock_verified_code(0, MockExtension::new(8))); - assert_eq!(global_cache_manager.module_cache.num_modules(), 1); - - assert_ok!(global_cache_manager.mark_block_execution_start(&state_view, previous_block_id)); - assert!(!global_cache_manager.ready_for_next_block()); - - if recorded_previous_block_id.is_some() && recorded_previous_block_id == previous_block_id { - // In this case both IDs match, no cache flushing. - assert_eq!(global_cache_manager.module_cache.num_modules(), 1); - } else { - // If previous block IDs do not match, or are unknown, caches must be flushed! - assert_eq!(global_cache_manager.module_cache.num_modules(), 0); - } - } - - #[test] - fn mark_execution_start_concurrent() { - let state_view = Box::new(MockStateView::empty()); - let state_view: &'static _ = Box::leak(state_view); - - let global_cache_manager = Arc::new(GlobalCacheManagerInner::< - u32, - MockDeserializedCode, - MockVerifiedCode, - MockExtension, - >::new_with_default_config()); - assert!(global_cache_manager.ready_for_next_block()); - - let mut handles = vec![]; - for _ in 0..32 { - let handle = thread::spawn({ - let global_cache_manager = global_cache_manager.clone(); - move || global_cache_manager.mark_block_execution_start(state_view, None) - }); - handles.push(handle); - } - join_and_assert_single_ok(handles); - } - - #[test_case(None)] - #[test_case(Some(HashValue::from_u64(0)))] - fn mark_block_execution_end(block_id: Option) { - let global_cache_manager = GlobalCacheManagerInner::< - u32, - MockDeserializedCode, - MockVerifiedCode, - MockExtension, - >::new_with_default_config(); - assert!(global_cache_manager.previous_block_id.lock().is_unset()); - - // The global cache is ready, so we cannot mark execution end. - assert_err!(global_cache_manager.mark_block_execution_end(block_id)); - - global_cache_manager.mark_not_ready_for_next_block(); - let previous_block_id = *global_cache_manager.previous_block_id.lock(); - assert!(previous_block_id.is_unset()); - assert_ok!(global_cache_manager.mark_block_execution_end(block_id)); - - // The previous block ID should be set now, and the state is ready. - let new_block_id = *global_cache_manager.previous_block_id.lock(); - assert_eq!(new_block_id, BlockId::Set(block_id)); - assert!(global_cache_manager.ready_for_next_block()); - - global_cache_manager.mark_not_ready_for_next_block(); - let next_block_id = Some(HashValue::from_u64(1)); - assert_ok!(global_cache_manager.mark_block_execution_end(next_block_id)); - - // Previous block ID is again reset. - let new_block_id = *global_cache_manager.previous_block_id.lock(); - assert_eq!(new_block_id, BlockId::Set(next_block_id)); - } - - #[test] - fn mark_block_execution_end_concurrent() { - let global_cache_manager = Arc::new(GlobalCacheManagerInner::< - u32, - MockDeserializedCode, - MockVerifiedCode, - MockExtension, - >::new_with_default_config()); - global_cache_manager.mark_not_ready_for_next_block(); - - let mut handles = vec![]; - for _ in 0..32 { - let handle = thread::spawn({ - let global_cache_manager = global_cache_manager.clone(); - move || global_cache_manager.mark_block_execution_end(None) - }); - handles.push(handle); - } - join_and_assert_single_ok(handles); - } -} diff --git a/aptos-move/aptos-transaction-benchmarks/Cargo.toml b/aptos-move/aptos-transaction-benchmarks/Cargo.toml index 43674345b0732..3fe147d12d47a 100644 --- a/aptos-move/aptos-transaction-benchmarks/Cargo.toml +++ b/aptos-move/aptos-transaction-benchmarks/Cargo.toml @@ -17,7 +17,6 @@ aptos-bitvec = { workspace = true } aptos-block-executor = { workspace = true } aptos-block-partitioner = { workspace = true } aptos-crypto = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } diff --git a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs index 1e177095ef345..a6f4b68a9ce1d 100644 --- a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs +++ b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs @@ -8,7 +8,6 @@ use aptos_block_partitioner::{ v2::config::PartitionerV2Config, BlockPartitioner, PartitionerConfig, }; use aptos_crypto::HashValue; -use aptos_global_cache_manager::GlobalCacheManager; use aptos_language_e2e_tests::{ account_universe::{AUTransactionGen, AccountPickStyle, AccountUniverse, AccountUniverseGen}, data_store::FakeDataStore, @@ -212,11 +211,6 @@ where maybe_block_gas_limit: Option, ) -> (Vec, usize) { let block_size = transactions.len(); - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - global_cache_manager - .mark_block_execution_start(self.state_view.as_ref(), None) - .unwrap(); - let timer = Instant::now(); let output = BlockAptosVM::execute_block::< _, @@ -224,14 +218,13 @@ where >( transactions, self.state_view.as_ref(), - &global_cache_manager, + None, BlockExecutorConfig::new_maybe_block_limit(1, maybe_block_gas_limit), None, ) .expect("VM should not fail to start") .into_transaction_outputs_forced(); let exec_time = timer.elapsed().as_millis(); - global_cache_manager.mark_block_execution_end(None).unwrap(); (output, block_size * 1000 / exec_time as usize) } @@ -267,11 +260,6 @@ where maybe_block_gas_limit: Option, ) -> (Vec, usize) { let block_size = transactions.len(); - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - global_cache_manager - .mark_block_execution_start(self.state_view.as_ref(), None) - .unwrap(); - let timer = Instant::now(); let output = BlockAptosVM::execute_block::< _, @@ -279,7 +267,7 @@ where >( transactions, self.state_view.as_ref(), - &global_cache_manager, + None, BlockExecutorConfig::new_maybe_block_limit( concurrency_level_per_shard, maybe_block_gas_limit, @@ -290,8 +278,6 @@ where .into_transaction_outputs_forced(); let exec_time = timer.elapsed().as_millis(); - global_cache_manager.mark_block_execution_end(None).unwrap(); - (output, block_size * 1000 / exec_time as usize) } @@ -301,7 +287,7 @@ where partitioned_txns: Option, run_par: bool, run_seq: bool, - conurrency_level_per_shard: usize, + concurrency_level_per_shard: usize, maybe_block_gas_limit: Option, ) -> (usize, usize) { let (output, par_tps) = if run_par { @@ -309,13 +295,13 @@ where let (output, tps) = if self.is_shareded() { self.execute_benchmark_sharded( partitioned_txns.unwrap(), - conurrency_level_per_shard, + concurrency_level_per_shard, maybe_block_gas_limit, ) } else { self.execute_benchmark_parallel( &transactions, - conurrency_level_per_shard, + concurrency_level_per_shard, maybe_block_gas_limit, ) }; diff --git a/aptos-move/aptos-transactional-test-harness/Cargo.toml b/aptos-move/aptos-transactional-test-harness/Cargo.toml index af96620411278..c0e44b746718d 100644 --- a/aptos-move/aptos-transactional-test-harness/Cargo.toml +++ b/aptos-move/aptos-transactional-test-harness/Cargo.toml @@ -19,7 +19,6 @@ aptos-cached-packages = { workspace = true } aptos-crypto = { workspace = true } aptos-framework = { workspace = true } aptos-gas-schedule = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-resource-viewer = { workspace = true } aptos-storage-interface = { workspace = true } diff --git a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs index e07aec25a6fc5..a96506db49b47 100644 --- a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs +++ b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs @@ -515,25 +515,9 @@ impl<'a> AptosTestAdapter<'a> { fn run_transaction(&mut self, txn: Transaction) -> Result { let txn_block = vec![txn]; let sig_verified_block = into_signature_verified_block(txn_block); - let onchain_config = BlockExecutorConfigFromOnchain { - // TODO fetch values from state? - // Or should we just use execute_block_no_limit ? - block_gas_limit_type: BlockGasLimitType::Limit(30000), - }; - let (mut outputs, _) = AptosVMBlockExecutor::new() - .execute_block(&sig_verified_block, &self.storage.clone(), onchain_config)? - .into_inner(); - - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - global_cache_manager.mark_block_execution_start(&state_view, None)?; - let result = AptosVM::execute_block_no_limit( - &sig_verified_block, - &state_view, - &global_cache_manager, - ); - global_cache_manager.mark_block_execution_end(None)?; + let mut outputs = AptosVMBlockExecutor::new() + .execute_block_no_limit(&sig_verified_block, &self.storage.clone())?; - let mut outputs = result?; assert_eq!(outputs.len(), 1); let output = outputs.pop().unwrap(); diff --git a/aptos-move/aptos-vm-profiling/Cargo.toml b/aptos-move/aptos-vm-profiling/Cargo.toml index dcb4c5701e4c5..68bfadfcd39ea 100644 --- a/aptos-move/aptos-vm-profiling/Cargo.toml +++ b/aptos-move/aptos-vm-profiling/Cargo.toml @@ -19,7 +19,6 @@ smallvec = { workspace = true } aptos-cached-packages = { workspace = true } aptos-gas-schedule = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-move-stdlib = { workspace = true } aptos-native-interface = { workspace = true } diff --git a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs index 3ad2a722eab4a..02a5ed3c37a8c 100644 --- a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs +++ b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs @@ -48,13 +48,9 @@ fn main() -> Result<()> { }) .collect(); - let res = AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store)?; + let outputs = AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store)?; for i in 0..NUM_TXNS { - assert!(result.as_ref().unwrap()[i as usize] - .status() - .status() - .unwrap() - .is_success()); + assert!(outputs[i as usize].status().status().unwrap().is_success()); } Ok(()) diff --git a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs index 0e78bfce3d1b9..9a40888290199 100644 --- a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs +++ b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs @@ -84,8 +84,7 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, } /// Drains cached verified modules from the code storage, transforming them into format used by - /// global caches (i.e., with extension and no versioning). Should only be called when the code - /// storage borrows [StateView]. + /// global caches. Should only be called when the code storage borrows [StateView]. pub fn into_verified_module_code_iter( self, ) -> Result< @@ -134,7 +133,6 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, PanicError::CodeInvariantError(msg) })?; - // We are using storage version here. let module = ModuleCode::from_verified_ref(verified_code, Arc::new(extension)); modules_to_add.push((key, Arc::new(module))) } diff --git a/aptos-move/aptos-vm/Cargo.toml b/aptos-move/aptos-vm/Cargo.toml index abae28d9434ef..94897e9368d97 100644 --- a/aptos-move/aptos-vm/Cargo.toml +++ b/aptos-move/aptos-vm/Cargo.toml @@ -24,7 +24,6 @@ aptos-framework = { workspace = true } aptos-gas-algebra = { workspace = true } aptos-gas-meter = { workspace = true } aptos-gas-schedule = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } aptos-memory-usage-tracker = { workspace = true } @@ -64,7 +63,7 @@ serde = { workspace = true } aptos-aggregator = { workspace = true, features = ["testing"] } aptos-block-executor = { workspace = true, features = ["testing"] } aptos-language-e2e-tests = { workspace = true } -aptos-types = { workspace = true, features = ["fuzzing"] } +aptos-types = { workspace = true, features = ["fuzzing", "testing"] } claims = { workspace = true } move-vm-types = { workspace = true, features = ["testing"] } proptest = { workspace = true } diff --git a/aptos-move/aptos-vm/src/aptos_vm.rs b/aptos-move/aptos-vm/src/aptos_vm.rs index 8f5f411416101..e85934e3fbaa3 100644 --- a/aptos-move/aptos-vm/src/aptos_vm.rs +++ b/aptos-move/aptos-vm/src/aptos_vm.rs @@ -27,7 +27,9 @@ use crate::{ VMBlockExecutor, VMValidator, }; use anyhow::anyhow; -use aptos_block_executor::txn_commit_hook::NoOpTransactionCommitHook; +use aptos_block_executor::{ + code_cache_global_manager::ModuleCacheManager, txn_commit_hook::NoOpTransactionCommitHook, +}; use aptos_crypto::HashValue; use aptos_framework::{ natives::{code::PublishRequest, randomness::RandomnessContext}, @@ -36,7 +38,6 @@ use aptos_framework::{ use aptos_gas_algebra::{Gas, GasQuantity, NumBytes, Octa}; use aptos_gas_meter::{AptosGasMeter, GasAlgebra}; use aptos_gas_schedule::{AptosGasParameters, VMGasParameters}; -use aptos_global_cache_manager::GlobalCacheManager; use aptos_logger::{enabled, prelude::*, Level}; use aptos_metrics_core::TimerHelper; #[cfg(any(test, feature = "testing"))] @@ -44,7 +45,10 @@ use aptos_types::state_store::StateViewId; use aptos_types::{ account_config::{self, new_block_event_key, AccountResource}, block_executor::{ - config::{BlockExecutorConfig, BlockExecutorConfigFromOnchain, BlockExecutorLocalConfig}, + config::{ + BlockExecutorConfig, BlockExecutorConfigFromOnchain, BlockExecutorLocalConfig, + BlockExecutorModuleCacheLocalConfig, + }, partitioner::PartitionedTransactions, }, block_metadata::BlockMetadata, @@ -66,6 +70,7 @@ use aptos_types::{ TransactionAuxiliaryData, TransactionOutput, TransactionPayload, TransactionStatus, VMValidatorResult, ViewFunctionOutput, WriteSetPayload, }, + vm::modules::AptosModuleExtension, vm_status::{AbortLocation, StatusCode, VMStatus}, }; use aptos_utils::aptos_try; @@ -112,7 +117,7 @@ use move_vm_metrics::{Timer, VM_TIMER}; use move_vm_runtime::{ logging::expect_no_verification_errors, module_traversal::{TraversalContext, TraversalStorage}, - RuntimeEnvironment, WithRuntimeEnvironment, + Module, RuntimeEnvironment, WithRuntimeEnvironment, }; use move_vm_types::gas::{GasMeter, UnmeteredGasMeter}; use num_cpus; @@ -2774,29 +2779,32 @@ impl AptosVM { /// Transaction execution: AptosVM /// Executing conflicts: in the input order, via BlockSTM, /// State: BlockSTM-provided MVHashMap-based view with caching -pub struct AptosVMBlockExecutor; +pub struct AptosVMBlockExecutor { + /// Manages module cache and execution environment of this block executor. Users of executor + /// must use manager's API to ensure the correct state of caches. + module_cache_manager: + ModuleCacheManager, +} -// Executor external API impl VMBlockExecutor for AptosVMBlockExecutor { - // NOTE: At the moment there are no persistent caches that live past the end of a block (that's - // why AptosVMBlockExecutor has no state) - // There are some cache invalidation issues around transactions publishing code that need to be - // sorted out before that's possible. - fn new() -> Self { - Self + Self { + module_cache_manager: ModuleCacheManager::new(), + } + } + + fn module_cache_manager( + &self, + ) -> Option< + &ModuleCacheManager, + > { + Some(&self.module_cache_manager) } - /// Execute a block of `transactions`. The output vector will have the exact same length as the - /// input vector. The discarded transactions will be marked as `TransactionStatus::Discard` and - /// have an empty `WriteSet`. Also `state_view` is immutable, and does not have interior - /// mutability. Writes to be applied to the data view are encoded in the write set part of a - /// transaction output. fn execute_block( &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), - global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { fail_point!("move_adapter::execute_block", |_| { @@ -2819,12 +2827,13 @@ impl VMBlockExecutor for AptosVMBlockExecutor { >( transactions, state_view, - global_cache_manager, + Some(&self.module_cache_manager), BlockExecutorConfig { local: BlockExecutorLocalConfig { concurrency_level: AptosVM::get_concurrency_level(), allow_fallback: true, discard_failed_blocks: AptosVM::get_discard_failed_blocks(), + module_cache_config: BlockExecutorModuleCacheLocalConfig::default(), }, onchain: onchain_config, }, diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index 5fae5f7faec66..2246ec02b6c9b 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -12,12 +12,14 @@ use aptos_aggregator::{ delayed_change::DelayedChange, delta_change_set::DeltaOp, resolver::TAggregatorV1View, }; use aptos_block_executor::{ + code_cache_global::GlobalModuleCache, code_cache_global_manager::ModuleCacheManager, errors::BlockExecutionError, executor::BlockExecutor, task::TransactionOutput as BlockExecutorTransactionOutput, txn_commit_hook::TransactionCommitHook, types::InputOutputKey, }; -use aptos_global_cache_manager::GlobalCacheManager; +use aptos_crypto::HashValue; use aptos_infallible::Mutex; +use aptos_logger::error; use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::ContractEvent, @@ -29,18 +31,29 @@ use aptos_types::{ signature_verified_transaction::SignatureVerifiedTransaction, BlockOutput, TransactionOutput, TransactionStatus, }, + vm::modules::AptosModuleExtension, write_set::WriteOp, }; -use aptos_vm_logging::{flush_speculative_logs, init_speculative_logs}; +use aptos_vm_environment::environment::AptosEnvironment; +use aptos_vm_logging::{ + alert, flush_speculative_logs, init_speculative_logs, prelude::CRITICAL_ERRORS, +}; use aptos_vm_types::{ - abstract_write_op::AbstractResourceWriteOp, module_write_set::ModuleWrite, output::VMOutput, - resolver::ResourceGroupSize, + abstract_write_op::AbstractResourceWriteOp, module_and_script_storage::AsAptosCodeStorage, + module_write_set::ModuleWrite, output::VMOutput, resolver::ResourceGroupSize, +}; +use move_binary_format::{ + errors::{Location, VMError}, + CompiledModule, }; use move_core_types::{ - language_storage::StructTag, + account_address::AccountAddress, + ident_str, + language_storage::{ModuleId, StructTag}, value::MoveTypeLayout, vm_status::{StatusCode, VMStatus}, }; +use move_vm_runtime::{Module, ModuleStorage, WithRuntimeEnvironment}; use move_vm_types::delayed_values::delayed_field_id::DelayedFieldID; use once_cell::sync::{Lazy, OnceCell}; use std::{ @@ -389,18 +402,21 @@ impl BlockExecutorTransactionOutput for AptosTransactionOutput { pub struct BlockAptosVM; impl BlockAptosVM { - fn execute_block_on_thread_pool< + pub fn execute_block_on_thread_pool< S: StateView + Sync, L: TransactionCommitHook, >( executor_thread_pool: Arc, signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, - global_cache_manager: &GlobalCacheManager, + module_cache_manager: Option< + &ModuleCacheManager, + >, config: BlockExecutorConfig, transaction_commit_listener: Option, ) -> Result, VMStatus> { let _timer = BLOCK_EXECUTOR_EXECUTE_BLOCK_SECONDS.start_timer(); + let num_txns = signature_verified_block.len(); if state_view.id() != StateViewId::Miscellaneous { // Speculation is disabled in Miscellaneous context, which is used by testing and @@ -410,6 +426,56 @@ impl BlockAptosVM { BLOCK_EXECUTOR_CONCURRENCY.set(config.local.concurrency_level as i64); + // We should be checking different module cache configurations here. + let module_cache_config = &config.local.module_cache_config; + + let (environment, module_cache) = match module_cache_manager { + Some(module_cache_manager) if !module_cache_manager.mark_executing() => { + let environment = + module_cache_manager.get_or_initialize_environment_unchecked(state_view); + let module_cache = module_cache_manager.module_cache(); + (environment, module_cache) + }, + _ => { + // Either we do not have global caches , in which case we can create new ones, or + // something went wrong, and we were not able to mark the state as executing. In + // this case, fallback to empty caches. Note that the alert should have been raised + // during marking. + let environment = + AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); + let module_cache = Arc::new(GlobalModuleCache::empty()); + (environment, module_cache) + }, + }; + + // Check 1: struct re-indexing map is not too large. If it is, we flush the cache. Also, we + // need to flush modules because they store indices into re-indexing map. + let runtime_environment = environment.runtime_environment(); + let struct_name_index_map_size = runtime_environment + .struct_name_index_map_size() + .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; + if struct_name_index_map_size > module_cache_config.max_struct_name_index_map_size { + module_cache.flush_unchecked(); + runtime_environment.flush_struct_name_and_info_caches(); + } + + // Check 2: If the module cache is too big, flush it. + if module_cache + .size_in_bytes_is_greater_than(module_cache_config.max_module_cache_size_in_bytes) + { + module_cache.flush_unchecked(); + } + + // Finally, to avoid cold starts, fetch the framework code prior to block execution. + if module_cache.num_modules() == 0 && module_cache_config.prefetch_framework_code { + prefetch_aptos_framework(environment.clone(), state_view, &module_cache).map_err( + |err| { + alert!("Failed to load Aptos framework to module cache: {:?}", err); + VMError::from(err).into_vm_status() + }, + )?; + } + let executor = BlockExecutor::< SignatureVerifiedTransaction, AptosExecutorTask, @@ -419,12 +485,22 @@ impl BlockAptosVM { >::new( config, executor_thread_pool, - global_cache_manager.module_cache(), + module_cache, transaction_commit_listener, ); - let environment = global_cache_manager.environment()?; let ret = executor.execute_block(environment, signature_verified_block, state_view); + if let Some(module_cache_manager) = module_cache_manager { + if !module_cache_manager.mark_done() { + // Something is wrong as we were not able to mark execution as done. Return an + // error. + return Err(VMStatus::error( + StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some("Unable to mark block execution as done".to_string()), + )); + } + } + match ret { Ok(block_output) => { let (transaction_outputs, block_end_info) = block_output.into_inner(); @@ -455,31 +531,6 @@ impl BlockAptosVM { } } - pub fn execute_block_on_thread_pool_without_global_caches< - S: StateView + Sync, - L: TransactionCommitHook, - >( - executor_thread_pool: Arc, - signature_verified_block: &[SignatureVerifiedTransaction], - state_view: &S, - config: BlockExecutorConfig, - transaction_commit_listener: Option, - ) -> Result, VMStatus> { - let global_cache_manager = GlobalCacheManager::new_with_default_config(); - global_cache_manager.mark_block_execution_start(state_view, None)?; - - let result = Self::execute_block_on_thread_pool::( - executor_thread_pool, - signature_verified_block, - state_view, - &global_cache_manager, - config, - transaction_commit_listener, - ); - global_cache_manager.mark_block_execution_end(None)?; - result - } - /// Uses shared thread pool to execute blocks. pub fn execute_block< S: StateView + Sync, @@ -487,7 +538,9 @@ impl BlockAptosVM { >( signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, - global_cache_manager: &GlobalCacheManager, + module_cache_manager: Option< + &ModuleCacheManager, + >, config: BlockExecutorConfig, transaction_commit_listener: Option, ) -> Result, VMStatus> { @@ -495,9 +548,73 @@ impl BlockAptosVM { Arc::clone(&RAYON_EXEC_POOL), signature_verified_block, state_view, - global_cache_manager, + module_cache_manager, config, transaction_commit_listener, ) } } + +/// If Aptos framework exists, loads "transaction_validation.move" and all its transitive +/// dependencies from storage into provided module cache. If loading fails for any reason, a panic +/// error is returned. +fn prefetch_aptos_framework( + environment: AptosEnvironment, + state_view: &impl StateView, + module_cache: &GlobalModuleCache, +) -> Result<(), PanicError> { + let code_storage = state_view.as_aptos_code_storage(environment); + + // If framework code exists in storage, the transitive closure will be verified and cached. + let maybe_loaded = code_storage + .fetch_verified_module(&AccountAddress::ONE, ident_str!("transaction_validation")) + .map_err(|err| { + // There should be no errors when pre-fetching the framework, if there are, we + // better return an error here. + PanicError::CodeInvariantError(format!("Unable to fetch Aptos framework: {:?}", err)) + })?; + + if let Some(module) = maybe_loaded { + drop(module); + + // Framework must have been loaded. Drain verified modules from local cache into + // global cache. + let verified_module_code_iter = code_storage.into_verified_module_code_iter()?; + module_cache.insert_verified_unchecked(verified_module_code_iter)?; + } + Ok(()) +} + +#[cfg(test)] +mod test { + use super::*; + use aptos_language_e2e_tests::{data_store::FakeDataStore, executor::FakeExecutor}; + + #[test] + fn test_prefetch_existing_aptos_framework() { + let executor = FakeExecutor::from_head_genesis(); + let state_view = executor.get_state_view(); + + let environment = AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); + let module_cache = GlobalModuleCache::empty(); + assert_eq!(module_cache.num_modules(), 0); + + let result = prefetch_aptos_framework(environment, state_view, &module_cache); + assert!(result.is_ok()); + assert!(module_cache.num_modules() > 0); + } + + #[test] + fn test_prefetch_non_existing_aptos_framework() { + let state_view = FakeDataStore::default(); + + let environment = + AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); + let module_cache = GlobalModuleCache::empty(); + assert_eq!(module_cache.num_modules(), 0); + + let result = prefetch_aptos_framework(environment, &state_view, &module_cache); + assert!(result.is_ok()); + assert_eq!(module_cache.num_modules(), 0); + } +} diff --git a/aptos-move/aptos-vm/src/lib.rs b/aptos-move/aptos-vm/src/lib.rs index e99784753a552..977287304b10c 100644 --- a/aptos-move/aptos-vm/src/lib.rs +++ b/aptos-move/aptos-vm/src/lib.rs @@ -126,6 +126,8 @@ pub mod verifier; pub use crate::aptos_vm::{AptosSimulationVM, AptosVM}; use crate::sharded_block_executor::{executor_client::ExecutorClient, ShardedBlockExecutor}; +use aptos_block_executor::code_cache_global_manager::ModuleCacheManager; +use aptos_crypto::HashValue; use aptos_types::{ block_executor::{ config::BlockExecutorConfigFromOnchain, partitioner::PartitionedTransactions, @@ -135,9 +137,13 @@ use aptos_types::{ signature_verified_transaction::SignatureVerifiedTransaction, BlockOutput, SignedTransaction, TransactionOutput, VMValidatorResult, }, + vm::modules::AptosModuleExtension, vm_status::VMStatus, }; use aptos_vm_types::module_and_script_storage::code_storage::AptosCodeStorage; +use move_binary_format::CompiledModule; +use move_core_types::language_storage::ModuleId; +use move_vm_runtime::Module; use std::{marker::Sync, sync::Arc}; pub use verifier::view_function::determine_is_view; @@ -152,13 +158,24 @@ pub trait VMValidator { ) -> VMValidatorResult; } -/// This trait describes the VM's execution interface. +/// This trait describes the block executor interface. pub trait VMBlockExecutor: Send + Sync { - /// Be careful if any state is kept in VMBlockExecutor, as all validations are implementers responsibility - /// (and state_view passed in execute_block can go both backwards and forwards in time). - /// TODO: Currently, production uses new() on every block, and only executor-benchmark reuses across. + /// Be careful if any state (such as caches) is kept in [VMBlockExecutor]. It is the + /// responsibility of the implementation to ensure the state is valid across multiple + /// executions. For example, the same executor may be used to run on a new state, and then on + /// an old one. fn new() -> Self; + /// Returns the cache manager responsible for keeping module caches in sync. By default, is + /// [None]. + fn module_cache_manager( + &self, + ) -> Option< + &ModuleCacheManager, + > { + None + } + /// Executes a block of transactions and returns output for each one of them. fn execute_block( &self, @@ -173,7 +190,6 @@ pub trait VMBlockExecutor: Send + Sync { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), - global_cache_manager: &GlobalCacheManager, ) -> Result, VMStatus> { self.execute_block( transactions, diff --git a/aptos-move/aptos-vm/src/sharded_block_executor/global_executor.rs b/aptos-move/aptos-vm/src/sharded_block_executor/global_executor.rs index 90a12556deb57..239bbaba7bb24 100644 --- a/aptos-move/aptos-vm/src/sharded_block_executor/global_executor.rs +++ b/aptos-move/aptos-vm/src/sharded_block_executor/global_executor.rs @@ -60,11 +60,9 @@ impl GlobalExecutor { GLOBAL_ROUND_ID, state_view, BlockExecutorConfig { - local: BlockExecutorLocalConfig { - concurrency_level: self.concurrency_level, - allow_fallback: true, - discard_failed_blocks: false, - }, + local: BlockExecutorLocalConfig::default_with_concurrency_level( + self.concurrency_level, + ), onchain: onchain_config, }, ) diff --git a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs index 5ad1dc5e602f6..193302692725d 100644 --- a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs +++ b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs @@ -135,10 +135,12 @@ impl ShardedExecutorService { ); }); s.spawn(move |_| { - let ret = BlockAptosVM::execute_block_on_thread_pool_without_global_caches( + let ret = BlockAptosVM::execute_block_on_thread_pool( executor_thread_pool, &signature_verified_transactions, aggr_overridden_state_view.as_ref(), + // Since we execute blocks in parallel, we cannot share module caches. + None, config, cross_shard_commit_sender, ) @@ -230,11 +232,9 @@ impl ShardedExecutorService { transactions, state_view.as_ref(), BlockExecutorConfig { - local: BlockExecutorLocalConfig { - concurrency_level: concurrency_level_per_shard, - allow_fallback: true, - discard_failed_blocks: false, - }, + local: BlockExecutorLocalConfig::default_with_concurrency_level( + concurrency_level_per_shard, + ), onchain: onchain_config, }, ); diff --git a/aptos-move/aptos-vm/tests/sharded_block_executor.rs b/aptos-move/aptos-vm/tests/sharded_block_executor.rs index 2f5c08eaa86a1..5968d0a495ab9 100644 --- a/aptos-move/aptos-vm/tests/sharded_block_executor.rs +++ b/aptos-move/aptos-vm/tests/sharded_block_executor.rs @@ -187,7 +187,6 @@ fn test_partitioner_v2_connected_component_sharded_block_executor_with_random_tr mod test_utils { use aptos_block_partitioner::BlockPartitioner; - use aptos_global_cache_manager::GlobalCacheManager; use aptos_language_e2e_tests::{ account::AccountData, common_transactions::peer_to_peer_txn, data_store::FakeDataStore, executor::FakeExecutor, diff --git a/aptos-move/block-executor/Cargo.toml b/aptos-move/block-executor/Cargo.toml index aefd0ae6f13cc..958c09a5787c2 100644 --- a/aptos-move/block-executor/Cargo.toml +++ b/aptos-move/block-executor/Cargo.toml @@ -22,6 +22,7 @@ aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } aptos-mvhashmap = { workspace = true } aptos-types = { workspace = true } +aptos-vm-environment = { workspace = true } aptos-vm-logging = { workspace = true } aptos-vm-types = { workspace = true } arc-swap = { workspace = true } diff --git a/aptos-move/block-executor/src/captured_reads.rs b/aptos-move/block-executor/src/captured_reads.rs index 7abe1da817a9a..c1214baa633db 100644 --- a/aptos-move/block-executor/src/captured_reads.rs +++ b/aptos-move/block-executor/src/captured_reads.rs @@ -1,7 +1,10 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use crate::{types::InputOutputKey, value_exchange::filter_value_for_exchange}; +use crate::{ + code_cache_global::GlobalModuleCache, types::InputOutputKey, + value_exchange::filter_value_for_exchange, +}; use anyhow::bail; use aptos_aggregator::{ delta_math::DeltaHistory, @@ -19,7 +22,6 @@ use aptos_mvhashmap::{ use aptos_types::{ error::{code_invariant_error, PanicError, PanicOr}, executable::ModulePath, - read_only_module_cache::ReadOnlyModuleCache, state_store::state_value::StateValueMetadata, transaction::BlockExecutableTransaction as Transaction, write_set::TransactionWrite, @@ -651,7 +653,7 @@ where /// 3. Entries that were in per-block cache have the same commit index. pub(crate) fn validate_module_reads( &self, - global_module_cache: &ReadOnlyModuleCache, + global_module_cache: &GlobalModuleCache, per_block_module_cache: &SyncModuleCache>, ) -> bool { if self.non_delayed_field_speculative_failure { @@ -872,7 +874,10 @@ where #[cfg(test)] mod test { use super::*; - use crate::proptest_types::types::{raw_metadata, KeyType, MockEvent, ValueType}; + use crate::{ + code_cache_global::GlobalModuleCache, + proptest_types::types::{raw_metadata, KeyType, MockEvent, ValueType}, + }; use aptos_mvhashmap::{types::StorageVersion, MVHashMap}; use aptos_types::executable::ExecutableTestType; use claims::{ @@ -1520,7 +1525,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - let global_module_cache = ReadOnlyModuleCache::empty(); + let global_module_cache = GlobalModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); @@ -1555,7 +1560,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - let global_module_cache = ReadOnlyModuleCache::empty(); + let global_module_cache = GlobalModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); global_module_cache.insert(0, mock_verified_code(0, MockExtension::new(8))); @@ -1632,7 +1637,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - let global_module_cache = ReadOnlyModuleCache::empty(); + let global_module_cache = GlobalModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); let a = mock_deserialized_code(0, MockExtension::new(8)); @@ -1692,7 +1697,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - let global_module_cache = ReadOnlyModuleCache::empty(); + let global_module_cache = GlobalModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); // Module exists in global cache. diff --git a/aptos-move/block-executor/src/code_cache.rs b/aptos-move/block-executor/src/code_cache.rs index 7ffe0d78b2ef4..8d875dca1ce2d 100644 --- a/aptos-move/block-executor/src/code_cache.rs +++ b/aptos-move/block-executor/src/code_cache.rs @@ -54,6 +54,8 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCodeB .map_err(|err| err.finish(Location::Undefined))? .map(|state_value| { let extension = Arc::new(AptosModuleExtension::new(state_value)); + + // TODO(loader_v2): This recomputes module hash twice, we should avoid it. let (compiled_module, _, _) = self .runtime_environment() .deserialize_into_compiled_module(extension.bytes())?; diff --git a/types/src/read_only_module_cache.rs b/aptos-move/block-executor/src/code_cache_global.rs similarity index 91% rename from types/src/read_only_module_cache.rs rename to aptos-move/block-executor/src/code_cache_global.rs index a7d70138822aa..d8fa927a60008 100644 --- a/types/src/read_only_module_cache.rs +++ b/aptos-move/block-executor/src/code_cache_global.rs @@ -1,7 +1,8 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use crate::{error::PanicError, explicit_sync_wrapper::ExplicitSyncWrapper}; +use crate::explicit_sync_wrapper::ExplicitSyncWrapper; +use aptos_types::error::PanicError; use crossbeam::utils::CachePadded; use hashbrown::HashMap; use move_vm_types::code::{ModuleCode, WithSize}; @@ -14,7 +15,7 @@ use std::{ }, }; -/// Entry stored in [ReadOnlyModuleCache]. +/// Entry stored in [GlobalModuleCache]. struct Entry { /// True if this code is "valid" within the block execution context (i.e., there has been no /// republishing of this module so far). If false, executor needs to read the module from the @@ -61,14 +62,14 @@ where /// A read-only module cache for verified code, that can be accessed concurrently within the block. /// Can only be modified safely at block boundaries. -pub struct ReadOnlyModuleCache { +pub struct GlobalModuleCache { /// Module cache containing the verified code. module_cache: ExplicitSyncWrapper>>, /// Sum of serialized sizes (in bytes) of all cached modules. size: AtomicUsize, } -impl ReadOnlyModuleCache +impl GlobalModuleCache where K: Hash + Eq + Clone, VC: Deref>, @@ -124,6 +125,12 @@ where self.size.load(Ordering::Relaxed) } + /// Returns true if the sum of serialized sizes of modules stored in cache is greater than the + /// specified value. + pub fn size_in_bytes_is_greater_than(&self, size: usize) -> bool { + self.size_in_bytes() > size + } + /// Inserts modules into the cache. Should never be called throughout block-execution. Use with /// caution. /// @@ -214,7 +221,7 @@ mod test { #[test] fn test_cache_contains_valid_and_get() { - let cache = ReadOnlyModuleCache::empty(); + let cache = GlobalModuleCache::empty(); // Set the state. cache.insert(0, mock_verified_code(0, MockExtension::new(8))); @@ -233,8 +240,8 @@ mod test { } #[test] - fn test_num_modules_and_flush_unchecked() { - let cache = ReadOnlyModuleCache::empty(); + fn test_cache_sizes_and_flush_unchecked() { + let cache = GlobalModuleCache::empty(); assert_eq!(cache.num_modules(), 0); assert_eq!(cache.size_in_bytes(), 0); @@ -248,6 +255,10 @@ mod test { assert_eq!(cache.num_modules(), 2); assert_eq!(cache.size_in_bytes(), 24); + assert!(cache.size_in_bytes_is_greater_than(23)); + assert!(!cache.size_in_bytes_is_greater_than(24)); + assert!(!cache.size_in_bytes_is_greater_than(25)); + cache.flush_unchecked(); assert_eq!(cache.num_modules(), 0); assert_eq!(cache.size_in_bytes(), 0); @@ -255,7 +266,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked() { - let cache = ReadOnlyModuleCache::empty(); + let cache = GlobalModuleCache::empty(); let mut new_modules = vec![]; for i in 0..10 { @@ -271,7 +282,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked_does_not_add_deserialized_code() { - let cache = ReadOnlyModuleCache::empty(); + let cache = GlobalModuleCache::empty(); let deserialized_modules = vec![(0, mock_deserialized_code(0, MockExtension::new(8)))]; assert_ok!(cache.insert_verified_unchecked(deserialized_modules.into_iter())); @@ -282,7 +293,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked_does_not_override_valid_modules() { - let cache = ReadOnlyModuleCache::empty(); + let cache = GlobalModuleCache::empty(); cache.insert(0, mock_verified_code(0, MockExtension::new(8))); assert_eq!(cache.num_modules(), 1); @@ -294,7 +305,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked_overrides_invalid_modules() { - let cache = ReadOnlyModuleCache::empty(); + let cache = GlobalModuleCache::empty(); cache.insert(0, mock_verified_code(0, MockExtension::new(8))); cache.mark_invalid_if_contains(&0); diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs new file mode 100644 index 0000000000000..d7bd8b771c344 --- /dev/null +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -0,0 +1,530 @@ +// Copyright © Aptos Foundation +// SPDX-License-Identifier: Apache-2.0 + +use crate::{code_cache_global::GlobalModuleCache, explicit_sync_wrapper::ExplicitSyncWrapper}; +use aptos_types::state_store::StateView; +use aptos_vm_environment::environment::AptosEnvironment; +use move_vm_runtime::WithRuntimeEnvironment; +use move_vm_types::code::WithSize; +use parking_lot::Mutex; +use std::{ + fmt::Debug, + hash::Hash, + ops::{Deref, DerefMut}, + sync::Arc, +}; + +/// Raises an alert with the specified message. In case we run in testing mode, instead prints the +/// message to standard output. +macro_rules! alert_or_println { + ($($arg:tt)*) => { + if cfg!(any(test, feature = "testing")) { + println!($($arg)*) + } else { + use aptos_vm_logging::{alert, prelude::CRITICAL_ERRORS}; + use aptos_logger::error; + alert!($($arg)*); + } + }; +} + +/// Represents the state of [GlobalModuleCache]. The following transitions are allowed: +/// 1. [State::Clean] --> [State::Ready]. +/// 2. [State::Ready] --> [State::Executing]. +/// 3. [State::Executing] --> [State::Done]. +/// 4. [State::Done] --> [State::Ready]. +#[derive(Clone, Debug, Eq, PartialEq)] +enum State { + Clean, + Ready(T), + Executing(T), + Done(T), +} + +impl State { + /// If the state is [State::Clean] returns true, and false otherwise. + fn is_clean(&self) -> bool { + match self { + State::Clean => true, + State::Ready(_) | State::Executing(_) | State::Done(_) => false, + } + } + + /// If the state is [State::Done], returns true. + fn is_done(&self) -> bool { + match self { + State::Done(_) => true, + State::Clean | State::Ready(_) | State::Executing(_) => false, + } + } + + /// If the state is [State::Done] and its value equals the one provided, returns true. In other + /// cases, returns false. + fn is_done_with_value(&self, value: &T) -> bool { + match self { + State::Done(v) => v == value, + State::Clean | State::Executing(_) | State::Ready(_) => false, + } + } + + /// If the state is [State::Ready], returns its value. Otherwise, returns [None]. + fn value_from_ready(&self) -> Option { + match self { + State::Ready(v) => Some(v.clone()), + State::Clean | State::Executing(_) | State::Done(_) => None, + } + } + + /// If the state is [State::Executing], returns its value. Otherwise, returns [None]. + fn value_from_executing(&self) -> Option { + match self { + State::Executing(v) => Some(v.clone()), + State::Clean | State::Ready(_) | State::Done(_) => None, + } + } + + /// Sets the current state to [State::Ready]. + fn set_ready(&mut self, value: T) { + *self = Self::Ready(value); + } + + /// Sets the current state to [State::Executing]. + fn set_executing(&mut self, value: T) { + *self = Self::Executing(value); + } + + /// Sets the current state to [State::Done]. + fn set_done(&mut self, value: T) { + *self = Self::Done(value); + } +} + +/// Manages module caches and the execution environment, possible across multiple blocks. +pub struct ModuleCacheManager { + /// The state of global caches. + state: Mutex>, + + /// During concurrent executions, this module cache is read-only. However, it can be mutated + /// when it is known that there are no concurrent accesses. [ModuleCacheManager] must ensure + /// the safety. + module_cache: Arc>, + /// The execution environment, initially set to [None]. The environment, as long as it does not + /// change, can be kept for multiple block executions. + environment: ExplicitSyncWrapper>, +} + +impl ModuleCacheManager +where + T: Clone + Debug + Eq, + K: Hash + Eq + Clone, + VC: Deref>, + E: WithSize, +{ + /// Returns a new instance of [ModuleCacheManager]. + #[allow(clippy::new_without_default)] + pub fn new() -> Self { + Self { + state: Mutex::new(State::Clean), + module_cache: Arc::new(GlobalModuleCache::empty()), + environment: ExplicitSyncWrapper::new(None), + } + } + + /// If state is [State::Clean], or [State::Ready] with matching previous value, sets the state + /// to [State::Ready] with the current value and returns true. Otherwise, raises an alert and + /// returns false. + pub fn mark_ready(&self, previous: &T, current: T) -> bool { + let mut state = self.state.lock(); + + if state.is_clean() || state.is_done_with_value(previous) { + state.set_ready(current); + return true; + } + + if state.is_done() { + // If the state is done, but the values to not match, we still set the state as ready, but + // also flush global caches because they execute not on top of the previous state. + self.module_cache.flush_unchecked(); + if let Some(environment) = self.environment.acquire().as_ref() { + environment + .runtime_environment() + .flush_struct_name_and_info_caches(); + } + + state.set_ready(current); + return true; + } + + alert_or_println!( + "Unable to mark ready, state: {:?}, previous: {:?}, current: {:?}", + state, + previous, + current + ); + false + } + + /// If state is [State::Ready], changes it to [State::Executing] with the same value, returning + /// true. Otherwise, returns false indicating that state transition failed, also raising an + /// alert. + pub fn mark_executing(&self) -> bool { + let mut state = self.state.lock(); + if let Some(value) = state.value_from_ready() { + state.set_executing(value); + return true; + } + + alert_or_println!("Unable to mark executing, state: {:?}", state); + false + } + + /// If state is [State::Executing], changes it to [State::Done] with the same value, returning + /// true. Otherwise, returns false indicating that state transition failed, also raising an + /// alert. + pub fn mark_done(&self) -> bool { + let mut state = self.state.lock(); + if let Some(value) = state.value_from_executing() { + state.set_done(value); + return true; + } + + alert_or_println!("Unable to mark done, state: {:?}", state); + false + } + + /// Returns the cached global environment if it already exists, and matches the one in storage. + /// If it does not exist, or does not match, the new environment is initialized from the given + /// state, cached, and returned. + pub fn get_or_initialize_environment_unchecked( + &self, + state_view: &impl StateView, + ) -> AptosEnvironment { + let _lock = self.state.lock(); + + let new_environment = + AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); + + let mut guard = self.environment.acquire(); + let existing_environment = guard.deref_mut(); + + let (environment, is_new) = match existing_environment.as_ref() { + None => { + *existing_environment = Some(new_environment.clone()); + (new_environment, true) + }, + Some(environment) => { + if environment == &new_environment { + (environment.clone(), false) + } else { + *existing_environment = Some(new_environment.clone()); + (new_environment, true) + } + }, + }; + + // If this environment has been (re-)initialized, we need to flush the module cache because + // it can contain now out-dated code. + if is_new { + self.module_cache.flush_unchecked(); + } + + environment + } + + /// Returns the global module cache. + pub fn module_cache(&self) -> Arc> { + self.module_cache.clone() + } +} + +#[cfg(test)] +mod test { + use super::*; + use aptos_types::{ + on_chain_config::{FeatureFlag, Features, OnChainConfig}, + state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, + }; + use claims::assert_matches; + use move_vm_types::code::{ + mock_verified_code, MockDeserializedCode, MockExtension, MockVerifiedCode, + }; + use std::{collections::HashMap, thread, thread::JoinHandle}; + use test_case::test_case; + + #[test] + fn test_clean_state() { + let state = State::Clean; + + assert!(state.is_clean()); + assert!(!state.is_done()); + assert!(!state.is_done_with_value(&0)); + assert!(state.value_from_ready().is_none()); + assert!(state.value_from_executing().is_none()); + } + + #[test] + fn test_ready_state() { + let state = State::Ready(0); + + assert!(!state.is_clean()); + assert!(!state.is_done()); + assert!(!state.is_done_with_value(&0)); + assert_eq!(state.value_from_ready(), Some(0)); + assert!(state.value_from_executing().is_none()); + } + + #[test] + fn test_executing_state() { + let state = State::Executing(0); + + assert!(!state.is_clean()); + assert!(!state.is_done()); + assert!(!state.is_done_with_value(&0)); + assert!(state.value_from_ready().is_none()); + assert_eq!(state.value_from_executing(), Some(0)); + } + + #[test] + fn test_done_state() { + let state = State::Done(0); + + assert!(!state.is_clean()); + assert!(state.is_done()); + assert!(state.is_done_with_value(&0)); + assert!(!state.is_done_with_value(&10)); + assert!(state.value_from_ready().is_none()); + assert!(state.value_from_executing().is_none()); + } + + #[test] + fn test_set_state() { + let mut state = State::Clean; + + state.set_ready(0); + assert_matches!(state, State::Ready(0)); + + state.set_executing(10); + assert_matches!(state, State::Executing(10)); + + state.set_done(100); + assert_matches!(state, State::Done(100)); + } + + #[test_case(true)] + #[test_case(false)] + fn test_marking(with_different_value_for_done: bool) { + let module_cache_manager = ModuleCacheManager::new(); + assert!(module_cache_manager.state.lock().is_clean()); + + // Pre-populate module cache to test flushing. + module_cache_manager + .module_cache + .insert(0, mock_verified_code(0, MockExtension::new(8))); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + + // Can only go to ready state from clean state. + assert!(!module_cache_manager.mark_executing()); + assert!(!module_cache_manager.mark_done()); + assert!(module_cache_manager.mark_ready(&0, 1)); + + assert_matches!(module_cache_manager.state.lock().deref(), State::Ready(1)); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + + // Can only go to executing state from ready state. + assert!(!module_cache_manager.mark_done()); + assert!(!module_cache_manager.mark_ready(&0, 1)); + assert!(module_cache_manager.mark_executing()); + + assert_matches!( + module_cache_manager.state.lock().deref(), + State::Executing(1) + ); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + + // Can only go to done state from executing state. + assert!(!module_cache_manager.mark_executing()); + assert!(!module_cache_manager.mark_ready(&0, 1)); + assert!(module_cache_manager.mark_done()); + + assert_matches!(module_cache_manager.state.lock().deref(), State::Done(1)); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + + // Can only go to ready state from done state. + assert!(!module_cache_manager.mark_executing()); + assert!(!module_cache_manager.mark_done()); + + if with_different_value_for_done { + // Does not match! Caches should be flushed, but state reset. + assert!(module_cache_manager.mark_ready(&10, 11)); + assert_matches!(module_cache_manager.state.lock().deref(), State::Ready(11)); + assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + } else { + assert!(module_cache_manager.mark_ready(&1, 2)); + assert_matches!(module_cache_manager.state.lock().deref(), State::Ready(2)); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + } + } + + /// Joins threads. Succeeds only if a single handle evaluates to [Ok] and the rest are [Err]s. + fn join_and_assert_single_true(handles: Vec>) { + let mut num_true = 0; + let mut num_false = 0; + + let num_handles = handles.len(); + for handle in handles { + if handle.join().unwrap() { + num_true += 1; + } else { + num_false += 1; + } + } + assert_eq!(num_true, 1); + assert_eq!(num_false, num_handles - 1); + } + + #[test_case(true)] + #[test_case(false)] + fn test_mark_ready_concurrent(start_from_clean_state: bool) { + let global_cache_manager = Arc::new(ModuleCacheManager::< + _, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new()); + if !start_from_clean_state { + assert!(global_cache_manager.mark_ready(&0, 1)); + assert!(global_cache_manager.mark_executing()); + assert!(global_cache_manager.mark_done()); + // We are at done with value of 1. + } + + let mut handles = vec![]; + for _ in 0..32 { + let handle = thread::spawn({ + let global_cache_manager = global_cache_manager.clone(); + move || global_cache_manager.mark_ready(&1, 2) + }); + handles.push(handle); + } + join_and_assert_single_true(handles); + } + + #[test] + fn test_mark_executing_concurrent() { + let global_cache_manager = Arc::new(ModuleCacheManager::< + _, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new()); + assert!(global_cache_manager.mark_ready(&0, 1)); + + let mut handles = vec![]; + for _ in 0..32 { + let handle = thread::spawn({ + let global_cache_manager = global_cache_manager.clone(); + move || global_cache_manager.mark_executing() + }); + handles.push(handle); + } + join_and_assert_single_true(handles); + } + + #[test] + fn test_mark_done_concurrent() { + let global_cache_manager = Arc::new(ModuleCacheManager::< + _, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new()); + assert!(global_cache_manager.mark_ready(&0, 1)); + assert!(global_cache_manager.mark_executing()); + + let mut handles = vec![]; + for _ in 0..32 { + let handle = thread::spawn({ + let global_cache_manager = global_cache_manager.clone(); + move || global_cache_manager.mark_done() + }); + handles.push(handle); + } + join_and_assert_single_true(handles); + } + + fn state_view_with_changed_feature_flag( + feature_flag: Option, + ) -> MockStateView { + // Tweak feature flags to force a different config. + let mut features = Features::default(); + + if let Some(feature_flag) = feature_flag { + if features.is_enabled(feature_flag) { + features.disable(feature_flag); + } else { + features.enable(feature_flag); + } + } + + MockStateView::new(HashMap::from([( + StateKey::resource(Features::address(), &Features::struct_tag()).unwrap(), + StateValue::new_legacy(bcs::to_bytes(&features).unwrap().into()), + )])) + } + + #[test] + fn mark_execution_start_when_different_environment() { + let module_cache_manager = ModuleCacheManager::::new(); + + module_cache_manager + .module_cache + .insert(0, mock_verified_code(0, MockExtension::new(8))); + module_cache_manager + .module_cache + .insert(1, mock_verified_code(1, MockExtension::new(8))); + assert_eq!(module_cache_manager.module_cache.num_modules(), 2); + assert!(module_cache_manager.environment.acquire().is_none()); + + // Environment has to be set to the same value, cache flushed. + let state_view = state_view_with_changed_feature_flag(None); + let environment = module_cache_manager.get_or_initialize_environment_unchecked(&state_view); + assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert!(module_cache_manager + .environment + .acquire() + .as_ref() + .is_some_and(|cached_environment| cached_environment == &environment)); + + module_cache_manager + .module_cache + .insert(2, mock_verified_code(2, MockExtension::new(8))); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert!(module_cache_manager.environment.acquire().is_some()); + + // Environment has to be re-set to the new value, cache flushed. + let state_view = + state_view_with_changed_feature_flag(Some(FeatureFlag::CODE_DEPENDENCY_CHECK)); + let environment = module_cache_manager.get_or_initialize_environment_unchecked(&state_view); + assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert!(module_cache_manager + .environment + .acquire() + .as_ref() + .is_some_and(|cached_environment| cached_environment == &environment)); + + module_cache_manager + .module_cache + .insert(3, mock_verified_code(3, MockExtension::new(8))); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert!(module_cache_manager.environment.acquire().is_some()); + + // Environment is kept, and module caches are not flushed. + let new_environment = + module_cache_manager.get_or_initialize_environment_unchecked(&state_view); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert!(environment == new_environment); + } +} diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 05521aea84c5f..151dc89522013 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -3,6 +3,7 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{ + code_cache_global::GlobalModuleCache, counters::{ self, BLOCK_EXECUTOR_INNER_EXECUTE_BLOCK, PARALLEL_EXECUTION_SECONDS, RAYON_EXECUTION_SECONDS, TASK_EXECUTE_SECONDS, TASK_VALIDATE_SECONDS, VM_INIT_SECONDS, @@ -10,6 +11,7 @@ use crate::{ }, errors::*, executor_utilities::*, + explicit_sync_wrapper::ExplicitSyncWrapper, limit_processor::BlockGasLimitProcessor, scheduler::{DependencyStatus, ExecutionTaskType, Scheduler, SchedulerTask, Wave}, task::{ExecutionStatus, ExecutorTask, TransactionOutput}, @@ -34,9 +36,7 @@ use aptos_types::{ block_executor::config::BlockExecutorConfig, error::{code_invariant_error, expect_ok, PanicError, PanicOr}, executable::Executable, - explicit_sync_wrapper::ExplicitSyncWrapper, on_chain_config::BlockGasLimitType, - read_only_module_cache::ReadOnlyModuleCache, state_store::{state_value::StateValue, TStateView}, transaction::{ block_epilogue::BlockEndInfo, BlockExecutableTransaction as Transaction, BlockOutput, @@ -75,7 +75,7 @@ pub struct BlockExecutor { config: BlockExecutorConfig, executor_thread_pool: Arc, global_module_cache: - Arc>, + Arc>, transaction_commit_hook: Option, phantom: PhantomData<(T, E, S, L, X)>, } @@ -94,7 +94,7 @@ where config: BlockExecutorConfig, executor_thread_pool: Arc, global_module_cache: Arc< - ReadOnlyModuleCache, + GlobalModuleCache, >, transaction_commit_hook: Option, ) -> Self { @@ -120,7 +120,7 @@ where versioned_cache: &MVHashMap, executor: &E, base_view: &S, - global_module_cache: &ReadOnlyModuleCache< + global_module_cache: &GlobalModuleCache< ModuleId, CompiledModule, Module, @@ -402,7 +402,7 @@ where fn validate( idx_to_validate: TxnIndex, last_input_output: &TxnLastInputOutput, - global_module_cache: &ReadOnlyModuleCache< + global_module_cache: &GlobalModuleCache< ModuleId, CompiledModule, Module, @@ -755,7 +755,7 @@ where fn publish_module_writes( txn_idx: TxnIndex, module_write_set: BTreeMap>, - global_module_cache: &ReadOnlyModuleCache< + global_module_cache: &GlobalModuleCache< ModuleId, CompiledModule, Module, @@ -1198,7 +1198,7 @@ where write: ModuleWrite, txn_idx: TxnIndex, runtime_environment: &RuntimeEnvironment, - global_module_cache: &ReadOnlyModuleCache< + global_module_cache: &GlobalModuleCache< ModuleId, CompiledModule, Module, @@ -1247,7 +1247,7 @@ where fn apply_output_sequential( txn_idx: TxnIndex, runtime_environment: &RuntimeEnvironment, - global_module_cache: &ReadOnlyModuleCache< + global_module_cache: &GlobalModuleCache< ModuleId, CompiledModule, Module, diff --git a/types/src/explicit_sync_wrapper.rs b/aptos-move/block-executor/src/explicit_sync_wrapper.rs similarity index 98% rename from types/src/explicit_sync_wrapper.rs rename to aptos-move/block-executor/src/explicit_sync_wrapper.rs index 7f57b2ae81c00..c088c71f88958 100644 --- a/types/src/explicit_sync_wrapper.rs +++ b/aptos-move/block-executor/src/explicit_sync_wrapper.rs @@ -1,8 +1,6 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -#![allow(unsafe_code)] - use std::{ cell::UnsafeCell, ops::{Deref, DerefMut}, diff --git a/aptos-move/block-executor/src/lib.rs b/aptos-move/block-executor/src/lib.rs index 902fe9caa0876..f2d388f769008 100644 --- a/aptos-move/block-executor/src/lib.rs +++ b/aptos-move/block-executor/src/lib.rs @@ -141,10 +141,13 @@ extern crate scopeguard; mod captured_reads; mod code_cache; +pub mod code_cache_global; +pub mod code_cache_global_manager; pub mod counters; pub mod errors; pub mod executor; mod executor_utilities; +pub mod explicit_sync_wrapper; mod limit_processor; #[cfg(any(test, feature = "fuzzing"))] pub mod proptest_types; diff --git a/aptos-move/block-executor/src/proptest_types/bencher.rs b/aptos-move/block-executor/src/proptest_types/bencher.rs index d4b989c041c98..527c926f1b636 100644 --- a/aptos-move/block-executor/src/proptest_types/bencher.rs +++ b/aptos-move/block-executor/src/proptest_types/bencher.rs @@ -3,6 +3,7 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{ + code_cache_global::GlobalModuleCache, executor::BlockExecutor, proptest_types::{ baseline::BaselineOutput, @@ -15,8 +16,7 @@ use crate::{ }; use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, - executable::ExecutableTestType, read_only_module_cache::ReadOnlyModuleCache, - state_store::MockStateView, + executable::ExecutableTestType, state_store::MockStateView, }; use criterion::{BatchSize, Bencher as CBencher}; use num_cpus; @@ -126,7 +126,7 @@ where .build() .unwrap(), ); - let global_module_cache = Arc::new(ReadOnlyModuleCache::empty()); + let global_module_cache = Arc::new(GlobalModuleCache::empty()); let config = BlockExecutorConfig::new_no_block_limit(num_cpus::get()); let env = MockEnvironment::new(); diff --git a/aptos-move/block-executor/src/proptest_types/tests.rs b/aptos-move/block-executor/src/proptest_types/tests.rs index f065ae3e307b1..5d83c2fe50578 100644 --- a/aptos-move/block-executor/src/proptest_types/tests.rs +++ b/aptos-move/block-executor/src/proptest_types/tests.rs @@ -3,6 +3,7 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{ + code_cache_global::GlobalModuleCache, errors::SequentialBlockExecutionError, executor::BlockExecutor, proptest_types::{ @@ -17,8 +18,7 @@ use crate::{ }; use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, - executable::ExecutableTestType, read_only_module_cache::ReadOnlyModuleCache, - state_store::MockStateView, + executable::ExecutableTestType, state_store::MockStateView, }; use claims::{assert_matches, assert_ok}; use num_cpus; @@ -80,7 +80,7 @@ fn run_transactions( >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &state_view); @@ -217,7 +217,7 @@ fn deltas_writes_mixed_with_block_gas_limit(num_txns: usize, maybe_block_gas_lim >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -270,7 +270,7 @@ fn deltas_resolver_with_block_gas_limit(num_txns: usize, maybe_block_gas_limit: >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -428,7 +428,7 @@ fn publishing_fixed_params_with_block_gas_limit( >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool, - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -475,7 +475,7 @@ fn publishing_fixed_params_with_block_gas_limit( Some(max(w_index, r_index) as u64 * MAX_GAS_PER_TXN + 1), ), executor_thread_pool.clone(), - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) // Ensure enough gas limit to commit the module txns (4 is maximum gas per txn) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -556,7 +556,7 @@ fn non_empty_group( >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool.clone(), - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); @@ -575,7 +575,7 @@ fn non_empty_group( >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool.clone(), - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_sequential(&env, &transactions, &data_view, false); diff --git a/aptos-move/block-executor/src/scheduler.rs b/aptos-move/block-executor/src/scheduler.rs index 796d024619075..0d27eb94ba8f5 100644 --- a/aptos-move/block-executor/src/scheduler.rs +++ b/aptos-move/block-executor/src/scheduler.rs @@ -2,12 +2,10 @@ // Parts of the project are originally copyright © Meta Platforms, Inc. // SPDX-License-Identifier: Apache-2.0 +use crate::explicit_sync_wrapper::ExplicitSyncWrapper; use aptos_infallible::Mutex; use aptos_mvhashmap::types::{Incarnation, TxnIndex}; -use aptos_types::{ - error::{code_invariant_error, PanicError}, - explicit_sync_wrapper::ExplicitSyncWrapper, -}; +use aptos_types::error::{code_invariant_error, PanicError}; use concurrent_queue::{ConcurrentQueue, PopError}; use crossbeam::utils::CachePadded; use parking_lot::{RwLock, RwLockUpgradableReadGuard}; diff --git a/aptos-move/block-executor/src/txn_last_input_output.rs b/aptos-move/block-executor/src/txn_last_input_output.rs index e8391b3ea6986..fcfbde7452aa7 100644 --- a/aptos-move/block-executor/src/txn_last_input_output.rs +++ b/aptos-move/block-executor/src/txn_last_input_output.rs @@ -4,6 +4,7 @@ use crate::{ captured_reads::CapturedReads, errors::ParallelBlockExecutionError, + explicit_sync_wrapper::ExplicitSyncWrapper, task::{ExecutionStatus, TransactionOutput}, types::{InputOutputKey, ReadWriteSummary}, }; @@ -11,7 +12,6 @@ use aptos_logger::error; use aptos_mvhashmap::types::TxnIndex; use aptos_types::{ error::{code_invariant_error, PanicError}, - explicit_sync_wrapper::ExplicitSyncWrapper, fee_statement::FeeStatement, state_store::state_value::StateValueMetadata, transaction::BlockExecutableTransaction as Transaction, diff --git a/aptos-move/block-executor/src/unit_tests/mod.rs b/aptos-move/block-executor/src/unit_tests/mod.rs index 39dca409555c5..0cb4a946dbc27 100644 --- a/aptos-move/block-executor/src/unit_tests/mod.rs +++ b/aptos-move/block-executor/src/unit_tests/mod.rs @@ -5,6 +5,7 @@ mod code_cache_tests; use crate::{ + code_cache_global::GlobalModuleCache, errors::SequentialBlockExecutionError, executor::BlockExecutor, proptest_types::{ @@ -29,7 +30,6 @@ use aptos_types::{ block_executor::config::BlockExecutorConfig, contract_event::TransactionEvent, executable::{ExecutableTestType, ModulePath}, - read_only_module_cache::ReadOnlyModuleCache, state_store::state_value::StateValueMetadata, write_set::WriteOpKind, }; @@ -87,7 +87,7 @@ fn test_resource_group_deletion() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ); @@ -154,7 +154,7 @@ fn resource_group_bcs_fallback() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ); @@ -254,7 +254,7 @@ fn block_output_err_precedence() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ); @@ -294,7 +294,7 @@ fn skip_rest_gas_limit() { >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), Some(5)), executor_thread_pool, - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ); @@ -330,7 +330,7 @@ where >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(ReadOnlyModuleCache::empty()), + Arc::new(GlobalModuleCache::empty()), None, ) .execute_transactions_parallel(&env, &transactions, &data_view); diff --git a/aptos-move/block-executor/src/view.rs b/aptos-move/block-executor/src/view.rs index 96956b91d6886..0659662c09e89 100644 --- a/aptos-move/block-executor/src/view.rs +++ b/aptos-move/block-executor/src/view.rs @@ -8,6 +8,7 @@ use crate::{ CapturedReads, DataRead, DelayedFieldRead, DelayedFieldReadKind, GroupRead, ReadKind, UnsyncReadSet, }, + code_cache_global::GlobalModuleCache, counters, scheduler::{DependencyResult, DependencyStatus, Scheduler, TWaitForDependency}, value_exchange::{ @@ -35,7 +36,6 @@ use aptos_mvhashmap::{ use aptos_types::{ error::{code_invariant_error, expect_ok, PanicError, PanicOr}, executable::{Executable, ModulePath}, - read_only_module_cache::ReadOnlyModuleCache, state_store::{ errors::StateviewError, state_storage_usage::StateStorageUsage, @@ -991,7 +991,7 @@ impl<'a, T: Transaction, X: Executable> ViewState<'a, T, X> { pub(crate) struct LatestView<'a, T: Transaction, S: TStateView, X: Executable> { base_view: &'a S, pub(crate) global_module_cache: - &'a ReadOnlyModuleCache, + &'a GlobalModuleCache, pub(crate) runtime_environment: &'a RuntimeEnvironment, pub(crate) latest_view: ViewState<'a, T, X>, pub(crate) txn_idx: TxnIndex, @@ -1000,7 +1000,7 @@ pub(crate) struct LatestView<'a, T: Transaction, S: TStateView, X: impl<'a, T: Transaction, S: TStateView, X: Executable> LatestView<'a, T, S, X> { pub(crate) fn new( base_view: &'a S, - global_module_cache: &'a ReadOnlyModuleCache< + global_module_cache: &'a GlobalModuleCache< ModuleId, CompiledModule, Module, @@ -2491,7 +2491,7 @@ mod test { let base_view = MockStateView::empty(); let start_counter = 5; let runtime_environment = RuntimeEnvironment::new(vec![]); - let global_module_cache = ReadOnlyModuleCache::empty(); + let global_module_cache = GlobalModuleCache::empty(); let latest_view = LatestView::>, MockExecutable>::new( @@ -2761,7 +2761,7 @@ mod test { counter: RefCell, base_view: MockStateView>, empty_global_module_cache: - ReadOnlyModuleCache, + GlobalModuleCache, runtime_environment: RuntimeEnvironment, } @@ -2775,7 +2775,7 @@ mod test { unsync_map, counter, base_view, - empty_global_module_cache: ReadOnlyModuleCache::empty(), + empty_global_module_cache: GlobalModuleCache::empty(), runtime_environment, } } diff --git a/aptos-move/e2e-tests/src/executor.rs b/aptos-move/e2e-tests/src/executor.rs index 6c64a10fffa30..c98c4e352006e 100644 --- a/aptos-move/e2e-tests/src/executor.rs +++ b/aptos-move/e2e-tests/src/executor.rs @@ -28,6 +28,7 @@ use aptos_types::{ }, block_executor::config::{ BlockExecutorConfig, BlockExecutorConfigFromOnchain, BlockExecutorLocalConfig, + BlockExecutorModuleCacheLocalConfig, }, block_metadata::BlockMetadata, chain_id::ChainId, @@ -633,16 +634,19 @@ impl FakeExecutor { }, allow_fallback: self.allow_block_executor_fallback, discard_failed_blocks: false, + module_cache_config: BlockExecutorModuleCacheLocalConfig::default(), }, onchain: onchain_config, }; - BlockAptosVM::execute_block_on_thread_pool_without_global_caches::< + BlockAptosVM::execute_block_on_thread_pool::< _, NoOpTransactionCommitHook, >( self.executor_thread_pool.clone(), txn_block, &state_view, + // Do not use module caches in tests. + None, config, None, ) diff --git a/execution/executor-benchmark/Cargo.toml b/execution/executor-benchmark/Cargo.toml index 6c572c0b7d329..4f99fe0a77268 100644 --- a/execution/executor-benchmark/Cargo.toml +++ b/execution/executor-benchmark/Cargo.toml @@ -25,7 +25,6 @@ aptos-executor-types = { workspace = true } aptos-experimental-ptx-executor = { workspace = true } aptos-experimental-runtimes = { workspace = true } aptos-genesis = { workspace = true, features = ["testing"] } -aptos-global-cache-manager = { workspace = true } aptos-jellyfish-merkle = { workspace = true } aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } diff --git a/execution/executor-benchmark/src/native_executor.rs b/execution/executor-benchmark/src/native_executor.rs index e25bb063d2fbc..f90eb1498f79e 100644 --- a/execution/executor-benchmark/src/native_executor.rs +++ b/execution/executor-benchmark/src/native_executor.rs @@ -358,7 +358,6 @@ impl VMBlockExecutor for NativeExecutor { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), - _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { let transaction_outputs = NATIVE_EXECUTOR_POOL diff --git a/execution/executor-service/Cargo.toml b/execution/executor-service/Cargo.toml index 5d631700f1af2..e590b54912c69 100644 --- a/execution/executor-service/Cargo.toml +++ b/execution/executor-service/Cargo.toml @@ -15,7 +15,6 @@ rust-version = { workspace = true } [dependencies] aptos-block-partitioner = { workspace = true } aptos-config = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-infallible = { workspace = true } aptos-language-e2e-tests = { workspace = true } aptos-logger = { workspace = true } diff --git a/execution/executor/Cargo.toml b/execution/executor/Cargo.toml index 39ab296d4f35e..7fc27599e38fd 100644 --- a/execution/executor/Cargo.toml +++ b/execution/executor/Cargo.toml @@ -20,7 +20,6 @@ aptos-drop-helper = { workspace = true } aptos-executor-service = { workspace = true } aptos-executor-types = { workspace = true } aptos-experimental-runtimes = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-indexer-grpc-table-info = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } diff --git a/execution/executor/src/block_executor/mod.rs b/execution/executor/src/block_executor/mod.rs index 553804df81d68..6599ee4a0af3b 100644 --- a/execution/executor/src/block_executor/mod.rs +++ b/execution/executor/src/block_executor/mod.rs @@ -232,6 +232,16 @@ where ))) }); + // In case block executor has a cache manager, we need to mark it as ready for + // execution. If for some reason this fails, return an error. + if let Some(module_cache_manager) = self.block_executor.module_cache_manager() { + if !module_cache_manager.mark_ready(&parent_block_id, block_id) { + return Err(ExecutorError::internal_err( + "Unable to mark module cache manager as ready", + )); + } + } + DoGetExecutionOutput::by_transaction_execution( &self.block_executor, transactions, diff --git a/execution/executor/src/chunk_executor/mod.rs b/execution/executor/src/chunk_executor/mod.rs index db8c2052dc288..26391ae28acd6 100644 --- a/execution/executor/src/chunk_executor/mod.rs +++ b/execution/executor/src/chunk_executor/mod.rs @@ -605,7 +605,6 @@ impl ChunkExecutorInner { BlockExecutorConfigFromOnchain::new_no_block_limit(), None, )?; - // not `zip_eq`, deliberately for (version, txn_out, txn_info, write_set, events) in multizip(( begin_version..end_version, diff --git a/execution/executor/src/chunk_executor/transaction_chunk.rs b/execution/executor/src/chunk_executor/transaction_chunk.rs index aca1d387e2ddc..26f873162eb61 100644 --- a/execution/executor/src/chunk_executor/transaction_chunk.rs +++ b/execution/executor/src/chunk_executor/transaction_chunk.rs @@ -87,7 +87,6 @@ impl TransactionChunk for ChunkToExecute { &V::new(), sig_verified_txns.into(), state_view, - &global_cache_manager, BlockExecutorConfigFromOnchain::new_no_block_limit(), None, ) diff --git a/execution/executor/src/db_bootstrapper/mod.rs b/execution/executor/src/db_bootstrapper/mod.rs index 0445db5a6a2ff..1d9a3c2742cc8 100644 --- a/execution/executor/src/db_bootstrapper/mod.rs +++ b/execution/executor/src/db_bootstrapper/mod.rs @@ -138,7 +138,6 @@ pub fn calculate_genesis( BlockExecutorConfigFromOnchain::new_no_block_limit(), None, )?; - ensure!( execution_output.num_transactions_to_commit() != 0, "Genesis txn execution failed." diff --git a/execution/executor/src/fuzzing.rs b/execution/executor/src/fuzzing.rs index 2ea96225c51ed..67639b5831107 100644 --- a/execution/executor/src/fuzzing.rs +++ b/execution/executor/src/fuzzing.rs @@ -78,7 +78,6 @@ impl VMBlockExecutor for FakeVM { &self, _transactions: &[SignatureVerifiedTransaction], _state_view: &impl StateView, - _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { Ok(BlockOutput::new(vec![], None)) diff --git a/execution/executor/src/tests/mock_vm/mock_vm_test.rs b/execution/executor/src/tests/mock_vm/mock_vm_test.rs index fcf97bb230bad..4df0ef06d0665 100644 --- a/execution/executor/src/tests/mock_vm/mock_vm_test.rs +++ b/execution/executor/src/tests/mock_vm/mock_vm_test.rs @@ -25,8 +25,11 @@ fn test_mock_vm_different_senders() { txns.push(encode_mint_transaction(gen_address(i), amount)); } - let outputs = MockVM - .execute_block_no_limit(&into_signature_verified_block(txns.clone()), &MockStateView::empty()) + let outputs = MockVM::new() + .execute_block_no_limit( + &into_signature_verified_block(txns.clone()), + &MockStateView::empty(), + ) .expect("MockVM should not fail to start"); for (output, txn) in itertools::zip_eq(outputs.iter(), txns.iter()) { @@ -62,8 +65,11 @@ fn test_mock_vm_same_sender() { txns.push(encode_mint_transaction(sender, amount)); } - let outputs = MockVM - .execute_block_no_limit(&into_signature_verified_block(txns), &MockStateView::empty()) + let outputs = MockVM::new() + .execute_block_no_limit( + &into_signature_verified_block(txns), + &MockStateView::empty(), + ) .expect("MockVM should not fail to start"); for (i, output) in outputs.iter().enumerate() { @@ -97,8 +103,11 @@ fn test_mock_vm_payment() { encode_transfer_transaction(gen_address(0), gen_address(1), 50), ]; - let output = MockVM - .execute_block_no_limit(&into_signature_verified_block(txns), &MockStateView::empty()) + let output = MockVM::new() + .execute_block_no_limit( + &into_signature_verified_block(txns), + &MockStateView::empty(), + ) .expect("MockVM should not fail to start"); let mut output_iter = output.iter(); diff --git a/execution/executor/src/tests/mock_vm/mod.rs b/execution/executor/src/tests/mock_vm/mod.rs index 608c45076976e..bb9ea70a99393 100644 --- a/execution/executor/src/tests/mock_vm/mod.rs +++ b/execution/executor/src/tests/mock_vm/mod.rs @@ -52,7 +52,6 @@ enum MockVMTransaction { pub static KEEP_STATUS: Lazy = Lazy::new(|| TransactionStatus::Keep(ExecutionStatus::Success)); -// We use 10 as the assertion error code for insufficient balance within the Aptos coin contract. pub static DISCARD_STATUS: Lazy = Lazy::new(|| TransactionStatus::Discard(StatusCode::INSUFFICIENT_BALANCE_FOR_TRANSACTION_FEE)); @@ -67,7 +66,6 @@ impl VMBlockExecutor for MockVM { &self, transactions: &[SignatureVerifiedTransaction], state_view: &impl StateView, - _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { // output_cache is used to store the output of transactions so they are visible to later diff --git a/execution/executor/src/tests/mod.rs b/execution/executor/src/tests/mod.rs index ba76568c38f7e..8ca28af1a83ff 100644 --- a/execution/executor/src/tests/mod.rs +++ b/execution/executor/src/tests/mod.rs @@ -675,20 +675,23 @@ fn run_transactions_naive( ) -> HashValue { let executor = TestExecutor::new(); let db = &executor.db; - let global_cache_manager = GlobalCacheManager::new_with_default_config(); for txn in transactions { let ledger_view: ExecutedTrees = db.reader.get_latest_executed_trees().unwrap(); let out = DoGetExecutionOutput::by_transaction_execution( &MockVM::new(), vec![txn].into(), - state_view, - &global_cache_manager, + ledger_view + .verified_state_view( + StateViewId::Miscellaneous, + Arc::clone(&db.reader), + Arc::new(AsyncProofFetcher::new(db.reader.clone())), + ) + .unwrap(), block_executor_onchain_config.clone(), None, ) .unwrap(); - let output = ApplyExecutionOutput::run(out, &ledger_view).unwrap(); db.writer .save_transactions( diff --git a/execution/executor/src/workflow/do_get_execution_output.rs b/execution/executor/src/workflow/do_get_execution_output.rs index cb1c76c96b284..4fad63d2e8261 100644 --- a/execution/executor/src/workflow/do_get_execution_output.rs +++ b/execution/executor/src/workflow/do_get_execution_output.rs @@ -48,7 +48,6 @@ impl DoGetExecutionOutput { executor: &V, transactions: ExecutableTransactions, state_view: CachedStateView, - global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, append_state_checkpoint_to_block: Option, ) -> Result { @@ -58,7 +57,6 @@ impl DoGetExecutionOutput { executor, txns, state_view, - global_cache_manager, onchain_config, append_state_checkpoint_to_block, )? @@ -90,7 +88,6 @@ impl DoGetExecutionOutput { executor: &V, transactions: Vec, state_view: CachedStateView, - global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, append_state_checkpoint_to_block: Option, ) -> Result { @@ -204,7 +201,6 @@ impl DoGetExecutionOutput { executor: &V, transactions: &[SignatureVerifiedTransaction], state_view: &CachedStateView, - global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, ) -> Result> { let _timer = OTHER_TIMERS.timer_with(&["vm_execute_block"]); @@ -220,7 +216,6 @@ impl DoGetExecutionOutput { executor: &V, transactions: &[SignatureVerifiedTransaction], state_view: &CachedStateView, - global_cache_manager: &GlobalCacheManager, onchain_config: BlockExecutorConfigFromOnchain, ) -> Result> { use aptos_types::{ diff --git a/experimental/execution/ptx-executor/Cargo.toml b/experimental/execution/ptx-executor/Cargo.toml index 20b9e6a3a28e3..0da896d31500a 100644 --- a/experimental/execution/ptx-executor/Cargo.toml +++ b/experimental/execution/ptx-executor/Cargo.toml @@ -14,7 +14,6 @@ rust-version = { workspace = true } [dependencies] aptos-experimental-runtimes = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } aptos-metrics-core = { workspace = true } diff --git a/experimental/execution/ptx-executor/src/lib.rs b/experimental/execution/ptx-executor/src/lib.rs index a080223eb7159..9ed83b7d7a3f6 100644 --- a/experimental/execution/ptx-executor/src/lib.rs +++ b/experimental/execution/ptx-executor/src/lib.rs @@ -22,7 +22,6 @@ use crate::{ scheduler::PtxScheduler, sorter::PtxSorter, state_reader::PtxStateReader, }; use aptos_experimental_runtimes::thread_manager::THREAD_MANAGER; -use aptos_global_cache_manager::GlobalCacheManager; use aptos_infallible::Mutex; use aptos_metrics_core::TimerHelper; use aptos_types::{ @@ -53,7 +52,6 @@ impl VMBlockExecutor for PtxBlockExecutor { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), - _global_cache_manager: &GlobalCacheManager, _onchain_config: BlockExecutorConfigFromOnchain, ) -> Result, VMStatus> { let _timer = TIMER.timer_with(&["block_total"]); diff --git a/storage/db-tool/Cargo.toml b/storage/db-tool/Cargo.toml index 0f9f772776d00..b2feeb952750d 100644 --- a/storage/db-tool/Cargo.toml +++ b/storage/db-tool/Cargo.toml @@ -18,7 +18,6 @@ aptos-config = { workspace = true } aptos-db = { workspace = true, features = ["db-debugger"] } aptos-executor = { workspace = true } aptos-executor-types = { workspace = true } -aptos-global-cache-manager = { workspace = true } aptos-logger = { workspace = true } aptos-storage-interface = { workspace = true } aptos-temppath = { workspace = true } diff --git a/storage/db-tool/src/replay_on_archive.rs b/storage/db-tool/src/replay_on_archive.rs index 09f873ef2c434..5bc13308bee98 100644 --- a/storage/db-tool/src/replay_on_archive.rs +++ b/storage/db-tool/src/replay_on_archive.rs @@ -29,7 +29,6 @@ use std::{ sync::{atomic::AtomicU64, Arc}, time::Instant, }; - // Replay Verify controller is responsible for providing legit range with start and end versions. #[derive(Parser)] pub struct Opt { @@ -298,7 +297,9 @@ impl Verifier { .map(|txn| SignatureVerifiedTransaction::from(txn.clone())) .collect::>() .as_slice(), - &state_view, + &self + .arc_db + .state_view_at_version(start_version.checked_sub(1))?, )?; let mut failed_txns = Vec::new(); diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs index 578b3690d3cb6..8138bac66e02c 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs @@ -170,10 +170,11 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> UnsyncModuleStorage<' .module_cache .into_modules_iter() .flat_map(|(key, module)| { - module - .code() - .is_verified() - .then(|| (key, module.code().verified().clone())) + module.code().is_verified().then(|| { + // TODO(loader_v2): + // We should be able to take ownership here, instead of clones. + (key, module.code().verified().clone()) + }) }) } diff --git a/types/Cargo.toml b/types/Cargo.toml index 79cb83ee9e9cc..088b34699ac06 100644 --- a/types/Cargo.toml +++ b/types/Cargo.toml @@ -32,7 +32,6 @@ arr_macro = { workspace = true } base64 = { workspace = true } bcs = { workspace = true } bytes = { workspace = true } -crossbeam = { workspace = true } dashmap = { workspace = true } derivative = { workspace = true } fixed = { workspace = true } diff --git a/types/src/block_executor/config.rs b/types/src/block_executor/config.rs index 586b76e8c7942..95e66d1609850 100644 --- a/types/src/block_executor/config.rs +++ b/types/src/block_executor/config.rs @@ -4,6 +4,31 @@ use crate::on_chain_config::BlockGasLimitType; use serde::{Deserialize, Serialize}; +/// Local, per-node configurations for module cache. While caches can be persisted across multiple +/// block executions, these configurations allow to specify cache sizes, etc. +#[derive(Clone, Debug)] +pub struct BlockExecutorModuleCacheLocalConfig { + /// If true, when global caches are empty, Aptos framework is prefetched into module cache. + pub prefetch_framework_code: bool, + /// The maximum size of module cache (the sum of serialized sizes of all cached modules in + /// bytes). + pub max_module_cache_size_in_bytes: usize, + /// The maximum size (in terms of entries) of struct name re-indexing map stored in the runtime + /// environment. + pub max_struct_name_index_map_size: usize, +} + +impl Default for BlockExecutorModuleCacheLocalConfig { + fn default() -> Self { + Self { + prefetch_framework_code: true, + // Use 50 Mb for now, should be large enough to cache many modules. + max_module_cache_size_in_bytes: 50 * 1024 * 1024, + max_struct_name_index_map_size: 100_000, + } + } +} + /// Local, per-node configuration. #[derive(Clone, Debug)] pub struct BlockExecutorLocalConfig { @@ -14,6 +39,24 @@ pub struct BlockExecutorLocalConfig { // If true, we will discard the failed blocks and continue with the next block. // (allow_fallback needs to be set) pub discard_failed_blocks: bool, + + /// Various cache configurations, see [BlockExecutorModuleCacheLocalConfig] for more details. + pub module_cache_config: BlockExecutorModuleCacheLocalConfig, +} + +impl BlockExecutorLocalConfig { + /// Returns a new config with specified concurrency level and: + /// - Allowed fallback to sequential execution from parallel. + /// - Not allowed discards of failed blocks. + /// - Default module cache configs. + pub fn default_with_concurrency_level(concurrency_level: usize) -> Self { + Self { + concurrency_level, + allow_fallback: true, + discard_failed_blocks: false, + module_cache_config: BlockExecutorModuleCacheLocalConfig::default(), + } + } } /// Configuration from on-chain configuration, that is @@ -40,7 +83,7 @@ impl BlockExecutorConfigFromOnchain { pub const fn on_but_large_for_test() -> Self { Self { block_gas_limit_type: - // present, so code is excercised, but large to not limit blocks + // present, so code is exercised, but large to not limit blocks BlockGasLimitType::ComplexLimitV1 { effective_block_gas_limit: 1_000_000_000, execution_gas_effective_multiplier: 1, @@ -69,11 +112,7 @@ pub struct BlockExecutorConfig { impl BlockExecutorConfig { pub fn new_no_block_limit(concurrency_level: usize) -> Self { Self { - local: BlockExecutorLocalConfig { - concurrency_level, - allow_fallback: true, - discard_failed_blocks: false, - }, + local: BlockExecutorLocalConfig::default_with_concurrency_level(concurrency_level), onchain: BlockExecutorConfigFromOnchain::new_no_block_limit(), } } @@ -83,11 +122,7 @@ impl BlockExecutorConfig { maybe_block_gas_limit: Option, ) -> Self { Self { - local: BlockExecutorLocalConfig { - concurrency_level, - allow_fallback: true, - discard_failed_blocks: false, - }, + local: BlockExecutorLocalConfig::default_with_concurrency_level(concurrency_level), onchain: BlockExecutorConfigFromOnchain::new_maybe_block_limit(maybe_block_gas_limit), } } diff --git a/types/src/lib.rs b/types/src/lib.rs index bec1414ad4104..9081b6c0f0a4d 100644 --- a/types/src/lib.rs +++ b/types/src/lib.rs @@ -2,7 +2,7 @@ // Parts of the project are originally copyright © Meta Platforms, Inc. // SPDX-License-Identifier: Apache-2.0 -#![deny(unsafe_code)] +#![forbid(unsafe_code)] pub mod access_path; pub mod account_address; @@ -18,7 +18,6 @@ pub mod epoch_state; pub mod error; pub mod event; pub mod executable; -pub mod explicit_sync_wrapper; pub mod fee_statement; pub mod governance; pub mod indexer; @@ -36,7 +35,6 @@ pub mod proof; #[cfg(any(test, feature = "fuzzing"))] pub mod proptest_types; pub mod randomness; -pub mod read_only_module_cache; pub mod serde_helper; pub mod stake_pool; pub mod staking_contract; From 7c2dbf733796165407188c0d2766c9fdf0ae73e0 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Sat, 9 Nov 2024 18:04:30 +0000 Subject: [PATCH 11/25] prior to addressing comments: - Fixed a bug where on successful marking as executing we were re-creating environment - Removed optional manager in BlockAptosVM, instead a new instance can be provided - Removed clean state, instead manager is initialized to Done(None) and optional values need to be provided. --- .../aptos-debugger/src/aptos_debugger.rs | 6 +- .../src/transaction_bench_state.rs | 8 +- .../src/aptos_test_harness.rs | 10 +- aptos-move/aptos-vm/src/aptos_vm.rs | 2 +- aptos-move/aptos-vm/src/block_executor/mod.rs | 66 ++-- .../sharded_executor_service.rs | 6 +- .../aptos-vm/tests/sharded_block_executor.rs | 18 +- .../src/code_cache_global_manager.rs | 306 +++++++++--------- aptos-move/e2e-tests/src/executor.rs | 8 +- .../aptos-framework/doc/function_info.md | 104 ------ execution/executor-service/src/test_utils.rs | 12 +- execution/executor/src/block_executor/mod.rs | 134 ++++---- execution/executor/src/chunk_executor/mod.rs | 8 +- .../src/chunk_executor/transaction_chunk.rs | 8 +- execution/executor/src/db_bootstrapper/mod.rs | 5 + 15 files changed, 319 insertions(+), 382 deletions(-) diff --git a/aptos-move/aptos-debugger/src/aptos_debugger.rs b/aptos-move/aptos-debugger/src/aptos_debugger.rs index 0b4946fce95dc..6a57d69b33154 100644 --- a/aptos-move/aptos-debugger/src/aptos_debugger.rs +++ b/aptos-move/aptos-debugger/src/aptos_debugger.rs @@ -2,7 +2,9 @@ // SPDX-License-Identifier: Apache-2.0 use anyhow::{bail, format_err, Result}; -use aptos_block_executor::txn_commit_hook::NoOpTransactionCommitHook; +use aptos_block_executor::{ + code_cache_global_manager::ModuleCacheManager, txn_commit_hook::NoOpTransactionCommitHook, +}; use aptos_gas_profiling::{GasProfiler, TransactionGasLog}; use aptos_rest_client::Client; use aptos_types::{ @@ -431,7 +433,7 @@ fn execute_block_no_limit( BlockAptosVM::execute_block::<_, NoOpTransactionCommitHook>( sig_verified_txns, state_view, - None, + &ModuleCacheManager::new(), BlockExecutorConfig { local: BlockExecutorLocalConfig::default_with_concurrency_level(concurrency_level), onchain: BlockExecutorConfigFromOnchain::new_no_block_limit(), diff --git a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs index a6f4b68a9ce1d..9a18b17c35c96 100644 --- a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs +++ b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs @@ -3,7 +3,9 @@ use crate::transactions; use aptos_bitvec::BitVec; -use aptos_block_executor::txn_commit_hook::NoOpTransactionCommitHook; +use aptos_block_executor::{ + code_cache_global_manager::ModuleCacheManager, txn_commit_hook::NoOpTransactionCommitHook, +}; use aptos_block_partitioner::{ v2::config::PartitionerV2Config, BlockPartitioner, PartitionerConfig, }; @@ -218,7 +220,7 @@ where >( transactions, self.state_view.as_ref(), - None, + &ModuleCacheManager::new(), BlockExecutorConfig::new_maybe_block_limit(1, maybe_block_gas_limit), None, ) @@ -267,7 +269,7 @@ where >( transactions, self.state_view.as_ref(), - None, + &ModuleCacheManager::new(), BlockExecutorConfig::new_maybe_block_limit( concurrency_level_per_shard, maybe_block_gas_limit, diff --git a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs index a96506db49b47..aff52ca04c4e9 100644 --- a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs +++ b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs @@ -515,8 +515,14 @@ impl<'a> AptosTestAdapter<'a> { fn run_transaction(&mut self, txn: Transaction) -> Result { let txn_block = vec![txn]; let sig_verified_block = into_signature_verified_block(txn_block); - let mut outputs = AptosVMBlockExecutor::new() - .execute_block_no_limit(&sig_verified_block, &self.storage.clone())?; + + let executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + + let mut outputs = + executor.execute_block_no_limit(&sig_verified_block, &self.storage.clone())?; assert_eq!(outputs.len(), 1); diff --git a/aptos-move/aptos-vm/src/aptos_vm.rs b/aptos-move/aptos-vm/src/aptos_vm.rs index e85934e3fbaa3..30305923add99 100644 --- a/aptos-move/aptos-vm/src/aptos_vm.rs +++ b/aptos-move/aptos-vm/src/aptos_vm.rs @@ -2827,7 +2827,7 @@ impl VMBlockExecutor for AptosVMBlockExecutor { >( transactions, state_view, - Some(&self.module_cache_manager), + &self.module_cache_manager, BlockExecutorConfig { local: BlockExecutorLocalConfig { concurrency_level: AptosVM::get_concurrency_level(), diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index 2246ec02b6c9b..9f2f7a546f1dc 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -409,8 +409,12 @@ impl BlockAptosVM { executor_thread_pool: Arc, signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, - module_cache_manager: Option< - &ModuleCacheManager, + module_cache_manager: &ModuleCacheManager< + HashValue, + ModuleId, + CompiledModule, + Module, + AptosModuleExtension, >, config: BlockExecutorConfig, transaction_commit_listener: Option, @@ -426,28 +430,24 @@ impl BlockAptosVM { BLOCK_EXECUTOR_CONCURRENCY.set(config.local.concurrency_level as i64); + let (environment, module_cache) = if module_cache_manager.mark_executing() { + let environment = module_cache_manager.get_or_initialize_environment(state_view); + let module_cache = module_cache_manager.module_cache(); + (environment, module_cache) + } else { + // Either we do not have global caches , in which case we can create new ones, or + // something went wrong, and we were not able to mark the state as executing. In + // this case, fallback to empty caches. Note that the alert should have been raised + // during marking. + let environment = + AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); + let module_cache = Arc::new(GlobalModuleCache::empty()); + (environment, module_cache) + }; + // We should be checking different module cache configurations here. let module_cache_config = &config.local.module_cache_config; - let (environment, module_cache) = match module_cache_manager { - Some(module_cache_manager) if !module_cache_manager.mark_executing() => { - let environment = - module_cache_manager.get_or_initialize_environment_unchecked(state_view); - let module_cache = module_cache_manager.module_cache(); - (environment, module_cache) - }, - _ => { - // Either we do not have global caches , in which case we can create new ones, or - // something went wrong, and we were not able to mark the state as executing. In - // this case, fallback to empty caches. Note that the alert should have been raised - // during marking. - let environment = - AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); - let module_cache = Arc::new(GlobalModuleCache::empty()); - (environment, module_cache) - }, - }; - // Check 1: struct re-indexing map is not too large. If it is, we flush the cache. Also, we // need to flush modules because they store indices into re-indexing map. let runtime_environment = environment.runtime_environment(); @@ -490,15 +490,13 @@ impl BlockAptosVM { ); let ret = executor.execute_block(environment, signature_verified_block, state_view); - if let Some(module_cache_manager) = module_cache_manager { - if !module_cache_manager.mark_done() { - // Something is wrong as we were not able to mark execution as done. Return an - // error. - return Err(VMStatus::error( - StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, - Some("Unable to mark block execution as done".to_string()), - )); - } + if !module_cache_manager.mark_done() { + // Something is wrong as we were not able to mark execution as done. Return an + // error. + return Err(VMStatus::error( + StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some("Unable to mark block execution as done".to_string()), + )); } match ret { @@ -538,8 +536,12 @@ impl BlockAptosVM { >( signature_verified_block: &[SignatureVerifiedTransaction], state_view: &S, - module_cache_manager: Option< - &ModuleCacheManager, + module_cache_manager: &ModuleCacheManager< + HashValue, + ModuleId, + CompiledModule, + Module, + AptosModuleExtension, >, config: BlockExecutorConfig, transaction_commit_listener: Option, diff --git a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs index 193302692725d..15c627732cca0 100644 --- a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs +++ b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs @@ -16,6 +16,7 @@ use crate::{ ExecutorShardCommand, }, }; +use aptos_block_executor::code_cache_global_manager::ModuleCacheManager; use aptos_logger::{info, trace}; use aptos_types::{ block_executor::{ @@ -139,8 +140,9 @@ impl ShardedExecutorService { executor_thread_pool, &signature_verified_transactions, aggr_overridden_state_view.as_ref(), - // Since we execute blocks in parallel, we cannot share module caches. - None, + // Since we execute blocks in parallel, we cannot share module caches, so each + // thread has its own caches. + &ModuleCacheManager::new(), config, cross_shard_commit_sender, ) diff --git a/aptos-move/aptos-vm/tests/sharded_block_executor.rs b/aptos-move/aptos-vm/tests/sharded_block_executor.rs index 5968d0a495ab9..da872b4049367 100644 --- a/aptos-move/aptos-vm/tests/sharded_block_executor.rs +++ b/aptos-move/aptos-vm/tests/sharded_block_executor.rs @@ -308,7 +308,11 @@ mod test_utils { .into_iter() .map(|t| t.into_txn()) .collect(); - let unsharded_txn_output = AptosVMBlockExecutor::new() + let executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + let unsharded_txn_output = executor .execute_block_no_limit(&ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); @@ -358,7 +362,11 @@ mod test_utils { ) .unwrap(); - let unsharded_txn_output = AptosVMBlockExecutor::new() + let executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + let unsharded_txn_output = executor .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); @@ -412,7 +420,11 @@ mod test_utils { ) .unwrap(); - let unsharded_txn_output = AptosVMBlockExecutor::new() + let executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + let unsharded_txn_output = executor .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs index d7bd8b771c344..7fabddeb4242e 100644 --- a/aptos-move/block-executor/src/code_cache_global_manager.rs +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -29,72 +29,53 @@ macro_rules! alert_or_println { } /// Represents the state of [GlobalModuleCache]. The following transitions are allowed: -/// 1. [State::Clean] --> [State::Ready]. /// 2. [State::Ready] --> [State::Executing]. /// 3. [State::Executing] --> [State::Done]. /// 4. [State::Done] --> [State::Ready]. #[derive(Clone, Debug, Eq, PartialEq)] enum State { - Clean, - Ready(T), - Executing(T), - Done(T), + Ready(Option), + Executing(Option), + Done(Option), } impl State { - /// If the state is [State::Clean] returns true, and false otherwise. - fn is_clean(&self) -> bool { - match self { - State::Clean => true, - State::Ready(_) | State::Executing(_) | State::Done(_) => false, - } - } - - /// If the state is [State::Done], returns true. - fn is_done(&self) -> bool { - match self { - State::Done(_) => true, - State::Clean | State::Ready(_) | State::Executing(_) => false, - } - } - - /// If the state is [State::Done] and its value equals the one provided, returns true. In other - /// cases, returns false. - fn is_done_with_value(&self, value: &T) -> bool { - match self { - State::Done(v) => v == value, - State::Clean | State::Executing(_) | State::Ready(_) => false, - } - } - /// If the state is [State::Ready], returns its value. Otherwise, returns [None]. - fn value_from_ready(&self) -> Option { + fn value_from_ready(&self) -> Option> { match self { State::Ready(v) => Some(v.clone()), - State::Clean | State::Executing(_) | State::Done(_) => None, + _ => None, } } /// If the state is [State::Executing], returns its value. Otherwise, returns [None]. - fn value_from_executing(&self) -> Option { + fn value_from_executing(&self) -> Option> { match self { State::Executing(v) => Some(v.clone()), - State::Clean | State::Ready(_) | State::Done(_) => None, + _ => None, + } + } + + /// If the state is [State::Done], returns its value. Otherwise, returns [None]. + fn value_from_done(&self) -> Option> { + match self { + State::Done(v) => Some(v.clone()), + _ => None, } } /// Sets the current state to [State::Ready]. - fn set_ready(&mut self, value: T) { + fn set_ready(&mut self, value: Option) { *self = Self::Ready(value); } /// Sets the current state to [State::Executing]. - fn set_executing(&mut self, value: T) { + fn set_executing(&mut self, value: Option) { *self = Self::Executing(value); } /// Sets the current state to [State::Done]. - fn set_done(&mut self, value: T) { + fn set_done(&mut self, value: Option) { *self = Self::Done(value); } } @@ -120,48 +101,55 @@ where VC: Deref>, E: WithSize, { - /// Returns a new instance of [ModuleCacheManager]. + /// Returns a new instance of [ModuleCacheManager] in a [State::Done] state with uninitialized + /// current value. #[allow(clippy::new_without_default)] pub fn new() -> Self { Self { - state: Mutex::new(State::Clean), + state: Mutex::new(State::Done(None)), module_cache: Arc::new(GlobalModuleCache::empty()), environment: ExplicitSyncWrapper::new(None), } } - /// If state is [State::Clean], or [State::Ready] with matching previous value, sets the state - /// to [State::Ready] with the current value and returns true. Otherwise, raises an alert and - /// returns false. - pub fn mark_ready(&self, previous: &T, current: T) -> bool { + /// If state is [State::Done], sets the state to [State::Ready] with the current value and + /// returns true. Otherwise, raises an alert and returns false. Additionally, synchronizes + /// module and environment caches based on the provided previous value. + pub fn mark_ready(&self, previous: Option<&T>, current: Option) -> bool { let mut state = self.state.lock(); - if state.is_clean() || state.is_done_with_value(previous) { - state.set_ready(current); - return true; - } - - if state.is_done() { - // If the state is done, but the values to not match, we still set the state as ready, but - // also flush global caches because they execute not on top of the previous state. - self.module_cache.flush_unchecked(); - if let Some(environment) = self.environment.acquire().as_ref() { - environment - .runtime_environment() - .flush_struct_name_and_info_caches(); - } + let recorded_previous = state.value_from_done(); + match (recorded_previous, previous) { + (None, _) => { + // We are not in the done state, this is an error. + alert_or_println!( + "Unable to mark ready, state: {:?}, previous: {:?}, current: {:?}", + state, + previous, + current + ); + false + }, + (Some(Some(recorded_previous)), Some(previous)) if recorded_previous.eq(previous) => { + // We are in done state with matching values. Can mark ready. + state.set_ready(current); + true + }, + _ => { + // If the state is done, but the values do not exist or do not match, we still set + // the state as ready, but also flush global caches because they execute on top of + // unknown state (or on top of some different to previous state). + self.module_cache.flush_unchecked(); + if let Some(environment) = self.environment.acquire().as_ref() { + environment + .runtime_environment() + .flush_struct_name_and_info_caches(); + } - state.set_ready(current); - return true; + state.set_ready(current); + true + }, } - - alert_or_println!( - "Unable to mark ready, state: {:?}, previous: {:?}, current: {:?}", - state, - previous, - current - ); - false } /// If state is [State::Ready], changes it to [State::Executing] with the same value, returning @@ -195,10 +183,7 @@ where /// Returns the cached global environment if it already exists, and matches the one in storage. /// If it does not exist, or does not match, the new environment is initialized from the given /// state, cached, and returned. - pub fn get_or_initialize_environment_unchecked( - &self, - state_view: &impl StateView, - ) -> AptosEnvironment { + pub fn get_or_initialize_environment(&self, state_view: &impl StateView) -> AptosEnvironment { let _lock = self.state.lock(); let new_environment = @@ -251,70 +236,64 @@ mod test { use std::{collections::HashMap, thread, thread::JoinHandle}; use test_case::test_case; - #[test] - fn test_clean_state() { - let state = State::Clean; + #[test_case(None)] + #[test_case(Some(0))] + fn test_ready_state(value: Option) { + let state = State::Ready(value); - assert!(state.is_clean()); - assert!(!state.is_done()); - assert!(!state.is_done_with_value(&0)); - assert!(state.value_from_ready().is_none()); + assert_eq!(state.value_from_ready(), Some(value)); assert!(state.value_from_executing().is_none()); + assert!(state.value_from_done().is_none()); } - #[test] - fn test_ready_state() { - let state = State::Ready(0); + #[test_case(None)] + #[test_case(Some(0))] + fn test_executing_state(value: Option) { + let state = State::Executing(value); - assert!(!state.is_clean()); - assert!(!state.is_done()); - assert!(!state.is_done_with_value(&0)); - assert_eq!(state.value_from_ready(), Some(0)); - assert!(state.value_from_executing().is_none()); - } - - #[test] - fn test_executing_state() { - let state = State::Executing(0); - - assert!(!state.is_clean()); - assert!(!state.is_done()); - assert!(!state.is_done_with_value(&0)); assert!(state.value_from_ready().is_none()); - assert_eq!(state.value_from_executing(), Some(0)); + assert_eq!(state.value_from_executing(), Some(value)); + assert!(state.value_from_done().is_none()); } - #[test] - fn test_done_state() { - let state = State::Done(0); + #[test_case(None)] + #[test_case(Some(0))] + fn test_done_state(value: Option) { + let state = State::Done(value); - assert!(!state.is_clean()); - assert!(state.is_done()); - assert!(state.is_done_with_value(&0)); - assert!(!state.is_done_with_value(&10)); assert!(state.value_from_ready().is_none()); assert!(state.value_from_executing().is_none()); + assert_eq!(state.value_from_done(), Some(value)); } #[test] fn test_set_state() { - let mut state = State::Clean; + let mut state = State::Done(None); - state.set_ready(0); - assert_matches!(state, State::Ready(0)); + state.set_ready(Some(0)); + assert_matches!(state, State::Ready(Some(0))); - state.set_executing(10); - assert_matches!(state, State::Executing(10)); + state.set_executing(Some(10)); + assert_matches!(state, State::Executing(Some(10))); - state.set_done(100); - assert_matches!(state, State::Done(100)); + state.set_done(Some(100)); + assert_matches!(state, State::Done(Some(100))); + + state.set_ready(Some(1000)); + assert_matches!(state, State::Ready(Some(1000))); } - #[test_case(true)] - #[test_case(false)] - fn test_marking(with_different_value_for_done: bool) { + #[test_case(None, None)] + #[test_case(None, Some(1))] + #[test_case(Some(0), None)] + #[test_case(Some(0), Some(1))] + #[test_case(Some(0), Some(0))] + fn test_mark_ready(recorded_previous: Option, previous: Option) { let module_cache_manager = ModuleCacheManager::new(); - assert!(module_cache_manager.state.lock().is_clean()); + module_cache_manager + .state + .lock() + .set_done(recorded_previous); // Pre-populate module cache to test flushing. module_cache_manager @@ -322,47 +301,60 @@ mod test { .insert(0, mock_verified_code(0, MockExtension::new(8))); assert_eq!(module_cache_manager.module_cache.num_modules(), 1); - // Can only go to ready state from clean state. assert!(!module_cache_manager.mark_executing()); assert!(!module_cache_manager.mark_done()); - assert!(module_cache_manager.mark_ready(&0, 1)); - assert_matches!(module_cache_manager.state.lock().deref(), State::Ready(1)); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert!(module_cache_manager.mark_ready(previous.as_ref(), Some(77))); + + // Only in matching case the module cache is not flushed. + if recorded_previous.is_some() && recorded_previous == previous { + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + } else { + assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + } + + let state = module_cache_manager.state.lock().clone(); + assert_eq!(state, State::Ready(Some(77))) + } - // Can only go to executing state from ready state. + #[test] + fn test_mark_executing() { + let module_cache_manager = ModuleCacheManager::< + _, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new(); + module_cache_manager.state.lock().set_ready(Some(100)); + + assert!(!module_cache_manager.mark_ready(Some(&76), Some(77))); assert!(!module_cache_manager.mark_done()); - assert!(!module_cache_manager.mark_ready(&0, 1)); - assert!(module_cache_manager.mark_executing()); - assert_matches!( - module_cache_manager.state.lock().deref(), - State::Executing(1) - ); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert!(module_cache_manager.mark_executing()); - // Can only go to done state from executing state. - assert!(!module_cache_manager.mark_executing()); - assert!(!module_cache_manager.mark_ready(&0, 1)); - assert!(module_cache_manager.mark_done()); + let state = module_cache_manager.state.lock().clone(); + assert_eq!(state, State::Executing(Some(100))) + } - assert_matches!(module_cache_manager.state.lock().deref(), State::Done(1)); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + #[test] + fn test_mark_done() { + let module_cache_manager = ModuleCacheManager::< + _, + u32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new(); + module_cache_manager.state.lock().set_executing(Some(100)); - // Can only go to ready state from done state. + assert!(!module_cache_manager.mark_ready(Some(&76), Some(77))); assert!(!module_cache_manager.mark_executing()); - assert!(!module_cache_manager.mark_done()); - if with_different_value_for_done { - // Does not match! Caches should be flushed, but state reset. - assert!(module_cache_manager.mark_ready(&10, 11)); - assert_matches!(module_cache_manager.state.lock().deref(), State::Ready(11)); - assert_eq!(module_cache_manager.module_cache.num_modules(), 0); - } else { - assert!(module_cache_manager.mark_ready(&1, 2)); - assert_matches!(module_cache_manager.state.lock().deref(), State::Ready(2)); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); - } + assert!(module_cache_manager.mark_done()); + + let state = module_cache_manager.state.lock().clone(); + assert_eq!(state, State::Done(Some(100))) } /// Joins threads. Succeeds only if a single handle evaluates to [Ok] and the rest are [Err]s. @@ -382,9 +374,8 @@ mod test { assert_eq!(num_false, num_handles - 1); } - #[test_case(true)] - #[test_case(false)] - fn test_mark_ready_concurrent(start_from_clean_state: bool) { + #[test] + fn test_mark_ready_concurrent() { let global_cache_manager = Arc::new(ModuleCacheManager::< _, u32, @@ -392,18 +383,12 @@ mod test { MockVerifiedCode, MockExtension, >::new()); - if !start_from_clean_state { - assert!(global_cache_manager.mark_ready(&0, 1)); - assert!(global_cache_manager.mark_executing()); - assert!(global_cache_manager.mark_done()); - // We are at done with value of 1. - } let mut handles = vec![]; for _ in 0..32 { let handle = thread::spawn({ let global_cache_manager = global_cache_manager.clone(); - move || global_cache_manager.mark_ready(&1, 2) + move || global_cache_manager.mark_ready(Some(&1), Some(2)) }); handles.push(handle); } @@ -419,7 +404,7 @@ mod test { MockVerifiedCode, MockExtension, >::new()); - assert!(global_cache_manager.mark_ready(&0, 1)); + assert!(global_cache_manager.mark_ready(Some(&0), Some(1))); let mut handles = vec![]; for _ in 0..32 { @@ -441,7 +426,7 @@ mod test { MockVerifiedCode, MockExtension, >::new()); - assert!(global_cache_manager.mark_ready(&0, 1)); + assert!(global_cache_manager.mark_ready(Some(&0), Some(1))); assert!(global_cache_manager.mark_executing()); let mut handles = vec![]; @@ -476,7 +461,7 @@ mod test { } #[test] - fn mark_execution_start_when_different_environment() { + fn test_get_or_initialize_environment() { let module_cache_manager = ModuleCacheManager::::new(); module_cache_manager @@ -490,7 +475,7 @@ mod test { // Environment has to be set to the same value, cache flushed. let state_view = state_view_with_changed_feature_flag(None); - let environment = module_cache_manager.get_or_initialize_environment_unchecked(&state_view); + let environment = module_cache_manager.get_or_initialize_environment(&state_view); assert_eq!(module_cache_manager.module_cache.num_modules(), 0); assert!(module_cache_manager .environment @@ -507,7 +492,7 @@ mod test { // Environment has to be re-set to the new value, cache flushed. let state_view = state_view_with_changed_feature_flag(Some(FeatureFlag::CODE_DEPENDENCY_CHECK)); - let environment = module_cache_manager.get_or_initialize_environment_unchecked(&state_view); + let environment = module_cache_manager.get_or_initialize_environment(&state_view); assert_eq!(module_cache_manager.module_cache.num_modules(), 0); assert!(module_cache_manager .environment @@ -522,8 +507,7 @@ mod test { assert!(module_cache_manager.environment.acquire().is_some()); // Environment is kept, and module caches are not flushed. - let new_environment = - module_cache_manager.get_or_initialize_environment_unchecked(&state_view); + let new_environment = module_cache_manager.get_or_initialize_environment(&state_view); assert_eq!(module_cache_manager.module_cache.num_modules(), 1); assert!(environment == new_environment); } diff --git a/aptos-move/e2e-tests/src/executor.rs b/aptos-move/e2e-tests/src/executor.rs index c98c4e352006e..d8a34e73cf2b1 100644 --- a/aptos-move/e2e-tests/src/executor.rs +++ b/aptos-move/e2e-tests/src/executor.rs @@ -14,7 +14,9 @@ use crate::{ }; use aptos_abstract_gas_usage::CalibrationAlgebra; use aptos_bitvec::BitVec; -use aptos_block_executor::txn_commit_hook::NoOpTransactionCommitHook; +use aptos_block_executor::{ + code_cache_global_manager::ModuleCacheManager, txn_commit_hook::NoOpTransactionCommitHook, +}; use aptos_crypto::HashValue; use aptos_framework::ReleaseBundle; use aptos_gas_algebra::DynamicExpression; @@ -645,8 +647,8 @@ impl FakeExecutor { self.executor_thread_pool.clone(), txn_block, &state_view, - // Do not use module caches in tests. - None, + // Do not use shared module caches in tests. + &ModuleCacheManager::new(), config, None, ) diff --git a/aptos-move/framework/aptos-framework/doc/function_info.md b/aptos-move/framework/aptos-framework/doc/function_info.md index 65840ecd59f8b..e55ff672c939e 100644 --- a/aptos-move/framework/aptos-framework/doc/function_info.md +++ b/aptos-move/framework/aptos-framework/doc/function_info.md @@ -16,12 +16,7 @@ The function_info - [Function `is_identifier`](#0x1_function_info_is_identifier) - [Function `load_function_impl`](#0x1_function_info_load_function_impl) - [Specification](#@Specification_1) - - [Function `new_function_info`](#@Specification_1_new_function_info) - - [Function `new_function_info_from_address`](#@Specification_1_new_function_info_from_address) - - [Function `check_dispatch_type_compatibility`](#@Specification_1_check_dispatch_type_compatibility) - - [Function `load_module_from_function`](#@Specification_1_load_module_from_function) - [Function `check_dispatch_type_compatibility_impl`](#@Specification_1_check_dispatch_type_compatibility_impl) - - [Function `is_identifier`](#@Specification_1_is_identifier) - [Function `load_function_impl`](#@Specification_1_load_function_impl) @@ -328,88 +323,7 @@ if such module isn't accessed previously in the transaction. - - - -
fun spec_is_identifier(s: vector<u8>): bool;
-
- - - - - -### Function `new_function_info` - - -
public fun new_function_info(module_signer: &signer, module_name: string::String, function_name: string::String): function_info::FunctionInfo
-
- - - - -
aborts_if !spec_is_identifier(string::bytes(module_name));
-aborts_if !spec_is_identifier(string::bytes(function_name));
-ensures result == FunctionInfo {
-    module_address: signer::address_of(module_signer),
-    module_name,
-    function_name,
-};
-
- - - - - -### Function `new_function_info_from_address` - - -
public(friend) fun new_function_info_from_address(module_address: address, module_name: string::String, function_name: string::String): function_info::FunctionInfo
-
- - - - -
aborts_if !spec_is_identifier(string::bytes(module_name));
-aborts_if !spec_is_identifier(string::bytes(function_name));
-ensures result == FunctionInfo {
-    module_address,
-    module_name,
-    function_name,
-};
-
- - - - - -### Function `check_dispatch_type_compatibility` - - -
public(friend) fun check_dispatch_type_compatibility(framework_function: &function_info::FunctionInfo, dispatch_target: &function_info::FunctionInfo): bool
-
- - - - -
pragma verify = false;
-pragma opaque;
-
- - - - - -### Function `load_module_from_function` - - -
public(friend) fun load_module_from_function(f: &function_info::FunctionInfo)
-
- - - -
pragma verify = false;
-pragma opaque;
 
@@ -430,24 +344,6 @@ if such module isn't accessed previously in the transaction. - - -### Function `is_identifier` - - -
fun is_identifier(s: &vector<u8>): bool
-
- - - - -
pragma opaque;
-aborts_if [abstract] false;
-ensures [abstract] result == spec_is_identifier(s);
-
- - - ### Function `load_function_impl` diff --git a/execution/executor-service/src/test_utils.rs b/execution/executor-service/src/test_utils.rs index a1d6e8673290d..150d13b5b5f4c 100644 --- a/execution/executor-service/src/test_utils.rs +++ b/execution/executor-service/src/test_utils.rs @@ -137,7 +137,11 @@ pub fn test_sharded_block_executor_no_conflict> .into_iter() .map(|t| t.into_txn()) .collect(); - let unsharded_txn_output = AptosVMBlockExecutor::new() + let executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + let unsharded_txn_output = executor .execute_block_no_limit(&txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); @@ -192,7 +196,11 @@ pub fn sharded_block_executor_with_conflict>( ) .unwrap(); - let unsharded_txn_output = AptosVMBlockExecutor::new() + let executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + let unsharded_txn_output = executor .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); diff --git a/execution/executor/src/block_executor/mod.rs b/execution/executor/src/block_executor/mod.rs index 6599ee4a0af3b..1595de3eeb4ba 100644 --- a/execution/executor/src/block_executor/mod.rs +++ b/execution/executor/src/block_executor/mod.rs @@ -198,73 +198,77 @@ where "execute_block" ); let committed_block_id = self.committed_block_id(); - let (execution_output, state_checkpoint_output) = - if parent_block_id != committed_block_id && parent_output.has_reconfiguration() { - // ignore reconfiguration suffix, even if the block is non-empty - info!( - LogSchema::new(LogEntry::BlockExecutor).block_id(block_id), - "reconfig_descendant_block_received" - ); - ( - parent_output.execution_output.reconfig_suffix(), - parent_output - .expect_state_checkpoint_output() - .reconfig_suffix(), - ) - } else { - let state_view = { - let _timer = OTHER_TIMERS.timer_with(&["verified_state_view"]); - - CachedStateView::new( - StateViewId::BlockExecution { block_id }, - Arc::clone(&self.db.reader), - parent_output.execution_output.next_version(), - parent_output.expect_result_state().current.clone(), - Arc::new(AsyncProofFetcher::new(self.db.reader.clone())), - )? - }; - - let execution_output = { - let _timer = GET_BLOCK_EXECUTION_OUTPUT_BY_EXECUTING.start_timer(); - fail_point!("executor::block_executor_execute_block", |_| { - Err(ExecutorError::from(anyhow::anyhow!( - "Injected error in block_executor_execute_block" - ))) - }); - - // In case block executor has a cache manager, we need to mark it as ready for - // execution. If for some reason this fails, return an error. - if let Some(module_cache_manager) = self.block_executor.module_cache_manager() { - if !module_cache_manager.mark_ready(&parent_block_id, block_id) { - return Err(ExecutorError::internal_err( - "Unable to mark module cache manager as ready", - )); - } - } + let (execution_output, state_checkpoint_output) = if parent_block_id != committed_block_id + && parent_output.has_reconfiguration() + { + // ignore reconfiguration suffix, even if the block is non-empty + info!( + LogSchema::new(LogEntry::BlockExecutor).block_id(block_id), + "reconfig_descendant_block_received" + ); + ( + parent_output.execution_output.reconfig_suffix(), + parent_output + .expect_state_checkpoint_output() + .reconfig_suffix(), + ) + } else { + let state_view = { + let _timer = OTHER_TIMERS.timer_with(&["verified_state_view"]); + + CachedStateView::new( + StateViewId::BlockExecution { block_id }, + Arc::clone(&self.db.reader), + parent_output.execution_output.next_version(), + parent_output.expect_result_state().current.clone(), + Arc::new(AsyncProofFetcher::new(self.db.reader.clone())), + )? + }; - DoGetExecutionOutput::by_transaction_execution( - &self.block_executor, - transactions, - state_view, - onchain_config.clone(), - Some(block_id), - )? - }; - - let _timer = OTHER_TIMERS.timer_with(&["state_checkpoint"]); - - let state_checkpoint_output = THREAD_MANAGER.get_exe_cpu_pool().install(|| { - fail_point!("executor::block_state_checkpoint", |_| { - Err(anyhow::anyhow!("Injected error in block state checkpoint.")) - }); - DoStateCheckpoint::run( - &execution_output, - parent_output.expect_result_state(), - Option::>::None, - ) - })?; - (execution_output, state_checkpoint_output) + let execution_output = { + let _timer = GET_BLOCK_EXECUTION_OUTPUT_BY_EXECUTING.start_timer(); + fail_point!("executor::block_executor_execute_block", |_| { + Err(ExecutorError::from(anyhow::anyhow!( + "Injected error in block_executor_execute_block" + ))) + }); + + // In case block executor has a cache manager, we need to mark it as ready for + // execution. If for some reason this fails, return an error. + if let Some(module_cache_manager) = self.block_executor.module_cache_manager() { + // TODO(loader_v2): + // Refactor to be able to move this into AptosVM block executor. This will + // also allow us to remove all ready markings in other places. + if !module_cache_manager.mark_ready(Some(&parent_block_id), Some(block_id)) { + return Err(ExecutorError::internal_err( + "Unable to mark module cache manager as ready", + )); + } + } + + DoGetExecutionOutput::by_transaction_execution( + &self.block_executor, + transactions, + state_view, + onchain_config.clone(), + Some(block_id), + )? }; + + let _timer = OTHER_TIMERS.timer_with(&["state_checkpoint"]); + + let state_checkpoint_output = THREAD_MANAGER.get_exe_cpu_pool().install(|| { + fail_point!("executor::block_state_checkpoint", |_| { + Err(anyhow::anyhow!("Injected error in block state checkpoint.")) + }); + DoStateCheckpoint::run( + &execution_output, + parent_output.expect_result_state(), + Option::>::None, + ) + })?; + (execution_output, state_checkpoint_output) + }; let output = PartialStateComputeResult::new(execution_output); output.set_state_checkpoint_output(state_checkpoint_output); diff --git a/execution/executor/src/chunk_executor/mod.rs b/execution/executor/src/chunk_executor/mod.rs index 26391ae28acd6..c97398fae868e 100644 --- a/execution/executor/src/chunk_executor/mod.rs +++ b/execution/executor/src/chunk_executor/mod.rs @@ -597,9 +597,15 @@ impl ChunkExecutorInner { .map(|t| t.into()) .collect::>(); + // For now, we create executor for each chunk. + let executor = V::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + // State sync executor shouldn't have block gas limit. let execution_output = DoGetExecutionOutput::by_transaction_execution::( - &V::new(), + &executor, txns.into(), state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), diff --git a/execution/executor/src/chunk_executor/transaction_chunk.rs b/execution/executor/src/chunk_executor/transaction_chunk.rs index 26f873162eb61..a51ec0f5c9bf8 100644 --- a/execution/executor/src/chunk_executor/transaction_chunk.rs +++ b/execution/executor/src/chunk_executor/transaction_chunk.rs @@ -83,8 +83,14 @@ impl TransactionChunk for ChunkToExecute { }; let _timer = VM_EXECUTE_CHUNK.start_timer(); + + let executor = V::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + DoGetExecutionOutput::by_transaction_execution::( - &V::new(), + &executor, sig_verified_txns.into(), state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), diff --git a/execution/executor/src/db_bootstrapper/mod.rs b/execution/executor/src/db_bootstrapper/mod.rs index 1d9a3c2742cc8..e69f696d48f1e 100644 --- a/execution/executor/src/db_bootstrapper/mod.rs +++ b/execution/executor/src/db_bootstrapper/mod.rs @@ -131,6 +131,11 @@ pub fn calculate_genesis( get_state_epoch(&base_state_view)? }; + let executor = V::new(); + if let Some(module_cache_manager) = executor.module_cache_manager() { + module_cache_manager.mark_ready(None, None); + } + let execution_output = DoGetExecutionOutput::by_transaction_execution::( &V::new(), vec![genesis_txn.clone().into()].into(), From 03a1ea20bd42dd3bdb9c414cb144d52a2a81c1c5 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Mon, 11 Nov 2024 14:33:27 +0000 Subject: [PATCH 12/25] [comments] Address Rati's and Zekun's comments --- .../state_view_adapter.rs | 87 ++++---- aptos-move/aptos-vm/src/block_executor/mod.rs | 51 ++--- .../aptos-vm/tests/sharded_block_executor.rs | 18 +- .../block-executor/src/captured_reads.rs | 10 +- .../block-executor/src/code_cache_global.rs | 97 ++++---- .../src/code_cache_global_manager.rs | 210 +++++------------- aptos-move/block-executor/src/executor.rs | 8 +- execution/executor-service/src/test_utils.rs | 12 +- .../implementations/unsync_module_storage.rs | 35 +-- .../types/src/code/cache/module_cache.rs | 6 +- .../types/src/code/cache/test_types.rs | 12 +- .../move-vm/types/src/code/cache/types.rs | 2 +- types/src/block_executor/config.rs | 6 +- 13 files changed, 231 insertions(+), 323 deletions(-) diff --git a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs index 9a40888290199..64627c634797d 100644 --- a/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs +++ b/aptos-move/aptos-vm-types/src/module_and_script_storage/state_view_adapter.rs @@ -5,7 +5,7 @@ use crate::module_and_script_storage::module_storage::AptosModuleStorage; use ambassador::Delegate; use aptos_types::{ error::PanicError, - state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView}, + state_store::{state_key::StateKey, state_value::StateValueMetadata, StateView, TStateView}, vm::modules::AptosModuleExtension, }; use bytes::Bytes; @@ -28,7 +28,7 @@ use move_vm_types::{ code::{ModuleBytesStorage, ModuleCode}, module_storage_error, }; -use std::sync::Arc; +use std::{ops::Deref, sync::Arc}; /// Avoids orphan rule to implement [ModuleBytesStorage] for [StateView]. struct StateViewAdapter<'s, S> { @@ -48,6 +48,14 @@ impl<'s, S: StateView> ModuleBytesStorage for StateViewAdapter<'s, S> { } } +impl<'s, S: StateView> Deref for StateViewAdapter<'s, S> { + type Target = S; + + fn deref(&self) -> &Self::Target { + &self.state_view + } +} + /// A (not thread-safe) implementation of code storage on top of a state view. It is never built /// directly by clients - only via [AsAptosCodeStorage] trait. Can be used to resolve both modules /// and cached scripts. @@ -84,7 +92,7 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, } /// Drains cached verified modules from the code storage, transforming them into format used by - /// global caches. Should only be called when the code storage borrows [StateView]. + /// global caches. pub fn into_verified_module_code_iter( self, ) -> Result< @@ -96,47 +104,42 @@ impl<'s, S: StateView, E: WithRuntimeEnvironment> AptosCodeStorageAdapter<'s, S, >, PanicError, > { - let state_view = match self.storage.module_storage().byte_storage().state_view { - BorrowedOrOwned::Borrowed(state_view) => state_view, - BorrowedOrOwned::Owned(_) => { - return Err(PanicError::CodeInvariantError( - "Verified modules should only be extracted from borrowed state".to_string(), - )) - }, - }; - - let mut modules_to_add = vec![]; - for (key, verified_code) in self + let (state_view, verified_modules_iter) = self .storage .into_module_storage() - .into_verified_modules_iter() - { - // We have cached the module previously, so we must be able to find it in storage. - let extension = state_view - .get_state_value(&StateKey::module_id(&key)) - .map_err(|err| { - let msg = format!( - "Failed to retrieve module {}::{} from storage {:?}", - key.address(), - key.name(), - err - ); - PanicError::CodeInvariantError(msg) - })? - .map(AptosModuleExtension::new) - .ok_or_else(|| { - let msg = format!( - "Module {}::{} should exist, but it does not anymore", - key.address(), - key.name() - ); - PanicError::CodeInvariantError(msg) - })?; - - let module = ModuleCode::from_verified_ref(verified_code, Arc::new(extension)); - modules_to_add.push((key, Arc::new(module))) - } - Ok(modules_to_add.into_iter()) + .unpack_into_verified_modules_iter(); + + Ok(verified_modules_iter + .map(|(key, verified_code)| { + // We have cached the module previously, so we must be able to find it in storage. + let extension = state_view + .get_state_value(&StateKey::module_id(&key)) + .map_err(|err| { + let msg = format!( + "Failed to retrieve module {}::{} from storage {:?}", + key.address(), + key.name(), + err + ); + PanicError::CodeInvariantError(msg) + })? + .map_or_else( + || { + let msg = format!( + "Module {}::{} should exist, but it does not anymore", + key.address(), + key.name() + ); + Err(PanicError::CodeInvariantError(msg)) + }, + |state_value| Ok(AptosModuleExtension::new(state_value)), + )?; + + let module = ModuleCode::from_arced_verified(verified_code, Arc::new(extension)); + Ok((key, Arc::new(module))) + }) + .collect::, PanicError>>()? + .into_iter()) } } diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index 9f2f7a546f1dc..68877952e80ee 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -39,8 +39,11 @@ use aptos_vm_logging::{ alert, flush_speculative_logs, init_speculative_logs, prelude::CRITICAL_ERRORS, }; use aptos_vm_types::{ - abstract_write_op::AbstractResourceWriteOp, module_and_script_storage::AsAptosCodeStorage, - module_write_set::ModuleWrite, output::VMOutput, resolver::ResourceGroupSize, + abstract_write_op::AbstractResourceWriteOp, + module_and_script_storage::{AptosCodeStorageAdapter, AsAptosCodeStorage}, + module_write_set::ModuleWrite, + output::VMOutput, + resolver::ResourceGroupSize, }; use move_binary_format::{ errors::{Location, VMError}, @@ -454,26 +457,23 @@ impl BlockAptosVM { let struct_name_index_map_size = runtime_environment .struct_name_index_map_size() .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; - if struct_name_index_map_size > module_cache_config.max_struct_name_index_map_size { - module_cache.flush_unchecked(); + if struct_name_index_map_size > module_cache_config.max_struct_name_index_map_num_entries { + module_cache.flush_unsync(); runtime_environment.flush_struct_name_and_info_caches(); } // Check 2: If the module cache is too big, flush it. - if module_cache - .size_in_bytes_is_greater_than(module_cache_config.max_module_cache_size_in_bytes) - { - module_cache.flush_unchecked(); + if module_cache.size_in_bytes() > module_cache_config.max_module_cache_size_in_bytes { + module_cache.flush_unsync(); } // Finally, to avoid cold starts, fetch the framework code prior to block execution. if module_cache.num_modules() == 0 && module_cache_config.prefetch_framework_code { - prefetch_aptos_framework(environment.clone(), state_view, &module_cache).map_err( - |err| { - alert!("Failed to load Aptos framework to module cache: {:?}", err); - VMError::from(err).into_vm_status() - }, - )?; + let code_storage = state_view.as_aptos_code_storage(environment.clone()); + prefetch_aptos_framework(code_storage, &module_cache).map_err(|err| { + alert!("Failed to load Aptos framework to module cache: {:?}", err); + VMError::from(err).into_vm_status() + })?; } let executor = BlockExecutor::< @@ -491,8 +491,6 @@ impl BlockAptosVM { let ret = executor.execute_block(environment, signature_verified_block, state_view); if !module_cache_manager.mark_done() { - // Something is wrong as we were not able to mark execution as done. Return an - // error. return Err(VMStatus::error( StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, Some("Unable to mark block execution as done".to_string()), @@ -560,13 +558,10 @@ impl BlockAptosVM { /// If Aptos framework exists, loads "transaction_validation.move" and all its transitive /// dependencies from storage into provided module cache. If loading fails for any reason, a panic /// error is returned. -fn prefetch_aptos_framework( - environment: AptosEnvironment, - state_view: &impl StateView, +fn prefetch_aptos_framework( + code_storage: AptosCodeStorageAdapter, module_cache: &GlobalModuleCache, ) -> Result<(), PanicError> { - let code_storage = state_view.as_aptos_code_storage(environment); - // If framework code exists in storage, the transitive closure will be verified and cached. let maybe_loaded = code_storage .fetch_verified_module(&AccountAddress::ONE, ident_str!("transaction_validation")) @@ -576,13 +571,11 @@ fn prefetch_aptos_framework( PanicError::CodeInvariantError(format!("Unable to fetch Aptos framework: {:?}", err)) })?; - if let Some(module) = maybe_loaded { - drop(module); - + if maybe_loaded.is_some() { // Framework must have been loaded. Drain verified modules from local cache into // global cache. let verified_module_code_iter = code_storage.into_verified_module_code_iter()?; - module_cache.insert_verified_unchecked(verified_module_code_iter)?; + module_cache.insert_verified_unsync(verified_module_code_iter)?; } Ok(()) } @@ -598,10 +591,12 @@ mod test { let state_view = executor.get_state_view(); let environment = AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); + let code_storage = state_view.as_aptos_code_storage(environment); + let module_cache = GlobalModuleCache::empty(); assert_eq!(module_cache.num_modules(), 0); - let result = prefetch_aptos_framework(environment, state_view, &module_cache); + let result = prefetch_aptos_framework(code_storage, &module_cache); assert!(result.is_ok()); assert!(module_cache.num_modules() > 0); } @@ -612,10 +607,12 @@ mod test { let environment = AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); + let code_storage = state_view.as_aptos_code_storage(environment); + let module_cache = GlobalModuleCache::empty(); assert_eq!(module_cache.num_modules(), 0); - let result = prefetch_aptos_framework(environment, &state_view, &module_cache); + let result = prefetch_aptos_framework(code_storage, &module_cache); assert!(result.is_ok()); assert_eq!(module_cache.num_modules(), 0); } diff --git a/aptos-move/aptos-vm/tests/sharded_block_executor.rs b/aptos-move/aptos-vm/tests/sharded_block_executor.rs index da872b4049367..1dcfbea2dd0f9 100644 --- a/aptos-move/aptos-vm/tests/sharded_block_executor.rs +++ b/aptos-move/aptos-vm/tests/sharded_block_executor.rs @@ -308,11 +308,11 @@ mod test_utils { .into_iter() .map(|t| t.into_txn()) .collect(); - let executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { + let block_executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = block_executor.module_cache_manager() { module_cache_manager.mark_ready(None, None); } - let unsharded_txn_output = executor + let unsharded_txn_output = block_executor .execute_block_no_limit(&ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); @@ -362,11 +362,11 @@ mod test_utils { ) .unwrap(); - let executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { + let block_executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = block_executor.module_cache_manager() { module_cache_manager.mark_ready(None, None); } - let unsharded_txn_output = executor + let unsharded_txn_output = block_executor .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); @@ -420,11 +420,11 @@ mod test_utils { ) .unwrap(); - let executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { + let block_executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = block_executor.module_cache_manager() { module_cache_manager.mark_ready(None, None); } - let unsharded_txn_output = executor + let unsharded_txn_output = block_executor .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); diff --git a/aptos-move/block-executor/src/captured_reads.rs b/aptos-move/block-executor/src/captured_reads.rs index c1214baa633db..834becdcd4c94 100644 --- a/aptos-move/block-executor/src/captured_reads.rs +++ b/aptos-move/block-executor/src/captured_reads.rs @@ -295,11 +295,11 @@ impl DelayedFieldRead { /// Represents a module read, either from immutable cross-block cache, or from code [SyncCodeCache] /// used by block executor (per-block cache). This way, when transaction needs to read a module /// from [SyncCodeCache] it can first check the read-set here. -enum ModuleRead { +enum ModuleRead { /// Read from the cross-block module cache. GlobalCache, /// Read from per-block cache ([SyncCodeCache]) used by parallel execution. - PerBlockCache(Option<(Arc>, V)>), + PerBlockCache(Option<(Arc>, Option)>), } /// Represents a result of a read from [CapturedReads] when they are used as the transaction-level @@ -326,7 +326,7 @@ pub(crate) struct CapturedReads { #[deprecated] pub(crate) deprecated_module_reads: Vec, - module_reads: hashbrown::HashMap>>, + module_reads: hashbrown::HashMap>, /// If there is a speculative failure (e.g. delta application failure, or an observed /// inconsistency), the transaction output is irrelevant (must be discarded and transaction @@ -1578,12 +1578,12 @@ mod test { assert!(!valid); // Without invalid module (and if it is not captured), validation should pass. - global_module_cache.remove(&1); + assert!(global_module_cache.remove(&1)); captured_reads.module_reads.remove(&1); assert!(captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache)); // Validation fails if we captured a cross-block module which does not exist anymore. - global_module_cache.remove(&0); + assert!(global_module_cache.remove(&0)); let valid = captured_reads.validate_module_reads(&global_module_cache, &per_block_module_cache); assert!(!valid); diff --git a/aptos-move/block-executor/src/code_cache_global.rs b/aptos-move/block-executor/src/code_cache_global.rs index d8fa927a60008..fef08f5f5c940 100644 --- a/aptos-move/block-executor/src/code_cache_global.rs +++ b/aptos-move/block-executor/src/code_cache_global.rs @@ -3,7 +3,6 @@ use crate::explicit_sync_wrapper::ExplicitSyncWrapper; use aptos_types::error::PanicError; -use crossbeam::utils::CachePadded; use hashbrown::HashMap; use move_vm_types::code::{ModuleCode, WithSize}; use std::{ @@ -16,22 +15,22 @@ use std::{ }; /// Entry stored in [GlobalModuleCache]. -struct Entry { +struct Entry { /// True if this code is "valid" within the block execution context (i.e., there has been no /// republishing of this module so far). If false, executor needs to read the module from the /// sync/unsync module caches. - valid: CachePadded, + valid: AtomicBool, /// Cached verified module. Must always be verified. - module: CachePadded>>, + module: Arc>, } -impl Entry +impl Entry where - VC: Deref>, - E: WithSize, + Verified: Deref>, + Extension: WithSize, { /// Returns a new valid module. Returns a (panic) error if the module is not verified. - fn new(module: Arc>) -> Result { + fn new(module: Arc>) -> Result { if !module.code().is_verified() { return Err(PanicError::CodeInvariantError( "Module code is not verified".to_string(), @@ -39,8 +38,8 @@ where } Ok(Self { - valid: CachePadded::new(AtomicBool::new(true)), - module: CachePadded::new(module), + valid: AtomicBool::new(true), + module, }) } @@ -55,24 +54,24 @@ where } /// Returns the module code stored is this [Entry]. - fn module_code(&self) -> &Arc> { - self.module.deref() + fn module_code(&self) -> &Arc> { + &self.module } } -/// A read-only module cache for verified code, that can be accessed concurrently within the block. -/// Can only be modified safely at block boundaries. -pub struct GlobalModuleCache { +/// A global module cache for verified code that is read-only and concurrently accessed during the +/// block execution. Modified safely only at block boundaries. +pub struct GlobalModuleCache { /// Module cache containing the verified code. - module_cache: ExplicitSyncWrapper>>, + module_cache: ExplicitSyncWrapper>>, /// Sum of serialized sizes (in bytes) of all cached modules. size: AtomicUsize, } -impl GlobalModuleCache +impl GlobalModuleCache where K: Hash + Eq + Clone, - VC: Deref>, + V: Deref>, E: WithSize, { /// Returns new empty module cache. @@ -92,27 +91,21 @@ where } /// Marks the cached module (if it exists) as invalid. As a result, all subsequent calls to the - /// cache for the associated key will result in a cache miss. If an entry does not to exist, is - /// a no-op. + /// cache for the associated key will result in a cache miss. If an entry does not to exist, it + /// is a no-op. pub fn mark_invalid_if_contains(&self, key: &K) { if let Some(entry) = self.module_cache.acquire().get(key) { entry.mark_invalid(); } } - /// Returns the module stored in cache. If the module has not been cached, or it exists but it - /// is not valid, [None] is returned. - pub fn get(&self, key: &K) -> Option>> { + /// Returns the module stored in cache. If the module has not been cached, or it exists but is + /// not valid, [None] is returned. + pub fn get(&self, key: &K) -> Option>> { self.module_cache .acquire() .get(key) - .and_then(|entry| entry.is_valid().then(|| entry.module_code().clone())) - } - - /// Flushes the cache. Should never be called throughout block-execution. Use with caution. - pub fn flush_unchecked(&self) { - self.module_cache.acquire().clear(); - self.size.store(0, Ordering::Relaxed); + .and_then(|entry| entry.is_valid().then(|| Arc::clone(entry.module_code()))) } /// Returns the number of entries in the cache. @@ -125,22 +118,24 @@ where self.size.load(Ordering::Relaxed) } - /// Returns true if the sum of serialized sizes of modules stored in cache is greater than the - /// specified value. - pub fn size_in_bytes_is_greater_than(&self, size: usize) -> bool { - self.size_in_bytes() > size + /// **Use with caution: should never be called during block execution.** + /// + /// Flushes the module cache. + pub fn flush_unsync(&self) { + self.module_cache.acquire().clear(); + self.size.store(0, Ordering::Relaxed); } - /// Inserts modules into the cache. Should never be called throughout block-execution. Use with - /// caution. + /// **Use with caution: should never be called during block execution.** /// + /// Inserts modules into the cache. /// Notes: /// 1. Only verified modules are inserted. /// 2. Valid modules should not be removed, and new modules should have unique ownership. If /// these constraints are violated, a panic error is returned. - pub fn insert_verified_unchecked( + pub fn insert_verified_unsync( &self, - modules: impl Iterator>)>, + modules: impl Iterator>)>, ) -> Result<(), PanicError> { use hashbrown::hash_map::Entry::*; @@ -177,7 +172,7 @@ where /// Insert the module to cache. Used for tests only. #[cfg(any(test, feature = "testing"))] - pub fn insert(&self, key: K, module: Arc>) { + pub fn insert(&self, key: K, module: Arc>) { self.size .fetch_add(module.extension().size_in_bytes(), Ordering::Relaxed); self.module_cache.acquire().insert( @@ -186,14 +181,18 @@ where ); } - /// Removes the module from cache. Used for tests only. + /// Removes the module from cache and returns true. If the module does not exist for the + /// associated key, returns false. Used for tests only. #[cfg(any(test, feature = "testing"))] - pub fn remove(&self, key: &K) { + pub fn remove(&self, key: &K) -> bool { if let Some(entry) = self.module_cache.acquire().remove(key) { self.size.fetch_sub( entry.module_code().extension().size_in_bytes(), Ordering::Relaxed, ); + true + } else { + false } } } @@ -251,15 +250,11 @@ mod test { assert_eq!(cache.num_modules(), 3); assert_eq!(cache.size_in_bytes(), 32); - cache.remove(&2); + assert!(cache.remove(&2)); assert_eq!(cache.num_modules(), 2); assert_eq!(cache.size_in_bytes(), 24); - assert!(cache.size_in_bytes_is_greater_than(23)); - assert!(!cache.size_in_bytes_is_greater_than(24)); - assert!(!cache.size_in_bytes_is_greater_than(25)); - - cache.flush_unchecked(); + cache.flush_unsync(); assert_eq!(cache.num_modules(), 0); assert_eq!(cache.size_in_bytes(), 0); } @@ -273,7 +268,7 @@ mod test { new_modules.push((i, mock_verified_code(i, MockExtension::new(8)))); } assert!(cache - .insert_verified_unchecked(new_modules.into_iter()) + .insert_verified_unsync(new_modules.into_iter()) .is_ok()); assert_eq!(cache.num_modules(), 10); @@ -285,7 +280,7 @@ mod test { let cache = GlobalModuleCache::empty(); let deserialized_modules = vec![(0, mock_deserialized_code(0, MockExtension::new(8)))]; - assert_ok!(cache.insert_verified_unchecked(deserialized_modules.into_iter())); + assert_ok!(cache.insert_verified_unsync(deserialized_modules.into_iter())); assert_eq!(cache.num_modules(), 0); assert_eq!(cache.size_in_bytes(), 0); @@ -300,7 +295,7 @@ mod test { assert_eq!(cache.size_in_bytes(), 8); let new_modules = vec![(0, mock_verified_code(100, MockExtension::new(32)))]; - assert_err!(cache.insert_verified_unchecked(new_modules.into_iter())); + assert_err!(cache.insert_verified_unsync(new_modules.into_iter())); } #[test] @@ -313,7 +308,7 @@ mod test { assert_eq!(cache.size_in_bytes(), 8); let new_modules = vec![(0, mock_verified_code(100, MockExtension::new(32)))]; - assert_ok!(cache.insert_verified_unchecked(new_modules.into_iter())); + assert_ok!(cache.insert_verified_unsync(new_modules.into_iter())); assert_eq!(cache.num_modules(), 1); assert_eq!(cache.size_in_bytes(), 32); diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs index 7fabddeb4242e..5793feab4c8b0 100644 --- a/aptos-move/block-executor/src/code_cache_global_manager.rs +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -10,6 +10,7 @@ use parking_lot::Mutex; use std::{ fmt::Debug, hash::Hash, + mem, ops::{Deref, DerefMut}, sync::Arc, }; @@ -32,6 +33,13 @@ macro_rules! alert_or_println { /// 2. [State::Ready] --> [State::Executing]. /// 3. [State::Executing] --> [State::Done]. /// 4. [State::Done] --> [State::Ready]. +/// The optional value stored in variants is propagated during state transitions. When a full cycle +/// is reached (just before [State::Done] to [State::Ready] transition), the user can check if the +/// value is expected and continue with a new one. For instance: +/// ```text +/// Ready(Some(0)) --> Executing(Some(0)) --> Done(Some(0)) --> Ready(Some(1)) is allowed. +/// Ready(Some(0)) --> Executing(Some(0)) --> Done(Some(0)) --> Ready(Some(2)) is not allowed. +/// ``` #[derive(Clone, Debug, Eq, PartialEq)] enum State { Ready(Option), @@ -39,47 +47,6 @@ enum State { Done(Option), } -impl State { - /// If the state is [State::Ready], returns its value. Otherwise, returns [None]. - fn value_from_ready(&self) -> Option> { - match self { - State::Ready(v) => Some(v.clone()), - _ => None, - } - } - - /// If the state is [State::Executing], returns its value. Otherwise, returns [None]. - fn value_from_executing(&self) -> Option> { - match self { - State::Executing(v) => Some(v.clone()), - _ => None, - } - } - - /// If the state is [State::Done], returns its value. Otherwise, returns [None]. - fn value_from_done(&self) -> Option> { - match self { - State::Done(v) => Some(v.clone()), - _ => None, - } - } - - /// Sets the current state to [State::Ready]. - fn set_ready(&mut self, value: Option) { - *self = Self::Ready(value); - } - - /// Sets the current state to [State::Executing]. - fn set_executing(&mut self, value: Option) { - *self = Self::Executing(value); - } - - /// Sets the current state to [State::Done]. - fn set_done(&mut self, value: Option) { - *self = Self::Done(value); - } -} - /// Manages module caches and the execution environment, possible across multiple blocks. pub struct ModuleCacheManager { /// The state of global caches. @@ -96,7 +63,7 @@ pub struct ModuleCacheManager { impl ModuleCacheManager where - T: Clone + Debug + Eq, + T: Debug + Eq, K: Hash + Eq + Clone, VC: Deref>, E: WithSize, @@ -118,37 +85,35 @@ where pub fn mark_ready(&self, previous: Option<&T>, current: Option) -> bool { let mut state = self.state.lock(); - let recorded_previous = state.value_from_done(); - match (recorded_previous, previous) { - (None, _) => { - // We are not in the done state, this is an error. - alert_or_println!( - "Unable to mark ready, state: {:?}, previous: {:?}, current: {:?}", - state, - previous, - current - ); - false - }, - (Some(Some(recorded_previous)), Some(previous)) if recorded_previous.eq(previous) => { - // We are in done state with matching values. Can mark ready. - state.set_ready(current); - true - }, - _ => { - // If the state is done, but the values do not exist or do not match, we still set - // the state as ready, but also flush global caches because they execute on top of - // unknown state (or on top of some different to previous state). - self.module_cache.flush_unchecked(); + if let State::Done(recorded_previous) = state.deref() { + // If the state is done, but the values do not exist or do not match, we flush global + // caches because they execute on top of unknown state (or on top of some different to + // the previous state). + if !recorded_previous + .as_ref() + .is_some_and(|r| previous.is_some_and(|p| r == p)) + { if let Some(environment) = self.environment.acquire().as_ref() { environment .runtime_environment() .flush_struct_name_and_info_caches(); + self.module_cache.flush_unsync(); + } else { + debug_assert!(self.module_cache.num_modules() == 0); } + } - state.set_ready(current); - true - }, + *state = State::Ready(current); + true + } else { + // We are not in the done state, this is an error. + alert_or_println!( + "Unable to mark ready, state: {:?}, previous: {:?}, current: {:?}", + state, + previous, + current + ); + false } } @@ -157,13 +122,13 @@ where /// alert. pub fn mark_executing(&self) -> bool { let mut state = self.state.lock(); - if let Some(value) = state.value_from_ready() { - state.set_executing(value); - return true; + if let State::Ready(v) = state.deref_mut() { + *state = State::Executing(mem::take(v)); + true + } else { + alert_or_println!("Unable to mark executing, state: {:?}", state); + false } - - alert_or_println!("Unable to mark executing, state: {:?}", state); - false } /// If state is [State::Executing], changes it to [State::Done] with the same value, returning @@ -171,13 +136,13 @@ where /// alert. pub fn mark_done(&self) -> bool { let mut state = self.state.lock(); - if let Some(value) = state.value_from_executing() { - state.set_done(value); - return true; + if let State::Executing(v) = state.deref_mut() { + *state = State::Done(mem::take(v)); + true + } else { + alert_or_println!("Unable to mark done, state: {:?}", state); + false } - - alert_or_println!("Unable to mark done, state: {:?}", state); - false } /// Returns the cached global environment if it already exists, and matches the one in storage. @@ -192,28 +157,20 @@ where let mut guard = self.environment.acquire(); let existing_environment = guard.deref_mut(); - let (environment, is_new) = match existing_environment.as_ref() { - None => { - *existing_environment = Some(new_environment.clone()); - (new_environment, true) - }, - Some(environment) => { - if environment == &new_environment { - (environment.clone(), false) - } else { - *existing_environment = Some(new_environment.clone()); - (new_environment, true) - } - }, - }; + let environment_requires_update = existing_environment + .as_ref() + .map_or(true, |environment| environment == &new_environment); + if environment_requires_update { + *existing_environment = Some(new_environment); - // If this environment has been (re-)initialized, we need to flush the module cache because - // it can contain now out-dated code. - if is_new { - self.module_cache.flush_unchecked(); + // If this environment has been (re-)initialized, we need to flush the module cache + // because it can contain now out-dated code. + self.module_cache.flush_unsync(); } - environment + existing_environment + .clone() + .expect("Environment must be set") } /// Returns the global module cache. @@ -229,60 +186,12 @@ mod test { on_chain_config::{FeatureFlag, Features, OnChainConfig}, state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, }; - use claims::assert_matches; use move_vm_types::code::{ mock_verified_code, MockDeserializedCode, MockExtension, MockVerifiedCode, }; use std::{collections::HashMap, thread, thread::JoinHandle}; use test_case::test_case; - #[test_case(None)] - #[test_case(Some(0))] - fn test_ready_state(value: Option) { - let state = State::Ready(value); - - assert_eq!(state.value_from_ready(), Some(value)); - assert!(state.value_from_executing().is_none()); - assert!(state.value_from_done().is_none()); - } - - #[test_case(None)] - #[test_case(Some(0))] - fn test_executing_state(value: Option) { - let state = State::Executing(value); - - assert!(state.value_from_ready().is_none()); - assert_eq!(state.value_from_executing(), Some(value)); - assert!(state.value_from_done().is_none()); - } - - #[test_case(None)] - #[test_case(Some(0))] - fn test_done_state(value: Option) { - let state = State::Done(value); - - assert!(state.value_from_ready().is_none()); - assert!(state.value_from_executing().is_none()); - assert_eq!(state.value_from_done(), Some(value)); - } - - #[test] - fn test_set_state() { - let mut state = State::Done(None); - - state.set_ready(Some(0)); - assert_matches!(state, State::Ready(Some(0))); - - state.set_executing(Some(10)); - assert_matches!(state, State::Executing(Some(10))); - - state.set_done(Some(100)); - assert_matches!(state, State::Done(Some(100))); - - state.set_ready(Some(1000)); - assert_matches!(state, State::Ready(Some(1000))); - } - #[test_case(None, None)] #[test_case(None, Some(1))] #[test_case(Some(0), None)] @@ -290,10 +199,7 @@ mod test { #[test_case(Some(0), Some(0))] fn test_mark_ready(recorded_previous: Option, previous: Option) { let module_cache_manager = ModuleCacheManager::new(); - module_cache_manager - .state - .lock() - .set_done(recorded_previous); + *module_cache_manager.state.lock() = State::Done(recorded_previous); // Pre-populate module cache to test flushing. module_cache_manager @@ -326,7 +232,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - module_cache_manager.state.lock().set_ready(Some(100)); + *module_cache_manager.state.lock() = State::Ready(Some(100)); assert!(!module_cache_manager.mark_ready(Some(&76), Some(77))); assert!(!module_cache_manager.mark_done()); @@ -346,7 +252,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - module_cache_manager.state.lock().set_executing(Some(100)); + *module_cache_manager.state.lock() = State::Executing(Some(100)); assert!(!module_cache_manager.mark_ready(Some(&76), Some(77))); assert!(!module_cache_manager.mark_executing()); diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 151dc89522013..603bce0fbe1ab 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -618,9 +618,9 @@ where // Publish modules before we decrease validation index so that validations observe // the new module writes as well. if runtime_environment.vm_config().use_loader_v2 { - executed_at_commit = true; let module_write_set = last_input_output.module_write_set(txn_idx); if !module_write_set.is_empty() { + executed_at_commit = true; Self::publish_module_writes( txn_idx, module_write_set, @@ -1169,7 +1169,7 @@ where counters::update_state_counters(versioned_cache.stats(), true); self.global_module_cache - .insert_verified_unchecked(versioned_cache.take_modules_iter()) + .insert_verified_unsync(versioned_cache.take_modules_iter()) .map_err(|err| { alert!("[BlockSTM] Encountered panic error: {:?}", err); })?; @@ -1647,7 +1647,7 @@ where counters::update_state_counters(unsync_map.stats(), false); self.global_module_cache - .insert_verified_unchecked(unsync_map.into_modules_iter())?; + .insert_verified_unsync(unsync_map.into_modules_iter())?; let block_end_info = if self .config @@ -1709,7 +1709,7 @@ where // Flush the cache and the environment to re-run from the "clean" state. env.runtime_environment() .flush_struct_name_and_info_caches(); - self.global_module_cache.flush_unchecked(); + self.global_module_cache.flush_unsync(); info!("parallel execution requiring fallback"); } diff --git a/execution/executor-service/src/test_utils.rs b/execution/executor-service/src/test_utils.rs index 150d13b5b5f4c..bfd2c36cd1776 100644 --- a/execution/executor-service/src/test_utils.rs +++ b/execution/executor-service/src/test_utils.rs @@ -137,11 +137,11 @@ pub fn test_sharded_block_executor_no_conflict> .into_iter() .map(|t| t.into_txn()) .collect(); - let executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { + let block_executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = block_executor.module_cache_manager() { module_cache_manager.mark_ready(None, None); } - let unsharded_txn_output = executor + let unsharded_txn_output = block_executor .execute_block_no_limit(&txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); @@ -196,11 +196,11 @@ pub fn sharded_block_executor_with_conflict>( ) .unwrap(); - let executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { + let block_executor = AptosVMBlockExecutor::new(); + if let Some(module_cache_manager) = block_executor.module_cache_manager() { module_cache_manager.mark_ready(None, None); } - let unsharded_txn_output = executor + let unsharded_txn_output = block_executor .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); diff --git a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs index 8138bac66e02c..186b0b86f875f 100644 --- a/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs +++ b/third_party/move/move-vm/runtime/src/storage/implementations/unsync_module_storage.rs @@ -165,17 +165,24 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> UnsyncModuleStorage<' } /// Returns an iterator of all modules that have been cached and verified. - pub fn into_verified_modules_iter(self) -> impl Iterator)> { - self.0 - .module_cache - .into_modules_iter() - .flat_map(|(key, module)| { - module.code().is_verified().then(|| { - // TODO(loader_v2): - // We should be able to take ownership here, instead of clones. - (key, module.code().verified().clone()) - }) - }) + pub fn unpack_into_verified_modules_iter( + self, + ) -> ( + BorrowedOrOwned<'s, S>, + impl Iterator)>, + ) { + let verified_modules_iter = + self.0 + .module_cache + .into_modules_iter() + .flat_map(|(key, module)| { + module.code().is_verified().then(|| { + // TODO(loader_v2): + // We should be able to take ownership here, instead of clones. + (key, module.code().verified().clone()) + }) + }); + (self.0.base_storage, verified_modules_iter) } /// Test-only method that checks the state of the module cache. @@ -185,15 +192,17 @@ impl<'s, S: ModuleBytesStorage, E: WithRuntimeEnvironment> UnsyncModuleStorage<' deserialized: Vec<&'b ModuleId>, verified: Vec<&'b ModuleId>, ) { + use claims::*; + assert_eq!(self.0.num_modules(), deserialized.len() + verified.len()); for id in deserialized { let result = self.0.get_module_or_build_with(id, &self.0); - let module = claims::assert_some!(claims::assert_ok!(result)).0; + let module = assert_some!(assert_ok!(result)).0; assert!(!module.code().is_verified()) } for id in verified { let result = self.0.get_module_or_build_with(id, &self.0); - let module = claims::assert_some!(claims::assert_ok!(result)).0; + let module = assert_some!(assert_ok!(result)).0; assert!(module.code().is_verified()) } } diff --git a/third_party/move/move-vm/types/src/code/cache/module_cache.rs b/third_party/move/move-vm/types/src/code/cache/module_cache.rs index e3663c4f388da..7697c20e3a8c6 100644 --- a/third_party/move/move-vm/types/src/code/cache/module_cache.rs +++ b/third_party/move/move-vm/types/src/code/cache/module_cache.rs @@ -32,13 +32,13 @@ where /// Creates new [ModuleCode] from verified code. pub fn from_verified(verified_code: VC, extension: Arc) -> Self { - Self::from_verified_ref(Arc::new(verified_code), extension) + Self::from_arced_verified(Arc::new(verified_code), extension) } /// Creates new [ModuleCode] from [Arc]ed verified code. - pub fn from_verified_ref(verified_code: Arc, extension: Arc) -> Self { + pub fn from_arced_verified(verified_code: Arc, extension: Arc) -> Self { Self { - code: Code::from_verified_ref(verified_code), + code: Code::from_arced_verified(verified_code), extension, } } diff --git a/third_party/move/move-vm/types/src/code/cache/test_types.rs b/third_party/move/move-vm/types/src/code/cache/test_types.rs index 793d5f11951ef..1ad64e4bb1590 100644 --- a/third_party/move/move-vm/types/src/code/cache/test_types.rs +++ b/third_party/move/move-vm/types/src/code/cache/test_types.rs @@ -56,21 +56,21 @@ pub fn mock_verified_code( #[derive(Clone, Debug)] pub struct MockExtension { - size: usize, + mock_size: usize, } impl MockExtension { - pub fn new(size: usize) -> Self { - Self { size } + pub fn new(mock_size: usize) -> Self { + Self { mock_size } } } impl WithSize for MockExtension { fn size_in_bytes(&self) -> usize { - self.size + self.mock_size } } -pub fn mock_extension(size: usize) -> Arc { - Arc::new(MockExtension::new(size)) +pub fn mock_extension(mock_size: usize) -> Arc { + Arc::new(MockExtension::new(mock_size)) } diff --git a/third_party/move/move-vm/types/src/code/cache/types.rs b/third_party/move/move-vm/types/src/code/cache/types.rs index 09f51096b9cb9..12cd078c89d43 100644 --- a/third_party/move/move-vm/types/src/code/cache/types.rs +++ b/third_party/move/move-vm/types/src/code/cache/types.rs @@ -68,7 +68,7 @@ where } /// Returns new verified code from [Arc]ed instance. - pub fn from_verified_ref(verified_code: Arc) -> Self { + pub fn from_arced_verified(verified_code: Arc) -> Self { Self::Verified(verified_code) } diff --git a/types/src/block_executor/config.rs b/types/src/block_executor/config.rs index 95e66d1609850..1342eba0de53f 100644 --- a/types/src/block_executor/config.rs +++ b/types/src/block_executor/config.rs @@ -15,7 +15,7 @@ pub struct BlockExecutorModuleCacheLocalConfig { pub max_module_cache_size_in_bytes: usize, /// The maximum size (in terms of entries) of struct name re-indexing map stored in the runtime /// environment. - pub max_struct_name_index_map_size: usize, + pub max_struct_name_index_map_num_entries: usize, } impl Default for BlockExecutorModuleCacheLocalConfig { @@ -24,7 +24,7 @@ impl Default for BlockExecutorModuleCacheLocalConfig { prefetch_framework_code: true, // Use 50 Mb for now, should be large enough to cache many modules. max_module_cache_size_in_bytes: 50 * 1024 * 1024, - max_struct_name_index_map_size: 100_000, + max_struct_name_index_map_num_entries: 100_000, } } } @@ -39,8 +39,6 @@ pub struct BlockExecutorLocalConfig { // If true, we will discard the failed blocks and continue with the next block. // (allow_fallback needs to be set) pub discard_failed_blocks: bool, - - /// Various cache configurations, see [BlockExecutorModuleCacheLocalConfig] for more details. pub module_cache_config: BlockExecutorModuleCacheLocalConfig, } From 28116bcad5d51f2d13e881d9f4ee6ff4d1c45723 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Mon, 11 Nov 2024 21:14:40 +0000 Subject: [PATCH 13/25] [fix] Add done --> ready transition for unit tests --- .../sharded_block_executor/sharded_executor_service.rs | 9 ++++++--- aptos-move/e2e-tests/src/executor.rs | 8 ++++++-- execution/executor/src/db_bootstrapper/mod.rs | 2 +- 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs index 15c627732cca0..d6b09906a50a2 100644 --- a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs +++ b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs @@ -136,13 +136,16 @@ impl ShardedExecutorService { ); }); s.spawn(move |_| { + // Since we execute blocks in parallel, we cannot share module caches, so each + // thread has its own caches. + let module_cache_manager = ModuleCacheManager::new(); + module_cache_manager.mark_ready(None, None); + let ret = BlockAptosVM::execute_block_on_thread_pool( executor_thread_pool, &signature_verified_transactions, aggr_overridden_state_view.as_ref(), - // Since we execute blocks in parallel, we cannot share module caches, so each - // thread has its own caches. - &ModuleCacheManager::new(), + &module_cache_manager, config, cross_shard_commit_sender, ) diff --git a/aptos-move/e2e-tests/src/executor.rs b/aptos-move/e2e-tests/src/executor.rs index d8a34e73cf2b1..cff14879c0c52 100644 --- a/aptos-move/e2e-tests/src/executor.rs +++ b/aptos-move/e2e-tests/src/executor.rs @@ -640,6 +640,11 @@ impl FakeExecutor { }, onchain: onchain_config, }; + + // Do not use shared module caches in tests. + let module_cache_manager = ModuleCacheManager::new(); + module_cache_manager.mark_ready(None, None); + BlockAptosVM::execute_block_on_thread_pool::< _, NoOpTransactionCommitHook, @@ -647,8 +652,7 @@ impl FakeExecutor { self.executor_thread_pool.clone(), txn_block, &state_view, - // Do not use shared module caches in tests. - &ModuleCacheManager::new(), + &module_cache_manager, config, None, ) diff --git a/execution/executor/src/db_bootstrapper/mod.rs b/execution/executor/src/db_bootstrapper/mod.rs index e69f696d48f1e..d9dcd513fe82d 100644 --- a/execution/executor/src/db_bootstrapper/mod.rs +++ b/execution/executor/src/db_bootstrapper/mod.rs @@ -137,7 +137,7 @@ pub fn calculate_genesis( } let execution_output = DoGetExecutionOutput::by_transaction_execution::( - &V::new(), + &executor, vec![genesis_txn.clone().into()].into(), base_state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), From d45df1f74f8c5d226f297dfb922f5dc90eb04019 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 07:32:25 +0000 Subject: [PATCH 14/25] Move ready checks to cache manager, better encapsulation by passing hash values throughout --- Cargo.lock | 1 + .../aptos-debugger/src/aptos_debugger.rs | 2 + .../src/data_collection.rs | 2 +- .../src/transaction_bench_state.rs | 4 + .../src/aptos_test_harness.rs | 14 +- .../src/bins/run_aptos_p2p.rs | 3 +- aptos-move/aptos-vm/src/aptos_vm.rs | 12 +- aptos-move/aptos-vm/src/block_executor/mod.rs | 62 +++--- aptos-move/aptos-vm/src/lib.rs | 21 +- .../sharded_executor_service.rs | 11 +- .../aptos-vm/tests/sharded_block_executor.rs | 24 +-- aptos-move/block-executor/Cargo.toml | 1 + .../src/code_cache_global_manager.rs | 202 +++++++++++++++--- aptos-move/e2e-tests/src/executor.rs | 10 +- .../executor-benchmark/src/native_executor.rs | 3 + execution/executor-service/src/test_utils.rs | 16 +- execution/executor/src/block_executor/mod.rs | 127 +++++------ execution/executor/src/chunk_executor/mod.rs | 9 +- .../src/chunk_executor/transaction_chunk.rs | 9 +- execution/executor/src/db_bootstrapper/mod.rs | 8 +- execution/executor/src/fuzzing.rs | 2 + .../src/tests/mock_vm/mock_vm_test.rs | 6 + execution/executor/src/tests/mock_vm/mod.rs | 4 +- execution/executor/src/tests/mod.rs | 1 + .../src/workflow/do_get_execution_output.rs | 45 +++- .../execution/ptx-executor/Cargo.toml | 1 + .../execution/ptx-executor/src/lib.rs | 3 + storage/db-tool/src/replay_on_archive.rs | 2 + .../runtime/src/storage/environment.rs | 14 +- 29 files changed, 375 insertions(+), 244 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a61a62bab4508..fb51fb9c3bf92 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1646,6 +1646,7 @@ dependencies = [ name = "aptos-experimental-ptx-executor" version = "0.1.0" dependencies = [ + "aptos-crypto", "aptos-experimental-runtimes", "aptos-infallible", "aptos-logger", diff --git a/aptos-move/aptos-debugger/src/aptos_debugger.rs b/aptos-move/aptos-debugger/src/aptos_debugger.rs index 6a57d69b33154..9e9f93c62adf1 100644 --- a/aptos-move/aptos-debugger/src/aptos_debugger.rs +++ b/aptos-move/aptos-debugger/src/aptos_debugger.rs @@ -439,6 +439,8 @@ fn execute_block_no_limit( onchain: BlockExecutorConfigFromOnchain::new_no_block_limit(), }, None, + None, + None, ) .map(BlockOutput::into_transaction_outputs_forced) } diff --git a/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs b/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs index c5c3e3b153ab0..6276683e8fd18 100644 --- a/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs +++ b/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs @@ -93,7 +93,7 @@ impl DataCollection { let val = debugger_state_view.get_state_value(TOTAL_SUPPLY_STATE_KEY.deref()); assert!(val.is_ok() && val.unwrap().is_some()); AptosVMBlockExecutor::new() - .execute_block_no_limit(&sig_verified_txns, debugger_state_view) + .execute_block_no_limit(&sig_verified_txns, debugger_state_view, None, None) .map_err(|err| format_err!("Unexpected VM Error: {:?}", err)) } diff --git a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs index 9a18b17c35c96..03355bab525f6 100644 --- a/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs +++ b/aptos-move/aptos-transaction-benchmarks/src/transaction_bench_state.rs @@ -223,6 +223,8 @@ where &ModuleCacheManager::new(), BlockExecutorConfig::new_maybe_block_limit(1, maybe_block_gas_limit), None, + None, + None, ) .expect("VM should not fail to start") .into_transaction_outputs_forced(); @@ -275,6 +277,8 @@ where maybe_block_gas_limit, ), None, + None, + None, ) .expect("VM should not fail to start") .into_transaction_outputs_forced(); diff --git a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs index aff52ca04c4e9..811a000248bc5 100644 --- a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs +++ b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs @@ -515,14 +515,12 @@ impl<'a> AptosTestAdapter<'a> { fn run_transaction(&mut self, txn: Transaction) -> Result { let txn_block = vec![txn]; let sig_verified_block = into_signature_verified_block(txn_block); - - let executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - - let mut outputs = - executor.execute_block_no_limit(&sig_verified_block, &self.storage.clone())?; + let mut outputs = AptosVMBlockExecutor::new().execute_block_no_limit( + &sig_verified_block, + &self.storage.clone(), + None, + None, + )?; assert_eq!(outputs.len(), 1); diff --git a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs index 02a5ed3c37a8c..282156b481881 100644 --- a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs +++ b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs @@ -48,7 +48,8 @@ fn main() -> Result<()> { }) .collect(); - let outputs = AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store)?; + let outputs = + AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store, None, None)?; for i in 0..NUM_TXNS { assert!(outputs[i as usize].status().status().unwrap().is_success()); } diff --git a/aptos-move/aptos-vm/src/aptos_vm.rs b/aptos-move/aptos-vm/src/aptos_vm.rs index 30305923add99..d50b5c5371047 100644 --- a/aptos-move/aptos-vm/src/aptos_vm.rs +++ b/aptos-move/aptos-vm/src/aptos_vm.rs @@ -2793,19 +2793,13 @@ impl VMBlockExecutor for AptosVMBlockExecutor { } } - fn module_cache_manager( - &self, - ) -> Option< - &ModuleCacheManager, - > { - Some(&self.module_cache_manager) - } - fn execute_block( &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), onchain_config: BlockExecutorConfigFromOnchain, + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result, VMStatus> { fail_point!("move_adapter::execute_block", |_| { Err(VMStatus::error( @@ -2837,6 +2831,8 @@ impl VMBlockExecutor for AptosVMBlockExecutor { }, onchain: onchain_config, }, + parent_block, + current_block, None, ); if ret.is_ok() { diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index 68877952e80ee..b19981e5aa937 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -45,10 +45,7 @@ use aptos_vm_types::{ output::VMOutput, resolver::ResourceGroupSize, }; -use move_binary_format::{ - errors::{Location, VMError}, - CompiledModule, -}; +use move_binary_format::{errors::VMError, CompiledModule}; use move_core_types::{ account_address::AccountAddress, ident_str, @@ -56,7 +53,7 @@ use move_core_types::{ value::MoveTypeLayout, vm_status::{StatusCode, VMStatus}, }; -use move_vm_runtime::{Module, ModuleStorage, WithRuntimeEnvironment}; +use move_vm_runtime::{Module, ModuleStorage}; use move_vm_types::delayed_values::delayed_field_id::DelayedFieldID; use once_cell::sync::{Lazy, OnceCell}; use std::{ @@ -420,6 +417,8 @@ impl BlockAptosVM { AptosModuleExtension, >, config: BlockExecutorConfig, + parent_block: Option<&HashValue>, + current_block: Option, transaction_commit_listener: Option, ) -> Result, VMStatus> { let _timer = BLOCK_EXECUTOR_EXECUTE_BLOCK_SECONDS.start_timer(); @@ -433,42 +432,19 @@ impl BlockAptosVM { BLOCK_EXECUTOR_CONCURRENCY.set(config.local.concurrency_level as i64); - let (environment, module_cache) = if module_cache_manager.mark_executing() { - let environment = module_cache_manager.get_or_initialize_environment(state_view); - let module_cache = module_cache_manager.module_cache(); - (environment, module_cache) - } else { - // Either we do not have global caches , in which case we can create new ones, or - // something went wrong, and we were not able to mark the state as executing. In - // this case, fallback to empty caches. Note that the alert should have been raised - // during marking. - let environment = - AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); - let module_cache = Arc::new(GlobalModuleCache::empty()); - (environment, module_cache) - }; - - // We should be checking different module cache configurations here. - let module_cache_config = &config.local.module_cache_config; - - // Check 1: struct re-indexing map is not too large. If it is, we flush the cache. Also, we - // need to flush modules because they store indices into re-indexing map. - let runtime_environment = environment.runtime_environment(); - let struct_name_index_map_size = runtime_environment - .struct_name_index_map_size() - .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; - if struct_name_index_map_size > module_cache_config.max_struct_name_index_map_num_entries { - module_cache.flush_unsync(); - runtime_environment.flush_struct_name_and_info_caches(); - } - - // Check 2: If the module cache is too big, flush it. - if module_cache.size_in_bytes() > module_cache_config.max_module_cache_size_in_bytes { - module_cache.flush_unsync(); + if !module_cache_manager.mark_ready(parent_block, current_block) { + return Err(VMStatus::error( + StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some("Unable to mark module caches for block execution as ready".to_string()), + )); } + let (environment, module_cache) = module_cache_manager + .check_ready_and_get_caches(state_view, &config.local.module_cache_config)?; // Finally, to avoid cold starts, fetch the framework code prior to block execution. - if module_cache.num_modules() == 0 && module_cache_config.prefetch_framework_code { + if module_cache.num_modules() == 0 + && config.local.module_cache_config.prefetch_framework_code + { let code_storage = state_view.as_aptos_code_storage(environment.clone()); prefetch_aptos_framework(code_storage, &module_cache).map_err(|err| { alert!("Failed to load Aptos framework to module cache: {:?}", err); @@ -489,6 +465,12 @@ impl BlockAptosVM { transaction_commit_listener, ); + if !module_cache_manager.mark_executing() { + return Err(VMStatus::error( + StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some("Unable to mark block execution start".to_string()), + )); + } let ret = executor.execute_block(environment, signature_verified_block, state_view); if !module_cache_manager.mark_done() { return Err(VMStatus::error( @@ -542,6 +524,8 @@ impl BlockAptosVM { AptosModuleExtension, >, config: BlockExecutorConfig, + parent_block: Option<&HashValue>, + current_block: Option, transaction_commit_listener: Option, ) -> Result, VMStatus> { Self::execute_block_on_thread_pool::( @@ -550,6 +534,8 @@ impl BlockAptosVM { state_view, module_cache_manager, config, + parent_block, + current_block, transaction_commit_listener, ) } diff --git a/aptos-move/aptos-vm/src/lib.rs b/aptos-move/aptos-vm/src/lib.rs index 977287304b10c..34b1c8085d1bf 100644 --- a/aptos-move/aptos-vm/src/lib.rs +++ b/aptos-move/aptos-vm/src/lib.rs @@ -126,7 +126,6 @@ pub mod verifier; pub use crate::aptos_vm::{AptosSimulationVM, AptosVM}; use crate::sharded_block_executor::{executor_client::ExecutorClient, ShardedBlockExecutor}; -use aptos_block_executor::code_cache_global_manager::ModuleCacheManager; use aptos_crypto::HashValue; use aptos_types::{ block_executor::{ @@ -137,13 +136,9 @@ use aptos_types::{ signature_verified_transaction::SignatureVerifiedTransaction, BlockOutput, SignedTransaction, TransactionOutput, VMValidatorResult, }, - vm::modules::AptosModuleExtension, vm_status::VMStatus, }; use aptos_vm_types::module_and_script_storage::code_storage::AptosCodeStorage; -use move_binary_format::CompiledModule; -use move_core_types::language_storage::ModuleId; -use move_vm_runtime::Module; use std::{marker::Sync, sync::Arc}; pub use verifier::view_function::determine_is_view; @@ -166,22 +161,14 @@ pub trait VMBlockExecutor: Send + Sync { /// an old one. fn new() -> Self; - /// Returns the cache manager responsible for keeping module caches in sync. By default, is - /// [None]. - fn module_cache_manager( - &self, - ) -> Option< - &ModuleCacheManager, - > { - None - } - /// Executes a block of transactions and returns output for each one of them. fn execute_block( &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), onchain_config: BlockExecutorConfigFromOnchain, + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result, VMStatus>; /// Executes a block of transactions and returns output for each one of them, without applying @@ -190,11 +177,15 @@ pub trait VMBlockExecutor: Send + Sync { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result, VMStatus> { self.execute_block( transactions, state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), + parent_block, + current_block, ) .map(BlockOutput::into_transaction_outputs_forced) } diff --git a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs index d6b09906a50a2..4c52ba947339f 100644 --- a/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs +++ b/aptos-move/aptos-vm/src/sharded_block_executor/sharded_executor_service.rs @@ -136,17 +136,16 @@ impl ShardedExecutorService { ); }); s.spawn(move |_| { - // Since we execute blocks in parallel, we cannot share module caches, so each - // thread has its own caches. - let module_cache_manager = ModuleCacheManager::new(); - module_cache_manager.mark_ready(None, None); - let ret = BlockAptosVM::execute_block_on_thread_pool( executor_thread_pool, &signature_verified_transactions, aggr_overridden_state_view.as_ref(), - &module_cache_manager, + // Since we execute blocks in parallel, we cannot share module caches, so each + // thread has its own caches. + &ModuleCacheManager::new(), config, + None, + None, cross_shard_commit_sender, ) .map(BlockOutput::into_transaction_outputs_forced); diff --git a/aptos-move/aptos-vm/tests/sharded_block_executor.rs b/aptos-move/aptos-vm/tests/sharded_block_executor.rs index 1dcfbea2dd0f9..bbe6862f99163 100644 --- a/aptos-move/aptos-vm/tests/sharded_block_executor.rs +++ b/aptos-move/aptos-vm/tests/sharded_block_executor.rs @@ -308,12 +308,8 @@ mod test_utils { .into_iter() .map(|t| t.into_txn()) .collect(); - let block_executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = block_executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - let unsharded_txn_output = block_executor - .execute_block_no_limit(&ordered_txns, executor.data_store()) + let unsharded_txn_output = AptosVMBlockExecutor::new() + .execute_block_no_limit(&ordered_txns, executor.data_store(), None, None) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); } @@ -362,12 +358,8 @@ mod test_utils { ) .unwrap(); - let block_executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = block_executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - let unsharded_txn_output = block_executor - .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) + let unsharded_txn_output = AptosVMBlockExecutor::new() + .execute_block_no_limit(&execution_ordered_txns, executor.data_store(), None, None) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); } @@ -420,12 +412,8 @@ mod test_utils { ) .unwrap(); - let block_executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = block_executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - let unsharded_txn_output = block_executor - .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) + let unsharded_txn_output = AptosVMBlockExecutor::new() + .execute_block_no_limit(&execution_ordered_txns, executor.data_store(), None, None) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); } diff --git a/aptos-move/block-executor/Cargo.toml b/aptos-move/block-executor/Cargo.toml index 958c09a5787c2..a1aefe7441e45 100644 --- a/aptos-move/block-executor/Cargo.toml +++ b/aptos-move/block-executor/Cargo.toml @@ -55,6 +55,7 @@ aptos-types = { workspace = true, features = ["testing"] } criterion = { workspace = true } fail = { workspace = true, features = ["failpoints"] } itertools = { workspace = true } +move-vm-runtime = { workspace = true, features = ["testing"] } move-vm-types = { workspace = true, features = ["testing"] } proptest = { workspace = true } proptest-derive = { workspace = true } diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs index 5793feab4c8b0..380462a482035 100644 --- a/aptos-move/block-executor/src/code_cache_global_manager.rs +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -2,8 +2,12 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{code_cache_global::GlobalModuleCache, explicit_sync_wrapper::ExplicitSyncWrapper}; -use aptos_types::state_store::StateView; +use aptos_types::{ + block_executor::config::BlockExecutorModuleCacheLocalConfig, state_store::StateView, +}; use aptos_vm_environment::environment::AptosEnvironment; +use move_binary_format::errors::Location; +use move_core_types::vm_status::{StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, VMStatus}; use move_vm_runtime::WithRuntimeEnvironment; use move_vm_types::code::WithSize; use parking_lot::Mutex; @@ -97,10 +101,8 @@ where environment .runtime_environment() .flush_struct_name_and_info_caches(); - self.module_cache.flush_unsync(); - } else { - debug_assert!(self.module_cache.num_modules() == 0); } + self.module_cache.flush_unsync(); } *state = State::Ready(current); @@ -117,6 +119,52 @@ where } } + /// When in [State::Ready], runs different checks on cached modules and environment: + /// 1. If the environment is not initialized, or is different from the one in storage, it is + /// re-initialized, and module caches are flushed. + /// 2. If too many struct names have been cached in re-indexing map in runtime environment, + /// struct type caches and module caches are flushed. + /// 3. If module cache size is too large (in bytes), it is flushed. + /// The final environment and module caches are returned. + pub fn check_ready_and_get_caches( + &self, + state_view: &impl StateView, + config: &BlockExecutorModuleCacheLocalConfig, + ) -> Result<(AptosEnvironment, Arc>), VMStatus> { + let state = self.state.lock(); + if !matches!(state.deref(), State::Ready(_)) { + let msg = format!( + "Expected ready state to check caches, got {:?}", + state.deref() + ); + return Err(VMStatus::error( + UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some(msg), + )); + } + + let environment = self.get_or_initialize_environment(state_view); + let module_cache = self.module_cache.clone(); + + // Check 1: struct re-indexing map is not too large. If it is, we flush the cache. Also, we + // need to flush modules because they store indices into re-indexing map. + let runtime_environment = environment.runtime_environment(); + let struct_name_index_map_size = runtime_environment + .struct_name_index_map_size() + .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; + if struct_name_index_map_size > config.max_struct_name_index_map_num_entries { + module_cache.flush_unsync(); + runtime_environment.flush_struct_name_and_info_caches(); + } + + // Check 2: If the module cache is too big, flush it. + if module_cache.size_in_bytes() > config.max_module_cache_size_in_bytes { + module_cache.flush_unsync(); + } + + Ok((environment, module_cache)) + } + /// If state is [State::Ready], changes it to [State::Executing] with the same value, returning /// true. Otherwise, returns false indicating that state transition failed, also raising an /// alert. @@ -147,10 +195,8 @@ where /// Returns the cached global environment if it already exists, and matches the one in storage. /// If it does not exist, or does not match, the new environment is initialized from the given - /// state, cached, and returned. - pub fn get_or_initialize_environment(&self, state_view: &impl StateView) -> AptosEnvironment { - let _lock = self.state.lock(); - + /// state, cached, and returned. Should be called when in [State::Ready] state, under lock. + fn get_or_initialize_environment(&self, state_view: &impl StateView) -> AptosEnvironment { let new_environment = AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); @@ -159,7 +205,7 @@ where let environment_requires_update = existing_environment .as_ref() - .map_or(true, |environment| environment == &new_environment); + .map_or(true, |environment| environment != &new_environment); if environment_requires_update { *existing_environment = Some(new_environment); @@ -172,11 +218,6 @@ where .clone() .expect("Environment must be set") } - - /// Returns the global module cache. - pub fn module_cache(&self) -> Arc> { - self.module_cache.clone() - } } #[cfg(test)] @@ -186,8 +227,13 @@ mod test { on_chain_config::{FeatureFlag, Features, OnChainConfig}, state_store::{state_key::StateKey, state_value::StateValue, MockStateView}, }; - use move_vm_types::code::{ - mock_verified_code, MockDeserializedCode, MockExtension, MockVerifiedCode, + use claims::assert_ok; + use move_core_types::{ + account_address::AccountAddress, identifier::Identifier, language_storage::ModuleId, + }; + use move_vm_types::{ + code::{mock_verified_code, MockDeserializedCode, MockExtension, MockVerifiedCode}, + loaded_data::runtime_types::StructIdentifier, }; use std::{collections::HashMap, thread, thread::JoinHandle}; use test_case::test_case; @@ -209,7 +255,6 @@ mod test { assert!(!module_cache_manager.mark_executing()); assert!(!module_cache_manager.mark_done()); - assert!(module_cache_manager.mark_ready(previous.as_ref(), Some(77))); // Only in matching case the module cache is not flushed. @@ -223,11 +268,105 @@ mod test { assert_eq!(state, State::Ready(Some(77))) } + #[test] + fn test_check_ready() { + let state_view = MockStateView::empty(); + let config = BlockExecutorModuleCacheLocalConfig { + prefetch_framework_code: false, + max_module_cache_size_in_bytes: 8, + max_struct_name_index_map_num_entries: 2, + }; + + let module_cache_manager = ModuleCacheManager::< + i32, + i32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new(); + + // Set up the state and the environment. + *module_cache_manager.state.lock() = State::Ready(None); + let environment = module_cache_manager.get_or_initialize_environment(&state_view); + + module_cache_manager + .module_cache + .insert(0, mock_verified_code(0, MockExtension::new(16))); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + + let runtime_environment = environment.runtime_environment(); + let dummy_struct_name = StructIdentifier { + module: ModuleId::new(AccountAddress::ONE, Identifier::new("foo").unwrap()), + name: Identifier::new("Bar").unwrap(), + }; + assert!(runtime_environment + .struct_name_to_idx_for_test(dummy_struct_name) + .is_ok()); + assert_eq!( + assert_ok!(runtime_environment.struct_name_index_map_size()), + 1 + ); + + // Module cache size in bytes is too large, should be flushed (but not struct types). + assert!(module_cache_manager + .check_ready_and_get_caches(&state_view, &config) + .is_ok()); + assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!( + assert_ok!(runtime_environment.struct_name_index_map_size()), + 1 + ); + + module_cache_manager + .module_cache + .insert(0, mock_verified_code(0, MockExtension::new(4))); + + // This time size is less than the one specified in config. No flushing. + assert!(module_cache_manager + .check_ready_and_get_caches(&state_view, &config) + .is_ok()); + assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!( + assert_ok!(runtime_environment.struct_name_index_map_size()), + 1 + ); + + let dummy_struct_names = [ + StructIdentifier { + module: ModuleId::new(AccountAddress::ONE, Identifier::new("foo").unwrap()), + name: Identifier::new("Foo").unwrap(), + }, + StructIdentifier { + module: ModuleId::new(AccountAddress::ONE, Identifier::new("foo").unwrap()), + name: Identifier::new("Baz").unwrap(), + }, + ]; + for dummy_struct_name in dummy_struct_names { + assert!(runtime_environment + .struct_name_to_idx_for_test(dummy_struct_name) + .is_ok()); + } + assert_eq!( + assert_ok!(runtime_environment.struct_name_index_map_size()), + 3 + ); + + // Too many struct names cached. + assert!(module_cache_manager + .check_ready_and_get_caches(&state_view, &config) + .is_ok()); + assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!( + assert_ok!(runtime_environment.struct_name_index_map_size()), + 0 + ); + } + #[test] fn test_mark_executing() { let module_cache_manager = ModuleCacheManager::< - _, - u32, + i32, + i32, MockDeserializedCode, MockVerifiedCode, MockExtension, @@ -246,8 +385,8 @@ mod test { #[test] fn test_mark_done() { let module_cache_manager = ModuleCacheManager::< - _, - u32, + i32, + i32, MockDeserializedCode, MockVerifiedCode, MockExtension, @@ -283,8 +422,8 @@ mod test { #[test] fn test_mark_ready_concurrent() { let global_cache_manager = Arc::new(ModuleCacheManager::< - _, - u32, + i32, + i32, MockDeserializedCode, MockVerifiedCode, MockExtension, @@ -304,8 +443,8 @@ mod test { #[test] fn test_mark_executing_concurrent() { let global_cache_manager = Arc::new(ModuleCacheManager::< - _, - u32, + i32, + i32, MockDeserializedCode, MockVerifiedCode, MockExtension, @@ -326,8 +465,8 @@ mod test { #[test] fn test_mark_done_concurrent() { let global_cache_manager = Arc::new(ModuleCacheManager::< - _, - u32, + i32, + i32, MockDeserializedCode, MockVerifiedCode, MockExtension, @@ -368,7 +507,14 @@ mod test { #[test] fn test_get_or_initialize_environment() { - let module_cache_manager = ModuleCacheManager::::new(); + let module_cache_manager = ModuleCacheManager::< + i32, + i32, + MockDeserializedCode, + MockVerifiedCode, + MockExtension, + >::new(); + *module_cache_manager.state.lock() = State::Ready(None); module_cache_manager .module_cache diff --git a/aptos-move/e2e-tests/src/executor.rs b/aptos-move/e2e-tests/src/executor.rs index cff14879c0c52..da4598abf98c5 100644 --- a/aptos-move/e2e-tests/src/executor.rs +++ b/aptos-move/e2e-tests/src/executor.rs @@ -640,11 +640,6 @@ impl FakeExecutor { }, onchain: onchain_config, }; - - // Do not use shared module caches in tests. - let module_cache_manager = ModuleCacheManager::new(); - module_cache_manager.mark_ready(None, None); - BlockAptosVM::execute_block_on_thread_pool::< _, NoOpTransactionCommitHook, @@ -652,9 +647,12 @@ impl FakeExecutor { self.executor_thread_pool.clone(), txn_block, &state_view, - &module_cache_manager, + // Do not use shared module caches in tests. + &ModuleCacheManager::new(), config, None, + None, + None, ) .map(BlockOutput::into_transaction_outputs_forced) } diff --git a/execution/executor-benchmark/src/native_executor.rs b/execution/executor-benchmark/src/native_executor.rs index f90eb1498f79e..38e1b08b4c393 100644 --- a/execution/executor-benchmark/src/native_executor.rs +++ b/execution/executor-benchmark/src/native_executor.rs @@ -6,6 +6,7 @@ use crate::{ metrics::TIMER, }; use anyhow::Result; +use aptos_crypto::HashValue; use aptos_types::{ account_address::AccountAddress, account_config::{DepositEvent, WithdrawEvent}, @@ -359,6 +360,8 @@ impl VMBlockExecutor for NativeExecutor { transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), _onchain_config: BlockExecutorConfigFromOnchain, + _parent_block: Option<&HashValue>, + _current_block: Option, ) -> Result, VMStatus> { let transaction_outputs = NATIVE_EXECUTOR_POOL .install(|| { diff --git a/execution/executor-service/src/test_utils.rs b/execution/executor-service/src/test_utils.rs index bfd2c36cd1776..15e7e3ea10761 100644 --- a/execution/executor-service/src/test_utils.rs +++ b/execution/executor-service/src/test_utils.rs @@ -137,12 +137,8 @@ pub fn test_sharded_block_executor_no_conflict> .into_iter() .map(|t| t.into_txn()) .collect(); - let block_executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = block_executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - let unsharded_txn_output = block_executor - .execute_block_no_limit(&txns, executor.data_store()) + let unsharded_txn_output = AptosVMBlockExecutor::new() + .execute_block_no_limit(&txns, executor.data_store(), None, None) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); sharded_block_executor.shutdown(); @@ -196,12 +192,8 @@ pub fn sharded_block_executor_with_conflict>( ) .unwrap(); - let block_executor = AptosVMBlockExecutor::new(); - if let Some(module_cache_manager) = block_executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - let unsharded_txn_output = block_executor - .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) + let unsharded_txn_output = AptosVMBlockExecutor::new() + .execute_block_no_limit(&execution_ordered_txns, executor.data_store(), None, None) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); sharded_block_executor.shutdown(); diff --git a/execution/executor/src/block_executor/mod.rs b/execution/executor/src/block_executor/mod.rs index 1595de3eeb4ba..45738dd6ca6a5 100644 --- a/execution/executor/src/block_executor/mod.rs +++ b/execution/executor/src/block_executor/mod.rs @@ -198,77 +198,64 @@ where "execute_block" ); let committed_block_id = self.committed_block_id(); - let (execution_output, state_checkpoint_output) = if parent_block_id != committed_block_id - && parent_output.has_reconfiguration() - { - // ignore reconfiguration suffix, even if the block is non-empty - info!( - LogSchema::new(LogEntry::BlockExecutor).block_id(block_id), - "reconfig_descendant_block_received" - ); - ( - parent_output.execution_output.reconfig_suffix(), - parent_output - .expect_state_checkpoint_output() - .reconfig_suffix(), - ) - } else { - let state_view = { - let _timer = OTHER_TIMERS.timer_with(&["verified_state_view"]); - - CachedStateView::new( - StateViewId::BlockExecution { block_id }, - Arc::clone(&self.db.reader), - parent_output.execution_output.next_version(), - parent_output.expect_result_state().current.clone(), - Arc::new(AsyncProofFetcher::new(self.db.reader.clone())), - )? - }; - - let execution_output = { - let _timer = GET_BLOCK_EXECUTION_OUTPUT_BY_EXECUTING.start_timer(); - fail_point!("executor::block_executor_execute_block", |_| { - Err(ExecutorError::from(anyhow::anyhow!( - "Injected error in block_executor_execute_block" - ))) - }); - - // In case block executor has a cache manager, we need to mark it as ready for - // execution. If for some reason this fails, return an error. - if let Some(module_cache_manager) = self.block_executor.module_cache_manager() { - // TODO(loader_v2): - // Refactor to be able to move this into AptosVM block executor. This will - // also allow us to remove all ready markings in other places. - if !module_cache_manager.mark_ready(Some(&parent_block_id), Some(block_id)) { - return Err(ExecutorError::internal_err( - "Unable to mark module cache manager as ready", - )); - } - } - - DoGetExecutionOutput::by_transaction_execution( - &self.block_executor, - transactions, - state_view, - onchain_config.clone(), - Some(block_id), - )? - }; - - let _timer = OTHER_TIMERS.timer_with(&["state_checkpoint"]); - - let state_checkpoint_output = THREAD_MANAGER.get_exe_cpu_pool().install(|| { - fail_point!("executor::block_state_checkpoint", |_| { - Err(anyhow::anyhow!("Injected error in block state checkpoint.")) - }); - DoStateCheckpoint::run( - &execution_output, - parent_output.expect_result_state(), - Option::>::None, + let (execution_output, state_checkpoint_output) = + if parent_block_id != committed_block_id && parent_output.has_reconfiguration() { + // ignore reconfiguration suffix, even if the block is non-empty + info!( + LogSchema::new(LogEntry::BlockExecutor).block_id(block_id), + "reconfig_descendant_block_received" + ); + ( + parent_output.execution_output.reconfig_suffix(), + parent_output + .expect_state_checkpoint_output() + .reconfig_suffix(), ) - })?; - (execution_output, state_checkpoint_output) - }; + } else { + let state_view = { + let _timer = OTHER_TIMERS.timer_with(&["verified_state_view"]); + + CachedStateView::new( + StateViewId::BlockExecution { block_id }, + Arc::clone(&self.db.reader), + parent_output.execution_output.next_version(), + parent_output.expect_result_state().current.clone(), + Arc::new(AsyncProofFetcher::new(self.db.reader.clone())), + )? + }; + + let execution_output = { + let _timer = GET_BLOCK_EXECUTION_OUTPUT_BY_EXECUTING.start_timer(); + fail_point!("executor::block_executor_execute_block", |_| { + Err(ExecutorError::from(anyhow::anyhow!( + "Injected error in block_executor_execute_block" + ))) + }); + + DoGetExecutionOutput::by_transaction_execution( + &self.block_executor, + transactions, + state_view, + onchain_config.clone(), + Some(&parent_block_id), + Some(block_id), + )? + }; + + let _timer = OTHER_TIMERS.timer_with(&["state_checkpoint"]); + + let state_checkpoint_output = THREAD_MANAGER.get_exe_cpu_pool().install(|| { + fail_point!("executor::block_state_checkpoint", |_| { + Err(anyhow::anyhow!("Injected error in block state checkpoint.")) + }); + DoStateCheckpoint::run( + &execution_output, + parent_output.expect_result_state(), + Option::>::None, + ) + })?; + (execution_output, state_checkpoint_output) + }; let output = PartialStateComputeResult::new(execution_output); output.set_state_checkpoint_output(state_checkpoint_output); diff --git a/execution/executor/src/chunk_executor/mod.rs b/execution/executor/src/chunk_executor/mod.rs index c97398fae868e..55c8e005ede28 100644 --- a/execution/executor/src/chunk_executor/mod.rs +++ b/execution/executor/src/chunk_executor/mod.rs @@ -597,19 +597,14 @@ impl ChunkExecutorInner { .map(|t| t.into()) .collect::>(); - // For now, we create executor for each chunk. - let executor = V::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - // State sync executor shouldn't have block gas limit. let execution_output = DoGetExecutionOutput::by_transaction_execution::( - &executor, + &V::new(), txns.into(), state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), None, + None, )?; // not `zip_eq`, deliberately for (version, txn_out, txn_info, write_set, events) in multizip(( diff --git a/execution/executor/src/chunk_executor/transaction_chunk.rs b/execution/executor/src/chunk_executor/transaction_chunk.rs index a51ec0f5c9bf8..46d84d4fc1ae1 100644 --- a/execution/executor/src/chunk_executor/transaction_chunk.rs +++ b/execution/executor/src/chunk_executor/transaction_chunk.rs @@ -83,18 +83,13 @@ impl TransactionChunk for ChunkToExecute { }; let _timer = VM_EXECUTE_CHUNK.start_timer(); - - let executor = V::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - DoGetExecutionOutput::by_transaction_execution::( - &executor, + &V::new(), sig_verified_txns.into(), state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), None, + None, ) } } diff --git a/execution/executor/src/db_bootstrapper/mod.rs b/execution/executor/src/db_bootstrapper/mod.rs index d9dcd513fe82d..ccb6bc6195014 100644 --- a/execution/executor/src/db_bootstrapper/mod.rs +++ b/execution/executor/src/db_bootstrapper/mod.rs @@ -131,17 +131,13 @@ pub fn calculate_genesis( get_state_epoch(&base_state_view)? }; - let executor = V::new(); - if let Some(module_cache_manager) = executor.module_cache_manager() { - module_cache_manager.mark_ready(None, None); - } - let execution_output = DoGetExecutionOutput::by_transaction_execution::( - &executor, + &V::new(), vec![genesis_txn.clone().into()].into(), base_state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), None, + None, )?; ensure!( execution_output.num_transactions_to_commit() != 0, diff --git a/execution/executor/src/fuzzing.rs b/execution/executor/src/fuzzing.rs index 67639b5831107..d73e272557dab 100644 --- a/execution/executor/src/fuzzing.rs +++ b/execution/executor/src/fuzzing.rs @@ -79,6 +79,8 @@ impl VMBlockExecutor for FakeVM { _transactions: &[SignatureVerifiedTransaction], _state_view: &impl StateView, _onchain_config: BlockExecutorConfigFromOnchain, + _parent_block: Option<&HashValue>, + _current_block: Option, ) -> Result, VMStatus> { Ok(BlockOutput::new(vec![], None)) } diff --git a/execution/executor/src/tests/mock_vm/mock_vm_test.rs b/execution/executor/src/tests/mock_vm/mock_vm_test.rs index 4df0ef06d0665..427897e3c9e52 100644 --- a/execution/executor/src/tests/mock_vm/mock_vm_test.rs +++ b/execution/executor/src/tests/mock_vm/mock_vm_test.rs @@ -29,6 +29,8 @@ fn test_mock_vm_different_senders() { .execute_block_no_limit( &into_signature_verified_block(txns.clone()), &MockStateView::empty(), + None, + None, ) .expect("MockVM should not fail to start"); @@ -69,6 +71,8 @@ fn test_mock_vm_same_sender() { .execute_block_no_limit( &into_signature_verified_block(txns), &MockStateView::empty(), + None, + None, ) .expect("MockVM should not fail to start"); @@ -107,6 +111,8 @@ fn test_mock_vm_payment() { .execute_block_no_limit( &into_signature_verified_block(txns), &MockStateView::empty(), + None, + None, ) .expect("MockVM should not fail to start"); diff --git a/execution/executor/src/tests/mock_vm/mod.rs b/execution/executor/src/tests/mock_vm/mod.rs index bb9ea70a99393..701c2e1d332fc 100644 --- a/execution/executor/src/tests/mock_vm/mod.rs +++ b/execution/executor/src/tests/mock_vm/mod.rs @@ -6,7 +6,7 @@ mod mock_vm_test; use anyhow::Result; -use aptos_crypto::{ed25519::Ed25519PrivateKey, PrivateKey, Uniform}; +use aptos_crypto::{ed25519::Ed25519PrivateKey, HashValue, PrivateKey, Uniform}; use aptos_types::{ account_address::AccountAddress, account_config::NEW_EPOCH_EVENT_V2_MOVE_TYPE_TAG, @@ -67,6 +67,8 @@ impl VMBlockExecutor for MockVM { transactions: &[SignatureVerifiedTransaction], state_view: &impl StateView, _onchain_config: BlockExecutorConfigFromOnchain, + _parent_block: Option<&HashValue>, + _current_block: Option, ) -> Result, VMStatus> { // output_cache is used to store the output of transactions so they are visible to later // transactions. diff --git a/execution/executor/src/tests/mod.rs b/execution/executor/src/tests/mod.rs index 8ca28af1a83ff..366c891ff1760 100644 --- a/execution/executor/src/tests/mod.rs +++ b/execution/executor/src/tests/mod.rs @@ -690,6 +690,7 @@ fn run_transactions_naive( .unwrap(), block_executor_onchain_config.clone(), None, + None, ) .unwrap(); let output = ApplyExecutionOutput::run(out, &ledger_view).unwrap(); diff --git a/execution/executor/src/workflow/do_get_execution_output.rs b/execution/executor/src/workflow/do_get_execution_output.rs index 4fad63d2e8261..5e7987b95098f 100644 --- a/execution/executor/src/workflow/do_get_execution_output.rs +++ b/execution/executor/src/workflow/do_get_execution_output.rs @@ -49,7 +49,8 @@ impl DoGetExecutionOutput { transactions: ExecutableTransactions, state_view: CachedStateView, onchain_config: BlockExecutorConfigFromOnchain, - append_state_checkpoint_to_block: Option, + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result { let out = match transactions { ExecutableTransactions::Unsharded(txns) => { @@ -58,14 +59,15 @@ impl DoGetExecutionOutput { txns, state_view, onchain_config, - append_state_checkpoint_to_block, + parent_block, + current_block, )? }, ExecutableTransactions::Sharded(txns) => Self::by_transaction_execution_sharded::( txns, state_view, onchain_config, - append_state_checkpoint_to_block, + current_block, )?, }; @@ -89,10 +91,17 @@ impl DoGetExecutionOutput { transactions: Vec, state_view: CachedStateView, onchain_config: BlockExecutorConfigFromOnchain, - append_state_checkpoint_to_block: Option, + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result { - let block_output = - Self::execute_block::(executor, &transactions, &state_view, onchain_config)?; + let block_output = Self::execute_block::( + executor, + &transactions, + &state_view, + onchain_config, + parent_block, + current_block, + )?; let (transaction_outputs, block_end_info) = block_output.into_inner(); Parser::parse( @@ -101,7 +110,7 @@ impl DoGetExecutionOutput { transaction_outputs, state_view.into_state_cache(), block_end_info, - append_state_checkpoint_to_block, + current_block, ) } @@ -202,9 +211,17 @@ impl DoGetExecutionOutput { transactions: &[SignatureVerifiedTransaction], state_view: &CachedStateView, onchain_config: BlockExecutorConfigFromOnchain, + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result> { let _timer = OTHER_TIMERS.timer_with(&["vm_execute_block"]); - Ok(executor.execute_block(transactions, state_view, onchain_config)?) + Ok(executor.execute_block( + transactions, + state_view, + onchain_config, + parent_block, + current_block, + )?) } /// In consensus-only mode, executes the block of [Transaction]s using the @@ -217,6 +234,8 @@ impl DoGetExecutionOutput { transactions: &[SignatureVerifiedTransaction], state_view: &CachedStateView, onchain_config: BlockExecutorConfigFromOnchain, + parent_block: Option<&HashValue>, + current_block: Option, ) -> Result> { use aptos_types::{ state_store::{StateViewId, TStateView}, @@ -226,9 +245,13 @@ impl DoGetExecutionOutput { let transaction_outputs = match state_view.id() { // this state view ID implies a genesis block in non-test cases. - StateViewId::Miscellaneous => { - executor.execute_block(transactions, state_view, onchain_config)? - }, + StateViewId::Miscellaneous => executor.execute_block( + transactions, + state_view, + onchain_config, + parent_block, + current_block, + )?, _ => BlockOutput::new( transactions .iter() diff --git a/experimental/execution/ptx-executor/Cargo.toml b/experimental/execution/ptx-executor/Cargo.toml index 0da896d31500a..4ed757a8bba81 100644 --- a/experimental/execution/ptx-executor/Cargo.toml +++ b/experimental/execution/ptx-executor/Cargo.toml @@ -13,6 +13,7 @@ repository = { workspace = true } rust-version = { workspace = true } [dependencies] +aptos-crypto = { workspace = true } aptos-experimental-runtimes = { workspace = true } aptos-infallible = { workspace = true } aptos-logger = { workspace = true } diff --git a/experimental/execution/ptx-executor/src/lib.rs b/experimental/execution/ptx-executor/src/lib.rs index 9ed83b7d7a3f6..649c0651af63a 100644 --- a/experimental/execution/ptx-executor/src/lib.rs +++ b/experimental/execution/ptx-executor/src/lib.rs @@ -21,6 +21,7 @@ use crate::{ analyzer::PtxAnalyzer, finalizer::PtxFinalizer, metrics::TIMER, runner::PtxRunner, scheduler::PtxScheduler, sorter::PtxSorter, state_reader::PtxStateReader, }; +use aptos_crypto::HashValue; use aptos_experimental_runtimes::thread_manager::THREAD_MANAGER; use aptos_infallible::Mutex; use aptos_metrics_core::TimerHelper; @@ -53,6 +54,8 @@ impl VMBlockExecutor for PtxBlockExecutor { transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), _onchain_config: BlockExecutorConfigFromOnchain, + _parent_block: Option<&HashValue>, + _current_block: Option, ) -> Result, VMStatus> { let _timer = TIMER.timer_with(&["block_total"]); diff --git a/storage/db-tool/src/replay_on_archive.rs b/storage/db-tool/src/replay_on_archive.rs index 5bc13308bee98..a5df5787a20f2 100644 --- a/storage/db-tool/src/replay_on_archive.rs +++ b/storage/db-tool/src/replay_on_archive.rs @@ -300,6 +300,8 @@ impl Verifier { &self .arc_db .state_view_at_version(start_version.checked_sub(1))?, + None, + None, )?; let mut failed_txns = Vec::new(); diff --git a/third_party/move/move-vm/runtime/src/storage/environment.rs b/third_party/move/move-vm/runtime/src/storage/environment.rs index 62d885c36a96a..bc152dd1cc06b 100644 --- a/third_party/move/move-vm/runtime/src/storage/environment.rs +++ b/third_party/move/move-vm/runtime/src/storage/environment.rs @@ -25,7 +25,10 @@ use move_core_types::{ identifier::{IdentStr, Identifier}, vm_status::{sub_status::unknown_invariant_violation::EPARANOID_FAILURE, StatusCode}, }; -use move_vm_types::sha3_256; +use move_vm_types::{ + loaded_data::runtime_types::{StructIdentifier, StructNameIndex}, + sha3_256, +}; use std::sync::Arc; /// [MoveVM] runtime environment encapsulating different configurations. Shared between the VM and @@ -284,6 +287,15 @@ impl RuntimeEnvironment { self.flush_struct_info_cache(); self.struct_name_index_map.flush(); } + + /// Test-only function to be able to populate [StructNameIndexMap] outside of this crate. + #[cfg(any(test, feature = "testing"))] + pub fn struct_name_to_idx_for_test( + &self, + struct_name: StructIdentifier, + ) -> PartialVMResult { + self.struct_name_index_map.struct_name_to_idx(struct_name) + } } impl Clone for RuntimeEnvironment { From f317dcfb71357097d00ff48a4ea77110b81ea692 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 08:09:59 +0000 Subject: [PATCH 15/25] [cleanup] Set execute_no_limit to always use Nones for block IDs --- .../aptos-e2e-comparison-testing/src/data_collection.rs | 2 +- .../src/aptos_test_harness.rs | 8 ++------ aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs | 3 +-- aptos-move/aptos-vm/src/lib.rs | 7 +++---- aptos-move/aptos-vm/tests/sharded_block_executor.rs | 6 +++--- execution/executor-service/src/test_utils.rs | 4 ++-- execution/executor/src/tests/mock_vm/mock_vm_test.rs | 6 ------ storage/db-tool/src/replay_on_archive.rs | 2 -- 8 files changed, 12 insertions(+), 26 deletions(-) diff --git a/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs b/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs index 6276683e8fd18..c5c3e3b153ab0 100644 --- a/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs +++ b/aptos-move/aptos-e2e-comparison-testing/src/data_collection.rs @@ -93,7 +93,7 @@ impl DataCollection { let val = debugger_state_view.get_state_value(TOTAL_SUPPLY_STATE_KEY.deref()); assert!(val.is_ok() && val.unwrap().is_some()); AptosVMBlockExecutor::new() - .execute_block_no_limit(&sig_verified_txns, debugger_state_view, None, None) + .execute_block_no_limit(&sig_verified_txns, debugger_state_view) .map_err(|err| format_err!("Unexpected VM Error: {:?}", err)) } diff --git a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs index 811a000248bc5..a96506db49b47 100644 --- a/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs +++ b/aptos-move/aptos-transactional-test-harness/src/aptos_test_harness.rs @@ -515,12 +515,8 @@ impl<'a> AptosTestAdapter<'a> { fn run_transaction(&mut self, txn: Transaction) -> Result { let txn_block = vec![txn]; let sig_verified_block = into_signature_verified_block(txn_block); - let mut outputs = AptosVMBlockExecutor::new().execute_block_no_limit( - &sig_verified_block, - &self.storage.clone(), - None, - None, - )?; + let mut outputs = AptosVMBlockExecutor::new() + .execute_block_no_limit(&sig_verified_block, &self.storage.clone())?; assert_eq!(outputs.len(), 1); diff --git a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs index 282156b481881..02a5ed3c37a8c 100644 --- a/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs +++ b/aptos-move/aptos-vm-profiling/src/bins/run_aptos_p2p.rs @@ -48,8 +48,7 @@ fn main() -> Result<()> { }) .collect(); - let outputs = - AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store, None, None)?; + let outputs = AptosVMBlockExecutor::new().execute_block_no_limit(&txns, &state_store)?; for i in 0..NUM_TXNS { assert!(outputs[i as usize].status().status().unwrap().is_success()); } diff --git a/aptos-move/aptos-vm/src/lib.rs b/aptos-move/aptos-vm/src/lib.rs index 34b1c8085d1bf..25f807a499f95 100644 --- a/aptos-move/aptos-vm/src/lib.rs +++ b/aptos-move/aptos-vm/src/lib.rs @@ -177,15 +177,14 @@ pub trait VMBlockExecutor: Send + Sync { &self, transactions: &[SignatureVerifiedTransaction], state_view: &(impl StateView + Sync), - parent_block: Option<&HashValue>, - current_block: Option, ) -> Result, VMStatus> { self.execute_block( transactions, state_view, BlockExecutorConfigFromOnchain::new_no_block_limit(), - parent_block, - current_block, + // For all use cases, we run on an unknown state. Hence, defaulting to None here. + None, + None, ) .map(BlockOutput::into_transaction_outputs_forced) } diff --git a/aptos-move/aptos-vm/tests/sharded_block_executor.rs b/aptos-move/aptos-vm/tests/sharded_block_executor.rs index bbe6862f99163..5968d0a495ab9 100644 --- a/aptos-move/aptos-vm/tests/sharded_block_executor.rs +++ b/aptos-move/aptos-vm/tests/sharded_block_executor.rs @@ -309,7 +309,7 @@ mod test_utils { .map(|t| t.into_txn()) .collect(); let unsharded_txn_output = AptosVMBlockExecutor::new() - .execute_block_no_limit(&ordered_txns, executor.data_store(), None, None) + .execute_block_no_limit(&ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); } @@ -359,7 +359,7 @@ mod test_utils { .unwrap(); let unsharded_txn_output = AptosVMBlockExecutor::new() - .execute_block_no_limit(&execution_ordered_txns, executor.data_store(), None, None) + .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); } @@ -413,7 +413,7 @@ mod test_utils { .unwrap(); let unsharded_txn_output = AptosVMBlockExecutor::new() - .execute_block_no_limit(&execution_ordered_txns, executor.data_store(), None, None) + .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); } diff --git a/execution/executor-service/src/test_utils.rs b/execution/executor-service/src/test_utils.rs index 15e7e3ea10761..a1d6e8673290d 100644 --- a/execution/executor-service/src/test_utils.rs +++ b/execution/executor-service/src/test_utils.rs @@ -138,7 +138,7 @@ pub fn test_sharded_block_executor_no_conflict> .map(|t| t.into_txn()) .collect(); let unsharded_txn_output = AptosVMBlockExecutor::new() - .execute_block_no_limit(&txns, executor.data_store(), None, None) + .execute_block_no_limit(&txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); sharded_block_executor.shutdown(); @@ -193,7 +193,7 @@ pub fn sharded_block_executor_with_conflict>( .unwrap(); let unsharded_txn_output = AptosVMBlockExecutor::new() - .execute_block_no_limit(&execution_ordered_txns, executor.data_store(), None, None) + .execute_block_no_limit(&execution_ordered_txns, executor.data_store()) .unwrap(); compare_txn_outputs(unsharded_txn_output, sharded_txn_output); sharded_block_executor.shutdown(); diff --git a/execution/executor/src/tests/mock_vm/mock_vm_test.rs b/execution/executor/src/tests/mock_vm/mock_vm_test.rs index 427897e3c9e52..4df0ef06d0665 100644 --- a/execution/executor/src/tests/mock_vm/mock_vm_test.rs +++ b/execution/executor/src/tests/mock_vm/mock_vm_test.rs @@ -29,8 +29,6 @@ fn test_mock_vm_different_senders() { .execute_block_no_limit( &into_signature_verified_block(txns.clone()), &MockStateView::empty(), - None, - None, ) .expect("MockVM should not fail to start"); @@ -71,8 +69,6 @@ fn test_mock_vm_same_sender() { .execute_block_no_limit( &into_signature_verified_block(txns), &MockStateView::empty(), - None, - None, ) .expect("MockVM should not fail to start"); @@ -111,8 +107,6 @@ fn test_mock_vm_payment() { .execute_block_no_limit( &into_signature_verified_block(txns), &MockStateView::empty(), - None, - None, ) .expect("MockVM should not fail to start"); diff --git a/storage/db-tool/src/replay_on_archive.rs b/storage/db-tool/src/replay_on_archive.rs index a5df5787a20f2..5bc13308bee98 100644 --- a/storage/db-tool/src/replay_on_archive.rs +++ b/storage/db-tool/src/replay_on_archive.rs @@ -300,8 +300,6 @@ impl Verifier { &self .arc_db .state_view_at_version(start_version.checked_sub(1))?, - None, - None, )?; let mut failed_txns = Vec::new(); From 03bb4dfa9142336c9364e233ab3500334c4ee434 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 09:21:13 +0000 Subject: [PATCH 16/25] Feature gating --- aptos-move/aptos-vm/src/block_executor/mod.rs | 31 +++++++++----- .../src/code_cache_global_manager.rs | 42 +++++++++++-------- 2 files changed, 44 insertions(+), 29 deletions(-) diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index b19981e5aa937..4b8bf88155b3c 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -432,17 +432,26 @@ impl BlockAptosVM { BLOCK_EXECUTOR_CONCURRENCY.set(config.local.concurrency_level as i64); - if !module_cache_manager.mark_ready(parent_block, current_block) { - return Err(VMStatus::error( - StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, - Some("Unable to mark module caches for block execution as ready".to_string()), - )); - } - let (environment, module_cache) = module_cache_manager - .check_ready_and_get_caches(state_view, &config.local.module_cache_config)?; + let environment = + AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); + let is_loader_v2_enabled = environment.features().is_loader_v2_enabled(); + + let (environment, module_cache) = if is_loader_v2_enabled { + if !module_cache_manager.mark_ready(parent_block, current_block) { + return Err(VMStatus::error( + StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, + Some("Unable to mark module caches for block execution as ready".to_string()), + )); + } + module_cache_manager + .check_ready_and_get_caches(environment, &config.local.module_cache_config)? + } else { + (environment, Arc::new(GlobalModuleCache::empty())) + }; // Finally, to avoid cold starts, fetch the framework code prior to block execution. - if module_cache.num_modules() == 0 + if is_loader_v2_enabled + && module_cache.num_modules() == 0 && config.local.module_cache_config.prefetch_framework_code { let code_storage = state_view.as_aptos_code_storage(environment.clone()); @@ -465,14 +474,14 @@ impl BlockAptosVM { transaction_commit_listener, ); - if !module_cache_manager.mark_executing() { + if is_loader_v2_enabled && !module_cache_manager.mark_executing() { return Err(VMStatus::error( StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, Some("Unable to mark block execution start".to_string()), )); } let ret = executor.execute_block(environment, signature_verified_block, state_view); - if !module_cache_manager.mark_done() { + if is_loader_v2_enabled && !module_cache_manager.mark_done() { return Err(VMStatus::error( StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, Some("Unable to mark block execution as done".to_string()), diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs index 380462a482035..e37187d36f0e7 100644 --- a/aptos-move/block-executor/src/code_cache_global_manager.rs +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -2,9 +2,7 @@ // SPDX-License-Identifier: Apache-2.0 use crate::{code_cache_global::GlobalModuleCache, explicit_sync_wrapper::ExplicitSyncWrapper}; -use aptos_types::{ - block_executor::config::BlockExecutorModuleCacheLocalConfig, state_store::StateView, -}; +use aptos_types::block_executor::config::BlockExecutorModuleCacheLocalConfig; use aptos_vm_environment::environment::AptosEnvironment; use move_binary_format::errors::Location; use move_core_types::vm_status::{StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, VMStatus}; @@ -128,7 +126,7 @@ where /// The final environment and module caches are returned. pub fn check_ready_and_get_caches( &self, - state_view: &impl StateView, + storage_environment: AptosEnvironment, config: &BlockExecutorModuleCacheLocalConfig, ) -> Result<(AptosEnvironment, Arc>), VMStatus> { let state = self.state.lock(); @@ -143,7 +141,7 @@ where )); } - let environment = self.get_or_initialize_environment(state_view); + let environment = self.get_or_initialize_environment(storage_environment); let module_cache = self.module_cache.clone(); // Check 1: struct re-indexing map is not too large. If it is, we flush the cache. Also, we @@ -196,18 +194,18 @@ where /// Returns the cached global environment if it already exists, and matches the one in storage. /// If it does not exist, or does not match, the new environment is initialized from the given /// state, cached, and returned. Should be called when in [State::Ready] state, under lock. - fn get_or_initialize_environment(&self, state_view: &impl StateView) -> AptosEnvironment { - let new_environment = - AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); - + fn get_or_initialize_environment( + &self, + storage_environment: AptosEnvironment, + ) -> AptosEnvironment { let mut guard = self.environment.acquire(); let existing_environment = guard.deref_mut(); let environment_requires_update = existing_environment .as_ref() - .map_or(true, |environment| environment != &new_environment); + .map_or(true, |environment| environment != &storage_environment); if environment_requires_update { - *existing_environment = Some(new_environment); + *existing_environment = Some(storage_environment); // If this environment has been (re-)initialized, we need to flush the module cache // because it can contain now out-dated code. @@ -287,7 +285,9 @@ mod test { // Set up the state and the environment. *module_cache_manager.state.lock() = State::Ready(None); - let environment = module_cache_manager.get_or_initialize_environment(&state_view); + let environment = module_cache_manager.get_or_initialize_environment( + AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), + ); module_cache_manager .module_cache @@ -309,7 +309,7 @@ mod test { // Module cache size in bytes is too large, should be flushed (but not struct types). assert!(module_cache_manager - .check_ready_and_get_caches(&state_view, &config) + .check_ready_and_get_caches(environment.clone(), &config) .is_ok()); assert_eq!(module_cache_manager.module_cache.num_modules(), 0); assert_eq!( @@ -323,7 +323,7 @@ mod test { // This time size is less than the one specified in config. No flushing. assert!(module_cache_manager - .check_ready_and_get_caches(&state_view, &config) + .check_ready_and_get_caches(environment.clone(), &config) .is_ok()); assert_eq!(module_cache_manager.module_cache.num_modules(), 1); assert_eq!( @@ -353,7 +353,7 @@ mod test { // Too many struct names cached. assert!(module_cache_manager - .check_ready_and_get_caches(&state_view, &config) + .check_ready_and_get_caches(environment.clone(), &config) .is_ok()); assert_eq!(module_cache_manager.module_cache.num_modules(), 0); assert_eq!( @@ -527,7 +527,9 @@ mod test { // Environment has to be set to the same value, cache flushed. let state_view = state_view_with_changed_feature_flag(None); - let environment = module_cache_manager.get_or_initialize_environment(&state_view); + let environment = module_cache_manager.get_or_initialize_environment( + AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), + ); assert_eq!(module_cache_manager.module_cache.num_modules(), 0); assert!(module_cache_manager .environment @@ -544,7 +546,9 @@ mod test { // Environment has to be re-set to the new value, cache flushed. let state_view = state_view_with_changed_feature_flag(Some(FeatureFlag::CODE_DEPENDENCY_CHECK)); - let environment = module_cache_manager.get_or_initialize_environment(&state_view); + let environment = module_cache_manager.get_or_initialize_environment( + AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), + ); assert_eq!(module_cache_manager.module_cache.num_modules(), 0); assert!(module_cache_manager .environment @@ -559,7 +563,9 @@ mod test { assert!(module_cache_manager.environment.acquire().is_some()); // Environment is kept, and module caches are not flushed. - let new_environment = module_cache_manager.get_or_initialize_environment(&state_view); + let new_environment = module_cache_manager.get_or_initialize_environment( + AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), + ); assert_eq!(module_cache_manager.module_cache.num_modules(), 1); assert!(environment == new_environment); } From 715f3a9acd7de9a14fc16e31822d0846b157a25b Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 13 Nov 2024 00:34:59 +0000 Subject: [PATCH 17/25] [comments] Addressing Igor's comments --- aptos-move/aptos-vm/src/lib.rs | 6 +++- .../block-executor/src/code_cache_global.rs | 1 + .../src/code_cache_global_manager.rs | 17 +++++++++-- aptos-move/block-executor/src/counters.rs | 28 +++++++++++++++++-- .../src/workflow/do_get_execution_output.rs | 1 + types/src/block_executor/config.rs | 7 +++-- 6 files changed, 52 insertions(+), 8 deletions(-) diff --git a/aptos-move/aptos-vm/src/lib.rs b/aptos-move/aptos-vm/src/lib.rs index 25f807a499f95..3f21e37f8e2b1 100644 --- a/aptos-move/aptos-vm/src/lib.rs +++ b/aptos-move/aptos-vm/src/lib.rs @@ -153,7 +153,11 @@ pub trait VMValidator { ) -> VMValidatorResult; } -/// This trait describes the block executor interface. +/// This trait describes the block executor interface which is responsible for executing a block of +/// transactions. In general, block execution returns a vector of transaction outputs. This vector +/// has the same length as the input vector of transactions. In case transactions are skipped or +/// discarded, they are still included - but their output is empty. The outputs are not applied to +/// the state directly. It is the responsibility of the caller to update the state accordingly. pub trait VMBlockExecutor: Send + Sync { /// Be careful if any state (such as caches) is kept in [VMBlockExecutor]. It is the /// responsibility of the implementation to ensure the state is valid across multiple diff --git a/aptos-move/block-executor/src/code_cache_global.rs b/aptos-move/block-executor/src/code_cache_global.rs index fef08f5f5c940..076dca358be7f 100644 --- a/aptos-move/block-executor/src/code_cache_global.rs +++ b/aptos-move/block-executor/src/code_cache_global.rs @@ -133,6 +133,7 @@ where /// 1. Only verified modules are inserted. /// 2. Valid modules should not be removed, and new modules should have unique ownership. If /// these constraints are violated, a panic error is returned. + // TODO(loader_v2): Use a trait for sync methods, and a concrete implementation for unsync. pub fn insert_verified_unsync( &self, modules: impl Iterator>)>, diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs index e37187d36f0e7..f616a85703e76 100644 --- a/aptos-move/block-executor/src/code_cache_global_manager.rs +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -1,7 +1,14 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use crate::{code_cache_global::GlobalModuleCache, explicit_sync_wrapper::ExplicitSyncWrapper}; +use crate::{ + code_cache_global::GlobalModuleCache, + counters::{ + GLOBAL_MODULE_CACHE_NUM_MODULES, GLOBAL_MODULE_CACHE_SIZE_IN_BYTES, + STRUCT_NAME_INDEX_MAP_NUM_ENTRIES, + }, + explicit_sync_wrapper::ExplicitSyncWrapper, +}; use aptos_types::block_executor::config::BlockExecutorModuleCacheLocalConfig; use aptos_vm_environment::environment::AptosEnvironment; use move_binary_format::errors::Location; @@ -150,13 +157,19 @@ where let struct_name_index_map_size = runtime_environment .struct_name_index_map_size() .map_err(|err| err.finish(Location::Undefined).into_vm_status())?; + STRUCT_NAME_INDEX_MAP_NUM_ENTRIES.set(struct_name_index_map_size as i64); + if struct_name_index_map_size > config.max_struct_name_index_map_num_entries { module_cache.flush_unsync(); runtime_environment.flush_struct_name_and_info_caches(); } // Check 2: If the module cache is too big, flush it. - if module_cache.size_in_bytes() > config.max_module_cache_size_in_bytes { + let module_cache_size_in_bytes = module_cache.size_in_bytes(); + GLOBAL_MODULE_CACHE_SIZE_IN_BYTES.set(module_cache_size_in_bytes as i64); + GLOBAL_MODULE_CACHE_NUM_MODULES.set(module_cache.num_modules() as i64); + + if module_cache_size_in_bytes > config.max_module_cache_size_in_bytes { module_cache.flush_unsync(); } diff --git a/aptos-move/block-executor/src/counters.rs b/aptos-move/block-executor/src/counters.rs index c3f1dc61f8fd3..e75d1b3354805 100644 --- a/aptos-move/block-executor/src/counters.rs +++ b/aptos-move/block-executor/src/counters.rs @@ -3,8 +3,8 @@ use aptos_metrics_core::{ exponential_buckets, register_avg_counter_vec, register_histogram, register_histogram_vec, - register_int_counter, register_int_counter_vec, Histogram, HistogramVec, IntCounter, - IntCounterVec, + register_int_counter, register_int_counter_vec, register_int_gauge, Histogram, HistogramVec, + IntCounter, IntCounterVec, IntGauge, }; use aptos_mvhashmap::BlockStateStats; use aptos_types::fee_statement::FeeStatement; @@ -333,3 +333,27 @@ pub(crate) fn update_state_counters(block_state_stats: BlockStateStats, is_paral .with_label_values(&[mode_str, "delayed_field"]) .observe(block_state_stats.base_delayed_fields_size as f64); } + +pub static GLOBAL_MODULE_CACHE_SIZE_IN_BYTES: Lazy = Lazy::new(|| { + register_int_gauge!( + "global_module_cache_size_in_bytes", + "Sum of sizes of all serialized modules stored in global module cache" + ) + .unwrap() +}); + +pub static GLOBAL_MODULE_CACHE_NUM_MODULES: Lazy = Lazy::new(|| { + register_int_gauge!( + "global_module_cache_num_modules", + "Number of modules cached in global module cache" + ) + .unwrap() +}); + +pub static STRUCT_NAME_INDEX_MAP_NUM_ENTRIES: Lazy = Lazy::new(|| { + register_int_gauge!( + "struct_name_index_map_num_entries", + "Number of struct names interned and cached in execution environment" + ) + .unwrap() +}); diff --git a/execution/executor/src/workflow/do_get_execution_output.rs b/execution/executor/src/workflow/do_get_execution_output.rs index 5e7987b95098f..5d2e962dff989 100644 --- a/execution/executor/src/workflow/do_get_execution_output.rs +++ b/execution/executor/src/workflow/do_get_execution_output.rs @@ -44,6 +44,7 @@ use std::{iter, sync::Arc}; pub struct DoGetExecutionOutput; impl DoGetExecutionOutput { + // Note: state checkpoint will be appended in when the current block is Some(..). pub fn by_transaction_execution( executor: &V, transactions: ExecutableTransactions, diff --git a/types/src/block_executor/config.rs b/types/src/block_executor/config.rs index 1342eba0de53f..ca578142e7295 100644 --- a/types/src/block_executor/config.rs +++ b/types/src/block_executor/config.rs @@ -22,9 +22,10 @@ impl Default for BlockExecutorModuleCacheLocalConfig { fn default() -> Self { Self { prefetch_framework_code: true, - // Use 50 Mb for now, should be large enough to cache many modules. - max_module_cache_size_in_bytes: 50 * 1024 * 1024, - max_struct_name_index_map_num_entries: 100_000, + // Use 512 Mb for now, should be large enough to cache all mainnet modules (at the time + // of writing this comment, 13.11.24). + max_module_cache_size_in_bytes: 512 * 1024 * 1024, + max_struct_name_index_map_num_entries: 1_000_000, } } } From 81e44f88cee9ce01db895972c086c31e70533d6e Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 13 Nov 2024 00:41:13 +0000 Subject: [PATCH 18/25] [congigs] Use 1 GB max size --- types/src/block_executor/config.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/types/src/block_executor/config.rs b/types/src/block_executor/config.rs index ca578142e7295..aed7e63346646 100644 --- a/types/src/block_executor/config.rs +++ b/types/src/block_executor/config.rs @@ -22,9 +22,9 @@ impl Default for BlockExecutorModuleCacheLocalConfig { fn default() -> Self { Self { prefetch_framework_code: true, - // Use 512 Mb for now, should be large enough to cache all mainnet modules (at the time + // Use 1 Gb for now, should be large enough to cache all mainnet modules (at the time // of writing this comment, 13.11.24). - max_module_cache_size_in_bytes: 512 * 1024 * 1024, + max_module_cache_size_in_bytes: 1024 * 1024 * 1024, max_struct_name_index_map_num_entries: 1_000_000, } } From 2f3708ed53344c879453ce2fe39747ddfec4d473 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 13 Nov 2024 01:02:50 +0000 Subject: [PATCH 19/25] format fix --- .../aptos-framework/doc/function_info.md | 104 ++++++++++++++++++ types/src/block_executor/config.rs | 2 +- 2 files changed, 105 insertions(+), 1 deletion(-) diff --git a/aptos-move/framework/aptos-framework/doc/function_info.md b/aptos-move/framework/aptos-framework/doc/function_info.md index e55ff672c939e..65840ecd59f8b 100644 --- a/aptos-move/framework/aptos-framework/doc/function_info.md +++ b/aptos-move/framework/aptos-framework/doc/function_info.md @@ -16,7 +16,12 @@ The function_info - [Function `is_identifier`](#0x1_function_info_is_identifier) - [Function `load_function_impl`](#0x1_function_info_load_function_impl) - [Specification](#@Specification_1) + - [Function `new_function_info`](#@Specification_1_new_function_info) + - [Function `new_function_info_from_address`](#@Specification_1_new_function_info_from_address) + - [Function `check_dispatch_type_compatibility`](#@Specification_1_check_dispatch_type_compatibility) + - [Function `load_module_from_function`](#@Specification_1_load_module_from_function) - [Function `check_dispatch_type_compatibility_impl`](#@Specification_1_check_dispatch_type_compatibility_impl) + - [Function `is_identifier`](#@Specification_1_is_identifier) - [Function `load_function_impl`](#@Specification_1_load_function_impl) @@ -323,7 +328,88 @@ if such module isn't accessed previously in the transaction. + + + +
fun spec_is_identifier(s: vector<u8>): bool;
+
+ + + + + +### Function `new_function_info` + + +
public fun new_function_info(module_signer: &signer, module_name: string::String, function_name: string::String): function_info::FunctionInfo
+
+ + + + +
aborts_if !spec_is_identifier(string::bytes(module_name));
+aborts_if !spec_is_identifier(string::bytes(function_name));
+ensures result == FunctionInfo {
+    module_address: signer::address_of(module_signer),
+    module_name,
+    function_name,
+};
+
+ + + + + +### Function `new_function_info_from_address` + + +
public(friend) fun new_function_info_from_address(module_address: address, module_name: string::String, function_name: string::String): function_info::FunctionInfo
+
+ + + + +
aborts_if !spec_is_identifier(string::bytes(module_name));
+aborts_if !spec_is_identifier(string::bytes(function_name));
+ensures result == FunctionInfo {
+    module_address,
+    module_name,
+    function_name,
+};
+
+ + + + + +### Function `check_dispatch_type_compatibility` + + +
public(friend) fun check_dispatch_type_compatibility(framework_function: &function_info::FunctionInfo, dispatch_target: &function_info::FunctionInfo): bool
+
+ + + + +
pragma verify = false;
+pragma opaque;
+
+ + + + + +### Function `load_module_from_function` + + +
public(friend) fun load_module_from_function(f: &function_info::FunctionInfo)
+
+ + + +
pragma verify = false;
+pragma opaque;
 
@@ -344,6 +430,24 @@ if such module isn't accessed previously in the transaction. + + +### Function `is_identifier` + + +
fun is_identifier(s: &vector<u8>): bool
+
+ + + + +
pragma opaque;
+aborts_if [abstract] false;
+ensures [abstract] result == spec_is_identifier(s);
+
+ + + ### Function `load_function_impl` diff --git a/types/src/block_executor/config.rs b/types/src/block_executor/config.rs index aed7e63346646..0ce382d3d94a5 100644 --- a/types/src/block_executor/config.rs +++ b/types/src/block_executor/config.rs @@ -22,7 +22,7 @@ impl Default for BlockExecutorModuleCacheLocalConfig { fn default() -> Self { Self { prefetch_framework_code: true, - // Use 1 Gb for now, should be large enough to cache all mainnet modules (at the time + // Use 1Gb for now, should be large enough to cache all mainnet modules (at the time // of writing this comment, 13.11.24). max_module_cache_size_in_bytes: 1024 * 1024 * 1024, max_struct_name_index_map_num_entries: 1_000_000, From 003eceaed52f3402bd61f6e52c35a3520ade80e3 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 13 Nov 2024 09:30:55 +0000 Subject: [PATCH 20/25] [comments] Added a comment about num_modules > 0 --- aptos-move/aptos-vm/src/block_executor/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index 4b8bf88155b3c..c71c363e936f1 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -449,7 +449,9 @@ impl BlockAptosVM { (environment, Arc::new(GlobalModuleCache::empty())) }; - // Finally, to avoid cold starts, fetch the framework code prior to block execution. + // Finally, to avoid cold starts, fetch the framework code prior to block execution. This + // ensures the state with 0 modules cached is not possible for block execution (as long as + // the config enables the framework prefetch). if is_loader_v2_enabled && module_cache.num_modules() == 0 && config.local.module_cache_config.prefetch_framework_code From 12292a29fe639190c8566731b9978eea01d75c79 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Wed, 13 Nov 2024 10:32:04 +0000 Subject: [PATCH 21/25] [comments] Use rw lock for module cache, but aquire before block executions --- Cargo.lock | 1 + aptos-move/aptos-vm/Cargo.toml | 1 + aptos-move/aptos-vm/src/block_executor/mod.rs | 34 ++++--- .../block-executor/src/captured_reads.rs | 4 +- aptos-move/block-executor/src/code_cache.rs | 2 +- .../block-executor/src/code_cache_global.rs | 80 ++++++--------- .../src/code_cache_global_manager.rs | 60 +++++++----- aptos-move/block-executor/src/executor.rs | 97 ++++++++++++++----- .../src/proptest_types/bencher.rs | 8 +- .../src/proptest_types/tests.rs | 67 ++++++++++--- .../block-executor/src/unit_tests/mod.rs | 73 +++++++++++--- 11 files changed, 284 insertions(+), 143 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index fb51fb9c3bf92..d79aaf60f9753 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4494,6 +4494,7 @@ dependencies = [ "num_cpus", "once_cell", "ouroboros", + "parking_lot 0.12.1", "proptest", "rand 0.7.3", "rand_core 0.5.1", diff --git a/aptos-move/aptos-vm/Cargo.toml b/aptos-move/aptos-vm/Cargo.toml index 94897e9368d97..0afe74bebc4fb 100644 --- a/aptos-move/aptos-vm/Cargo.toml +++ b/aptos-move/aptos-vm/Cargo.toml @@ -55,6 +55,7 @@ move-vm-types = { workspace = true } num_cpus = { workspace = true } once_cell = { workspace = true } ouroboros = { workspace = true } +parking_lot = { workspace = true } rand = { workspace = true } rayon = { workspace = true } serde = { workspace = true } diff --git a/aptos-move/aptos-vm/src/block_executor/mod.rs b/aptos-move/aptos-vm/src/block_executor/mod.rs index c71c363e936f1..9336a90ba028b 100644 --- a/aptos-move/aptos-vm/src/block_executor/mod.rs +++ b/aptos-move/aptos-vm/src/block_executor/mod.rs @@ -56,6 +56,7 @@ use move_core_types::{ use move_vm_runtime::{Module, ModuleStorage}; use move_vm_types::delayed_values::delayed_field_id::DelayedFieldID; use once_cell::sync::{Lazy, OnceCell}; +use parking_lot::RwLock; use std::{ collections::{BTreeMap, HashSet}, sync::Arc, @@ -446,18 +447,21 @@ impl BlockAptosVM { module_cache_manager .check_ready_and_get_caches(environment, &config.local.module_cache_config)? } else { - (environment, Arc::new(GlobalModuleCache::empty())) + ( + environment, + Arc::new(RwLock::new(GlobalModuleCache::empty())), + ) }; // Finally, to avoid cold starts, fetch the framework code prior to block execution. This // ensures the state with 0 modules cached is not possible for block execution (as long as // the config enables the framework prefetch). if is_loader_v2_enabled - && module_cache.num_modules() == 0 + && module_cache.read().num_modules() == 0 && config.local.module_cache_config.prefetch_framework_code { let code_storage = state_view.as_aptos_code_storage(environment.clone()); - prefetch_aptos_framework(code_storage, &module_cache).map_err(|err| { + prefetch_aptos_framework(code_storage, &mut module_cache.write()).map_err(|err| { alert!("Failed to load Aptos framework to module cache: {:?}", err); VMError::from(err).into_vm_status() })?; @@ -469,12 +473,7 @@ impl BlockAptosVM { S, L, ExecutableTestType, - >::new( - config, - executor_thread_pool, - module_cache, - transaction_commit_listener, - ); + >::new(config, executor_thread_pool, transaction_commit_listener); if is_loader_v2_enabled && !module_cache_manager.mark_executing() { return Err(VMStatus::error( @@ -482,7 +481,12 @@ impl BlockAptosVM { Some("Unable to mark block execution start".to_string()), )); } - let ret = executor.execute_block(environment, signature_verified_block, state_view); + let ret = executor.execute_block( + environment, + signature_verified_block, + state_view, + module_cache.as_ref(), + ); if is_loader_v2_enabled && !module_cache_manager.mark_done() { return Err(VMStatus::error( StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, @@ -557,7 +561,7 @@ impl BlockAptosVM { /// error is returned. fn prefetch_aptos_framework( code_storage: AptosCodeStorageAdapter, - module_cache: &GlobalModuleCache, + module_cache: &mut GlobalModuleCache, ) -> Result<(), PanicError> { // If framework code exists in storage, the transitive closure will be verified and cached. let maybe_loaded = code_storage @@ -590,10 +594,10 @@ mod test { let environment = AptosEnvironment::new_with_delayed_field_optimization_enabled(state_view); let code_storage = state_view.as_aptos_code_storage(environment); - let module_cache = GlobalModuleCache::empty(); + let mut module_cache = GlobalModuleCache::empty(); assert_eq!(module_cache.num_modules(), 0); - let result = prefetch_aptos_framework(code_storage, &module_cache); + let result = prefetch_aptos_framework(code_storage, &mut module_cache); assert!(result.is_ok()); assert!(module_cache.num_modules() > 0); } @@ -606,10 +610,10 @@ mod test { AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view); let code_storage = state_view.as_aptos_code_storage(environment); - let module_cache = GlobalModuleCache::empty(); + let mut module_cache = GlobalModuleCache::empty(); assert_eq!(module_cache.num_modules(), 0); - let result = prefetch_aptos_framework(code_storage, &module_cache); + let result = prefetch_aptos_framework(code_storage, &mut module_cache); assert!(result.is_ok()); assert_eq!(module_cache.num_modules(), 0); } diff --git a/aptos-move/block-executor/src/captured_reads.rs b/aptos-move/block-executor/src/captured_reads.rs index 834becdcd4c94..516b0855c39c4 100644 --- a/aptos-move/block-executor/src/captured_reads.rs +++ b/aptos-move/block-executor/src/captured_reads.rs @@ -1560,7 +1560,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - let global_module_cache = GlobalModuleCache::empty(); + let mut global_module_cache = GlobalModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); global_module_cache.insert(0, mock_verified_code(0, MockExtension::new(8))); @@ -1697,7 +1697,7 @@ mod test { MockVerifiedCode, MockExtension, >::new(); - let global_module_cache = GlobalModuleCache::empty(); + let mut global_module_cache = GlobalModuleCache::empty(); let per_block_module_cache = SyncModuleCache::empty(); // Module exists in global cache. diff --git a/aptos-move/block-executor/src/code_cache.rs b/aptos-move/block-executor/src/code_cache.rs index 8d875dca1ce2d..d9b236346dfe2 100644 --- a/aptos-move/block-executor/src/code_cache.rs +++ b/aptos-move/block-executor/src/code_cache.rs @@ -140,7 +140,7 @@ impl<'a, T: Transaction, S: TStateView, X: Executable> ModuleCache > { // First, look up the module in the cross-block global module cache. Record the read for // later validation in case the read module is republished. - if let Some(module) = self.global_module_cache.get(key) { + if let Some(module) = self.global_module_cache.get_valid(key) { match &self.latest_view { ViewState::Sync(state) => state .captured_reads diff --git a/aptos-move/block-executor/src/code_cache_global.rs b/aptos-move/block-executor/src/code_cache_global.rs index 076dca358be7f..3f68f24fe4239 100644 --- a/aptos-move/block-executor/src/code_cache_global.rs +++ b/aptos-move/block-executor/src/code_cache_global.rs @@ -1,7 +1,6 @@ // Copyright © Aptos Foundation // SPDX-License-Identifier: Apache-2.0 -use crate::explicit_sync_wrapper::ExplicitSyncWrapper; use aptos_types::error::PanicError; use hashbrown::HashMap; use move_vm_types::code::{ModuleCode, WithSize}; @@ -9,7 +8,7 @@ use std::{ hash::Hash, ops::Deref, sync::{ - atomic::{AtomicBool, AtomicUsize, Ordering}, + atomic::{AtomicBool, Ordering}, Arc, }, }; @@ -63,9 +62,9 @@ where /// block execution. Modified safely only at block boundaries. pub struct GlobalModuleCache { /// Module cache containing the verified code. - module_cache: ExplicitSyncWrapper>>, + module_cache: HashMap>, /// Sum of serialized sizes (in bytes) of all cached modules. - size: AtomicUsize, + size: usize, } impl GlobalModuleCache @@ -77,15 +76,14 @@ where /// Returns new empty module cache. pub fn empty() -> Self { Self { - module_cache: ExplicitSyncWrapper::new(HashMap::new()), - size: AtomicUsize::new(0), + module_cache: HashMap::new(), + size: 0, } } /// Returns true if the key exists in cache and the corresponding module is valid. pub fn contains_valid(&self, key: &K) -> bool { self.module_cache - .acquire() .get(key) .is_some_and(|entry| entry.is_valid()) } @@ -94,57 +92,48 @@ where /// cache for the associated key will result in a cache miss. If an entry does not to exist, it /// is a no-op. pub fn mark_invalid_if_contains(&self, key: &K) { - if let Some(entry) = self.module_cache.acquire().get(key) { + if let Some(entry) = self.module_cache.get(key) { entry.mark_invalid(); } } /// Returns the module stored in cache. If the module has not been cached, or it exists but is /// not valid, [None] is returned. - pub fn get(&self, key: &K) -> Option>> { + pub fn get_valid(&self, key: &K) -> Option>> { self.module_cache - .acquire() .get(key) .and_then(|entry| entry.is_valid().then(|| Arc::clone(entry.module_code()))) } /// Returns the number of entries in the cache. pub fn num_modules(&self) -> usize { - self.module_cache.acquire().len() + self.module_cache.len() } /// Returns the sum of serialized sizes of modules stored in cache. pub fn size_in_bytes(&self) -> usize { - self.size.load(Ordering::Relaxed) + self.size } - /// **Use with caution: should never be called during block execution.** - /// /// Flushes the module cache. - pub fn flush_unsync(&self) { - self.module_cache.acquire().clear(); - self.size.store(0, Ordering::Relaxed); + pub fn flush_unsync(&mut self) { + self.module_cache.clear(); + self.size = 0; } - /// **Use with caution: should never be called during block execution.** - /// /// Inserts modules into the cache. /// Notes: /// 1. Only verified modules are inserted. /// 2. Valid modules should not be removed, and new modules should have unique ownership. If /// these constraints are violated, a panic error is returned. - // TODO(loader_v2): Use a trait for sync methods, and a concrete implementation for unsync. pub fn insert_verified_unsync( - &self, + &mut self, modules: impl Iterator>)>, ) -> Result<(), PanicError> { use hashbrown::hash_map::Entry::*; - let mut guard = self.module_cache.acquire(); - let module_cache = guard.dereference_mut(); - for (key, module) in modules { - if let Occupied(entry) = module_cache.entry(key.clone()) { + if let Occupied(entry) = self.module_cache.entry(key.clone()) { if entry.get().is_valid() { return Err(PanicError::CodeInvariantError( "Should never overwrite a valid module".to_string(), @@ -152,17 +141,16 @@ where } else { // Otherwise, remove the invalid entry. let size = entry.get().module_code().extension().size_in_bytes(); - self.size.fetch_sub(size, Ordering::Relaxed); + self.size -= size; entry.remove(); } } if module.code().is_verified() { - self.size - .fetch_add(module.extension().size_in_bytes(), Ordering::Relaxed); + self.size += module.extension().size_in_bytes(); let entry = Entry::new(module).expect("Module has been checked and must be verified"); - let prev = module_cache.insert(key.clone(), entry); + let prev = self.module_cache.insert(key.clone(), entry); // At this point, we must have removed the entry, or returned a panic error. assert!(prev.is_none()) @@ -173,10 +161,9 @@ where /// Insert the module to cache. Used for tests only. #[cfg(any(test, feature = "testing"))] - pub fn insert(&self, key: K, module: Arc>) { - self.size - .fetch_add(module.extension().size_in_bytes(), Ordering::Relaxed); - self.module_cache.acquire().insert( + pub fn insert(&mut self, key: K, module: Arc>) { + self.size += module.extension().size_in_bytes(); + self.module_cache.insert( key, Entry::new(module).expect("Module code should be verified"), ); @@ -185,12 +172,9 @@ where /// Removes the module from cache and returns true. If the module does not exist for the /// associated key, returns false. Used for tests only. #[cfg(any(test, feature = "testing"))] - pub fn remove(&self, key: &K) -> bool { - if let Some(entry) = self.module_cache.acquire().remove(key) { - self.size.fetch_sub( - entry.module_code().extension().size_in_bytes(), - Ordering::Relaxed, - ); + pub fn remove(&mut self, key: &K) -> bool { + if let Some(entry) = self.module_cache.remove(key) { + self.size -= entry.module_code().extension().size_in_bytes(); true } else { false @@ -221,7 +205,7 @@ mod test { #[test] fn test_cache_contains_valid_and_get() { - let cache = GlobalModuleCache::empty(); + let mut cache = GlobalModuleCache::empty(); // Set the state. cache.insert(0, mock_verified_code(0, MockExtension::new(8))); @@ -234,14 +218,14 @@ mod test { assert!(!cache.contains_valid(&1)); assert!(!cache.contains_valid(&3)); - assert!(cache.get(&0).is_some()); - assert!(cache.get(&1).is_none()); - assert!(cache.get(&3).is_none()); + assert!(cache.get_valid(&0).is_some()); + assert!(cache.get_valid(&1).is_none()); + assert!(cache.get_valid(&3).is_none()); } #[test] fn test_cache_sizes_and_flush_unchecked() { - let cache = GlobalModuleCache::empty(); + let mut cache = GlobalModuleCache::empty(); assert_eq!(cache.num_modules(), 0); assert_eq!(cache.size_in_bytes(), 0); @@ -262,7 +246,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked() { - let cache = GlobalModuleCache::empty(); + let mut cache = GlobalModuleCache::empty(); let mut new_modules = vec![]; for i in 0..10 { @@ -278,7 +262,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked_does_not_add_deserialized_code() { - let cache = GlobalModuleCache::empty(); + let mut cache = GlobalModuleCache::empty(); let deserialized_modules = vec![(0, mock_deserialized_code(0, MockExtension::new(8)))]; assert_ok!(cache.insert_verified_unsync(deserialized_modules.into_iter())); @@ -289,7 +273,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked_does_not_override_valid_modules() { - let cache = GlobalModuleCache::empty(); + let mut cache = GlobalModuleCache::empty(); cache.insert(0, mock_verified_code(0, MockExtension::new(8))); assert_eq!(cache.num_modules(), 1); @@ -301,7 +285,7 @@ mod test { #[test] fn test_cache_insert_verified_unchecked_overrides_invalid_modules() { - let cache = GlobalModuleCache::empty(); + let mut cache = GlobalModuleCache::empty(); cache.insert(0, mock_verified_code(0, MockExtension::new(8))); cache.mark_invalid_if_contains(&0); diff --git a/aptos-move/block-executor/src/code_cache_global_manager.rs b/aptos-move/block-executor/src/code_cache_global_manager.rs index f616a85703e76..26fda4eb8a3be 100644 --- a/aptos-move/block-executor/src/code_cache_global_manager.rs +++ b/aptos-move/block-executor/src/code_cache_global_manager.rs @@ -15,7 +15,7 @@ use move_binary_format::errors::Location; use move_core_types::vm_status::{StatusCode::UNKNOWN_INVARIANT_VIOLATION_ERROR, VMStatus}; use move_vm_runtime::WithRuntimeEnvironment; use move_vm_types::code::WithSize; -use parking_lot::Mutex; +use parking_lot::{Mutex, RwLock}; use std::{ fmt::Debug, hash::Hash, @@ -64,7 +64,8 @@ pub struct ModuleCacheManager { /// During concurrent executions, this module cache is read-only. However, it can be mutated /// when it is known that there are no concurrent accesses. [ModuleCacheManager] must ensure /// the safety. - module_cache: Arc>, + // TODO(loader_v2): Remove this Arc when the feature is enabled on mainnet. + module_cache: Arc>>, /// The execution environment, initially set to [None]. The environment, as long as it does not /// change, can be kept for multiple block executions. environment: ExplicitSyncWrapper>, @@ -83,7 +84,7 @@ where pub fn new() -> Self { Self { state: Mutex::new(State::Done(None)), - module_cache: Arc::new(GlobalModuleCache::empty()), + module_cache: Arc::new(RwLock::new(GlobalModuleCache::empty())), environment: ExplicitSyncWrapper::new(None), } } @@ -107,7 +108,7 @@ where .runtime_environment() .flush_struct_name_and_info_caches(); } - self.module_cache.flush_unsync(); + self.module_cache.write().flush_unsync(); } *state = State::Ready(current); @@ -135,7 +136,13 @@ where &self, storage_environment: AptosEnvironment, config: &BlockExecutorModuleCacheLocalConfig, - ) -> Result<(AptosEnvironment, Arc>), VMStatus> { + ) -> Result< + ( + AptosEnvironment, + Arc>>, + ), + VMStatus, + > { let state = self.state.lock(); if !matches!(state.deref(), State::Ready(_)) { let msg = format!( @@ -160,17 +167,17 @@ where STRUCT_NAME_INDEX_MAP_NUM_ENTRIES.set(struct_name_index_map_size as i64); if struct_name_index_map_size > config.max_struct_name_index_map_num_entries { - module_cache.flush_unsync(); + module_cache.write().flush_unsync(); runtime_environment.flush_struct_name_and_info_caches(); } // Check 2: If the module cache is too big, flush it. - let module_cache_size_in_bytes = module_cache.size_in_bytes(); + let module_cache_size_in_bytes = module_cache.read().size_in_bytes(); GLOBAL_MODULE_CACHE_SIZE_IN_BYTES.set(module_cache_size_in_bytes as i64); - GLOBAL_MODULE_CACHE_NUM_MODULES.set(module_cache.num_modules() as i64); + GLOBAL_MODULE_CACHE_NUM_MODULES.set(module_cache.read().num_modules() as i64); if module_cache_size_in_bytes > config.max_module_cache_size_in_bytes { - module_cache.flush_unsync(); + module_cache.write().flush_unsync(); } Ok((environment, module_cache)) @@ -222,7 +229,7 @@ where // If this environment has been (re-)initialized, we need to flush the module cache // because it can contain now out-dated code. - self.module_cache.flush_unsync(); + self.module_cache.write().flush_unsync(); } existing_environment @@ -261,8 +268,9 @@ mod test { // Pre-populate module cache to test flushing. module_cache_manager .module_cache + .write() .insert(0, mock_verified_code(0, MockExtension::new(8))); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); assert!(!module_cache_manager.mark_executing()); assert!(!module_cache_manager.mark_done()); @@ -270,9 +278,9 @@ mod test { // Only in matching case the module cache is not flushed. if recorded_previous.is_some() && recorded_previous == previous { - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); } else { - assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 0); } let state = module_cache_manager.state.lock().clone(); @@ -304,8 +312,9 @@ mod test { module_cache_manager .module_cache + .write() .insert(0, mock_verified_code(0, MockExtension::new(16))); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); let runtime_environment = environment.runtime_environment(); let dummy_struct_name = StructIdentifier { @@ -324,7 +333,7 @@ mod test { assert!(module_cache_manager .check_ready_and_get_caches(environment.clone(), &config) .is_ok()); - assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 0); assert_eq!( assert_ok!(runtime_environment.struct_name_index_map_size()), 1 @@ -332,13 +341,14 @@ mod test { module_cache_manager .module_cache + .write() .insert(0, mock_verified_code(0, MockExtension::new(4))); // This time size is less than the one specified in config. No flushing. assert!(module_cache_manager .check_ready_and_get_caches(environment.clone(), &config) .is_ok()); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); assert_eq!( assert_ok!(runtime_environment.struct_name_index_map_size()), 1 @@ -368,7 +378,7 @@ mod test { assert!(module_cache_manager .check_ready_and_get_caches(environment.clone(), &config) .is_ok()); - assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 0); assert_eq!( assert_ok!(runtime_environment.struct_name_index_map_size()), 0 @@ -531,11 +541,13 @@ mod test { module_cache_manager .module_cache + .write() .insert(0, mock_verified_code(0, MockExtension::new(8))); module_cache_manager .module_cache + .write() .insert(1, mock_verified_code(1, MockExtension::new(8))); - assert_eq!(module_cache_manager.module_cache.num_modules(), 2); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 2); assert!(module_cache_manager.environment.acquire().is_none()); // Environment has to be set to the same value, cache flushed. @@ -543,7 +555,7 @@ mod test { let environment = module_cache_manager.get_or_initialize_environment( AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), ); - assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 0); assert!(module_cache_manager .environment .acquire() @@ -552,8 +564,9 @@ mod test { module_cache_manager .module_cache + .write() .insert(2, mock_verified_code(2, MockExtension::new(8))); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); assert!(module_cache_manager.environment.acquire().is_some()); // Environment has to be re-set to the new value, cache flushed. @@ -562,7 +575,7 @@ mod test { let environment = module_cache_manager.get_or_initialize_environment( AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), ); - assert_eq!(module_cache_manager.module_cache.num_modules(), 0); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 0); assert!(module_cache_manager .environment .acquire() @@ -571,15 +584,16 @@ mod test { module_cache_manager .module_cache + .write() .insert(3, mock_verified_code(3, MockExtension::new(8))); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); assert!(module_cache_manager.environment.acquire().is_some()); // Environment is kept, and module caches are not flushed. let new_environment = module_cache_manager.get_or_initialize_environment( AptosEnvironment::new_with_delayed_field_optimization_enabled(&state_view), ); - assert_eq!(module_cache_manager.module_cache.num_modules(), 1); + assert_eq!(module_cache_manager.module_cache.read().num_modules(), 1); assert!(environment == new_environment); } } diff --git a/aptos-move/block-executor/src/executor.rs b/aptos-move/block-executor/src/executor.rs index 603bce0fbe1ab..0f3890d0a6585 100644 --- a/aptos-move/block-executor/src/executor.rs +++ b/aptos-move/block-executor/src/executor.rs @@ -58,11 +58,13 @@ use move_core_types::{language_storage::ModuleId, value::MoveTypeLayout, vm_stat use move_vm_runtime::{Module, RuntimeEnvironment, WithRuntimeEnvironment}; use move_vm_types::code::ModuleCache; use num_cpus; +use parking_lot::RwLock; use rayon::ThreadPool; use std::{ cell::RefCell, collections::{BTreeMap, HashMap, HashSet}, marker::{PhantomData, Sync}, + ops::Deref, sync::{ atomic::{AtomicBool, AtomicU32, Ordering}, Arc, @@ -74,8 +76,6 @@ pub struct BlockExecutor { // threads that may be concurrently participating in parallel execution. config: BlockExecutorConfig, executor_thread_pool: Arc, - global_module_cache: - Arc>, transaction_commit_hook: Option, phantom: PhantomData<(T, E, S, L, X)>, } @@ -93,9 +93,6 @@ where pub fn new( config: BlockExecutorConfig, executor_thread_pool: Arc, - global_module_cache: Arc< - GlobalModuleCache, - >, transaction_commit_hook: Option, ) -> Self { assert!( @@ -106,7 +103,6 @@ where Self { config, executor_thread_pool, - global_module_cache, transaction_commit_hook, phantom: PhantomData, } @@ -560,6 +556,7 @@ where /// in outputs, which is heavier (due to serialization / deserialization, copies, etc). Moreover, /// since prepare_and_queue_commit_ready_txns takes care of synchronization in the flag-combining /// way, the materialization can be almost embarrassingly parallelizable. + #[allow(clippy::too_many_arguments)] fn prepare_and_queue_commit_ready_txns( &self, block_gas_limit_type: &BlockGasLimitType, @@ -569,6 +566,12 @@ where last_input_output: &TxnLastInputOutput, shared_commit_state: &ExplicitSyncWrapper>, base_view: &S, + global_module_cache: &GlobalModuleCache< + ModuleId, + CompiledModule, + Module, + AptosModuleExtension, + >, runtime_environment: &RuntimeEnvironment, start_shared_counter: u32, shared_counter: &AtomicU32, @@ -605,7 +608,7 @@ where versioned_cache, executor, base_view, - self.global_module_cache.as_ref(), + global_module_cache, runtime_environment, ParallelState::new( versioned_cache, @@ -624,7 +627,7 @@ where Self::publish_module_writes( txn_idx, module_write_set, - self.global_module_cache.as_ref(), + global_module_cache, versioned_cache, scheduler, runtime_environment, @@ -637,7 +640,7 @@ where let validation_result = Self::validate( txn_idx, last_input_output, - self.global_module_cache.as_ref(), + global_module_cache, versioned_cache, scheduler, ); @@ -666,7 +669,7 @@ where Self::publish_module_writes( txn_idx, module_write_set, - self.global_module_cache.as_ref(), + global_module_cache, versioned_cache, scheduler, runtime_environment, @@ -841,6 +844,12 @@ where shared_counter: &AtomicU32, last_input_output: &TxnLastInputOutput, base_view: &S, + global_module_cache: &GlobalModuleCache< + ModuleId, + CompiledModule, + Module, + AptosModuleExtension, + >, runtime_environment: &RuntimeEnvironment, final_results: &ExplicitSyncWrapper>, ) -> Result<(), PanicError> { @@ -852,7 +861,7 @@ where ); let latest_view = LatestView::new( base_view, - self.global_module_cache.as_ref(), + global_module_cache, runtime_environment, ViewState::Sync(parallel_state), txn_idx, @@ -946,6 +955,12 @@ where scheduler: &Scheduler, // TODO: should not need to pass base view. base_view: &S, + global_module_cache: &GlobalModuleCache< + ModuleId, + CompiledModule, + Module, + AptosModuleExtension, + >, start_shared_counter: u32, shared_counter: &AtomicU32, shared_commit_state: &ExplicitSyncWrapper>, @@ -974,6 +989,7 @@ where shared_counter, last_input_output, base_view, + global_module_cache, runtime_environment, final_results, )?; @@ -1001,6 +1017,7 @@ where last_input_output, shared_commit_state, base_view, + global_module_cache, runtime_environment, start_shared_counter, shared_counter, @@ -1018,7 +1035,7 @@ where let valid = Self::validate( txn_idx, last_input_output, - self.global_module_cache.as_ref(), + global_module_cache, versioned_cache, scheduler, ); @@ -1046,7 +1063,7 @@ where versioned_cache, &executor, base_view, - self.global_module_cache.as_ref(), + global_module_cache, runtime_environment, ParallelState::new( versioned_cache, @@ -1084,6 +1101,9 @@ where env: &E::Environment, signature_verified_block: &[T], base_view: &S, + global_module_cache: &RwLock< + GlobalModuleCache, + >, ) -> Result, ()> { let _timer = PARALLEL_EXECUTION_SECONDS.start_timer(); // Using parallel execution with 1 thread currently will not work as it @@ -1126,6 +1146,11 @@ where let scheduler = Scheduler::new(num_txns); let timer = RAYON_EXECUTION_SECONDS.start_timer(); + + // Read lock for execution because global cache is not mutated. + let global_module_cache_read_lock = global_module_cache.read(); + let immutable_global_module_cache = global_module_cache_read_lock.deref(); + self.executor_thread_pool.scope(|s| { for _ in 0..num_workers { s.spawn(|_| { @@ -1136,6 +1161,7 @@ where &versioned_cache, &scheduler, base_view, + immutable_global_module_cache, start_shared_counter, &shared_counter, &shared_commit_state, @@ -1156,6 +1182,7 @@ where }); } }); + drop(global_module_cache_read_lock); drop(timer); if !shared_maybe_error.load(Ordering::SeqCst) && scheduler.pop_from_commit_queue().is_ok() { @@ -1168,7 +1195,8 @@ where } counters::update_state_counters(versioned_cache.stats(), true); - self.global_module_cache + global_module_cache + .write() .insert_verified_unsync(versioned_cache.take_modules_iter()) .map_err(|err| { alert!("[BlockSTM] Encountered panic error: {:?}", err); @@ -1342,6 +1370,9 @@ where env: &E::Environment, signature_verified_block: &[T], base_view: &S, + global_module_cache: &RwLock< + GlobalModuleCache, + >, resource_group_bcs_fallback: bool, ) -> Result, SequentialBlockExecutionError> { let num_txns = signature_verified_block.len(); @@ -1363,10 +1394,14 @@ where let last_input_output: TxnLastInputOutput = TxnLastInputOutput::new(num_txns as TxnIndex); + // Global module cache is only readable throughout execution. + let global_module_cache_read_lock = global_module_cache.read(); + let immutable_global_module_cache = global_module_cache_read_lock.deref(); + for (idx, txn) in signature_verified_block.iter().enumerate() { let latest_view = LatestView::::new( base_view, - self.global_module_cache.as_ref(), + immutable_global_module_cache, runtime_environment, ViewState::Unsync(SequentialState::new(&unsync_map, start_counter, &counter)), idx as TxnIndex, @@ -1558,7 +1593,7 @@ where Self::apply_output_sequential( idx as TxnIndex, runtime_environment, - self.global_module_cache.as_ref(), + immutable_global_module_cache, &unsync_map, &output, resource_write_set.clone(), @@ -1646,7 +1681,12 @@ where ret.resize_with(num_txns, E::Output::skip_output); counters::update_state_counters(unsync_map.stats(), false); - self.global_module_cache + + // Release the read lock on the global module cache, and acquire a write lock to mutate + // it by adding cached modules from unsync map. + drop(global_module_cache_read_lock); + global_module_cache + .write() .insert_verified_unsync(unsync_map.into_modules_iter())?; let block_end_info = if self @@ -1686,12 +1726,19 @@ where env: E::Environment, signature_verified_block: &[T], base_view: &S, + global_module_cache: &RwLock< + GlobalModuleCache, + >, ) -> BlockExecutionResult, E::Error> { let _timer = BLOCK_EXECUTOR_INNER_EXECUTE_BLOCK.start_timer(); if self.config.local.concurrency_level > 1 { - let parallel_result = - self.execute_transactions_parallel(&env, signature_verified_block, base_view); + let parallel_result = self.execute_transactions_parallel( + &env, + signature_verified_block, + base_view, + global_module_cache, + ); // If parallel gave us result, return it if let Ok(output) = parallel_result { @@ -1709,14 +1756,19 @@ where // Flush the cache and the environment to re-run from the "clean" state. env.runtime_environment() .flush_struct_name_and_info_caches(); - self.global_module_cache.flush_unsync(); + global_module_cache.write().flush_unsync(); info!("parallel execution requiring fallback"); } // If we didn't run parallel, or it didn't finish successfully - run sequential - let sequential_result = - self.execute_transactions_sequential(&env, signature_verified_block, base_view, false); + let sequential_result = self.execute_transactions_sequential( + &env, + signature_verified_block, + base_view, + global_module_cache, + false, + ); // If sequential gave us result, return it let sequential_error = match sequential_result { @@ -1738,6 +1790,7 @@ where &env, signature_verified_block, base_view, + global_module_cache, true, ); diff --git a/aptos-move/block-executor/src/proptest_types/bencher.rs b/aptos-move/block-executor/src/proptest_types/bencher.rs index 527c926f1b636..ec9a6a7883765 100644 --- a/aptos-move/block-executor/src/proptest_types/bencher.rs +++ b/aptos-move/block-executor/src/proptest_types/bencher.rs @@ -20,6 +20,7 @@ use aptos_types::{ }; use criterion::{BatchSize, Bencher as CBencher}; use num_cpus; +use parking_lot::RwLock; use proptest::{ arbitrary::Arbitrary, collection::vec, @@ -126,18 +127,19 @@ where .build() .unwrap(), ); - let global_module_cache = Arc::new(GlobalModuleCache::empty()); let config = BlockExecutorConfig::new_no_block_limit(num_cpus::get()); let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, E>, MockTask, E>, MockStateView>, NoOpTransactionCommitHook, E>, usize>, ExecutableTestType, - >::new(config, executor_thread_pool, global_module_cache, None) - .execute_transactions_parallel(&env, &self.transactions, &state_view); + >::new(config, executor_thread_pool, None) + .execute_transactions_parallel(&env, &self.transactions, &state_view, &global_module_cache); self.baseline_output.assert_parallel_output(&output); } diff --git a/aptos-move/block-executor/src/proptest_types/tests.rs b/aptos-move/block-executor/src/proptest_types/tests.rs index 5d83c2fe50578..d80a1452fa19d 100644 --- a/aptos-move/block-executor/src/proptest_types/tests.rs +++ b/aptos-move/block-executor/src/proptest_types/tests.rs @@ -22,6 +22,7 @@ use aptos_types::{ }; use claims::{assert_matches, assert_ok}; use num_cpus; +use parking_lot::RwLock; use proptest::{ collection::vec, prelude::*, @@ -71,6 +72,8 @@ fn run_transactions( for _ in 0..num_repeat { let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, E>, MockTask, E>, @@ -80,10 +83,14 @@ fn run_transactions( >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &state_view); + .execute_transactions_parallel( + &env, + &transactions, + &state_view, + &global_module_cache, + ); if module_access.0 && module_access.1 { assert_matches!(output, Err(())); @@ -208,6 +215,8 @@ fn deltas_writes_mixed_with_block_gas_limit(num_txns: usize, maybe_block_gas_lim for _ in 0..20 { let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, MockEvent>, MockTask, MockEvent>, @@ -217,10 +226,14 @@ fn deltas_writes_mixed_with_block_gas_limit(num_txns: usize, maybe_block_gas_lim >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel( + &env, + &transactions, + &data_view, + &global_module_cache, + ); BaselineOutput::generate(&transactions, maybe_block_gas_limit) .assert_parallel_output(&output); @@ -261,6 +274,8 @@ fn deltas_resolver_with_block_gas_limit(num_txns: usize, maybe_block_gas_limit: for _ in 0..20 { let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, MockEvent>, MockTask, MockEvent>, @@ -270,10 +285,14 @@ fn deltas_resolver_with_block_gas_limit(num_txns: usize, maybe_block_gas_limit: >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool.clone(), - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel( + &env, + &transactions, + &data_view, + &global_module_cache, + ); BaselineOutput::generate(&transactions, maybe_block_gas_limit) .assert_parallel_output(&output); @@ -419,6 +438,8 @@ fn publishing_fixed_params_with_block_gas_limit( // Confirm still no intersection let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, MockEvent>, MockTask, MockEvent>, @@ -428,10 +449,9 @@ fn publishing_fixed_params_with_block_gas_limit( >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), maybe_block_gas_limit), executor_thread_pool, - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel(&env, &transactions, &data_view, &global_module_cache); assert_ok!(output); // Adjust the reads of txn indices[2] to contain module read to key 42. @@ -463,6 +483,8 @@ fn publishing_fixed_params_with_block_gas_limit( for _ in 0..200 { let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, MockEvent>, MockTask, MockEvent>, @@ -475,10 +497,14 @@ fn publishing_fixed_params_with_block_gas_limit( Some(max(w_index, r_index) as u64 * MAX_GAS_PER_TXN + 1), ), executor_thread_pool.clone(), - Arc::new(GlobalModuleCache::empty()), None, ) // Ensure enough gas limit to commit the module txns (4 is maximum gas per txn) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel( + &env, + &transactions, + &data_view, + &global_module_cache, + ); assert_matches!(output, Err(())); } @@ -547,6 +573,8 @@ fn non_empty_group( for _ in 0..num_repeat_parallel { let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, MockEvent>, MockTask, MockEvent>, @@ -556,16 +584,22 @@ fn non_empty_group( >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool.clone(), - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel( + &env, + &transactions, + &data_view, + &global_module_cache, + ); BaselineOutput::generate(&transactions, None).assert_parallel_output(&output); } for _ in 0..num_repeat_sequential { let env = MockEnvironment::new(); + let global_module_cache = RwLock::new(GlobalModuleCache::empty()); + let output = BlockExecutor::< MockTransaction, MockEvent>, MockTask, MockEvent>, @@ -575,10 +609,15 @@ fn non_empty_group( >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool.clone(), - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_sequential(&env, &transactions, &data_view, false); + .execute_transactions_sequential( + &env, + &transactions, + &data_view, + &global_module_cache, + false, + ); // TODO: test dynamic disabled as well. BaselineOutput::generate(&transactions, None).assert_output(&output.map_err(|e| match e { diff --git a/aptos-move/block-executor/src/unit_tests/mod.rs b/aptos-move/block-executor/src/unit_tests/mod.rs index 0cb4a946dbc27..7ef9fd793e4bf 100644 --- a/aptos-move/block-executor/src/unit_tests/mod.rs +++ b/aptos-move/block-executor/src/unit_tests/mod.rs @@ -35,6 +35,7 @@ use aptos_types::{ }; use claims::{assert_matches, assert_ok}; use fail::FailScenario; +use parking_lot::RwLock; use rand::{prelude::*, random}; use std::{ cmp::min, @@ -87,7 +88,6 @@ fn test_resource_group_deletion() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(GlobalModuleCache::empty()), None, ); @@ -96,9 +96,15 @@ fn test_resource_group_deletion() { &env, &transactions, &data_view, + &RwLock::new(GlobalModuleCache::empty()), false )); - assert_ok!(block_executor.execute_transactions_parallel(&env, &transactions, &data_view)); + assert_ok!(block_executor.execute_transactions_parallel( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()) + )); } #[test] @@ -154,13 +160,17 @@ fn resource_group_bcs_fallback() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(GlobalModuleCache::empty()), None, ); // Execute the block normally. let env = MockEnvironment::new(); - let output = block_executor.execute_transactions_parallel(&env, &transactions, &data_view); + let output = block_executor.execute_transactions_parallel( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + ); match output { Ok(block_output) => { let txn_outputs = block_output.into_transaction_outputs_forced(); @@ -179,12 +189,22 @@ fn resource_group_bcs_fallback() { assert!(!fail::list().is_empty()); let env = MockEnvironment::new(); - let par_output = block_executor.execute_transactions_parallel(&env, &transactions, &data_view); + let par_output = block_executor.execute_transactions_parallel( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + ); assert_matches!(par_output, Err(())); let env = MockEnvironment::new(); - let seq_output = - block_executor.execute_transactions_sequential(&env, &transactions, &data_view, false); + let seq_output = block_executor.execute_transactions_sequential( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + false, + ); assert_matches!( seq_output, Err(SequentialBlockExecutionError::ResourceGroupSerializationError) @@ -193,7 +213,13 @@ fn resource_group_bcs_fallback() { // Now execute with fallback handling for resource group serialization error: let env = MockEnvironment::new(); let fallback_output = block_executor - .execute_transactions_sequential(&env, &transactions, &data_view, true) + .execute_transactions_sequential( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + true, + ) .map_err(|e| match e { SequentialBlockExecutionError::ResourceGroupSerializationError => { panic!("Unexpected error") @@ -202,7 +228,12 @@ fn resource_group_bcs_fallback() { }); let env = MockEnvironment::new(); - let fallback_output_block = block_executor.execute_block(env, &transactions, &data_view); + let fallback_output_block = block_executor.execute_block( + env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + ); for output in [fallback_output, fallback_output_block] { match output { Ok(block_output) => { @@ -254,7 +285,6 @@ fn block_output_err_precedence() { >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(GlobalModuleCache::empty()), None, ); @@ -265,7 +295,12 @@ fn block_output_err_precedence() { // Pause the thread that processes the aborting txn1, so txn2 can halt the scheduler first. // Confirm that the fatal VM error is still detected and sequential fallback triggered. let env = MockEnvironment::new(); - let output = block_executor.execute_transactions_parallel(&env, &transactions, &data_view); + let output = block_executor.execute_transactions_parallel( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + ); assert_matches!(output, Err(())); scenario.teardown(); } @@ -294,13 +329,17 @@ fn skip_rest_gas_limit() { >::new( BlockExecutorConfig::new_maybe_block_limit(num_cpus::get(), Some(5)), executor_thread_pool, - Arc::new(GlobalModuleCache::empty()), None, ); // Should hit block limit on the skip transaction. let env = MockEnvironment::new(); - let _ = block_executor.execute_transactions_parallel(&env, &transactions, &data_view); + let _ = block_executor.execute_transactions_parallel( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + ); } // TODO: add unit test for block gas limit! @@ -330,10 +369,14 @@ where >::new( BlockExecutorConfig::new_no_block_limit(num_cpus::get()), executor_thread_pool, - Arc::new(GlobalModuleCache::empty()), None, ) - .execute_transactions_parallel(&env, &transactions, &data_view); + .execute_transactions_parallel( + &env, + &transactions, + &data_view, + &RwLock::new(GlobalModuleCache::empty()), + ); let baseline = BaselineOutput::generate(&transactions, None); baseline.assert_parallel_output(&output); From 878792e1ca93640fad3d3a9d32597d036da9d6d1 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 15:46:40 +0000 Subject: [PATCH 22/25] [benchmarks] Module loading benches (simple) --- crates/transaction-generator-lib/src/args.rs | 6 + .../src/publishing/module_simple.rs | 11 + .../src/publishing/raw_module_data.rs | 3149 ++++++++++++++++- .../Move.toml | 9 + .../sources/chain_dependencies.move | 1526 ++++++++ .../module_loading_chain_friends/Move.toml | 9 + .../sources/chain_friends.move | 1469 ++++++++ .../Move.toml | 9 + .../sources/star_dependencies.move | 1609 +++++++++ 9 files changed, 7783 insertions(+), 14 deletions(-) create mode 100644 testsuite/module-publish/src/packages/module_loading_chain_dependencies/Move.toml create mode 100644 testsuite/module-publish/src/packages/module_loading_chain_dependencies/sources/chain_dependencies.move create mode 100644 testsuite/module-publish/src/packages/module_loading_chain_friends/Move.toml create mode 100644 testsuite/module-publish/src/packages/module_loading_chain_friends/sources/chain_friends.move create mode 100644 testsuite/module-publish/src/packages/module_loading_star_dependencies/Move.toml create mode 100644 testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move diff --git a/crates/transaction-generator-lib/src/args.rs b/crates/transaction-generator-lib/src/args.rs index b19bf2651e8b4..c70fa28949e24 100644 --- a/crates/transaction-generator-lib/src/args.rs +++ b/crates/transaction-generator-lib/src/args.rs @@ -73,6 +73,9 @@ pub enum TransactionTypeArg { SmartTablePicture1MWith1KChangeExceedsLimit, DeserializeU256, SimpleScript, + ChainDependencies, + ChainFriends, + StarDependencies, } impl TransactionTypeArg { @@ -320,6 +323,9 @@ impl TransactionTypeArg { }, TransactionTypeArg::DeserializeU256 => call_custom_module(EntryPoints::DeserializeU256), TransactionTypeArg::SimpleScript => call_custom_module(EntryPoints::SimpleScript), + TransactionTypeArg::ChainDependencies => call_custom_module(EntryPoints::ChainDependencies), + TransactionTypeArg::ChainFriends => call_custom_module(EntryPoints::ChainFriends), + TransactionTypeArg::StarDependencies => call_custom_module(EntryPoints::StarDependencies), } } diff --git a/crates/transaction-generator-lib/src/publishing/module_simple.rs b/crates/transaction-generator-lib/src/publishing/module_simple.rs index 0bc7b959fb408..ce7afd699007e 100644 --- a/crates/transaction-generator-lib/src/publishing/module_simple.rs +++ b/crates/transaction-generator-lib/src/publishing/module_simple.rs @@ -272,6 +272,9 @@ pub enum EntryPoints { /// there to slow down deserialization & verification, effectively making it more expensive to /// load it into code cache. SimpleScript, + ChainDependencies, + ChainFriends, + StarDependencies, } impl EntryPoints { @@ -329,6 +332,9 @@ impl EntryPoints { EntryPoints::IncGlobalMilestoneAggV2 { .. } | EntryPoints::CreateGlobalMilestoneAggV2 { .. } => "aggregator_examples", EntryPoints::DeserializeU256 => "bcs_stream", + EntryPoints::ChainDependencies => "module_loading_chain_dependencies", + EntryPoints::ChainFriends => "module_loading_chain_friends", + EntryPoints::StarDependencies => "module_loading_star_dependencies", } } @@ -389,6 +395,9 @@ impl EntryPoints { EntryPoints::IncGlobalMilestoneAggV2 { .. } | EntryPoints::CreateGlobalMilestoneAggV2 { .. } => "counter_with_milestone", EntryPoints::DeserializeU256 => "bcs_stream", + EntryPoints::ChainDependencies => "module_loading_chain_dependencies", + EntryPoints::ChainFriends => "module_loading_chain_friends", + EntryPoints::StarDependencies => "module_loading_star_dependencies", } } @@ -732,6 +741,7 @@ impl EntryPoints { ], ) }, + EntryPoints::ChainDependencies | EntryPoints::ChainFriends | EntryPoints::StarDependencies => get_payload_void(module_id, ident_str!("run").to_owned()), } } @@ -840,6 +850,7 @@ impl EntryPoints { EntryPoints::DeserializeU256 => AutomaticArgs::None, EntryPoints::IncGlobalMilestoneAggV2 { .. } => AutomaticArgs::None, EntryPoints::CreateGlobalMilestoneAggV2 { .. } => AutomaticArgs::Signer, + EntryPoints::ChainDependencies | EntryPoints::ChainFriends | EntryPoints::StarDependencies => AutomaticArgs::None, } } } diff --git a/crates/transaction-generator-lib/src/publishing/raw_module_data.rs b/crates/transaction-generator-lib/src/publishing/raw_module_data.rs index 4deb1a72d1709..ba469c4cfcf58 100644 --- a/crates/transaction-generator-lib/src/publishing/raw_module_data.rs +++ b/crates/transaction-generator-lib/src/publishing/raw_module_data.rs @@ -15,6 +15,2121 @@ use once_cell::sync::Lazy; use std::collections::HashMap; +#[rustfmt::skip] +pub static PACKAGE_MODULE_LOADING_STAR_DEPENDENCIES_METADATA: Lazy> = Lazy::new(|| { + vec![ + 32, 109, 111, 100, 117, 108, 101, 95, 108, 111, 97, 100, 105, 110, 103, 95, 115, 116, + 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 1, 0, 0, + 0, 0, 0, 0, 0, 0, 64, 54, 50, 57, 66, 55, 56, 49, 50, 65, 52, 48, + 56, 69, 49, 52, 48, 51, 55, 66, 70, 57, 48, 55, 51, 56, 56, 66, 48, 53, + 51, 65, 48, 54, 66, 53, 54, 66, 65, 65, 68, 49, 53, 70, 50, 48, 67, 48, + 57, 70, 52, 67, 57, 57, 54, 54, 48, 54, 69, 70, 49, 54, 56, 55, 56, 170, + 1, 31, 139, 8, 0, 0, 0, 0, 0, 2, 255, 77, 141, 65, 14, 130, 48, 16, + 69, 247, 61, 69, 195, 134, 149, 21, 15, 224, 2, 245, 6, 46, 9, 105, 6, 58, + 98, 67, 233, 52, 29, 64, 19, 227, 221, 109, 13, 11, 146, 89, 76, 126, 222, 127, + 191, 9, 208, 143, 48, 96, 43, 60, 76, 40, 207, 178, 156, 200, 44, 14, 181, 35, + 48, 214, 15, 154, 103, 136, 218, 96, 64, 111, 208, 247, 22, 185, 20, 43, 70, 182, + 228, 51, 124, 82, 149, 170, 74, 33, 154, 61, 209, 138, 58, 204, 196, 247, 217, 56, + 219, 37, 234, 35, 29, 245, 224, 210, 87, 40, 117, 220, 31, 100, 238, 48, 209, 138, + 199, 71, 76, 251, 47, 138, 227, 22, 242, 191, 93, 200, 111, 146, 131, 49, 17, 153, + 179, 57, 44, 157, 179, 252, 196, 168, 183, 48, 91, 171, 119, 125, 185, 222, 10, 241, + 3, 239, 45, 19, 107, 205, 0, 0, 0, 21, 19, 116, 114, 101, 101, 95, 100, 101, + 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 57, 0, 0, 0, 19, 116, 114, + 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 56, 0, + 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 55, 0, 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, + 100, 101, 110, 99, 105, 101, 115, 95, 54, 0, 0, 0, 19, 116, 114, 101, 101, 95, + 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 53, 0, 0, 0, 19, + 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, + 52, 0, 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, + 99, 105, 101, 115, 95, 51, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, + 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 48, 0, 0, 0, 19, 116, 114, + 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 0, + 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 57, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 56, 0, 0, 0, 20, 116, 114, 101, + 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 55, 0, + 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 54, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 53, 0, 0, 0, 20, 116, 114, 101, + 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 52, 0, + 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 51, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 50, 0, 0, 0, 20, 116, 114, 101, + 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 49, 0, + 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 48, 0, 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 0, 0, 0, 17, 116, 114, 101, 101, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 0, 0, 0, 2, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 11, 65, 112, 116, 111, + 115, 83, 116, 100, 108, 105, 98, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1, 10, 77, 111, 118, 101, 83, 116, 100, 108, 105, 98, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_9: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 57, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_8: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 56, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_7: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 55, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_6: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 54, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_5: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 53, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_4: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 52, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_3: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 51, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_20: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 50, 48, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_2: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 50, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_19: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 57, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_18: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 56, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_17: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 55, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_16: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 54, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_15: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 53, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_14: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 52, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_13: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 51, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_12: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 50, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_11: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 49, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_10: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 204, 2, 13, 154, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 48, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, + 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, + 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, + 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, + 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, + 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, + 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, + 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, + 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, + 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, + 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, + 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, + 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, + 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, + 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, + 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, + 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, + 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, + 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, + 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, + 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_1: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, + 183, 2, 22, 12, 205, 2, 204, 2, 13, 153, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 49, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, + 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, + 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, + 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, + 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, + 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, + 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, + 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, + 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, + 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, + 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, + 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, + 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, + 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, + 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, + 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, + 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, + 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, + 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, + 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, + 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, + 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 7, 1, 0, 42, 3, 42, 126, 5, 168, 1, + 7, 7, 175, 1, 182, 3, 8, 229, 4, 32, 16, 133, 5, 31, 12, 164, 5, 130, + 2, 0, 0, 0, 2, 0, 4, 0, 5, 0, 6, 0, 7, 0, 8, 0, 9, 0, + 10, 0, 11, 0, 12, 0, 13, 0, 14, 0, 15, 0, 16, 0, 17, 0, 18, 0, + 19, 0, 20, 0, 21, 0, 22, 0, 1, 0, 0, 0, 1, 1, 3, 0, 1, 0, + 1, 2, 3, 0, 1, 0, 1, 3, 3, 0, 1, 0, 1, 4, 3, 0, 1, 0, + 1, 5, 3, 0, 1, 0, 1, 6, 3, 0, 1, 0, 1, 7, 3, 0, 1, 0, + 1, 8, 3, 0, 1, 0, 1, 9, 3, 0, 1, 0, 1, 10, 3, 0, 1, 0, + 1, 11, 3, 0, 1, 0, 1, 12, 3, 0, 1, 0, 1, 13, 3, 0, 1, 0, + 1, 14, 3, 0, 1, 0, 1, 15, 3, 0, 1, 0, 1, 16, 3, 0, 1, 0, + 1, 17, 3, 0, 1, 0, 1, 18, 3, 0, 1, 0, 1, 19, 3, 0, 1, 0, + 1, 20, 3, 0, 1, 0, 1, 0, 1, 3, 3, 3, 3, 3, 17, 116, 114, 101, + 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 3, 114, 117, 110, + 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, + 95, 49, 4, 110, 101, 120, 116, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, + 100, 101, 110, 99, 105, 101, 115, 95, 50, 19, 116, 114, 101, 101, 95, 100, 101, 112, + 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 51, 19, 116, 114, 101, 101, 95, 100, + 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 52, 19, 116, 114, 101, 101, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 53, 19, 116, 114, + 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 54, 19, + 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, + 55, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, + 115, 95, 56, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, + 105, 101, 115, 95, 57, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 49, 48, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 49, 20, 116, 114, 101, 101, 95, 100, + 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 50, 20, 116, 114, 101, + 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 51, 20, + 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, + 49, 52, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 53, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 49, 54, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 55, 20, 116, 114, 101, 101, 95, 100, + 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 56, 20, 116, 114, 101, + 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 57, 20, + 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, + 50, 48, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 20, 99, + 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, + 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 1, 4, 0, 2, 127, 17, + 1, 12, 1, 6, 0, 0, 0, 0, 0, 0, 0, 0, 11, 1, 22, 12, 0, 17, + 2, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 3, 12, 1, 11, 0, 11, 1, + 22, 12, 0, 17, 4, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 5, 12, 1, + 11, 0, 11, 1, 22, 12, 0, 17, 6, 12, 1, 11, 0, 11, 1, 22, 12, 0, + 17, 7, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 8, 12, 1, 11, 0, 11, + 1, 22, 12, 0, 17, 9, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 10, 12, + 1, 11, 0, 11, 1, 22, 12, 0, 17, 11, 12, 1, 11, 0, 11, 1, 22, 12, + 0, 17, 12, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 13, 12, 1, 11, 0, + 11, 1, 22, 12, 0, 17, 14, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 15, + 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 16, 12, 1, 11, 0, 11, 1, 22, + 12, 0, 17, 17, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, 18, 12, 1, 11, + 0, 11, 1, 22, 12, 0, 17, 19, 12, 1, 11, 0, 11, 1, 22, 12, 0, 17, + 20, 12, 1, 11, 0, 11, 1, 22, 12, 2, 11, 2, 6, 20, 0, 0, 0, 0, + 0, 0, 0, 33, 4, 125, 2, 6, 77, 0, 0, 0, 0, 0, 0, 0, 39, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULES_MODULE_LOADING_STAR_DEPENDENCIES: Lazy>> = Lazy::new(|| { vec![ + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_9.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_8.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_7.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_6.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_5.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_4.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_3.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_20.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_2.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_19.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_18.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_17.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_16.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_15.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_14.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_13.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_12.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_11.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_10.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_1.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES.to_vec(), +]}); + +#[rustfmt::skip] +pub static PACKAGE_MODULE_LOADING_CHAIN_DEPENDENCIES_METADATA: Lazy> = Lazy::new(|| { + vec![ + 33, 109, 111, 100, 117, 108, 101, 95, 108, 111, 97, 100, 105, 110, 103, 95, 99, 104, + 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 1, 0, + 0, 0, 0, 0, 0, 0, 0, 64, 48, 53, 57, 68, 48, 55, 53, 69, 53, 49, + 56, 65, 67, 49, 67, 54, 65, 66, 54, 56, 48, 70, 51, 50, 53, 67, 57, 67, + 65, 51, 48, 55, 68, 57, 55, 69, 49, 49, 70, 49, 53, 66, 65, 52, 54, 67, + 65, 49, 66, 52, 51, 67, 70, 53, 67, 53, 48, 56, 65, 51, 54, 53, 70, 70, + 171, 1, 31, 139, 8, 0, 0, 0, 0, 0, 2, 255, 77, 142, 65, 10, 131, 48, + 16, 69, 247, 57, 69, 112, 227, 170, 209, 30, 160, 11, 219, 222, 160, 75, 145, 48, + 38, 83, 13, 198, 36, 100, 212, 22, 74, 239, 222, 164, 184, 16, 102, 49, 124, 222, + 127, 252, 54, 128, 154, 96, 192, 142, 57, 152, 145, 95, 120, 57, 123, 189, 90, 148, + 214, 131, 54, 110, 144, 106, 4, 227, 164, 198, 128, 78, 163, 83, 6, 169, 100, 27, + 70, 50, 222, 101, 250, 44, 106, 81, 151, 140, 181, 71, 162, 99, 77, 88, 60, 61, + 22, 109, 77, 159, 168, 15, 183, 94, 129, 77, 95, 33, 68, 117, 60, 200, 220, 105, + 246, 27, 86, 207, 152, 6, 188, 124, 156, 246, 144, 254, 237, 130, 127, 147, 28, 180, + 142, 72, 148, 205, 97, 237, 173, 161, 17, 163, 220, 195, 108, 173, 223, 205, 245, 118, + 47, 216, 15, 168, 92, 114, 50, 206, 0, 0, 0, 21, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 48, 0, 0, + 0, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 57, 0, 0, 0, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, + 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 56, 0, 0, 0, 21, 99, 104, + 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, + 55, 0, 0, 0, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 49, 54, 0, 0, 0, 21, 99, 104, 97, 105, 110, 95, + 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 53, 0, 0, 0, + 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, + 115, 95, 49, 52, 0, 0, 0, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 51, 0, 0, 0, 21, 99, 104, 97, + 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 50, + 0, 0, 0, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, + 99, 105, 101, 115, 95, 49, 49, 0, 0, 0, 21, 99, 104, 97, 105, 110, 95, 100, + 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 48, 0, 0, 0, 20, + 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, + 95, 57, 0, 0, 0, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 56, 0, 0, 0, 20, 99, 104, 97, 105, 110, 95, + 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 55, 0, 0, 0, 20, + 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, + 95, 54, 0, 0, 0, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 53, 0, 0, 0, 20, 99, 104, 97, 105, 110, 95, + 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 52, 0, 0, 0, 20, + 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, + 95, 51, 0, 0, 0, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 50, 0, 0, 0, 20, 99, 104, 97, 105, 110, 95, + 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 0, 0, 0, 18, + 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, + 0, 0, 0, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, + 11, 65, 112, 116, 111, 115, 83, 116, 100, 108, 105, 98, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1, 10, 77, 111, 118, 101, 83, 116, 100, 108, 105, + 98, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_20: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 119, 8, 218, 1, 64, 16, 154, 2, 31, 10, + 185, 2, 22, 12, 207, 2, 204, 2, 13, 155, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, + 99, 105, 101, 115, 95, 50, 48, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, + 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, + 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, + 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, + 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, + 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, + 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, + 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, + 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, + 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, + 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, + 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, + 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, + 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, + 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, + 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, + 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, + 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, + 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, + 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, + 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, + 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, + 0, 4, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_19: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 57, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 50, 48, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_18: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 56, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 57, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_17: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 55, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 56, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_16: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 54, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 55, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_15: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 53, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 54, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_14: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 52, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 53, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_13: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 51, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 52, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_12: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 50, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 51, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_11: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 49, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 50, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_10: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 141, 1, 8, 248, 1, 64, 16, 184, 2, 31, + 10, 215, 2, 22, 12, 237, 2, 207, 2, 13, 188, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 21, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 48, 7, 67, + 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, + 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, + 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, + 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, + 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, + 4, 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, + 101, 110, 99, 105, 101, 115, 95, 49, 49, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_9: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 140, 1, 8, 247, 1, 64, 16, 183, 2, 31, + 10, 214, 2, 22, 12, 236, 2, 207, 2, 13, 187, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 57, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 21, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 49, 48, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, + 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, + 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, + 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, + 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, + 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, + 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, + 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, + 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, + 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, + 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, + 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, + 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, + 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, + 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, + 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, + 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, + 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, + 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, + 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, + 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, + 0, 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_8: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 56, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 57, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_7: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 55, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 56, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_6: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 54, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 55, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_5: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 53, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 54, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_4: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 52, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 53, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_3: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 51, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 52, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_2: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 51, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_1: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 8, 2, 8, 16, 3, 24, 31, + 4, 55, 2, 5, 57, 50, 7, 107, 139, 1, 8, 246, 1, 64, 16, 182, 2, 31, + 10, 213, 2, 22, 12, 235, 2, 207, 2, 13, 186, 5, 4, 0, 0, 1, 9, 1, + 12, 0, 15, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, + 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, + 0, 1, 0, 14, 4, 5, 0, 1, 3, 14, 4, 5, 0, 1, 2, 6, 4, 6, + 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, 1, 6, 3, 4, 6, 3, 6, + 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, 1, 10, 3, 1, 6, 9, 0, + 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, 2, 20, 99, 104, 97, 105, 110, + 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 7, 67, 111, + 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, 4, 100, + 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, 110, 97, + 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, 14, 99, + 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, 111, 112, + 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 4, + 110, 101, 120, 116, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, + 110, 99, 105, 101, 115, 95, 50, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, + 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, + 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, + 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, + 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, + 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, + 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, + 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, + 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, + 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, + 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, + 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, + 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, + 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, + 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, + 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, + 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, + 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, + 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, + 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, + 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 4, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 17, 4, 22, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 7, 1, 0, 4, 3, 4, 12, 5, 16, 3, + 7, 19, 49, 8, 68, 32, 16, 100, 31, 12, 131, 1, 31, 0, 0, 0, 2, 0, + 1, 0, 0, 0, 1, 1, 3, 0, 1, 0, 1, 0, 1, 3, 18, 99, 104, 97, + 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 3, 114, 117, + 110, 20, 99, 104, 97, 105, 110, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, + 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, + 0, 1, 4, 0, 0, 7, 17, 1, 6, 20, 0, 0, 0, 0, 0, 0, 0, 33, + 4, 5, 2, 6, 77, 0, 0, 0, 0, 0, 0, 0, 39, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULES_MODULE_LOADING_CHAIN_DEPENDENCIES: Lazy>> = Lazy::new(|| { vec![ + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_20.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_19.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_18.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_17.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_16.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_15.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_14.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_13.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_12.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_11.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_10.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_9.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_8.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_7.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_6.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_5.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_4.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_3.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_2.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES_1.to_vec(), + MODULE_MODULE_LOADING_CHAIN_DEPENDENCIES_CHAIN_DEPENDENCIES.to_vec(), +]}); + #[rustfmt::skip] pub static PACKAGE_COMPLEX_METADATA: Lazy> = Lazy::new(|| { vec![ @@ -735,14 +2850,1024 @@ pub static MODULES_COMPLEX: Lazy>> = Lazy::new(|| { vec![ MODULE_COMPLEX_VECTOR_PICTURE.to_vec(), ]}); +#[rustfmt::skip] +pub static PACKAGE_MODULE_LOADING_CHAIN_FRIENDS_METADATA: Lazy> = Lazy::new(|| { + vec![ + 28, 109, 111, 100, 117, 108, 101, 95, 108, 111, 97, 100, 105, 110, 103, 95, 99, 104, + 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 1, 0, 0, 0, 0, 0, 0, + 0, 0, 64, 56, 52, 57, 52, 57, 65, 49, 56, 67, 56, 68, 67, 55, 66, 69, + 67, 69, 69, 56, 51, 66, 56, 53, 48, 70, 68, 53, 68, 50, 51, 69, 57, 48, + 69, 65, 50, 66, 54, 51, 56, 65, 48, 52, 56, 55, 54, 55, 53, 52, 67, 54, + 54, 65, 50, 54, 55, 55, 51, 54, 50, 53, 54, 51, 69, 173, 1, 31, 139, 8, + 0, 0, 0, 0, 0, 2, 255, 77, 141, 49, 14, 131, 48, 12, 69, 247, 156, 34, + 98, 97, 106, 160, 7, 232, 64, 219, 27, 116, 68, 40, 50, 137, 129, 136, 144, 68, + 49, 208, 74, 85, 239, 222, 164, 98, 168, 228, 193, 250, 126, 126, 191, 13, 160, 102, + 24, 177, 99, 14, 22, 228, 23, 94, 46, 94, 111, 22, 165, 245, 160, 141, 27, 165, + 154, 192, 56, 57, 68, 131, 78, 83, 201, 118, 140, 100, 188, 203, 224, 89, 212, 162, + 46, 25, 107, 53, 134, 116, 68, 167, 12, 82, 199, 154, 176, 122, 122, 172, 218, 154, + 62, 81, 111, 110, 189, 2, 155, 182, 66, 136, 234, 127, 32, 115, 167, 197, 239, 88, + 13, 49, 117, 63, 125, 156, 143, 144, 126, 223, 5, 255, 36, 57, 104, 29, 145, 40, + 155, 195, 214, 91, 67, 19, 70, 121, 132, 217, 90, 191, 154, 235, 237, 94, 176, 47, + 31, 68, 110, 190, 201, 0, 0, 0, 21, 13, 99, 104, 97, 105, 110, 95, 102, 114, + 105, 101, 110, 100, 115, 0, 0, 0, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, + 101, 110, 100, 115, 95, 49, 0, 0, 0, 15, 99, 104, 97, 105, 110, 95, 102, 114, + 105, 101, 110, 100, 115, 95, 50, 0, 0, 0, 15, 99, 104, 97, 105, 110, 95, 102, + 114, 105, 101, 110, 100, 115, 95, 51, 0, 0, 0, 15, 99, 104, 97, 105, 110, 95, + 102, 114, 105, 101, 110, 100, 115, 95, 52, 0, 0, 0, 15, 99, 104, 97, 105, 110, + 95, 102, 114, 105, 101, 110, 100, 115, 95, 53, 0, 0, 0, 15, 99, 104, 97, 105, + 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 54, 0, 0, 0, 15, 99, 104, 97, + 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 55, 0, 0, 0, 15, 99, 104, + 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 56, 0, 0, 0, 15, 99, + 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 57, 0, 0, 0, 16, + 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 48, 0, 0, + 0, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 49, + 0, 0, 0, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, + 49, 50, 0, 0, 0, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, + 115, 95, 49, 51, 0, 0, 0, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, + 110, 100, 115, 95, 49, 52, 0, 0, 0, 16, 99, 104, 97, 105, 110, 95, 102, 114, + 105, 101, 110, 100, 115, 95, 49, 53, 0, 0, 0, 16, 99, 104, 97, 105, 110, 95, + 102, 114, 105, 101, 110, 100, 115, 95, 49, 54, 0, 0, 0, 16, 99, 104, 97, 105, + 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 55, 0, 0, 0, 16, 99, 104, + 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 56, 0, 0, 0, 16, + 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 57, 0, 0, + 0, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 50, 48, + 0, 0, 0, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, + 11, 65, 112, 116, 111, 115, 83, 116, 100, 108, 105, 98, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1, 10, 77, 111, 118, 101, 83, 116, 100, 108, 105, + 98, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 8, 1, 0, 2, 3, 2, 6, 5, 8, 1, + 7, 9, 34, 8, 43, 32, 16, 75, 31, 12, 106, 7, 15, 113, 2, 0, 0, 0, + 1, 0, 0, 0, 1, 0, 13, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, + 100, 115, 3, 114, 117, 110, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, + 100, 115, 95, 49, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, + 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, + 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 1, 4, 0, 0, + 1, 2, 0, 2, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_1: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 50, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_2: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 50, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 51, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_3: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 51, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 52, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_4: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 52, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 53, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_5: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 53, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 54, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_6: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 54, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 55, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_7: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 55, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 56, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_8: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 124, 8, 217, 1, 64, 16, 153, 2, 31, 10, + 184, 2, 22, 12, 206, 2, 188, 2, 13, 138, 5, 4, 15, 142, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 56, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 57, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, + 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, + 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, + 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, + 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, + 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, + 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, + 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, + 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, + 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, + 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, + 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, + 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, + 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, + 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, + 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, + 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, + 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_9: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 125, 8, 218, 1, 64, 16, 154, 2, 31, 10, + 185, 2, 22, 12, 207, 2, 188, 2, 13, 139, 5, 4, 15, 143, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 15, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 57, 7, + 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, 97, + 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, 4, + 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, 103, + 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, 111, + 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, + 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 48, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, + 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, + 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, 2, + 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, 16, + 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, 92, + 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, 20, + 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, 12, + 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, 5, + 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, 12, + 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, 5, + 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, 34, + 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, 0, + 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, 16, + 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, 64, + 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, 11, + 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, 0, + 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, 22, + 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, 18, + 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_10: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 48, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 49, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_11: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 49, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 50, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_12: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 50, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 51, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_13: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 51, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 52, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_14: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 52, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 53, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_15: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 53, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 54, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_16: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 54, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 55, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_17: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 55, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 56, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_18: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 56, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, + 57, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_19: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 12, 1, 0, 6, 2, 6, 16, 3, 22, 19, + 4, 41, 2, 5, 43, 50, 7, 93, 126, 8, 219, 1, 64, 16, 155, 2, 31, 10, + 186, 2, 22, 12, 208, 2, 188, 2, 13, 140, 5, 4, 15, 144, 5, 2, 0, 0, + 1, 9, 1, 12, 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, + 7, 0, 0, 10, 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, + 1, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, 8, 0, + 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, 1, 3, + 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, 3, 10, + 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 49, 57, + 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, 97, 116, + 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, 105, 100, + 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, 105, 110, + 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, 8, 108, + 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, 121, 116, + 101, 115, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, 50, + 48, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 171, 205, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, + 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, + 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, 3, 1, 2, 1, 4, 10, 2, 2, + 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, 0, 1, 0, 0, 2, 101, 10, 0, + 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, 11, 4, 20, 10, 5, 20, 35, 4, + 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, 5, 10, 5, 20, 10, 1, 16, 0, + 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, 11, 2, 1, 11, 4, 1, 11, 5, + 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, 20, 10, 5, 20, 35, 4, 62, 11, + 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, 33, 4, 57, 11, 5, 1, 11, 4, + 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, 12, 7, 5, 55, 11, 4, 1, 10, + 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, 16, 0, 12, 7, 10, 4, 11, 7, + 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, 2, 16, 1, 12, 4, 11, 1, 16, + 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, 1, 5, 37, 11, 5, 1, 10, 1, + 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, 5, 17, 1, 1, 0, 0, 10, 43, + 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, 4, 16, 13, 2, 10, 3, 68, 5, + 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 3, 5, 4, 6, 0, + 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, 0, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, 5, 11, 1, 12, 3, 14, 5, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, 20, 52, 12, 4, 11, 3, 11, 4, + 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 0, 5, + 18, 2, 2, 0, 0, 0, 0, 14, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_20: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, + 4, 47, 2, 5, 49, 50, 7, 99, 114, 8, 213, 1, 64, 16, 149, 2, 31, 10, + 180, 2, 22, 12, 202, 2, 204, 2, 13, 150, 5, 4, 0, 0, 1, 9, 1, 12, + 0, 1, 8, 0, 0, 3, 7, 0, 0, 5, 8, 0, 1, 8, 7, 0, 0, 10, + 0, 1, 0, 1, 0, 11, 3, 4, 0, 1, 2, 13, 7, 8, 1, 0, 1, 0, + 14, 4, 5, 0, 1, 2, 6, 4, 6, 8, 2, 6, 8, 2, 6, 8, 0, 6, + 8, 0, 1, 6, 3, 4, 6, 3, 6, 3, 6, 3, 6, 3, 2, 3, 3, 0, + 1, 3, 1, 10, 3, 1, 6, 9, 0, 1, 10, 2, 1, 2, 4, 10, 3, 3, + 3, 10, 2, 16, 99, 104, 97, 105, 110, 95, 102, 114, 105, 101, 110, 100, 115, 95, + 50, 48, 7, 67, 111, 117, 110, 116, 101, 114, 5, 99, 111, 117, 110, 116, 4, 68, + 97, 116, 97, 4, 100, 97, 116, 97, 8, 82, 101, 115, 111, 117, 114, 99, 101, 2, + 105, 100, 4, 110, 97, 109, 101, 6, 83, 116, 114, 105, 110, 103, 6, 115, 116, 114, + 105, 110, 103, 14, 99, 111, 112, 121, 95, 112, 97, 115, 116, 97, 95, 114, 101, 102, + 8, 108, 111, 111, 112, 95, 98, 99, 115, 3, 98, 99, 115, 8, 116, 111, 95, 98, + 121, 116, 101, 115, 4, 110, 101, 120, 116, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 171, 205, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, 116, 97, + 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 2, 1, 2, + 3, 1, 2, 1, 4, 10, 2, 2, 2, 3, 6, 3, 7, 8, 3, 4, 8, 1, + 0, 1, 0, 0, 2, 101, 10, 0, 16, 0, 12, 4, 10, 1, 16, 0, 12, 5, + 11, 4, 20, 10, 5, 20, 35, 4, 92, 11, 5, 12, 4, 10, 2, 16, 1, 12, + 5, 10, 5, 20, 10, 1, 16, 0, 20, 35, 4, 67, 11, 0, 1, 11, 1, 1, + 11, 2, 1, 11, 4, 1, 11, 5, 12, 4, 11, 3, 16, 1, 12, 5, 10, 4, + 20, 10, 5, 20, 35, 4, 62, 11, 5, 1, 10, 4, 12, 5, 10, 4, 10, 5, + 33, 4, 57, 11, 5, 1, 11, 4, 12, 7, 11, 7, 2, 11, 4, 1, 11, 5, + 12, 7, 5, 55, 11, 4, 1, 10, 5, 12, 4, 5, 47, 11, 3, 1, 11, 0, + 16, 0, 12, 7, 10, 4, 11, 7, 34, 4, 87, 11, 4, 1, 11, 5, 1, 11, + 2, 16, 1, 12, 4, 11, 1, 16, 0, 12, 5, 5, 37, 11, 1, 1, 11, 2, + 1, 5, 37, 11, 5, 1, 10, 1, 16, 0, 12, 4, 10, 3, 16, 1, 12, 5, + 5, 17, 1, 1, 0, 0, 10, 43, 64, 5, 0, 0, 0, 0, 0, 0, 0, 0, + 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 3, 10, 1, 35, + 4, 16, 13, 2, 10, 3, 68, 5, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, + 0, 22, 12, 3, 5, 4, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 1, 10, + 0, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 42, 14, 2, 56, 0, 12, + 5, 11, 1, 12, 3, 14, 5, 6, 0, 0, 0, 0, 0, 0, 0, 0, 66, 9, + 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 1, 11, 0, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 0, 5, 18, 2, 3, 1, 0, 0, 4, 2, 6, 1, + 0, 0, 0, 0, 0, 0, 0, 2, 2, 0, 0, 0, 0, + ] +}); + +#[rustfmt::skip] +pub static MODULES_MODULE_LOADING_CHAIN_FRIENDS: Lazy>> = Lazy::new(|| { vec![ + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_1.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_2.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_3.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_4.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_5.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_6.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_7.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_8.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_9.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_10.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_11.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_12.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_13.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_14.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_15.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_16.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_17.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_18.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_19.to_vec(), + MODULE_MODULE_LOADING_CHAIN_FRIENDS_CHAIN_FRIENDS_20.to_vec(), +]}); + #[rustfmt::skip] pub static PACKAGE_SIMPLE_METADATA: Lazy> = Lazy::new(|| { vec![ 13, 71, 101, 110, 101, 114, 105, 99, 77, 111, 100, 117, 108, 101, 1, 0, 0, 0, - 0, 0, 0, 0, 0, 64, 69, 53, 53, 57, 68, 57, 51, 68, 67, 55, 70, 65, - 56, 70, 70, 66, 70, 53, 70, 65, 57, 48, 49, 52, 57, 69, 49, 65, 56, 48, - 56, 55, 57, 67, 65, 52, 56, 50, 66, 67, 52, 67, 48, 50, 48, 51, 51, 50, - 53, 53, 52, 67, 50, 67, 48, 52, 49, 56, 67, 54, 56, 56, 53, 69, 132, 1, + 0, 0, 0, 0, 0, 64, 68, 48, 69, 53, 68, 55, 70, 65, 50, 66, 65, 54, + 68, 50, 65, 48, 57, 55, 55, 56, 51, 49, 51, 67, 57, 68, 69, 51, 56, 67, + 69, 69, 68, 49, 49, 49, 68, 65, 70, 49, 67, 66, 53, 53, 56, 54, 52, 65, + 67, 65, 52, 55, 56, 49, 53, 67, 65, 52, 53, 57, 66, 49, 66, 66, 132, 1, 31, 139, 8, 0, 0, 0, 0, 0, 2, 255, 77, 139, 59, 14, 194, 48, 16, 68, 251, 61, 133, 229, 30, 135, 11, 80, 208, 64, 197, 9, 162, 20, 43, 123, 64, 86, 156, 93, 203, 134, 80, 32, 238, 142, 45, 1, 138, 102, 154, 249, 188, 49, 179, 159, @@ -762,16 +3887,6 @@ pub static PACKAGE_SIMPLE_METADATA: Lazy> = Lazy::new(|| { ] }); -#[rustfmt::skip] -pub static SCRIPT_SIMPLE: Lazy> = Lazy::new(|| { - vec![ - 161, 28, 235, 11, 7, 0, 0, 10, 2, 5, 0, 4, 6, 4, 34, 1, 6, 12, - 0, 5, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, - 0, 1, 3, 11, 0, 1, 2, - ] -}); - #[rustfmt::skip] pub static MODULE_SIMPLE_SIMPLE: Lazy> = Lazy::new(|| { vec![ @@ -2159,7 +5274,10 @@ pub static MODULES_BCS_STREAM: Lazy>> = Lazy::new(|| { vec![ #[rustfmt::skip] pub static PACKAGE_TO_METADATA: Lazy>> = Lazy::new(|| { HashMap::from([ + ("module_loading_star_dependencies".to_string(), PACKAGE_MODULE_LOADING_STAR_DEPENDENCIES_METADATA.to_vec()), + ("module_loading_chain_dependencies".to_string(), PACKAGE_MODULE_LOADING_CHAIN_DEPENDENCIES_METADATA.to_vec()), ("complex".to_string(), PACKAGE_COMPLEX_METADATA.to_vec()), + ("module_loading_chain_friends".to_string(), PACKAGE_MODULE_LOADING_CHAIN_FRIENDS_METADATA.to_vec()), ("simple".to_string(), PACKAGE_SIMPLE_METADATA.to_vec()), ("framework_usecases".to_string(), PACKAGE_FRAMEWORK_USECASES_METADATA.to_vec()), ("ambassador_token".to_string(), PACKAGE_AMBASSADOR_TOKEN_METADATA.to_vec()), @@ -2169,7 +5287,10 @@ pub static PACKAGE_TO_METADATA: Lazy>> = Lazy::new(|| { #[rustfmt::skip] pub static PACKAGE_TO_MODULES: Lazy>>> = Lazy::new(|| { HashMap::from([ + ("module_loading_star_dependencies".to_string(), MODULES_MODULE_LOADING_STAR_DEPENDENCIES.to_vec()), + ("module_loading_chain_dependencies".to_string(), MODULES_MODULE_LOADING_CHAIN_DEPENDENCIES.to_vec()), ("complex".to_string(), MODULES_COMPLEX.to_vec()), + ("module_loading_chain_friends".to_string(), MODULES_MODULE_LOADING_CHAIN_FRIENDS.to_vec()), ("simple".to_string(), MODULES_SIMPLE.to_vec()), ("framework_usecases".to_string(), MODULES_FRAMEWORK_USECASES.to_vec()), ("ambassador_token".to_string(), MODULES_AMBASSADOR_TOKEN.to_vec()), diff --git a/testsuite/module-publish/src/packages/module_loading_chain_dependencies/Move.toml b/testsuite/module-publish/src/packages/module_loading_chain_dependencies/Move.toml new file mode 100644 index 0000000000000..5a4a2c24bf0e5 --- /dev/null +++ b/testsuite/module-publish/src/packages/module_loading_chain_dependencies/Move.toml @@ -0,0 +1,9 @@ +[package] +name = 'module_loading_chain_dependencies' +version = '1.0.0' + +[dependencies] +AptosStdlib = { local = "../../../../../aptos-move/framework/aptos-stdlib" } + +[addresses] +publisher_address = "0xABCD" diff --git a/testsuite/module-publish/src/packages/module_loading_chain_dependencies/sources/chain_dependencies.move b/testsuite/module-publish/src/packages/module_loading_chain_dependencies/sources/chain_dependencies.move new file mode 100644 index 0000000000000..d19aaa59cf0e3 --- /dev/null +++ b/testsuite/module-publish/src/packages/module_loading_chain_dependencies/sources/chain_dependencies.move @@ -0,0 +1,1526 @@ +module publisher_address::chain_dependencies { + + public entry fun run() { + let sum = publisher_address::chain_dependencies_1::next(); + assert!(sum == 20, 77); + } +} + +module publisher_address::chain_dependencies_1 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_2::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_2 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_3::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_3 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_4::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_4 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_5::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_5 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_6::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_6 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_7::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_7 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_8::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_8 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_9::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_9 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_10::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_10 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_11::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_11 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_12::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_12 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_13::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_13 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_14::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_14 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_15::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_15 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_16::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_16 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_17::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_17 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_18::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_18 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_19::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_19 { + + public fun next(): u64 { + 1 + publisher_address::chain_dependencies_20::next() + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_dependencies_20 { + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} diff --git a/testsuite/module-publish/src/packages/module_loading_chain_friends/Move.toml b/testsuite/module-publish/src/packages/module_loading_chain_friends/Move.toml new file mode 100644 index 0000000000000..fe6f6ab77c285 --- /dev/null +++ b/testsuite/module-publish/src/packages/module_loading_chain_friends/Move.toml @@ -0,0 +1,9 @@ +[package] +name = 'module_loading_chain_friends' +version = '1.0.0' + +[dependencies] +AptosStdlib = { local = "../../../../../aptos-move/framework/aptos-stdlib" } + +[addresses] +publisher_address = "0xABCD" diff --git a/testsuite/module-publish/src/packages/module_loading_chain_friends/sources/chain_friends.move b/testsuite/module-publish/src/packages/module_loading_chain_friends/sources/chain_friends.move new file mode 100644 index 0000000000000..1623e10fe5b4c --- /dev/null +++ b/testsuite/module-publish/src/packages/module_loading_chain_friends/sources/chain_friends.move @@ -0,0 +1,1469 @@ +module publisher_address::chain_friends { + friend publisher_address::chain_friends_1; + + public entry fun run() { + // no-op + } +} + +module publisher_address::chain_friends_1 { + friend publisher_address::chain_friends_2; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_2 { + friend publisher_address::chain_friends_3; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_3 { + friend publisher_address::chain_friends_4; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_4 { + friend publisher_address::chain_friends_5; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_5 { + friend publisher_address::chain_friends_6; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_6 { + friend publisher_address::chain_friends_7; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_7 { + friend publisher_address::chain_friends_8; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_8 { + friend publisher_address::chain_friends_9; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_9 { + friend publisher_address::chain_friends_10; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_10 { + friend publisher_address::chain_friends_11; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_11 { + friend publisher_address::chain_friends_12; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_12 { + friend publisher_address::chain_friends_13; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_13 { + friend publisher_address::chain_friends_14; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_14 { + friend publisher_address::chain_friends_15; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_15 { + friend publisher_address::chain_friends_16; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_16 { + friend publisher_address::chain_friends_17; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_17 { + friend publisher_address::chain_friends_18; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_18 { + friend publisher_address::chain_friends_19; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_19 { + friend publisher_address::chain_friends_20; + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::chain_friends_20 { + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} diff --git a/testsuite/module-publish/src/packages/module_loading_star_dependencies/Move.toml b/testsuite/module-publish/src/packages/module_loading_star_dependencies/Move.toml new file mode 100644 index 0000000000000..b4861ac4a5507 --- /dev/null +++ b/testsuite/module-publish/src/packages/module_loading_star_dependencies/Move.toml @@ -0,0 +1,9 @@ +[package] +name = 'module_loading_star_dependencies' +version = '1.0.0' + +[dependencies] +AptosStdlib = { local = "../../../../../aptos-move/framework/aptos-stdlib" } + +[addresses] +publisher_address = "0xABCD" diff --git a/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move b/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move new file mode 100644 index 0000000000000..91ac83d7da131 --- /dev/null +++ b/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move @@ -0,0 +1,1609 @@ +module publisher_address::tree_dependencies { + use publisher_address::tree_dependencies_1; + use publisher_address::tree_dependencies_2; + use publisher_address::tree_dependencies_3; + use publisher_address::tree_dependencies_4; + use publisher_address::tree_dependencies_5; + use publisher_address::tree_dependencies_6; + use publisher_address::tree_dependencies_7; + use publisher_address::tree_dependencies_8; + use publisher_address::tree_dependencies_9; + use publisher_address::tree_dependencies_10; + use publisher_address::tree_dependencies_11; + use publisher_address::tree_dependencies_12; + use publisher_address::tree_dependencies_13; + use publisher_address::tree_dependencies_14; + use publisher_address::tree_dependencies_15; + use publisher_address::tree_dependencies_16; + use publisher_address::tree_dependencies_17; + use publisher_address::tree_dependencies_18; + use publisher_address::tree_dependencies_19; + use publisher_address::tree_dependencies_20; + + public entry fun run() { + let sum = 0; + + sum = sum + tree_dependencies_1::next(); + sum = sum + tree_dependencies_2::next(); + sum = sum + tree_dependencies_3::next(); + sum = sum + tree_dependencies_4::next(); + sum = sum + tree_dependencies_5::next(); + sum = sum + tree_dependencies_6::next(); + sum = sum + tree_dependencies_7::next(); + sum = sum + tree_dependencies_8::next(); + sum = sum + tree_dependencies_9::next(); + sum = sum + tree_dependencies_10::next(); + sum = sum + tree_dependencies_11::next(); + sum = sum + tree_dependencies_12::next(); + sum = sum + tree_dependencies_13::next(); + sum = sum + tree_dependencies_14::next(); + sum = sum + tree_dependencies_15::next(); + sum = sum + tree_dependencies_16::next(); + sum = sum + tree_dependencies_17::next(); + sum = sum + tree_dependencies_18::next(); + sum = sum + tree_dependencies_19::next(); + sum = sum + tree_dependencies_20::next(); + + assert!(sum == 20, 77); + } +} + +module publisher_address::tree_dependencies_1 { + + const MAGIC: u64 = 1; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_2 { + + const MAGIC: u64 = 2; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_3 { + + const MAGIC: u64 = 3; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_4 { + + const MAGIC: u64 = 4; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_5 { + + const MAGIC: u64 = 5; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_6 { + + const MAGIC: u64 = 6; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_7 { + + const MAGIC: u64 = 7; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_8 { + + const MAGIC: u64 = 8; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_9 { + + const MAGIC: u64 = 9; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_10 { + + const MAGIC: u64 = 10; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_11 { + + const MAGIC: u64 = 11; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_12 { + + const MAGIC: u64 = 12; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_13 { + + const MAGIC: u64 = 13; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_14 { + + const MAGIC: u64 = 14; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_15 { + + const MAGIC: u64 = 15; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_16 { + + const MAGIC: u64 = 16; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_17 { + + const MAGIC: u64 = 17; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_18 { + + const MAGIC: u64 = 18; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_19 { + + const MAGIC: u64 = 19; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} + +module publisher_address::tree_dependencies_20 { + + const MAGIC: u64 = 20; + + public fun next(): u64 { + 1 + } + + // Functions bellow are used to make module verification a bit more expensive. + + struct Data has copy, drop, store { + data: vector, + } + + struct Resource has key { + id: u64, + name: aptos_std::string::String, + data: Data, + } + + struct Counter has key { + count: u64, + } + + public fun copy_pasta_ref( + r1: &Resource, + r2: &Resource, + c1: &Counter, + c2: &Counter, + ): &u64 { + let ret1 = &r1.id; + let ret2 = &r2.id; + if (*ret1 < *ret2) { + ret1 = ret2; + ret2 = &c1.count; + } else { + ret1 = &r2.id; + ret2 = &c2.count; + }; + if (*ret2 < r2.id) { + ret1 = ret2; + ret2 = &c2.count; + } else if (ret1 != &r1.id) { + ret1 = &c1.count; + ret2 = &r2.id; + }; + if (*ret1 < *ret2) { + ret2 = ret1; + ret1 + } else { + ret1 = ret2; + ret2 + }; + if (ret1 == ret2) { + ret1 + } else { + ret2 + } + } + + public fun loop_bcs(count: u64, len: u64) { + let vec = std::vector::empty(); + let i = 0; + while (i < len) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + } + } +} From fb126def614909b765e0665396cc59d12d541cd1 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 15:58:24 +0000 Subject: [PATCH 23/25] [fixes] Fix raw module data & add env vars --- .../aptos-vm-environment/src/environment.rs | 10 +- aptos-move/aptos-vm/src/move_vm_ext/vm.rs | 10 +- crates/transaction-generator-lib/src/args.rs | 8 +- .../src/publishing/module_simple.rs | 16 +- .../src/publishing/raw_module_data.rs | 252 +++++++++--------- .../sources/star_dependencies.move | 122 ++++----- .../move/move-vm/runtime/src/config.rs | 2 +- types/src/on_chain_config/aptos_features.rs | 5 + 8 files changed, 231 insertions(+), 194 deletions(-) diff --git a/aptos-move/aptos-vm-environment/src/environment.rs b/aptos-move/aptos-vm-environment/src/environment.rs index 151dc19247ef9..e441b4d4c7971 100644 --- a/aptos-move/aptos-vm-environment/src/environment.rs +++ b/aptos-move/aptos-vm-environment/src/environment.rs @@ -15,7 +15,8 @@ use aptos_native_interface::SafeNativeBuilder; use aptos_types::{ chain_id::ChainId, on_chain_config::{ - ConfigurationResource, Features, OnChainConfig, TimedFeatures, TimedFeaturesBuilder, + ConfigurationResource, FeatureFlag, Features, OnChainConfig, TimedFeatures, + TimedFeaturesBuilder, }, state_store::StateView, }; @@ -175,8 +176,13 @@ impl Environment { ) -> Self { // We compute and store a hash of configs in order to distinguish different environments. let mut sha3_256 = Sha3_256::new(); - let features = + let mut features = fetch_config_and_update_hash::(&mut sha3_256, state_view).unwrap_or_default(); + if std::env::var("USE_LOADER_V2").is_ok() { + features.enable(FeatureFlag::ENABLE_LOADER_V2); + } else { + features.disable(FeatureFlag::ENABLE_LOADER_V2); + } // If no chain ID is in storage, we assume we are in a testing environment. let chain_id = fetch_config_and_update_hash::(&mut sha3_256, state_view) diff --git a/aptos-move/aptos-vm/src/move_vm_ext/vm.rs b/aptos-move/aptos-vm/src/move_vm_ext/vm.rs index a5eb957339e5a..5218435cafe58 100644 --- a/aptos-move/aptos-vm/src/move_vm_ext/vm.rs +++ b/aptos-move/aptos-vm/src/move_vm_ext/vm.rs @@ -7,7 +7,7 @@ use aptos_gas_schedule::{MiscGasParameters, NativeGasParameters, LATEST_GAS_FEAT use aptos_native_interface::SafeNativeBuilder; use aptos_types::{ chain_id::ChainId, - on_chain_config::{Features, TimedFeaturesBuilder}, + on_chain_config::{FeatureFlag, Features, TimedFeaturesBuilder}, transaction::user_transaction_context::UserTransactionContext, }; use aptos_vm_environment::{ @@ -30,7 +30,13 @@ pub struct GenesisRuntimeBuilder { impl GenesisRuntimeBuilder { /// Returns a builder, capable of creating VM and runtime environment to run genesis. pub fn new(chain_id: ChainId) -> Self { - let features = Features::default(); + let mut features = Features::default(); + if std::env::var("USE_LOADER_V2").is_ok() { + features.enable(FeatureFlag::ENABLE_LOADER_V2); + } else { + features.disable(FeatureFlag::ENABLE_LOADER_V2); + } + let timed_features = TimedFeaturesBuilder::enable_all().build(); let vm_config = diff --git a/crates/transaction-generator-lib/src/args.rs b/crates/transaction-generator-lib/src/args.rs index c70fa28949e24..7a1e2330f3af6 100644 --- a/crates/transaction-generator-lib/src/args.rs +++ b/crates/transaction-generator-lib/src/args.rs @@ -323,9 +323,13 @@ impl TransactionTypeArg { }, TransactionTypeArg::DeserializeU256 => call_custom_module(EntryPoints::DeserializeU256), TransactionTypeArg::SimpleScript => call_custom_module(EntryPoints::SimpleScript), - TransactionTypeArg::ChainDependencies => call_custom_module(EntryPoints::ChainDependencies), + TransactionTypeArg::ChainDependencies => { + call_custom_module(EntryPoints::ChainDependencies) + }, TransactionTypeArg::ChainFriends => call_custom_module(EntryPoints::ChainFriends), - TransactionTypeArg::StarDependencies => call_custom_module(EntryPoints::StarDependencies), + TransactionTypeArg::StarDependencies => { + call_custom_module(EntryPoints::StarDependencies) + }, } } diff --git a/crates/transaction-generator-lib/src/publishing/module_simple.rs b/crates/transaction-generator-lib/src/publishing/module_simple.rs index ce7afd699007e..83be768e25d4e 100644 --- a/crates/transaction-generator-lib/src/publishing/module_simple.rs +++ b/crates/transaction-generator-lib/src/publishing/module_simple.rs @@ -395,9 +395,9 @@ impl EntryPoints { EntryPoints::IncGlobalMilestoneAggV2 { .. } | EntryPoints::CreateGlobalMilestoneAggV2 { .. } => "counter_with_milestone", EntryPoints::DeserializeU256 => "bcs_stream", - EntryPoints::ChainDependencies => "module_loading_chain_dependencies", - EntryPoints::ChainFriends => "module_loading_chain_friends", - EntryPoints::StarDependencies => "module_loading_star_dependencies", + EntryPoints::ChainDependencies => "chain_dependencies", + EntryPoints::ChainFriends => "chain_friends", + EntryPoints::StarDependencies => "star_dependencies", } } @@ -741,7 +741,11 @@ impl EntryPoints { ], ) }, - EntryPoints::ChainDependencies | EntryPoints::ChainFriends | EntryPoints::StarDependencies => get_payload_void(module_id, ident_str!("run").to_owned()), + EntryPoints::ChainDependencies + | EntryPoints::ChainFriends + | EntryPoints::StarDependencies => { + get_payload_void(module_id, ident_str!("run").to_owned()) + }, } } @@ -850,7 +854,9 @@ impl EntryPoints { EntryPoints::DeserializeU256 => AutomaticArgs::None, EntryPoints::IncGlobalMilestoneAggV2 { .. } => AutomaticArgs::None, EntryPoints::CreateGlobalMilestoneAggV2 { .. } => AutomaticArgs::Signer, - EntryPoints::ChainDependencies | EntryPoints::ChainFriends | EntryPoints::StarDependencies => AutomaticArgs::None, + EntryPoints::ChainDependencies + | EntryPoints::ChainFriends + | EntryPoints::StarDependencies => AutomaticArgs::None, } } } diff --git a/crates/transaction-generator-lib/src/publishing/raw_module_data.rs b/crates/transaction-generator-lib/src/publishing/raw_module_data.rs index ba469c4cfcf58..24e41f4699800 100644 --- a/crates/transaction-generator-lib/src/publishing/raw_module_data.rs +++ b/crates/transaction-generator-lib/src/publishing/raw_module_data.rs @@ -20,10 +20,10 @@ pub static PACKAGE_MODULE_LOADING_STAR_DEPENDENCIES_METADATA: Lazy> = La vec![ 32, 109, 111, 100, 117, 108, 101, 95, 108, 111, 97, 100, 105, 110, 103, 95, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 1, 0, 0, - 0, 0, 0, 0, 0, 0, 64, 54, 50, 57, 66, 55, 56, 49, 50, 65, 52, 48, - 56, 69, 49, 52, 48, 51, 55, 66, 70, 57, 48, 55, 51, 56, 56, 66, 48, 53, - 51, 65, 48, 54, 66, 53, 54, 66, 65, 65, 68, 49, 53, 70, 50, 48, 67, 48, - 57, 70, 52, 67, 57, 57, 54, 54, 48, 54, 69, 70, 49, 54, 56, 55, 56, 170, + 0, 0, 0, 0, 0, 0, 64, 55, 56, 67, 69, 65, 53, 49, 70, 70, 65, 56, + 51, 51, 70, 48, 66, 55, 52, 68, 55, 67, 57, 51, 66, 50, 70, 68, 52, 67, + 66, 56, 67, 54, 69, 69, 70, 55, 70, 48, 67, 50, 48, 70, 52, 56, 57, 65, + 65, 52, 68, 69, 57, 67, 66, 52, 51, 50, 68, 48, 69, 55, 55, 54, 53, 170, 1, 31, 139, 8, 0, 0, 0, 0, 0, 2, 255, 77, 141, 65, 14, 130, 48, 16, 69, 247, 61, 69, 195, 134, 149, 21, 15, 224, 2, 245, 6, 46, 9, 105, 6, 58, 98, 67, 233, 52, 29, 64, 19, 227, 221, 109, 13, 11, 146, 89, 76, 126, 222, 127, @@ -33,33 +33,33 @@ pub static PACKAGE_MODULE_LOADING_STAR_DEPENDENCIES_METADATA: Lazy> = La 219, 37, 234, 35, 29, 245, 224, 210, 87, 40, 117, 220, 31, 100, 238, 48, 209, 138, 199, 71, 76, 251, 47, 138, 227, 22, 242, 191, 93, 200, 111, 146, 131, 49, 17, 153, 179, 57, 44, 157, 179, 252, 196, 168, 183, 48, 91, 171, 119, 125, 185, 222, 10, 241, - 3, 239, 45, 19, 107, 205, 0, 0, 0, 21, 19, 116, 114, 101, 101, 95, 100, 101, - 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 57, 0, 0, 0, 19, 116, 114, - 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 56, 0, - 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, - 101, 115, 95, 55, 0, 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, - 100, 101, 110, 99, 105, 101, 115, 95, 54, 0, 0, 0, 19, 116, 114, 101, 101, 95, + 3, 239, 45, 19, 107, 205, 0, 0, 0, 21, 19, 115, 116, 97, 114, 95, 100, 101, + 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 57, 0, 0, 0, 19, 115, 116, + 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 56, 0, + 0, 0, 19, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 55, 0, 0, 0, 19, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, + 100, 101, 110, 99, 105, 101, 115, 95, 54, 0, 0, 0, 19, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 53, 0, 0, 0, 19, - 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, - 52, 0, 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, - 99, 105, 101, 115, 95, 51, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, - 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 48, 0, 0, 0, 19, 116, 114, - 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 0, - 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, - 101, 115, 95, 49, 57, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, - 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 56, 0, 0, 0, 20, 116, 114, 101, - 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 55, 0, - 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, - 101, 115, 95, 49, 54, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, - 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 53, 0, 0, 0, 20, 116, 114, 101, - 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 52, 0, - 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, - 101, 115, 95, 49, 51, 0, 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, - 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 50, 0, 0, 0, 20, 116, 114, 101, - 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 49, 0, - 0, 0, 20, 116, 114, 101, 101, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, - 101, 115, 95, 49, 48, 0, 0, 0, 19, 116, 114, 101, 101, 95, 100, 101, 112, 101, - 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 0, 0, 0, 17, 116, 114, 101, 101, + 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, + 52, 0, 0, 0, 19, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, + 99, 105, 101, 115, 95, 51, 0, 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, + 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 48, 0, 0, 0, 19, 115, 116, + 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 50, 0, + 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 57, 0, 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 56, 0, 0, 0, 20, 115, 116, 97, + 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 55, 0, + 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 54, 0, 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 53, 0, 0, 0, 20, 115, 116, 97, + 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 52, 0, + 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 51, 0, 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 50, 0, 0, 0, 20, 115, 116, 97, + 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 49, 0, + 0, 0, 20, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, + 101, 115, 95, 49, 48, 0, 0, 0, 19, 115, 116, 97, 114, 95, 100, 101, 112, 101, + 110, 100, 101, 110, 99, 105, 101, 115, 95, 49, 0, 0, 0, 17, 115, 116, 97, 114, 95, 100, 101, 112, 101, 110, 100, 101, 110, 99, 105, 101, 115, 0, 0, 0, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 11, 65, 112, 116, 111, @@ -70,7 +70,7 @@ pub static PACKAGE_MODULE_LOADING_STAR_DEPENDENCIES_METADATA: Lazy> = La }); #[rustfmt::skip] -pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_9: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_9: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -80,7 +80,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_9: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_8: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -126,7 +126,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_8: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_7: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -172,7 +172,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_7: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_6: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -218,7 +218,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_6: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_5: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -264,7 +264,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_5: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_4: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -310,7 +310,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_4: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_3: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -356,7 +356,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_3: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_20: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -402,7 +402,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_20: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_2: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -448,7 +448,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_2: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_19: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -494,7 +494,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_19: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_18: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -540,7 +540,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_18: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_17: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -586,7 +586,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_17: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_16: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -632,7 +632,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_16: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_15: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -678,7 +678,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_15: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_14: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -724,7 +724,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_14: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_13: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -770,7 +770,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_13: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_12: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -816,7 +816,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_12: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_11: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -862,7 +862,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_11: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_10: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 118, 8, 217, 1, 64, 16, 153, 2, 31, 10, @@ -908,7 +908,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_10: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_1: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 11, 1, 0, 6, 2, 6, 16, 3, 22, 25, 4, 47, 2, 5, 49, 50, 7, 99, 117, 8, 216, 1, 64, 16, 152, 2, 31, 10, @@ -954,7 +954,7 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_1: Lazy> = Lazy::new(|| { +pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES: Lazy> = Lazy::new(|| { vec![ 161, 28, 235, 11, 7, 0, 0, 10, 7, 1, 0, 42, 3, 42, 126, 5, 168, 1, 7, 7, 175, 1, 182, 3, 8, 229, 4, 32, 16, 133, 5, 31, 12, 164, 5, 130, @@ -1003,31 +1003,31 @@ pub static MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES: Lazy>> = Lazy::new(|| { vec![ - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_9.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_8.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_7.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_6.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_5.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_4.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_3.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_20.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_2.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_19.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_18.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_17.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_16.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_15.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_14.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_13.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_12.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_11.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_10.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES_1.to_vec(), - MODULE_MODULE_LOADING_STAR_DEPENDENCIES_TREE_DEPENDENCIES.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_9.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_8.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_7.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_6.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_5.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_4.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_3.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_20.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_2.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_19.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_18.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_17.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_16.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_15.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_14.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_13.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_12.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_11.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_10.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES_1.to_vec(), + MODULE_MODULE_LOADING_STAR_DEPENDENCIES_STAR_DEPENDENCIES.to_vec(), ]}); #[rustfmt::skip] @@ -3864,10 +3864,10 @@ pub static MODULES_MODULE_LOADING_CHAIN_FRIENDS: Lazy>> = Lazy::new( pub static PACKAGE_SIMPLE_METADATA: Lazy> = Lazy::new(|| { vec![ 13, 71, 101, 110, 101, 114, 105, 99, 77, 111, 100, 117, 108, 101, 1, 0, 0, 0, - 0, 0, 0, 0, 0, 64, 68, 48, 69, 53, 68, 55, 70, 65, 50, 66, 65, 54, - 68, 50, 65, 48, 57, 55, 55, 56, 51, 49, 51, 67, 57, 68, 69, 51, 56, 67, - 69, 69, 68, 49, 49, 49, 68, 65, 70, 49, 67, 66, 53, 53, 56, 54, 52, 65, - 67, 65, 52, 55, 56, 49, 53, 67, 65, 52, 53, 57, 66, 49, 66, 66, 132, 1, + 0, 0, 0, 0, 0, 64, 69, 53, 53, 57, 68, 57, 51, 68, 67, 55, 70, 65, + 56, 70, 70, 66, 70, 53, 70, 65, 57, 48, 49, 52, 57, 69, 49, 65, 56, 48, + 56, 55, 57, 67, 65, 52, 56, 50, 66, 67, 52, 67, 48, 50, 48, 51, 51, 50, + 53, 53, 52, 67, 50, 67, 48, 52, 49, 56, 67, 54, 56, 56, 53, 69, 132, 1, 31, 139, 8, 0, 0, 0, 0, 0, 2, 255, 77, 139, 59, 14, 194, 48, 16, 68, 251, 61, 133, 229, 30, 135, 11, 80, 208, 64, 197, 9, 162, 20, 43, 123, 64, 86, 156, 93, 203, 134, 80, 32, 238, 142, 45, 1, 138, 102, 154, 249, 188, 49, 179, 159, @@ -3887,6 +3887,16 @@ pub static PACKAGE_SIMPLE_METADATA: Lazy> = Lazy::new(|| { ] }); +#[rustfmt::skip] +pub static SCRIPT_SIMPLE: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 2, 5, 0, 4, 6, 4, 34, 1, 6, 12, + 0, 5, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, + 0, 1, 3, 11, 0, 1, 2, + ] +}); + #[rustfmt::skip] pub static MODULE_SIMPLE_SIMPLE: Lazy> = Lazy::new(|| { vec![ diff --git a/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move b/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move index 91ac83d7da131..d5eba39b43544 100644 --- a/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move +++ b/testsuite/module-publish/src/packages/module_loading_star_dependencies/sources/star_dependencies.move @@ -1,54 +1,54 @@ -module publisher_address::tree_dependencies { - use publisher_address::tree_dependencies_1; - use publisher_address::tree_dependencies_2; - use publisher_address::tree_dependencies_3; - use publisher_address::tree_dependencies_4; - use publisher_address::tree_dependencies_5; - use publisher_address::tree_dependencies_6; - use publisher_address::tree_dependencies_7; - use publisher_address::tree_dependencies_8; - use publisher_address::tree_dependencies_9; - use publisher_address::tree_dependencies_10; - use publisher_address::tree_dependencies_11; - use publisher_address::tree_dependencies_12; - use publisher_address::tree_dependencies_13; - use publisher_address::tree_dependencies_14; - use publisher_address::tree_dependencies_15; - use publisher_address::tree_dependencies_16; - use publisher_address::tree_dependencies_17; - use publisher_address::tree_dependencies_18; - use publisher_address::tree_dependencies_19; - use publisher_address::tree_dependencies_20; +module publisher_address::star_dependencies { + use publisher_address::star_dependencies_1; + use publisher_address::star_dependencies_2; + use publisher_address::star_dependencies_3; + use publisher_address::star_dependencies_4; + use publisher_address::star_dependencies_5; + use publisher_address::star_dependencies_6; + use publisher_address::star_dependencies_7; + use publisher_address::star_dependencies_8; + use publisher_address::star_dependencies_9; + use publisher_address::star_dependencies_10; + use publisher_address::star_dependencies_11; + use publisher_address::star_dependencies_12; + use publisher_address::star_dependencies_13; + use publisher_address::star_dependencies_14; + use publisher_address::star_dependencies_15; + use publisher_address::star_dependencies_16; + use publisher_address::star_dependencies_17; + use publisher_address::star_dependencies_18; + use publisher_address::star_dependencies_19; + use publisher_address::star_dependencies_20; public entry fun run() { let sum = 0; - sum = sum + tree_dependencies_1::next(); - sum = sum + tree_dependencies_2::next(); - sum = sum + tree_dependencies_3::next(); - sum = sum + tree_dependencies_4::next(); - sum = sum + tree_dependencies_5::next(); - sum = sum + tree_dependencies_6::next(); - sum = sum + tree_dependencies_7::next(); - sum = sum + tree_dependencies_8::next(); - sum = sum + tree_dependencies_9::next(); - sum = sum + tree_dependencies_10::next(); - sum = sum + tree_dependencies_11::next(); - sum = sum + tree_dependencies_12::next(); - sum = sum + tree_dependencies_13::next(); - sum = sum + tree_dependencies_14::next(); - sum = sum + tree_dependencies_15::next(); - sum = sum + tree_dependencies_16::next(); - sum = sum + tree_dependencies_17::next(); - sum = sum + tree_dependencies_18::next(); - sum = sum + tree_dependencies_19::next(); - sum = sum + tree_dependencies_20::next(); + sum = sum + star_dependencies_1::next(); + sum = sum + star_dependencies_2::next(); + sum = sum + star_dependencies_3::next(); + sum = sum + star_dependencies_4::next(); + sum = sum + star_dependencies_5::next(); + sum = sum + star_dependencies_6::next(); + sum = sum + star_dependencies_7::next(); + sum = sum + star_dependencies_8::next(); + sum = sum + star_dependencies_9::next(); + sum = sum + star_dependencies_10::next(); + sum = sum + star_dependencies_11::next(); + sum = sum + star_dependencies_12::next(); + sum = sum + star_dependencies_13::next(); + sum = sum + star_dependencies_14::next(); + sum = sum + star_dependencies_15::next(); + sum = sum + star_dependencies_16::next(); + sum = sum + star_dependencies_17::next(); + sum = sum + star_dependencies_18::next(); + sum = sum + star_dependencies_19::next(); + sum = sum + star_dependencies_20::next(); assert!(sum == 20, 77); } } -module publisher_address::tree_dependencies_1 { +module publisher_address::star_dependencies_1 { const MAGIC: u64 = 1; @@ -126,7 +126,7 @@ module publisher_address::tree_dependencies_1 { } } -module publisher_address::tree_dependencies_2 { +module publisher_address::star_dependencies_2 { const MAGIC: u64 = 2; @@ -204,7 +204,7 @@ module publisher_address::tree_dependencies_2 { } } -module publisher_address::tree_dependencies_3 { +module publisher_address::star_dependencies_3 { const MAGIC: u64 = 3; @@ -282,7 +282,7 @@ module publisher_address::tree_dependencies_3 { } } -module publisher_address::tree_dependencies_4 { +module publisher_address::star_dependencies_4 { const MAGIC: u64 = 4; @@ -360,7 +360,7 @@ module publisher_address::tree_dependencies_4 { } } -module publisher_address::tree_dependencies_5 { +module publisher_address::star_dependencies_5 { const MAGIC: u64 = 5; @@ -438,7 +438,7 @@ module publisher_address::tree_dependencies_5 { } } -module publisher_address::tree_dependencies_6 { +module publisher_address::star_dependencies_6 { const MAGIC: u64 = 6; @@ -516,7 +516,7 @@ module publisher_address::tree_dependencies_6 { } } -module publisher_address::tree_dependencies_7 { +module publisher_address::star_dependencies_7 { const MAGIC: u64 = 7; @@ -594,7 +594,7 @@ module publisher_address::tree_dependencies_7 { } } -module publisher_address::tree_dependencies_8 { +module publisher_address::star_dependencies_8 { const MAGIC: u64 = 8; @@ -672,7 +672,7 @@ module publisher_address::tree_dependencies_8 { } } -module publisher_address::tree_dependencies_9 { +module publisher_address::star_dependencies_9 { const MAGIC: u64 = 9; @@ -750,7 +750,7 @@ module publisher_address::tree_dependencies_9 { } } -module publisher_address::tree_dependencies_10 { +module publisher_address::star_dependencies_10 { const MAGIC: u64 = 10; @@ -828,7 +828,7 @@ module publisher_address::tree_dependencies_10 { } } -module publisher_address::tree_dependencies_11 { +module publisher_address::star_dependencies_11 { const MAGIC: u64 = 11; @@ -906,7 +906,7 @@ module publisher_address::tree_dependencies_11 { } } -module publisher_address::tree_dependencies_12 { +module publisher_address::star_dependencies_12 { const MAGIC: u64 = 12; @@ -984,7 +984,7 @@ module publisher_address::tree_dependencies_12 { } } -module publisher_address::tree_dependencies_13 { +module publisher_address::star_dependencies_13 { const MAGIC: u64 = 13; @@ -1062,7 +1062,7 @@ module publisher_address::tree_dependencies_13 { } } -module publisher_address::tree_dependencies_14 { +module publisher_address::star_dependencies_14 { const MAGIC: u64 = 14; @@ -1140,7 +1140,7 @@ module publisher_address::tree_dependencies_14 { } } -module publisher_address::tree_dependencies_15 { +module publisher_address::star_dependencies_15 { const MAGIC: u64 = 15; @@ -1218,7 +1218,7 @@ module publisher_address::tree_dependencies_15 { } } -module publisher_address::tree_dependencies_16 { +module publisher_address::star_dependencies_16 { const MAGIC: u64 = 16; @@ -1296,7 +1296,7 @@ module publisher_address::tree_dependencies_16 { } } -module publisher_address::tree_dependencies_17 { +module publisher_address::star_dependencies_17 { const MAGIC: u64 = 17; @@ -1374,7 +1374,7 @@ module publisher_address::tree_dependencies_17 { } } -module publisher_address::tree_dependencies_18 { +module publisher_address::star_dependencies_18 { const MAGIC: u64 = 18; @@ -1452,7 +1452,7 @@ module publisher_address::tree_dependencies_18 { } } -module publisher_address::tree_dependencies_19 { +module publisher_address::star_dependencies_19 { const MAGIC: u64 = 19; @@ -1530,7 +1530,7 @@ module publisher_address::tree_dependencies_19 { } } -module publisher_address::tree_dependencies_20 { +module publisher_address::star_dependencies_20 { const MAGIC: u64 = 20; diff --git a/third_party/move/move-vm/runtime/src/config.rs b/third_party/move/move-vm/runtime/src/config.rs index 5eebeba504cbe..81584ce0f8178 100644 --- a/third_party/move/move-vm/runtime/src/config.rs +++ b/third_party/move/move-vm/runtime/src/config.rs @@ -44,7 +44,7 @@ impl Default for VMConfig { ty_builder: TypeBuilder::with_limits(128, 20), disallow_dispatch_for_native: true, use_compatibility_checker_v2: true, - use_loader_v2: false, + use_loader_v2: std::env::var("USE_LOADER_V2").is_ok(), } } } diff --git a/types/src/on_chain_config/aptos_features.rs b/types/src/on_chain_config/aptos_features.rs index d1297014ad21f..ff069bab6559d 100644 --- a/types/src/on_chain_config/aptos_features.rs +++ b/types/src/on_chain_config/aptos_features.rs @@ -196,6 +196,11 @@ impl Default for Features { for feature in FeatureFlag::default_features() { features.enable(feature); } + if std::env::var("USE_LOADER_V2").is_ok() { + features.enable(FeatureFlag::ENABLE_LOADER_V2); + } else { + features.disable(FeatureFlag::ENABLE_LOADER_V2); + } features } } From e36de2d8a73486e8c2d2ef209f2c0c7c3677229a Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 18:10:46 +0000 Subject: [PATCH 24/25] [fix] Fix & improve script benchmark --- .../src/publishing/raw_module_data.rs | 59 ++++++++++++++----- testsuite/module-publish/src/main.rs | 13 +++- .../src/packages/simple/scripts/main.move | 51 +++++++++++++--- 3 files changed, 100 insertions(+), 23 deletions(-) diff --git a/crates/transaction-generator-lib/src/publishing/raw_module_data.rs b/crates/transaction-generator-lib/src/publishing/raw_module_data.rs index 24e41f4699800..f3edf4b7134d2 100644 --- a/crates/transaction-generator-lib/src/publishing/raw_module_data.rs +++ b/crates/transaction-generator-lib/src/publishing/raw_module_data.rs @@ -3864,10 +3864,10 @@ pub static MODULES_MODULE_LOADING_CHAIN_FRIENDS: Lazy>> = Lazy::new( pub static PACKAGE_SIMPLE_METADATA: Lazy> = Lazy::new(|| { vec![ 13, 71, 101, 110, 101, 114, 105, 99, 77, 111, 100, 117, 108, 101, 1, 0, 0, 0, - 0, 0, 0, 0, 0, 64, 69, 53, 53, 57, 68, 57, 51, 68, 67, 55, 70, 65, - 56, 70, 70, 66, 70, 53, 70, 65, 57, 48, 49, 52, 57, 69, 49, 65, 56, 48, - 56, 55, 57, 67, 65, 52, 56, 50, 66, 67, 52, 67, 48, 50, 48, 51, 51, 50, - 53, 53, 52, 67, 50, 67, 48, 52, 49, 56, 67, 54, 56, 56, 53, 69, 132, 1, + 0, 0, 0, 0, 0, 64, 57, 66, 56, 48, 68, 66, 66, 50, 48, 57, 69, 66, + 57, 49, 51, 54, 51, 56, 57, 55, 51, 65, 50, 67, 48, 67, 52, 70, 52, 67, + 51, 54, 69, 50, 49, 52, 48, 48, 57, 70, 56, 66, 53, 70, 68, 68, 53, 68, + 69, 51, 48, 65, 49, 54, 55, 67, 67, 66, 49, 57, 68, 55, 70, 68, 132, 1, 31, 139, 8, 0, 0, 0, 0, 0, 2, 255, 77, 139, 59, 14, 194, 48, 16, 68, 251, 61, 133, 229, 30, 135, 11, 80, 208, 64, 197, 9, 162, 20, 43, 123, 64, 86, 156, 93, 203, 134, 80, 32, 238, 142, 45, 1, 138, 102, 154, 249, 188, 49, 179, 159, @@ -3887,16 +3887,6 @@ pub static PACKAGE_SIMPLE_METADATA: Lazy> = Lazy::new(|| { ] }); -#[rustfmt::skip] -pub static SCRIPT_SIMPLE: Lazy> = Lazy::new(|| { - vec![ - 161, 28, 235, 11, 7, 0, 0, 10, 2, 5, 0, 4, 6, 4, 34, 1, 6, 12, - 0, 5, 32, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, - 0, 1, 3, 11, 0, 1, 2, - ] -}); - #[rustfmt::skip] pub static MODULE_SIMPLE_SIMPLE: Lazy> = Lazy::new(|| { vec![ @@ -4100,6 +4090,47 @@ pub static MODULE_SIMPLE_SIMPLE: Lazy> = Lazy::new(|| { ] }); +#[rustfmt::skip] +pub static SCRIPT_SIMPLE: Lazy> = Lazy::new(|| { + vec![ + 161, 28, 235, 11, 7, 0, 0, 10, 7, 1, 0, 2, 3, 2, 7, 4, 9, 2, + 5, 11, 32, 7, 43, 27, 8, 70, 64, 16, 134, 1, 31, 1, 2, 0, 3, 4, + 5, 1, 0, 1, 0, 3, 1, 6, 12, 0, 1, 3, 1, 10, 3, 1, 6, 9, + 0, 1, 10, 2, 1, 2, 11, 3, 3, 3, 3, 3, 3, 1, 1, 1, 10, 3, + 10, 2, 8, 60, 83, 69, 76, 70, 62, 95, 48, 4, 109, 97, 105, 110, 3, 98, + 99, 115, 8, 116, 111, 95, 98, 121, 116, 101, 115, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, + 255, 255, 255, 255, 255, 255, 255, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1, 20, 99, 111, 109, 112, 105, 108, 97, 116, 105, 111, 110, 95, 109, 101, + 116, 97, 100, 97, 116, 97, 9, 0, 3, 50, 46, 48, 3, 50, 46, 49, 0, 0, + 7, 139, 1, 6, 23, 0, 0, 0, 0, 0, 0, 0, 12, 1, 11, 0, 1, 10, + 1, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 13, 11, 1, 6, 1, 0, + 0, 0, 0, 0, 0, 0, 23, 12, 1, 5, 4, 6, 0, 0, 0, 0, 0, 0, + 0, 0, 12, 2, 6, 0, 0, 0, 0, 0, 0, 0, 0, 12, 3, 10, 1, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 86, 11, 1, 6, 1, 0, 0, 0, + 0, 0, 0, 0, 23, 12, 1, 11, 2, 6, 1, 0, 0, 0, 0, 0, 0, 0, + 22, 12, 4, 11, 3, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 5, 10, + 4, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 11, 4, 23, 12, 3, 10, 5, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 10, 5, 23, 12, 4, 10, 4, 6, + 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 2, 10, 4, 10, 2, 36, 4, 83, + 10, 2, 10, 5, 36, 12, 7, 11, 7, 4, 80, 11, 5, 10, 3, 36, 12, 8, + 11, 8, 4, 77, 10, 3, 11, 4, 36, 12, 9, 11, 9, 3, 72, 5, 17, 11, + 1, 6, 1, 0, 0, 0, 0, 0, 0, 0, 22, 12, 1, 5, 17, 9, 12, 9, + 5, 69, 9, 12, 8, 5, 63, 9, 12, 7, 5, 57, 6, 5, 0, 0, 0, 0, + 0, 0, 0, 12, 1, 64, 2, 0, 0, 0, 0, 0, 0, 0, 0, 12, 10, 6, + 0, 0, 0, 0, 0, 0, 0, 0, 12, 2, 10, 2, 6, 20, 0, 0, 0, 0, + 0, 0, 0, 35, 4, 104, 13, 10, 10, 2, 68, 2, 11, 2, 6, 1, 0, 0, + 0, 0, 0, 0, 0, 22, 12, 2, 5, 92, 6, 0, 0, 0, 0, 0, 0, 0, + 0, 12, 2, 10, 1, 6, 0, 0, 0, 0, 0, 0, 0, 0, 36, 4, 130, 1, + 14, 10, 56, 0, 12, 11, 11, 2, 12, 3, 14, 11, 6, 0, 0, 0, 0, 0, + 0, 0, 0, 66, 6, 20, 52, 12, 4, 11, 3, 11, 4, 22, 12, 2, 11, 1, + 6, 1, 0, 0, 0, 0, 0, 0, 0, 23, 12, 1, 5, 106, 11, 1, 6, 64, + 66, 15, 0, 0, 0, 0, 0, 33, 4, 135, 1, 5, 136, 1, 5, 138, 1, 5, + 136, 1, 5, 138, 1, 2, + ] +}); + #[rustfmt::skip] pub static MODULES_SIMPLE: Lazy>> = Lazy::new(|| { vec![ MODULE_SIMPLE_SIMPLE.to_vec(), diff --git a/testsuite/module-publish/src/main.rs b/testsuite/module-publish/src/main.rs index 7be0b3756122f..aac459c746366 100644 --- a/testsuite/module-publish/src/main.rs +++ b/testsuite/module-publish/src/main.rs @@ -150,7 +150,8 @@ fn write_package(file: &mut File, package_path: PathBuf, package_name: &str) -> // build package let package = BuiltPackage::build(package_path, BuildOptions::move_2()) .expect("building package must succeed"); - let code = package.extract_code(); + let modules = package.extract_code(); + let scripts = package.extract_script_code(); let package_metadata = package.extract_metadata().expect("Metadata must exist"); let metadata = bcs::to_bytes(&package_metadata).expect("Metadata must serialize"); @@ -164,7 +165,7 @@ fn write_package(file: &mut File, package_path: PathBuf, package_name: &str) -> let mut module_names = Vec::new(); // write out all modules - for module in &code { + for module in &modules { // this is an unfortunate way to find the module name but it is not // clear how to do it otherwise let compiled_module = CompiledModule::deserialize(module).expect("Module must deserialize"); @@ -180,6 +181,14 @@ fn write_package(file: &mut File, package_path: PathBuf, package_name: &str) -> module_names.push(name); } + // write out scripts (allow a single script for now) + assert!(scripts.len() <= 1); + for script in &scripts { + let name: String = format!("SCRIPT_{}", package_name.to_uppercase(),); + writeln!(file).expect("Empty line failed"); + write_lazy(file, name.as_str(), script); + } + writeln!(file).expect("Empty line failed"); writeln!(file, "#[rustfmt::skip]").expect("rustfmt skip failed"); writeln!( diff --git a/testsuite/module-publish/src/packages/simple/scripts/main.move b/testsuite/module-publish/src/packages/simple/scripts/main.move index 0c6c4767afb65..b41d1fc0de31e 100644 --- a/testsuite/module-publish/src/packages/simple/scripts/main.move +++ b/testsuite/module-publish/src/packages/simple/scripts/main.move @@ -2,16 +2,53 @@ script { // Note: this constant can be replaced in compiled script to make it hash to a different value. const SENDER: address = @0x1; - fun main(sender: &signer) { + fun main(_sender: &signer) { // The idea is to to ensure that this script takes some time to be deserialized and verified, but the actual // execution time is small (no-op). - if (false) { - 0xABCD::simple::loop_nop(sender, 0); - 0xABCD::simple::loop_arithmetic(sender, 0); - 0xABCD::simple::loop_bcs(sender, 0, 0); - if (false) { - while (true) {} + let count = 23; + while (count > 0) { + count = count - 1; + }; + + let a; + let b = 0; + let c; + let d = 0; + while (count > 0) { + count = count - 1; + + a = b + 1; + c = d + 1; + b = a + 1; + d = b - a; + b = c + 1; + a = b - c; + b = a + 1; + + // can never be true + if (a > b && b > c && c > d && d > a) { + count = count + 1; } + }; + + let count = 5; + let vec = std::vector::empty(); + let i = 0; + while (i < 20) { + std::vector::push_back(&mut vec, i); + i = i + 1; + }; + + let sum: u64 = 0; + + while (count > 0) { + let val = std::bcs::to_bytes(&vec); + sum = sum + ((*std::vector::borrow(&val, 0)) as u64); + count = count - 1; + }; + + if (count == 1000000) { + while (true) {} } } } From 9fad83dd9ce3f7864b3b2d7c66e1512dc2191f96 Mon Sep 17 00:00:00 2001 From: George Mitenkov Date: Tue, 12 Nov 2024 20:32:07 +0000 Subject: [PATCH 25/25] [hack] Cross-block debugger --- .../aptos-debugger/src/aptos_debugger.rs | 79 +++++++++---------- .../src/execute_pending_block.rs | 5 ++ aptos-move/aptos-vm/src/aptos_vm.rs | 6 +- 3 files changed, 48 insertions(+), 42 deletions(-) diff --git a/aptos-move/aptos-debugger/src/aptos_debugger.rs b/aptos-move/aptos-debugger/src/aptos_debugger.rs index 9e9f93c62adf1..5560977d713b7 100644 --- a/aptos-move/aptos-debugger/src/aptos_debugger.rs +++ b/aptos-move/aptos-debugger/src/aptos_debugger.rs @@ -2,16 +2,12 @@ // SPDX-License-Identifier: Apache-2.0 use anyhow::{bail, format_err, Result}; -use aptos_block_executor::{ - code_cache_global_manager::ModuleCacheManager, txn_commit_hook::NoOpTransactionCommitHook, -}; +use aptos_crypto::HashValue; use aptos_gas_profiling::{GasProfiler, TransactionGasLog}; use aptos_rest_client::Client; use aptos_types::{ account_address::AccountAddress, - block_executor::config::{ - BlockExecutorConfig, BlockExecutorConfigFromOnchain, BlockExecutorLocalConfig, - }, + block_executor::config::BlockExecutorConfigFromOnchain, contract_event::ContractEvent, state_store::TStateView, transaction::{ @@ -25,9 +21,7 @@ use aptos_validator_interface::{ AptosValidatorInterface, DBDebuggerInterface, DebuggerStateView, RestDebuggerInterface, }; use aptos_vm::{ - block_executor::{AptosTransactionOutput, BlockAptosVM}, - data_cache::AsMoveResolver, - AptosVM, + aptos_vm::AptosVMBlockExecutor, data_cache::AsMoveResolver, AptosVM, VMBlockExecutor, }; use aptos_vm_environment::environment::AptosEnvironment; use aptos_vm_logging::log_schema::AdapterLogSchema; @@ -56,10 +50,13 @@ impl AptosDebugger { pub fn execute_transactions_at_version( &self, + executor: &AptosVMBlockExecutor, version: Version, txns: Vec, repeat_execution_times: u64, concurrency_levels: &[usize], + parent: Option<&HashValue>, + current: Option, ) -> Result> { let sig_verified_txns: Vec = txns.into_iter().map(|x| x.into()).collect::>(); @@ -72,9 +69,16 @@ impl AptosDebugger { for concurrency_level in concurrency_levels { for i in 0..repeat_execution_times { let start_time = Instant::now(); - let cur_result = - execute_block_no_limit(&sig_verified_txns, &state_view, *concurrency_level) - .map_err(|err| format_err!("Unexpected VM Error: {:?}", err))?; + let cur_result = executor + .execute_block( + &sig_verified_txns, + &state_view, + BlockExecutorConfigFromOnchain::new_no_block_limit(), + parent, + current, + ) + .map(BlockOutput::into_transaction_outputs_forced) + .map_err(|err| format_err!("Unexpected VM Error: {:?}", err))?; println!( "[{} txns from {}] Finished execution round {}/{} with concurrency_level={} in {}ms", @@ -171,14 +175,12 @@ impl AptosDebugger { if use_same_block_boundaries { // when going block by block, no need to worry about epoch boundaries // as new epoch is always a new block. - Ok(self - .execute_transactions_by_block( - begin, - txns.clone(), - repeat_execution_times, - concurrency_levels, - ) - .await?) + Ok(self.execute_transactions_by_block( + begin, + txns.clone(), + repeat_execution_times, + concurrency_levels, + )?) } else { self.execute_transactions_by_epoch( limit, @@ -230,16 +232,20 @@ impl AptosDebugger { async fn execute_transactions_until_epoch_end( &self, + executor: &AptosVMBlockExecutor, begin: Version, txns: Vec, repeat_execution_times: u64, concurrency_levels: &[usize], ) -> Result> { let results = self.execute_transactions_at_version( + executor, begin, txns, repeat_execution_times, concurrency_levels, + None, + None, )?; let mut ret = vec![]; let mut is_reconfig = false; @@ -272,8 +278,10 @@ impl AptosDebugger { begin, limit ); + let executor = AptosVMBlockExecutor::new(); let mut epoch_result = self .execute_transactions_until_epoch_end( + &executor, begin, txns.clone(), repeat_execution_times, @@ -291,7 +299,7 @@ impl AptosDebugger { Ok(ret) } - async fn execute_transactions_by_block( + fn execute_transactions_by_block( &self, begin: Version, txns: Vec, @@ -301,14 +309,20 @@ impl AptosDebugger { let mut ret = vec![]; let mut cur = vec![]; let mut cur_version = begin; + + let hash = HashValue::zero(); + let executor = AptosVMBlockExecutor::new(); for txn in txns { if txn.is_block_start() && !cur.is_empty() { let to_execute = std::mem::take(&mut cur); let results = self.execute_transactions_at_version( + &executor, cur_version, to_execute, repeat_execution_times, concurrency_levels, + Some(&hash), + Some(hash), )?; cur_version += results.len() as u64; ret.extend(results); @@ -317,10 +331,13 @@ impl AptosDebugger { } if !cur.is_empty() { let results = self.execute_transactions_at_version( + &executor, cur_version, cur, repeat_execution_times, concurrency_levels, + Some(&hash), + Some(hash), )?; ret.extend(results); } @@ -424,23 +441,3 @@ fn is_reconfiguration(vm_output: &TransactionOutput) -> bool { .iter() .any(ContractEvent::is_new_epoch_event) } - -fn execute_block_no_limit( - sig_verified_txns: &[SignatureVerifiedTransaction], - state_view: &DebuggerStateView, - concurrency_level: usize, -) -> Result, VMStatus> { - BlockAptosVM::execute_block::<_, NoOpTransactionCommitHook>( - sig_verified_txns, - state_view, - &ModuleCacheManager::new(), - BlockExecutorConfig { - local: BlockExecutorLocalConfig::default_with_concurrency_level(concurrency_level), - onchain: BlockExecutorConfigFromOnchain::new_no_block_limit(), - }, - None, - None, - None, - ) - .map(BlockOutput::into_transaction_outputs_forced) -} diff --git a/aptos-move/aptos-debugger/src/execute_pending_block.rs b/aptos-move/aptos-debugger/src/execute_pending_block.rs index 7235dff6e9b01..b85f3382c3c2f 100644 --- a/aptos-move/aptos-debugger/src/execute_pending_block.rs +++ b/aptos-move/aptos-debugger/src/execute_pending_block.rs @@ -6,6 +6,7 @@ use anyhow::Result; use aptos_crypto::HashValue; use aptos_logger::info; use aptos_rest_client::Client; +use aptos_vm::{aptos_vm::AptosVMBlockExecutor, VMBlockExecutor}; use clap::Parser; use std::path::PathBuf; use url::Url; @@ -84,11 +85,15 @@ impl Command { user_txns }; + let executor = AptosVMBlockExecutor::new(); let txn_outputs = debugger.execute_transactions_at_version( + &executor, self.begin_version, block, self.repeat_execution_times.unwrap_or(1), &self.opts.concurrency_level, + None, + None, )?; println!("{txn_outputs:#?}"); diff --git a/aptos-move/aptos-vm/src/aptos_vm.rs b/aptos-move/aptos-vm/src/aptos_vm.rs index d50b5c5371047..03fb371b1601e 100644 --- a/aptos-move/aptos-vm/src/aptos_vm.rs +++ b/aptos-move/aptos-vm/src/aptos_vm.rs @@ -126,6 +126,7 @@ use std::{ cmp::{max, min}, collections::{BTreeMap, BTreeSet}, marker::Sync, + str::FromStr, sync::Arc, }; @@ -2815,6 +2816,9 @@ impl VMBlockExecutor for AptosVMBlockExecutor { ); let count = transactions.len(); + let concurrency_level = + usize::from_str(&std::env::var("CONCURRENCY_LEVEL").unwrap()).unwrap(); + let ret = BlockAptosVM::execute_block::< _, NoOpTransactionCommitHook, @@ -2824,7 +2828,7 @@ impl VMBlockExecutor for AptosVMBlockExecutor { &self.module_cache_manager, BlockExecutorConfig { local: BlockExecutorLocalConfig { - concurrency_level: AptosVM::get_concurrency_level(), + concurrency_level, allow_fallback: true, discard_failed_blocks: AptosVM::get_discard_failed_blocks(), module_cache_config: BlockExecutorModuleCacheLocalConfig::default(),