From 1157e657c29265a292cd9e55e8a8f85f58d271d4 Mon Sep 17 00:00:00 2001 From: Will Yang Date: Mon, 30 Sep 2024 19:03:48 -0700 Subject: [PATCH] PruningOptions -> RetentionPolicies make life easier, requires epochs_to_keep with optional overrides. otherwise, no retention policy at all strumming the pruning config. pruner.rs file will define prunable tables. the toml file when parsed into Rust config will warn if it tries to name any variants not supported by indexer code. additionally, there's also a check to make sure that prunable tables actually exist in the db --- Cargo.lock | 3 + .../sui-graphql-rpc/src/test_infra/cluster.rs | 6 +- crates/sui-indexer/Cargo.toml | 3 + crates/sui-indexer/src/config.rs | 178 +++++++++++++++++- crates/sui-indexer/src/db.rs | 54 +++++- crates/sui-indexer/src/handlers/pruner.rs | 132 +++++++++---- crates/sui-indexer/src/indexer.rs | 15 +- crates/sui-indexer/src/main.rs | 11 +- crates/sui-indexer/src/test_utils.rs | 9 +- .../src/test_adapter.rs | 13 +- 10 files changed, 359 insertions(+), 65 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 423e16e677663..75f4cd94ef67a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13659,6 +13659,8 @@ dependencies = [ "serde_json", "serde_with 3.9.0", "simulacrum", + "strum 0.24.1", + "strum_macros 0.24.3", "sui-archival", "sui-config", "sui-core", @@ -13687,6 +13689,7 @@ dependencies = [ "tokio", "tokio-stream", "tokio-util 0.7.10 (registry+https://github.com/rust-lang/crates.io-index)", + "toml 0.7.4", "tracing", "url", ] diff --git a/crates/sui-graphql-rpc/src/test_infra/cluster.rs b/crates/sui-graphql-rpc/src/test_infra/cluster.rs index 76ab5d4f9b789..84d182b4be58b 100644 --- a/crates/sui-graphql-rpc/src/test_infra/cluster.rs +++ b/crates/sui-graphql-rpc/src/test_infra/cluster.rs @@ -14,7 +14,7 @@ use std::path::PathBuf; use std::sync::Arc; use std::time::Duration; use sui_graphql_rpc_client::simple_client::SimpleClient; -use sui_indexer::config::PruningOptions; +pub use sui_indexer::config::RetentionPolicies; pub use sui_indexer::config::SnapshotLagConfig; use sui_indexer::errors::IndexerError; use sui_indexer::store::indexer_store::IndexerStore; @@ -151,7 +151,7 @@ pub async fn start_network_cluster() -> NetworkCluster { pub async fn serve_executor( executor: Arc, snapshot_config: Option, - epochs_to_keep: Option, + retention_policies: Option, data_ingestion_path: PathBuf, ) -> ExecutorCluster { let database = TempDb::new().unwrap(); @@ -183,7 +183,7 @@ pub async fn serve_executor( let (pg_store, pg_handle, _) = start_indexer_writer_for_testing( db_url, Some(snapshot_config.clone()), - Some(PruningOptions { epochs_to_keep }), + retention_policies, Some(data_ingestion_path), Some(cancellation_token.clone()), ) diff --git a/crates/sui-indexer/Cargo.toml b/crates/sui-indexer/Cargo.toml index 0ec76e6b2ff9a..1f4f76f415a3e 100644 --- a/crates/sui-indexer/Cargo.toml +++ b/crates/sui-indexer/Cargo.toml @@ -31,11 +31,14 @@ regex.workspace = true serde.workspace = true serde_json.workspace = true serde_with.workspace = true +strum.workspace = true +strum_macros.workspace = true tap.workspace = true tempfile.workspace = true thiserror.workspace = true tokio = { workspace = true, features = ["full"] } tokio-util = { workspace = true, features = ["rt"] } +toml.workspace = true tracing.workspace = true url.workspace = true diff --git a/crates/sui-indexer/src/config.rs b/crates/sui-indexer/src/config.rs index 9e42c57da65ea..2691d9cf25a63 100644 --- a/crates/sui-indexer/src/config.rs +++ b/crates/sui-indexer/src/config.rs @@ -1,14 +1,20 @@ // Copyright (c) Mysten Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use crate::backfill::BackfillTaskKind; use crate::db::ConnectionPoolConfig; +use crate::{backfill::BackfillTaskKind, handlers::pruner::PrunableTable}; use clap::{Args, Parser, Subcommand}; -use std::{net::SocketAddr, path::PathBuf}; +use serde::{Deserialize, Serialize}; +use std::{collections::HashMap, net::SocketAddr, path::PathBuf}; +use strum::IntoEnumIterator; use sui_json_rpc::name_service::NameServiceConfig; use sui_types::base_types::{ObjectID, SuiAddress}; use url::Url; +/// The primary purpose of objects_history is to serve consistency query. +/// A short retention is sufficient. +const OBJECTS_HISTORY_EPOCHS_TO_KEEP: u64 = 2; + #[derive(Parser, Clone, Debug)] #[clap( name = "Sui indexer", @@ -208,8 +214,93 @@ pub enum Command { #[derive(Args, Default, Debug, Clone)] pub struct PruningOptions { - #[arg(long, env = "EPOCHS_TO_KEEP")] - pub epochs_to_keep: Option, + /// Path to TOML file containing configuration for retention policies. + #[arg(long)] + pub pruning_config_path: Option, +} + +/// Represents the default retention policy and overrides for prunable tables. When `finalize` is +/// called, the `policies` field is updated with the default retention policy for all tables that do +/// not have an override specified. Instantiated only if `PruningOptions` is provided on indexer +/// start. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct RetentionPolicies { + /// Default retention policy for all tables. + pub epochs_to_keep: u64, + /// In Rust, a mapping of all `PrunableTable` variants to their respective retention policies. + /// From a TOML file, a user only needs to explicitly list tables that should override the + /// default retention. + #[serde(default, rename = "overrides")] + pub policies: HashMap, +} + +impl PruningOptions { + /// Load default retention policy and overrides from file. + pub fn load_from_file(&self) -> Option { + let Some(config_path) = self.pruning_config_path.as_ref() else { + return None; + }; + + let contents = std::fs::read_to_string(&config_path) + .expect("Failed to read default retention policy and overrides from file"); + let retention_with_overrides = toml::de::from_str::(&contents) + .expect("Failed to parse into RetentionPolicies struct"); + + let default_retention = retention_with_overrides.epochs_to_keep; + + assert!( + default_retention > 0, + "Default retention must be greater than 0" + ); + assert!( + retention_with_overrides + .policies + .values() + .all(|&policy| policy > 0), + "All retention overrides must be greater than 0" + ); + + Some(retention_with_overrides) + } +} + +impl RetentionPolicies { + /// Create a new `RetentionPolicies` with the specified default retention and overrides. Call + /// `finalize()` on the instance to update the `policies` field with the default retention + /// policy for all tables that do not have an override specified. + pub fn new(epochs_to_keep: u64, overrides: HashMap) -> Self { + Self { + epochs_to_keep, + policies: overrides, + } + } + + /// Create a new `RetentionPolicies` with only the default retention specified and the default + /// override for `objects_history`. Call `finalize()` on the instance to update the `policies` + /// field with the default retention policy for all tables that do not have an override + /// specified. + pub fn new_with_default_retention_only_for_testing(epochs_to_keep: u64) -> Self { + let mut overrides = HashMap::new(); + overrides.insert( + PrunableTable::ObjectsHistory, + OBJECTS_HISTORY_EPOCHS_TO_KEEP, + ); + + Self::new(epochs_to_keep, overrides) + } + + /// Updates the `policies` field with the default retention policy for all tables that do not + /// have an override specified. + pub fn finalize(mut self) -> Self { + for table in PrunableTable::iter() { + self.policies.entry(table).or_insert(self.epochs_to_keep); + } + self + } + + pub fn get(&self, table: &PrunableTable) -> Option { + self.policies.get(table).copied() + } } #[derive(Args, Debug, Clone)] @@ -290,7 +381,9 @@ impl Default for RestoreConfig { #[cfg(test)] mod test { use super::*; + use std::io::Write; use tap::Pipe; + use tempfile::NamedTempFile; fn parse_args<'a, T>(args: impl IntoIterator) -> Result where @@ -354,4 +447,81 @@ mod test { // fullnode rpc url must be present parse_args::([]).unwrap_err(); } + + #[test] + fn test_valid_pruning_config_file() { + let toml_str = r#" + epochs_to_keep = 5 + + [overrides] + objects_history = 10 + transactions = 20 + "#; + + let opts = toml::de::from_str::(toml_str).unwrap(); + assert_eq!(opts.epochs_to_keep, 5); + assert_eq!(opts.policies.get(&PrunableTable::ObjectsHistory), Some(&10)); + assert_eq!(opts.policies.get(&PrunableTable::Transactions), Some(&20)); + assert_eq!(opts.policies.len(), 2); + } + + #[test] + fn test_pruning_options_from_file() { + let mut temp_file = NamedTempFile::new().unwrap(); + let toml_content = r#" + epochs_to_keep = 5 + + [overrides] + objects_history = 10 + transactions = 20 + "#; + temp_file.write_all(toml_content.as_bytes()).unwrap(); + let temp_path: PathBuf = temp_file.path().to_path_buf(); + let pruning_options = PruningOptions { + pruning_config_path: Some(temp_path.clone()), + }; + let mut retention_policies = pruning_options.load_from_file().unwrap(); + + // Assert the parsed values + assert_eq!(retention_policies.epochs_to_keep, 5); + assert_eq!( + retention_policies.get(&PrunableTable::ObjectsHistory), + Some(10) + ); + assert_eq!( + retention_policies.get(&PrunableTable::Transactions), + Some(20) + ); + assert_eq!(retention_policies.policies.len(), 2); + + retention_policies = retention_policies.finalize(); + + assert!( + retention_policies.policies.len() > 2, + "Expected more than 2 policies, but got {}", + retention_policies.policies.len() + ); + } + + #[test] + fn test_invalid_pruning_config_file() { + let toml_str = r#" + epochs_to_keep = 5 + + [overrides] + objects_history = 10 + transactions = 20 + invalid_table = 30 + "#; + + let result = toml::from_str::(toml_str); + assert!(result.is_err(), "Expected an error, but parsing succeeded"); + + if let Err(e) = result { + assert!( + e.to_string().contains("unknown variant `invalid_table`"), + "Error message doesn't mention the invalid table" + ); + } + } } diff --git a/crates/sui-indexer/src/db.rs b/crates/sui-indexer/src/db.rs index 1abd5d31a956e..9937b61ce2655 100644 --- a/crates/sui-indexer/src/db.rs +++ b/crates/sui-indexer/src/db.rs @@ -3,14 +3,17 @@ use crate::database::Connection; use crate::errors::IndexerError; +use crate::handlers::pruner::PrunableTable; use clap::Args; use diesel::migration::{Migration, MigrationSource, MigrationVersion}; use diesel::pg::Pg; +use diesel::prelude::QueryableByName; use diesel::table; use diesel::QueryDsl; use diesel_migrations::{embed_migrations, EmbeddedMigrations}; -use std::collections::BTreeSet; +use std::collections::{BTreeSet, HashSet}; use std::time::Duration; +use strum::IntoEnumIterator; use tracing::info; table! { @@ -134,6 +137,55 @@ async fn check_db_migration_consistency_impl( ))) } +/// Check that prunable tables exist in the database. +pub async fn check_prunable_tables_valid(conn: &mut Connection<'_>) -> Result<(), IndexerError> { + info!("Starting compatibility check"); + + use diesel_async::RunQueryDsl; + + let select_parent_tables = r#" + SELECT c.relname AS table_name + FROM pg_class c + JOIN pg_namespace n ON n.oid = c.relnamespace + LEFT JOIN pg_partitioned_table pt ON pt.partrelid = c.oid + WHERE c.relkind IN ('r', 'p') -- 'r' for regular tables, 'p' for partitioned tables + AND n.nspname = 'public' + AND ( + pt.partrelid IS NOT NULL -- This is a partitioned (parent) table + OR NOT EXISTS ( -- This is not a partition (child table) + SELECT 1 + FROM pg_inherits i + WHERE i.inhrelid = c.oid + ) + ); + "#; + + #[derive(QueryableByName)] + struct TableName { + #[diesel(sql_type = diesel::sql_types::Text)] + table_name: String, + } + + let result: Vec = diesel::sql_query(select_parent_tables) + .load(conn) + .await + .map_err(|e| IndexerError::DbMigrationError(format!("Failed to fetch tables: {e}")))?; + + let parent_tables_from_db: HashSet<_> = result.into_iter().map(|t| t.table_name).collect(); + + for key in PrunableTable::iter() { + if !parent_tables_from_db.contains(key.as_ref()) { + return Err(IndexerError::GenericError(format!( + "Invalid retention policy override provided for table {}: does not exist in the database", + key + ))); + } + } + + info!("Compatibility check passed"); + Ok(()) +} + pub use setup_postgres::{reset_database, run_migrations}; pub mod setup_postgres { diff --git a/crates/sui-indexer/src/handlers/pruner.rs b/crates/sui-indexer/src/handlers/pruner.rs index cdbe274376f46..9f196a6d4f028 100644 --- a/crates/sui-indexer/src/handlers/pruner.rs +++ b/crates/sui-indexer/src/handlers/pruner.rs @@ -1,42 +1,99 @@ // Copyright (c) Mysten Labs, Inc. // SPDX-License-Identifier: Apache-2.0 -use std::collections::HashMap; -use std::time::Duration; -use tokio_util::sync::CancellationToken; -use tracing::{error, info}; - +use crate::config::RetentionPolicies; use crate::errors::IndexerError; use crate::store::pg_partition_manager::PgPartitionManager; use crate::store::PgIndexerStore; use crate::{metrics::IndexerMetrics, store::IndexerStore, types::IndexerResult}; - -/// The primary purpose of objects_history is to serve consistency query. -/// A short retention is sufficient. -const OBJECTS_HISTORY_EPOCHS_TO_KEEP: u64 = 2; +use serde::{Deserialize, Serialize}; +use std::collections::HashMap; +use std::time::Duration; +use strum_macros; +use tokio_util::sync::CancellationToken; +use tracing::{error, info}; pub struct Pruner { pub store: PgIndexerStore, pub partition_manager: PgPartitionManager, - pub epochs_to_keep: u64, + pub retention_policies: RetentionPolicies, pub metrics: IndexerMetrics, } +/// Enum representing tables that the pruner is allowed to prune. The pruner will ignore any table +/// that is not listed here. +#[derive( + Debug, + Eq, + PartialEq, + strum_macros::Display, + strum_macros::EnumString, + strum_macros::EnumIter, + strum_macros::AsRefStr, + Hash, + Serialize, + Deserialize, + Clone, +)] +#[strum(serialize_all = "snake_case")] +#[serde(rename_all = "snake_case")] +pub enum PrunableTable { + ObjectsHistory, + Transactions, + Events, + + EventEmitPackage, + EventEmitModule, + EventSenders, + EventStructInstantiation, + EventStructModule, + EventStructName, + EventStructPackage, + + TxAffectedAddresses, + TxAffectedObjects, + TxCallsPkg, + TxCallsMod, + TxCallsFun, + TxChangedObjects, + TxDigests, + TxInputObjects, + TxKinds, + TxRecipients, + TxSenders, + + Checkpoints, + PrunerCpWatermark, +} + impl Pruner { + /// Instantiates a pruner with default retention and overrides. Pruner will finalize the + /// retention policies so there is a value for every prunable table. pub fn new( store: PgIndexerStore, - epochs_to_keep: u64, + retention_policies: RetentionPolicies, metrics: IndexerMetrics, ) -> Result { let partition_manager = PgPartitionManager::new(store.pool())?; + Ok(Self { store, partition_manager, - epochs_to_keep, + retention_policies: retention_policies.finalize(), metrics, }) } + /// Given a table name, return the number of epochs to keep for that table. Return `None` if the + /// table is not prunable. + fn epochs_to_keep(&self, table_name: &str) -> Option { + if let Ok(variant) = table_name.parse::() { + self.retention_policies.get(&variant) + } else { + None + } + } + pub async fn start(&self, cancel: CancellationToken) -> IndexerResult<()> { let mut last_seen_max_epoch = 0; // The first epoch that has not yet been pruned. @@ -63,35 +120,38 @@ impl Pruner { .collect(); for (table_name, (min_partition, max_partition)) in &table_partitions { - if last_seen_max_epoch != *max_partition { - error!( - "Epochs are out of sync for table {}: max_epoch={}, max_partition={}", - table_name, last_seen_max_epoch, max_partition - ); - } + if let Some(epochs_to_keep) = self.epochs_to_keep(table_name) { + if last_seen_max_epoch != *max_partition { + error!( + "Epochs are out of sync for table {}: max_epoch={}, max_partition={}", + table_name, last_seen_max_epoch, max_partition + ); + } - let epochs_to_keep = if table_name == "objects_history" { - OBJECTS_HISTORY_EPOCHS_TO_KEEP - } else { - self.epochs_to_keep - }; - for epoch in *min_partition..last_seen_max_epoch.saturating_sub(epochs_to_keep - 1) - { - if cancel.is_cancelled() { - info!("Pruner task cancelled."); - return Ok(()); + for epoch in + *min_partition..last_seen_max_epoch.saturating_sub(epochs_to_keep - 1) + { + if cancel.is_cancelled() { + info!("Pruner task cancelled."); + return Ok(()); + } + self.partition_manager + .drop_table_partition(table_name.clone(), epoch) + .await?; + info!( + "Batch dropped table partition {} epoch {}", + table_name, epoch + ); } - self.partition_manager - .drop_table_partition(table_name.clone(), epoch) - .await?; - info!( - "Batch dropped table partition {} epoch {}", - table_name, epoch - ); } } - let prune_to_epoch = last_seen_max_epoch.saturating_sub(self.epochs_to_keep - 1); + // TODO: (wlmyng) Once we have the watermarks table, we can iterate through each row + // returned from `watermarks`, look it up against `retention_policies`, and process them + // independently. This also means that pruning overrides will only apply for + // epoch-partitioned tables right now. + let prune_to_epoch = + last_seen_max_epoch.saturating_sub(self.retention_policies.epochs_to_keep - 1); let prune_start_epoch = next_prune_epoch.unwrap_or(min_epoch); for epoch in prune_start_epoch..prune_to_epoch { if cancel.is_cancelled() { diff --git a/crates/sui-indexer/src/indexer.rs b/crates/sui-indexer/src/indexer.rs index 21ea094ab362f..f9f1d357364c9 100644 --- a/crates/sui-indexer/src/indexer.rs +++ b/crates/sui-indexer/src/indexer.rs @@ -19,7 +19,7 @@ use sui_data_ingestion_core::{ use sui_types::messages_checkpoint::CheckpointSequenceNumber; use crate::build_json_rpc_server; -use crate::config::{IngestionConfig, JsonRpcConfig, PruningOptions, SnapshotLagConfig}; +use crate::config::{IngestionConfig, JsonRpcConfig, RetentionPolicies, SnapshotLagConfig}; use crate::database::ConnectionPool; use crate::errors::IndexerError; use crate::handlers::checkpoint_handler::new_handlers; @@ -44,7 +44,7 @@ impl Indexer { store, metrics, snapshot_config, - PruningOptions::default(), + None, /* retention_policies */ CancellationToken::new(), ) .await @@ -55,7 +55,7 @@ impl Indexer { store: PgIndexerStore, metrics: IndexerMetrics, snapshot_config: SnapshotLagConfig, - pruning_options: PruningOptions, + retention_policies: Option, cancel: CancellationToken, ) -> Result<(), IndexerError> { info!( @@ -88,13 +88,8 @@ impl Indexer { ) .await?; - if let Some(epochs_to_keep) = pruning_options.epochs_to_keep { - info!( - "Starting indexer pruner with epochs to keep: {}", - epochs_to_keep - ); - assert!(epochs_to_keep > 0, "Epochs to keep must be positive"); - let pruner = Pruner::new(store.clone(), epochs_to_keep, metrics.clone())?; + if let Some(retention_policies) = retention_policies { + let pruner = Pruner::new(store.clone(), retention_policies, metrics.clone())?; let cancel_clone = cancel.clone(); spawn_monitored_task!(pruner.start(cancel_clone)); } diff --git a/crates/sui-indexer/src/main.rs b/crates/sui-indexer/src/main.rs index 7cd1a729dbefe..e5364bf00a136 100644 --- a/crates/sui-indexer/src/main.rs +++ b/crates/sui-indexer/src/main.rs @@ -5,7 +5,9 @@ use clap::Parser; use sui_indexer::backfill::backfill_runner::BackfillRunner; use sui_indexer::config::{Command, UploadOptions}; use sui_indexer::database::ConnectionPool; -use sui_indexer::db::{check_db_migration_consistency, reset_database, run_migrations}; +use sui_indexer::db::{ + check_db_migration_consistency, check_prunable_tables_valid, reset_database, run_migrations, +}; use sui_indexer::indexer::Indexer; use sui_indexer::metrics::{ spawn_connection_pool_metric_collector, start_prometheus_server, IndexerMetrics, @@ -45,6 +47,11 @@ async fn main() -> anyhow::Result<()> { } => { // Make sure to run all migrations on startup, and also serve as a compatibility check. run_migrations(pool.dedicated_connection().await?).await?; + let retention_policies = pruning_options.load_from_file(); + if retention_policies.is_some() { + check_prunable_tables_valid(&mut pool.get().await?).await?; + } + let store = PgIndexerStore::new(pool, upload_options, indexer_metrics.clone()); Indexer::start_writer_with_config( @@ -52,7 +59,7 @@ async fn main() -> anyhow::Result<()> { store, indexer_metrics, snapshot_config, - pruning_options, + retention_policies, CancellationToken::new(), ) .await?; diff --git a/crates/sui-indexer/src/test_utils.rs b/crates/sui-indexer/src/test_utils.rs index 92c5c9f68ebd8..776cf0f920855 100644 --- a/crates/sui-indexer/src/test_utils.rs +++ b/crates/sui-indexer/src/test_utils.rs @@ -9,7 +9,7 @@ use std::path::PathBuf; use std::time::Duration; use sui_json_rpc_types::SuiTransactionBlockResponse; -use crate::config::{IngestionConfig, PruningOptions, SnapshotLagConfig, UploadOptions}; +use crate::config::{IngestionConfig, RetentionPolicies, SnapshotLagConfig, UploadOptions}; use crate::database::Connection; use crate::database::ConnectionPool; use crate::db::ConnectionPoolConfig; @@ -62,11 +62,11 @@ pub async fn start_indexer_jsonrpc_for_testing( } /// Wrapper over `Indexer::start_writer_with_config` to make it easier to configure an indexer -/// writer for testing. +/// writer for testing. ` pub async fn start_indexer_writer_for_testing( db_url: String, snapshot_config: Option, - pruning_options: Option, + retention_policies: Option, data_ingestion_path: Option, cancel: Option, ) -> ( @@ -76,7 +76,6 @@ pub async fn start_indexer_writer_for_testing( ) { let token = cancel.unwrap_or_else(CancellationToken::new); let snapshot_config = snapshot_config.unwrap_or_default(); - let pruning_options = pruning_options.unwrap_or_default(); // Reduce the connection pool size to 10 for testing to prevent maxing out let pool_config = ConnectionPoolConfig { @@ -119,7 +118,7 @@ pub async fn start_indexer_writer_for_testing( store_clone, indexer_metrics, snapshot_config, - pruning_options, + retention_policies, token_clone, ) .await diff --git a/crates/sui-transactional-test-runner/src/test_adapter.rs b/crates/sui-transactional-test-runner/src/test_adapter.rs index 7b5bd91626593..00a7dd7803cf4 100644 --- a/crates/sui-transactional-test-runner/src/test_adapter.rs +++ b/crates/sui-transactional-test-runner/src/test_adapter.rs @@ -50,7 +50,7 @@ use sui_core::authority::test_authority_builder::TestAuthorityBuilder; use sui_core::authority::AuthorityState; use sui_framework::DEFAULT_FRAMEWORK_PATH; use sui_graphql_rpc::test_infra::cluster::ExecutorCluster; -use sui_graphql_rpc::test_infra::cluster::{serve_executor, SnapshotLagConfig}; +use sui_graphql_rpc::test_infra::cluster::{serve_executor, RetentionPolicies, SnapshotLagConfig}; use sui_json_rpc_api::QUERY_MAX_RESULT_LIMIT; use sui_json_rpc_types::{DevInspectResults, SuiExecutionStatus, SuiTransactionBlockEffectsAPI}; use sui_protocol_config::{Chain, ProtocolConfig}; @@ -330,6 +330,11 @@ impl<'a> MoveTestAdapter<'a> for SuiTestAdapter { }, cluster, ) = if is_simulator { + // TODO: (wlmyng) as of right now, we can't test per-table overrides until the pruner is + // updated + let retention_policies = + epochs_to_keep.map(RetentionPolicies::new_with_default_retention_only_for_testing); + init_sim_executor( rng, account_names, @@ -338,7 +343,7 @@ impl<'a> MoveTestAdapter<'a> for SuiTestAdapter { custom_validator_account, reference_gas_price, snapshot_config, - epochs_to_keep, + retention_policies, ) .await } else { @@ -2116,7 +2121,7 @@ async fn init_sim_executor( custom_validator_account: bool, reference_gas_price: Option, snapshot_config: SnapshotLagConfig, - epochs_to_keep: Option, + retention_policies: Option, ) -> ( Box, AccountSetup, @@ -2188,7 +2193,7 @@ async fn init_sim_executor( let cluster = serve_executor( Arc::new(read_replica), Some(snapshot_config), - epochs_to_keep, + retention_policies, data_ingestion_path, ) .await;