Removes the storage recycler (#118)

This commit is contained in:
Brooks 2024-03-11 15:38:34 -04:00 committed by GitHub
parent 209924d220
commit 5c1df15e92
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 31 additions and 481 deletions

View File

@ -106,7 +106,7 @@ use {
path::{Path, PathBuf},
sync::{
atomic::{AtomicBool, AtomicU32, AtomicU64, AtomicUsize, Ordering},
Arc, Condvar, Mutex, RwLock,
Arc, Condvar, Mutex,
},
thread::{sleep, Builder},
time::{Duration, Instant},
@ -115,7 +115,6 @@ use {
};
const PAGE_SIZE: u64 = 4 * 1024;
pub(crate) const MAX_RECYCLE_STORES: usize = 1000;
// when the accounts write cache exceeds this many bytes, we will flush it
// this can be specified on the command line, too (--accounts-db-cache-limit-mb)
const WRITE_CACHE_LIMIT_BYTES_DEFAULT: u64 = 15_000_000_000;
@ -1084,16 +1083,6 @@ impl AccountStorageEntry {
*count_and_status = (count, status);
}
pub fn recycle(&self, slot: Slot, id: AppendVecId) {
let mut count_and_status = self.count_and_status.lock_write();
self.accounts.reset();
*count_and_status = (0, AccountStorageStatus::Available);
self.slot.store(slot, Ordering::Release);
self.id.store(id, Ordering::Release);
self.approx_store_count.store(0, Ordering::Relaxed);
self.alive_bytes.store(0, Ordering::Release);
}
pub fn status(&self) -> AccountStorageStatus {
self.count_and_status.read().1
}
@ -1267,76 +1256,6 @@ impl StoreAccountsTiming {
}
}
#[derive(Debug, Default)]
struct RecycleStores {
entries: Vec<(Instant, Arc<AccountStorageEntry>)>,
total_bytes: u64,
}
// 30 min should be enough to be certain there won't be any prospective recycle uses for given
// store entry
// That's because it already processed ~2500 slots and ~25 passes of AccountsBackgroundService
pub const EXPIRATION_TTL_SECONDS: u64 = 1800;
impl RecycleStores {
fn add_entry(&mut self, new_entry: Arc<AccountStorageEntry>) {
self.total_bytes += new_entry.capacity();
self.entries.push((Instant::now(), new_entry))
}
fn iter(&self) -> std::slice::Iter<(Instant, Arc<AccountStorageEntry>)> {
self.entries.iter()
}
fn add_entries(&mut self, new_entries: Vec<Arc<AccountStorageEntry>>) {
let now = Instant::now();
for new_entry in new_entries {
self.total_bytes += new_entry.capacity();
self.entries.push((now, new_entry));
}
}
fn expire_old_entries(&mut self) -> Vec<Arc<AccountStorageEntry>> {
let mut expired = vec![];
let now = Instant::now();
let mut expired_bytes = 0;
self.entries.retain(|(recycled_time, entry)| {
if now.duration_since(*recycled_time).as_secs() > EXPIRATION_TTL_SECONDS {
if Arc::strong_count(entry) >= 2 {
warn!(
"Expiring still in-use recycled StorageEntry anyway...: id: {} slot: {}",
entry.append_vec_id(),
entry.slot(),
);
}
expired_bytes += entry.capacity();
expired.push(entry.clone());
false
} else {
true
}
});
self.total_bytes -= expired_bytes;
expired
}
fn remove_entry(&mut self, index: usize) -> Arc<AccountStorageEntry> {
let (_added_time, removed_entry) = self.entries.swap_remove(index);
self.total_bytes -= removed_entry.capacity();
removed_entry
}
fn entry_count(&self) -> usize {
self.entries.len()
}
fn total_bytes(&self) -> u64 {
self.total_bytes
}
}
/// Removing unrooted slots in Accounts Background Service needs to be synchronized with flushing
/// slots from the Accounts Cache. This keeps track of those slots and the Mutex + Condvar for
/// synchronization.
@ -1377,8 +1296,6 @@ pub struct AccountsDb {
sender_bg_hasher: Option<Sender<CachedAccount>>,
read_only_accounts_cache: ReadOnlyAccountsCache,
recycle_stores: RwLock<RecycleStores>,
/// distribute the accounts across storage lists
pub next_id: AtomicAppendVecId,
@ -1506,7 +1423,6 @@ pub struct AccountsStats {
pub stakes_cache_check_and_store_us: AtomicU64,
store_num_accounts: AtomicU64,
store_total_data: AtomicU64,
recycle_store_count: AtomicU64,
create_store_count: AtomicU64,
store_get_slot_store: AtomicU64,
store_find_existing: AtomicU64,
@ -1529,7 +1445,6 @@ pub struct PurgeStats {
total_removed_storage_entries: AtomicUsize,
total_removed_cached_bytes: AtomicU64,
total_removed_stored_bytes: AtomicU64,
recycle_stores_write_elapsed: AtomicU64,
scan_storages_elapsed: AtomicU64,
purge_accounts_index_elapsed: AtomicU64,
handle_reclaims_elapsed: AtomicU64,
@ -1591,11 +1506,6 @@ impl PurgeStats {
self.total_removed_stored_bytes.swap(0, Ordering::Relaxed) as i64,
i64
),
(
"recycle_stores_write_elapsed",
self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
i64
),
(
"scan_storages_elapsed",
self.scan_storages_elapsed.swap(0, Ordering::Relaxed) as i64,
@ -1972,7 +1882,6 @@ pub struct ShrinkStats {
unpackable_slots_count: AtomicU64,
newest_alive_packed_count: AtomicU64,
drop_storage_entries_elapsed: AtomicU64,
recycle_stores_write_elapsed: AtomicU64,
accounts_removed: AtomicUsize,
bytes_removed: AtomicU64,
bytes_written: AtomicU64,
@ -2038,11 +1947,6 @@ impl ShrinkStats {
self.drop_storage_entries_elapsed.swap(0, Ordering::Relaxed) as i64,
i64
),
(
"recycle_stores_write_time",
self.recycle_stores_write_elapsed.swap(0, Ordering::Relaxed) as i64,
i64
),
(
"accounts_removed",
self.accounts_removed.swap(0, Ordering::Relaxed) as i64,
@ -2169,13 +2073,6 @@ impl ShrinkAncientStats {
.swap(0, Ordering::Relaxed) as i64,
i64
),
(
"recycle_stores_write_time",
self.shrink_stats
.recycle_stores_write_elapsed
.swap(0, Ordering::Relaxed) as i64,
i64
),
(
"accounts_removed",
self.shrink_stats
@ -2425,7 +2322,6 @@ impl AccountsDb {
MAX_READ_ONLY_CACHE_DATA_SIZE,
READ_ONLY_CACHE_MS_TO_SKIP_LRU_UPDATE,
),
recycle_stores: RwLock::new(RecycleStores::default()),
uncleaned_pubkeys: DashMap::new(),
next_id: AtomicAppendVecId::new(0),
shrink_candidate_slots: Mutex::new(ShrinkCandidates::default()),
@ -3949,6 +3845,7 @@ impl AccountsDb {
shrink_in_progress,
shrink_can_be_active,
);
let dead_storages_len = dead_storages.len();
if !shrink_collect.all_are_zero_lamports {
self.add_uncleaned_pubkeys_after_shrink(
@ -3957,9 +3854,15 @@ impl AccountsDb {
);
}
self.drop_or_recycle_stores(dead_storages, stats);
let (_, drop_storage_entries_elapsed) = measure_us!(drop(dead_storages));
time.stop();
self.stats
.dropped_stores
.fetch_add(dead_storages_len as u64, Ordering::Relaxed);
stats
.drop_storage_entries_elapsed
.fetch_add(drop_storage_entries_elapsed, Ordering::Relaxed);
stats
.remove_old_stores_shrink_us
.fetch_add(time.as_us(), Ordering::Relaxed);
@ -4148,42 +4051,10 @@ impl AccountsDb {
dead_storages
}
pub fn drop_or_recycle_stores(
&self,
dead_storages: Vec<Arc<AccountStorageEntry>>,
stats: &ShrinkStats,
) {
let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
let mut recycle_stores = self.recycle_stores.write().unwrap();
recycle_stores_write_elapsed.stop();
let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
if recycle_stores.entry_count() < MAX_RECYCLE_STORES {
recycle_stores.add_entries(dead_storages);
drop(recycle_stores);
} else {
self.stats
.dropped_stores
.fetch_add(dead_storages.len() as u64, Ordering::Relaxed);
drop(recycle_stores);
drop(dead_storages);
}
drop_storage_entries_elapsed.stop();
stats
.drop_storage_entries_elapsed
.fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
stats
.recycle_stores_write_elapsed
.fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
}
/// return a store that can contain 'aligned_total' bytes
pub fn get_store_for_shrink(&self, slot: Slot, aligned_total: u64) -> ShrinkInProgress<'_> {
let shrunken_store = self
.try_recycle_store(slot, aligned_total, aligned_total + 1024)
.unwrap_or_else(|| {
self.create_store(slot, aligned_total, "shrink", self.shrink_paths.as_slice())
});
let shrunken_store =
self.create_store(slot, aligned_total, "shrink", self.shrink_paths.as_slice());
self.storage.shrinking_in_progress(slot, shrunken_store)
}
@ -5524,71 +5395,7 @@ impl AccountsDb {
}
}
fn try_recycle_and_insert_store(
&self,
slot: Slot,
min_size: u64,
max_size: u64,
) -> Option<Arc<AccountStorageEntry>> {
let store = self.try_recycle_store(slot, min_size, max_size)?;
self.insert_store(slot, store.clone());
Some(store)
}
fn try_recycle_store(
&self,
slot: Slot,
min_size: u64,
max_size: u64,
) -> Option<Arc<AccountStorageEntry>> {
let mut max = 0;
let mut min = std::u64::MAX;
let mut avail = 0;
let mut recycle_stores = self.recycle_stores.write().unwrap();
for (i, (_recycled_time, store)) in recycle_stores.iter().enumerate() {
if Arc::strong_count(store) == 1 {
max = std::cmp::max(store.accounts.capacity(), max);
min = std::cmp::min(store.accounts.capacity(), min);
avail += 1;
if store.accounts.is_recyclable()
&& store.accounts.capacity() >= min_size
&& store.accounts.capacity() < max_size
{
let ret = recycle_stores.remove_entry(i);
drop(recycle_stores);
let old_id = ret.append_vec_id();
ret.recycle(slot, self.next_id());
// This info shows the appendvec change history. It helps debugging
// the appendvec data corrupution issues related to recycling.
debug!(
"recycling store: old slot {}, old_id: {}, new slot {}, new id{}, path {:?} ",
slot,
old_id,
ret.slot(),
ret.append_vec_id(),
ret.get_path(),
);
self.stats
.recycle_store_count
.fetch_add(1, Ordering::Relaxed);
return Some(ret);
}
}
}
debug!(
"no recycle stores max: {} min: {} len: {} looking: {}, {} avail: {}",
max,
min,
recycle_stores.entry_count(),
min_size,
max_size,
avail,
);
None
}
fn find_storage_candidate(&self, slot: Slot, size: usize) -> Arc<AccountStorageEntry> {
fn find_storage_candidate(&self, slot: Slot) -> Arc<AccountStorageEntry> {
let mut get_slot_stores = Measure::start("get_slot_stores");
let store = self.storage.get_slot_storage_entry(slot);
get_slot_stores.stop();
@ -5612,11 +5419,7 @@ impl AccountsDb {
.store_find_existing
.fetch_add(find_existing.as_us(), Ordering::Relaxed);
let store = if let Some(store) = self.try_recycle_store(slot, size as u64, std::u64::MAX) {
store
} else {
self.create_store(slot, self.file_size, "store", &self.paths)
};
let store = self.create_store(slot, self.file_size, "store", &self.paths);
// try_available is like taking a lock on the store,
// preventing other threads from using it.
@ -5730,28 +5533,6 @@ impl AccountsDb {
self.purge_slots(std::iter::once(&slot));
}
fn recycle_slot_stores(
&self,
total_removed_storage_entries: usize,
slot_stores: &[Arc<AccountStorageEntry>],
) -> u64 {
let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_elapsed");
let mut recycle_stores = self.recycle_stores.write().unwrap();
recycle_stores_write_elapsed.stop();
for (recycled_count, store) in slot_stores.iter().enumerate() {
if recycle_stores.entry_count() > MAX_RECYCLE_STORES {
let dropped_count = total_removed_storage_entries - recycled_count;
self.stats
.dropped_stores
.fetch_add(dropped_count as u64, Ordering::Relaxed);
return recycle_stores_write_elapsed.as_us();
}
recycle_stores.add_entry(Arc::clone(store));
}
recycle_stores_write_elapsed.as_us()
}
/// Purges every slot in `removed_slots` from both the cache and storage. This includes
/// entries in the accounts index, cache entries, and any backing storage entries.
pub fn purge_slots_from_cache_and_store<'a>(
@ -5831,7 +5612,6 @@ impl AccountsDb {
.safety_checks_elapsed
.fetch_add(safety_checks_elapsed.as_us(), Ordering::Relaxed);
let mut total_removed_storage_entries = 0;
let mut total_removed_stored_bytes = 0;
let mut all_removed_slot_storages = vec![];
@ -5839,24 +5619,19 @@ impl AccountsDb {
for remove_slot in removed_slots {
// Remove the storage entries and collect some metrics
if let Some(store) = self.storage.remove(remove_slot, false) {
{
total_removed_storage_entries += 1;
total_removed_stored_bytes += store.accounts.capacity();
}
total_removed_stored_bytes += store.accounts.capacity();
all_removed_slot_storages.push(store);
}
}
remove_storage_entries_elapsed.stop();
let num_stored_slots_removed = all_removed_slot_storages.len();
let recycle_stores_write_elapsed =
self.recycle_slot_stores(total_removed_storage_entries, &all_removed_slot_storages);
let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
// Backing mmaps for removed storages entries explicitly dropped here outside
// of any locks
let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
drop(all_removed_slot_storages);
drop_storage_entries_elapsed.stop();
purge_stats
.remove_storage_entries_elapsed
.fetch_add(remove_storage_entries_elapsed.as_us(), Ordering::Relaxed);
@ -5868,13 +5643,13 @@ impl AccountsDb {
.fetch_add(num_stored_slots_removed, Ordering::Relaxed);
purge_stats
.total_removed_storage_entries
.fetch_add(total_removed_storage_entries, Ordering::Relaxed);
.fetch_add(num_stored_slots_removed, Ordering::Relaxed);
purge_stats
.total_removed_stored_bytes
.fetch_add(total_removed_stored_bytes, Ordering::Relaxed);
purge_stats
.recycle_stores_write_elapsed
.fetch_add(recycle_stores_write_elapsed, Ordering::Relaxed);
self.stats
.dropped_stores
.fetch_add(num_stored_slots_removed as u64, Ordering::Relaxed);
}
fn purge_slot_cache(&self, purged_slot: Slot, slot_cache: SlotCache) {
@ -6196,12 +5971,7 @@ impl AccountsDb {
accounts_and_meta_to_store.len()
);
let special_store_size = std::cmp::max(data_len * 2, self.file_size);
if self
.try_recycle_and_insert_store(slot, special_store_size, std::u64::MAX)
.is_none()
{
self.create_and_insert_store(slot, special_store_size, "large create");
}
self.create_and_insert_store(slot, special_store_size, "large create");
}
continue;
}
@ -6237,25 +6007,6 @@ impl AccountsDb {
self.accounts_cache.report_size();
}
pub fn expire_old_recycle_stores(&self) {
let mut recycle_stores_write_elapsed = Measure::start("recycle_stores_write_time");
let recycle_stores = self.recycle_stores.write().unwrap().expire_old_entries();
recycle_stores_write_elapsed.stop();
let mut drop_storage_entries_elapsed = Measure::start("drop_storage_entries_elapsed");
drop(recycle_stores);
drop_storage_entries_elapsed.stop();
self.clean_accounts_stats
.purge_stats
.drop_storage_entries_elapsed
.fetch_add(drop_storage_entries_elapsed.as_us(), Ordering::Relaxed);
self.clean_accounts_stats
.purge_stats
.recycle_stores_write_elapsed
.fetch_add(recycle_stores_write_elapsed.as_us(), Ordering::Relaxed);
}
// These functions/fields are only usable from a dev context (i.e. tests and benches)
#[cfg(feature = "dev-context-only-utils")]
pub fn flush_accounts_cache_slot_for_tests(&self, slot: Slot) {
@ -6779,11 +6530,6 @@ impl AccountsDb {
datapoint_info!(
"accounts_db-stores",
("total_count", total_count, i64),
(
"recycle_count",
self.recycle_stores.read().unwrap().entry_count() as u64,
i64
),
("total_bytes", total_bytes, i64),
("total_alive_bytes", total_alive_bytes, i64),
("total_alive_ratio", total_alive_ratio, f64),
@ -8410,7 +8156,7 @@ impl AccountsDb {
/// Store the account update.
/// only called by tests
pub fn store_uncached(&self, slot: Slot, accounts: &[(&Pubkey, &AccountSharedData)]) {
let storage = self.find_storage_candidate(slot, 1);
let storage = self.find_storage_candidate(slot);
self.store(
(slot, accounts),
&StoreTo::Storage(&storage),
@ -8568,24 +8314,8 @@ impl AccountsDb {
),
);
let recycle_stores = self.recycle_stores.read().unwrap();
datapoint_info!(
"accounts_db_store_timings2",
(
"recycle_store_count",
self.stats.recycle_store_count.swap(0, Ordering::Relaxed),
i64
),
(
"current_recycle_store_count",
recycle_stores.entry_count(),
i64
),
(
"current_recycle_store_bytes",
recycle_stores.total_bytes(),
i64
),
(
"create_store_count",
self.stats.create_store_count.swap(0, Ordering::Relaxed),
@ -9397,20 +9127,6 @@ impl AccountsDb {
pub fn print_accounts_stats(&self, label: &str) {
self.print_index(label);
self.print_count_and_status(label);
info!("recycle_stores:");
let recycle_stores = self.recycle_stores.read().unwrap();
for (recycled_time, entry) in recycle_stores.iter() {
info!(
" slot: {} id: {} count_and_status: {:?} approx_store_count: {} len: {} capacity: {} (recycled: {:?})",
entry.slot(),
entry.append_vec_id(),
entry.count_and_status.read(),
entry.approx_store_count.load(Ordering::Relaxed),
entry.accounts.len(),
entry.accounts.capacity(),
recycled_time,
);
}
}
fn print_index(&self, label: &str) {
@ -9766,7 +9482,7 @@ pub mod tests {
std::{
iter::FromIterator,
str::FromStr,
sync::atomic::AtomicBool,
sync::{atomic::AtomicBool, RwLock},
thread::{self, Builder, JoinHandle},
},
test_case::test_case,
@ -12522,7 +12238,7 @@ pub mod tests {
db.store_accounts_unfrozen(
(some_slot, &[(&key, &account)][..]),
Some(vec![&AccountHash(Hash::default())]),
&StoreTo::Storage(&db.find_storage_candidate(some_slot, 1)),
&StoreTo::Storage(&db.find_storage_candidate(some_slot)),
None,
StoreReclaims::Default,
UpdateIndexThreadSelection::PoolWithThreshold,
@ -12755,7 +12471,7 @@ pub mod tests {
db.store_accounts_unfrozen(
(some_slot, accounts),
Some(vec![&some_hash]),
&StoreTo::Storage(&db.find_storage_candidate(some_slot, 1)),
&StoreTo::Storage(&db.find_storage_candidate(some_slot)),
None,
StoreReclaims::Default,
UpdateIndexThreadSelection::PoolWithThreshold,
@ -13548,75 +13264,6 @@ pub mod tests {
assert_eq!(accounts.accounts_index.ref_count_from_storage(&pubkey1), 0);
}
#[test]
fn test_store_reuse() {
solana_logger::setup();
let accounts = AccountsDb {
file_size: 4096,
..AccountsDb::new_single_for_tests()
};
let size = 100;
let num_accounts: usize = 100;
let mut keys = Vec::new();
for i in 0..num_accounts {
let account = AccountSharedData::new((i + 1) as u64, size, &Pubkey::default());
let pubkey = solana_sdk::pubkey::new_rand();
accounts.store_cached((0 as Slot, &[(&pubkey, &account)][..]), None);
keys.push(pubkey);
}
// get delta hash to feed these accounts to clean
accounts.calculate_accounts_delta_hash(0);
accounts.add_root(0);
// we have to flush just slot 0
// if we slot 0 and 1 together, then they are cleaned and slot 0 doesn't contain the accounts
// this test wants to clean and then allow us to shrink
accounts.flush_accounts_cache(true, None);
for (i, key) in keys[1..].iter().enumerate() {
let account =
AccountSharedData::new((1 + i + num_accounts) as u64, size, &Pubkey::default());
accounts.store_cached((1 as Slot, &[(key, &account)][..]), None);
}
accounts.calculate_accounts_delta_hash(1);
accounts.add_root(1);
accounts.flush_accounts_cache(true, None);
accounts.clean_accounts_for_tests();
accounts.shrink_all_slots(false, None, &EpochSchedule::default());
// Clean again to flush the dirty stores
// and allow them to be recycled in the next step
accounts.clean_accounts_for_tests();
accounts.print_accounts_stats("post-shrink");
let num_stores = accounts.recycle_stores.read().unwrap().entry_count();
assert!(num_stores > 0);
let mut account_refs = Vec::new();
let num_to_store = 20;
for (i, key) in keys[..num_to_store].iter().enumerate() {
let account = AccountSharedData::new(
(1 + i + 2 * num_accounts) as u64,
i + 20,
&Pubkey::default(),
);
accounts.store_uncached(2, &[(key, &account)]);
account_refs.push(account);
}
assert!(accounts.recycle_stores.read().unwrap().entry_count() < num_stores);
accounts.print_accounts_stats("post-store");
let mut ancestors = Ancestors::default();
ancestors.insert(1, 0);
ancestors.insert(2, 1);
for (key, account_ref) in keys[..num_to_store].iter().zip(account_refs) {
assert_eq!(
accounts.load_without_fixed_root(&ancestors, key).unwrap().0,
account_ref
);
}
}
#[test]
#[should_panic(expected = "We've run out of storage ids!")]
fn test_wrapping_append_vec_id() {
@ -14870,77 +14517,6 @@ pub mod tests {
assert!(!db.storage.is_empty_entry(1));
}
#[test]
fn test_recycle_stores_expiration() {
solana_logger::setup();
let common_store_path = Path::new("");
let common_slot_id = 12;
let store_file_size = 1000;
let store1_id = 22;
let entry1 = Arc::new(AccountStorageEntry::new(
common_store_path,
common_slot_id,
store1_id,
store_file_size,
));
let store2_id = 44;
let entry2 = Arc::new(AccountStorageEntry::new(
common_store_path,
common_slot_id,
store2_id,
store_file_size,
));
let mut recycle_stores = RecycleStores::default();
recycle_stores.add_entry(entry1);
recycle_stores.add_entry(entry2);
assert_eq!(recycle_stores.entry_count(), 2);
// no expiration for newly added entries
let expired = recycle_stores.expire_old_entries();
assert_eq!(
expired
.iter()
.map(|e| e.append_vec_id())
.collect::<Vec<_>>(),
Vec::<AppendVecId>::new()
);
assert_eq!(
recycle_stores
.iter()
.map(|(_, e)| e.append_vec_id())
.collect::<Vec<_>>(),
vec![store1_id, store2_id]
);
assert_eq!(recycle_stores.entry_count(), 2);
assert_eq!(recycle_stores.total_bytes(), store_file_size * 2);
// expiration for only too old entries
recycle_stores.entries[0].0 = Instant::now()
.checked_sub(Duration::from_secs(EXPIRATION_TTL_SECONDS + 1))
.unwrap();
let expired = recycle_stores.expire_old_entries();
assert_eq!(
expired
.iter()
.map(|e| e.append_vec_id())
.collect::<Vec<_>>(),
vec![store1_id]
);
assert_eq!(
recycle_stores
.iter()
.map(|(_, e)| e.append_vec_id())
.collect::<Vec<_>>(),
vec![store2_id]
);
assert_eq!(recycle_stores.entry_count(), 1);
assert_eq!(recycle_stores.total_bytes(), store_file_size);
}
const RACY_SLEEP_MS: u64 = 10;
const RACE_TIME: u64 = 5;

View File

@ -985,7 +985,7 @@ pub mod tests {
create_db_with_storages_and_index, create_storages_and_update_index,
get_all_accounts, remove_account_for_tests, CAN_RANDOMLY_SHRINK_FALSE,
},
ShrinkCollectRefs, MAX_RECYCLE_STORES,
ShrinkCollectRefs,
},
accounts_index::UpsertReclaim,
append_vec::{aligned_stored_size, AppendVec, AppendVecStoredAccountMeta},
@ -3023,6 +3023,9 @@ pub mod tests {
#[test]
fn test_shrink_packed_ancient() {
// NOTE: The recycler has been removed. Creating this many extra storages is no longer
// necessary, but also does no harm either.
const MAX_RECYCLE_STORES: usize = 1000;
solana_logger::setup();
// When we pack ancient append vecs, the packed append vecs are recycled first if possible. This means they aren't dropped directly.

View File

@ -39,14 +39,6 @@ use {
const INTERVAL_MS: u64 = 100;
const CLEAN_INTERVAL_BLOCKS: u64 = 100;
// This value is chosen to spread the dropping cost over 3 expiration checks
// RecycleStores are fully populated almost all of its lifetime. So, otherwise
// this would drop MAX_RECYCLE_STORES mmaps at once in the worst case...
// (Anyway, the dropping part is outside the AccountsDb::recycle_stores lock
// and dropped in this AccountsBackgroundServe, so this shouldn't matter much)
const RECYCLE_STORE_EXPIRATION_INTERVAL_SECS: u64 =
solana_accounts_db::accounts_db::EXPIRATION_TTL_SECONDS / 3;
pub type SnapshotRequestSender = Sender<SnapshotRequest>;
pub type SnapshotRequestReceiver = Receiver<SnapshotRequest>;
pub type DroppedSlotsSender = Sender<(Slot, BankId)>;
@ -605,7 +597,6 @@ impl AccountsBackgroundService {
let mut last_cleaned_block_height = 0;
let mut removed_slots_count = 0;
let mut total_remove_slots_time = 0;
let mut last_expiration_check_time = Instant::now();
let t_background = Builder::new()
.name("solBgAccounts".to_string())
.spawn(move || {
@ -631,8 +622,6 @@ impl AccountsBackgroundService {
&mut total_remove_slots_time,
);
Self::expire_old_recycle_stores(&bank, &mut last_expiration_check_time);
let non_snapshot_time = last_snapshot_end_time
.map(|last_snapshot_end_time: Instant| {
last_snapshot_end_time.elapsed().as_micros()
@ -759,16 +748,6 @@ impl AccountsBackgroundService {
pub fn join(self) -> thread::Result<()> {
self.t_background.join()
}
fn expire_old_recycle_stores(bank: &Bank, last_expiration_check_time: &mut Instant) {
let now = Instant::now();
if now.duration_since(*last_expiration_check_time).as_secs()
> RECYCLE_STORE_EXPIRATION_INTERVAL_SECS
{
bank.expire_old_recycle_stores();
*last_expiration_check_time = now;
}
}
}
/// Get the AccountsPackageKind from a given SnapshotRequest

View File

@ -5864,10 +5864,6 @@ impl Bank {
.flush_accounts_cache(false, Some(self.slot()))
}
pub fn expire_old_recycle_stores(&self) {
self.rc.accounts.accounts_db.expire_old_recycle_stores()
}
/// Technically this issues (or even burns!) new lamports,
/// so be extra careful for its usage
fn store_account_and_update_capitalization(

View File

@ -239,12 +239,8 @@ impl<'a> SnapshotMinimizer<'a> {
measure!(self.purge_dead_slots(dead_slots), "purge dead slots");
info!("{purge_dead_slots_measure}");
let (_, drop_or_recycle_stores_measure) = measure!(
self.accounts_db()
.drop_or_recycle_stores(dead_storages, &self.accounts_db().shrink_stats),
"drop or recycle stores"
);
info!("{drop_or_recycle_stores_measure}");
let (_, drop_storages_measure) = measure!(drop(dead_storages), "drop storages");
info!("{drop_storages_measure}");
// Turn logging back on after minimization
self.accounts_db()