collect rent from accounts (take:2) (#6360)

* collect rent from credit debit accounts

* collect rent from credit only account

* rent_collector now can deduct partial rent + no mem copy + improved design

* adding a test to test credit only rent

* add bank level test for rent deduction

* add test to check if hash value changes or not

* adding test scenario for lamport circulation
This commit is contained in:
Parth 2019-10-16 07:45:47 +05:30 committed by GitHub
parent 78d5c1de9a
commit c1b401a04a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 605 additions and 67 deletions

View File

@ -18,7 +18,7 @@ use solana_sdk::transaction::{Transaction, TransactionError};
use std::collections::{HashMap, HashSet};
use std::io::{BufReader, Error as IOError, Read};
use std::path::Path;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
use std::sync::{Arc, Mutex, RwLock, RwLockReadGuard, RwLockWriteGuard};
use crate::transaction_utils::OrderedIterator;
@ -27,6 +27,7 @@ use crate::transaction_utils::OrderedIterator;
struct CreditOnlyLock {
credits: AtomicU64,
lock_count: Mutex<u64>,
rent_debtor: AtomicBool,
}
/// This structure handles synchronization for db
@ -39,7 +40,7 @@ pub struct Accounts {
account_locks: Mutex<HashSet<Pubkey>>,
/// Set of credit-only accounts which are currently in the pipeline, caching account balance
/// and number of locks. On commit_credits(), we do a take() on the option so that the hashmap
/// number of locks and whether or not account owes rent. On commit_credits(), we do a take() on the option so that the hashmap
/// is no longer available to be written to.
credit_only_account_locks: Arc<RwLock<Option<HashMap<Pubkey, CreditOnlyLock>>>>,
}
@ -118,7 +119,10 @@ impl Accounts {
.filter(|key| !message.program_ids().contains(&key))
{
let (account, rent) = AccountsDB::load(storage, ancestors, accounts_index, key)
.and_then(|(account, _)| rent_collector.update(account))
.and_then(|(mut account, _)| {
let rent_due = rent_collector.update(&mut account);
Some((account, rent_due))
})
.unwrap_or_default();
accounts.push(account);
@ -420,6 +424,7 @@ impl Accounts {
CreditOnlyLock {
credits: AtomicU64::new(0),
lock_count: Mutex::new(1),
rent_debtor: AtomicBool::new(false),
},
);
}
@ -550,49 +555,92 @@ impl Accounts {
// so will fail the lock
// 2) Any transaction that grabs a lock and then commit_credits clears the HashMap will find
// the HashMap is None on unlock_accounts, and will perform a no-op.
pub fn commit_credits(&self, ancestors: &HashMap<Fork, usize>, fork: Fork) {
pub fn commit_credits_and_rents(
&self,
rent_collector: &RentCollector,
ancestors: &HashMap<Fork, usize>,
fork: Fork,
) -> u64 {
// Clear the credit only hashmap so that no further transactions can modify it
let credit_only_account_locks = Self::take_credit_only(&self.credit_only_account_locks)
.expect("Credit only locks didn't exist in commit_credits");
self.store_credit_only_credits(credit_only_account_locks, ancestors, fork);
.expect("Credit only locks didn't exist in commit_credits_and_rents");
self.store_credit_only_credits_and_rents(
credit_only_account_locks,
rent_collector,
ancestors,
fork,
)
}
/// Used only for tests to store credit-only accounts after every transaction
pub fn commit_credits_unsafe(&self, ancestors: &HashMap<Fork, usize>, fork: Fork) {
pub fn commit_credits_and_rents_unsafe(
&self,
rent_collector: &RentCollector,
ancestors: &HashMap<Fork, usize>,
fork: Fork,
) -> u64 {
// Clear the credit only hashmap so that no further transactions can modify it
let mut w_credit_only_account_locks = self.credit_only_account_locks.write().unwrap();
let w_credit_only_account_locks =
Self::get_write_access_credit_only(&mut w_credit_only_account_locks)
.expect("Credit only locks didn't exist in commit_credits");
self.store_credit_only_credits(w_credit_only_account_locks.drain(), ancestors, fork);
self.store_credit_only_credits_and_rents(
w_credit_only_account_locks.drain(),
rent_collector,
ancestors,
fork,
)
}
fn store_credit_only_credits<I>(
fn store_credit_only_credits_and_rents<I>(
&self,
credit_only_account_locks: I,
rent_collector: &RentCollector,
ancestors: &HashMap<Fork, usize>,
fork: Fork,
) where
) -> u64
where
I: IntoIterator<Item = (Pubkey, CreditOnlyLock)>,
{
for (pubkey, lock) in credit_only_account_locks {
let lock_count = *lock.lock_count.lock().unwrap();
if lock_count != 0 {
warn!(
"dropping credit-only lock on {}, still has {} locks",
pubkey, lock_count
);
}
let credit = lock.credits.load(Ordering::Relaxed);
if credit > 0 {
let mut account = self
.load_slow(ancestors, &pubkey)
.map(|(account, _)| account)
.unwrap_or_default();
let mut accounts: Vec<(Pubkey, Account)> = vec![];
let mut total_rent_collected = 0;
{
let accounts_index = self.accounts_db.accounts_index.read().unwrap();
let storage = self.accounts_db.storage.read().unwrap();
for (pubkey, lock) in credit_only_account_locks {
let lock_count = *lock.lock_count.lock().unwrap();
if lock_count != 0 {
warn!(
"dropping credit-only lock on {}, still has {} locks",
pubkey, lock_count
);
}
let credit = lock.credits.load(Ordering::Relaxed);
let (mut account, _) =
AccountsDB::load(&storage, ancestors, &accounts_index, &pubkey)
.unwrap_or_default();
if lock.rent_debtor.load(Ordering::Relaxed) {
let rent_collected = rent_collector.update(&mut account);
total_rent_collected += rent_collected;
}
account.lamports += credit;
self.store_slow(fork, &pubkey, &account);
accounts.push((pubkey, account));
}
}
let account_to_store: Vec<(&Pubkey, &Account)> = accounts
.iter()
.map(|(key, account)| (key, account))
.collect();
self.accounts_db.store(fork, &account_to_store);
total_rent_collected
}
fn collect_accounts_to_store<'a>(
@ -623,16 +671,17 @@ impl Accounts {
{
if message.is_debitable(i) {
accounts.push((key, account));
}
if *credit > 0 {
// Increment credit-only account balance Atomic
self.credit_only_account_locks
.read()
.unwrap()
.as_ref()
} else {
let r_credit_only_account_locks =
self.credit_only_account_locks.read().unwrap();
let credit_only_lock = r_credit_only_account_locks.as_ref()
.expect("Collect accounts should only be called before a commit, and credit only account locks should exist before a commit")
.get(key)
.unwrap()
.unwrap();
credit_only_lock.rent_debtor.store(true, Ordering::Relaxed);
credit_only_lock
.credits
.fetch_add(*credit, Ordering::Relaxed);
}
@ -664,6 +713,7 @@ mod tests {
use solana_sdk::fee_calculator::FeeCalculator;
use solana_sdk::hash::Hash;
use solana_sdk::instruction::CompiledInstruction;
use solana_sdk::rent_calculator::RentCalculator;
use solana_sdk::signature::{Keypair, KeypairUtil};
use solana_sdk::sysvar;
use solana_sdk::transaction::Transaction;
@ -672,10 +722,11 @@ mod tests {
use std::{thread, time};
use tempfile::TempDir;
fn load_accounts_with_fee(
fn load_accounts_with_fee_and_rent(
tx: Transaction,
ka: &Vec<(Pubkey, Account)>,
fee_calculator: &FeeCalculator,
rent_collector: &RentCollector,
error_counters: &mut ErrorCounters,
) -> Vec<Result<TransactionLoadResult>> {
let mut hash_queue = BlockhashQueue::new(100);
@ -686,7 +737,6 @@ mod tests {
}
let ancestors = vec![(0, 0)].into_iter().collect();
let rent_collector = RentCollector::default();
let res = accounts.load_accounts(
&ancestors,
&[tx],
@ -705,7 +755,8 @@ mod tests {
error_counters: &mut ErrorCounters,
) -> Vec<Result<TransactionLoadResult>> {
let fee_calculator = FeeCalculator::default();
load_accounts_with_fee(tx, ka, &fee_calculator, error_counters)
let rent_collector: RentCollector = RentCollector::default();
load_accounts_with_fee_and_rent(tx, ka, &fee_calculator, &rent_collector, error_counters)
}
#[test]
@ -809,8 +860,15 @@ mod tests {
let fee_calculator = FeeCalculator::new(10, 0);
assert_eq!(fee_calculator.calculate_fee(tx.message()), 10);
let loaded_accounts =
load_accounts_with_fee(tx, &accounts, &fee_calculator, &mut error_counters);
let rent_collector = RentCollector::default();
let loaded_accounts = load_accounts_with_fee_and_rent(
tx,
&accounts,
&fee_calculator,
&rent_collector,
&mut error_counters,
);
assert_eq!(error_counters.insufficient_funds, 1);
assert_eq!(loaded_accounts.len(), 1);
@ -1414,17 +1472,29 @@ mod tests {
}
#[test]
fn test_commit_credits() {
fn test_commit_credits_and_rents() {
let pubkey0 = Pubkey::new_rand();
let pubkey1 = Pubkey::new_rand();
let pubkey2 = Pubkey::new_rand();
let overdue_account_pubkey = Pubkey::new_rand();
let account0 = Account::new(1, 0, &Pubkey::default());
let account1 = Account::new(2, 0, &Pubkey::default());
let account0 = Account::new(1, 1, &Pubkey::default());
let account1 = Account::new(2, 1, &Pubkey::default());
let overdue_account = Account::new(1, 2, &Pubkey::default());
let accounts = Accounts::new(None);
accounts.store_slow(0, &pubkey0, &account0);
accounts.store_slow(0, &pubkey1, &account1);
accounts.store_slow(0, &overdue_account_pubkey, &overdue_account);
let mut rent_collector = RentCollector::default();
rent_collector.epoch = 2;
rent_collector.slots_per_year = 400_f64;
rent_collector.rent_calculator = RentCalculator {
burn_percent: 10,
exemption_threshold: 8.0,
lamports_per_byte_year: 1,
};
{
let mut credit_only_account_locks = accounts.credit_only_account_locks.write().unwrap();
@ -1432,8 +1502,9 @@ mod tests {
credit_only_account_locks.insert(
pubkey0,
CreditOnlyLock {
credits: AtomicU64::new(0),
credits: AtomicU64::new(1),
lock_count: Mutex::new(1),
rent_debtor: AtomicBool::new(true),
},
);
credit_only_account_locks.insert(
@ -1441,6 +1512,7 @@ mod tests {
CreditOnlyLock {
credits: AtomicU64::new(5),
lock_count: Mutex::new(1),
rent_debtor: AtomicBool::new(true),
},
);
credit_only_account_locks.insert(
@ -1448,24 +1520,74 @@ mod tests {
CreditOnlyLock {
credits: AtomicU64::new(10),
lock_count: Mutex::new(1),
rent_debtor: AtomicBool::new(false),
},
);
credit_only_account_locks.insert(
overdue_account_pubkey,
CreditOnlyLock {
credits: AtomicU64::new(3),
lock_count: Mutex::new(1),
rent_debtor: AtomicBool::new(true),
},
);
}
let ancestors = vec![(0, 0)].into_iter().collect();
accounts.commit_credits_unsafe(&ancestors, 0);
// No change when CreditOnlyLock credits are 0
// This account has data length of 2
assert_eq!(
accounts
.load_slow(&ancestors, &overdue_account_pubkey)
.unwrap()
.0
.data
.len(),
2
);
// Total rent collected should be: rent from pubkey0(1) + rent from pubkey1(1)
assert_eq!(
accounts.commit_credits_and_rents_unsafe(&rent_collector, &ancestors, 0),
3
);
// New balance should be previous balance plus CreditOnlyLock credits - rent
// Balance(1) - Rent(1) + Credit(1)
assert_eq!(
accounts.load_slow(&ancestors, &pubkey0).unwrap().0.lamports,
1
);
// New balance should equal previous balance plus CreditOnlyLock credits
// New balance should equal previous balance plus CreditOnlyLock credits - rent
// Balance(2) - Rent(1) + Credit(5)
assert_eq!(
accounts.load_slow(&ancestors, &pubkey1).unwrap().0.lamports,
7
6
);
// New account should be created
// Rent overdue account, will be reseted and it's lamport will be consumed
// Balance(1) - Rent(1)(Due to insufficient balance) + Credit(3)
assert_eq!(
accounts
.load_slow(&ancestors, &overdue_account_pubkey)
.unwrap()
.0
.lamports,
3
);
// The reseted account should have data length of zero
assert_eq!(
accounts
.load_slow(&ancestors, &overdue_account_pubkey)
.unwrap()
.0
.data
.len(),
0
);
// New account should be created and no rent should be charged
assert_eq!(
accounts.load_slow(&ancestors, &pubkey2).unwrap().0.lamports,
10
@ -1551,6 +1673,7 @@ mod tests {
CreditOnlyLock {
credits: AtomicU64::new(0),
lock_count: Mutex::new(1),
rent_debtor: AtomicBool::new(false),
},
);
}

View File

@ -228,6 +228,11 @@ pub struct Bank {
/// latest rent collector, knows the epoch
rent_collector: RentCollector,
/// tallied credit-debit rent for this slot
#[serde(serialize_with = "serialize_atomicu64")]
#[serde(deserialize_with = "deserialize_atomicu64")]
tallied_credit_debit_rent: AtomicU64,
/// initialized from genesis
epoch_schedule: EpochSchedule,
@ -333,6 +338,7 @@ impl Bank {
parent_hash: parent.hash(),
collector_id: *collector_id,
collector_fees: AtomicU64::new(0),
tallied_credit_debit_rent: AtomicU64::new(0),
ancestors: HashMap::new(),
hash: RwLock::new(Hash::default()),
is_delta: AtomicBool::new(false),
@ -557,9 +563,11 @@ impl Bank {
if *hash == Hash::default() {
// finish up any deferred changes to account state
self.commit_credits();
self.collect_fees();
let collected_rent = self.commit_credits_and_rents();
self.distribute_rent(collected_rent);
// freeze is a one-way trip, idempotent
*hash = self.hash_internal_state();
true
@ -578,6 +586,10 @@ impl Bank {
&self.epoch_schedule
}
pub fn get_tallied_credit_debit_rent(&self) -> u64 {
self.tallied_credit_debit_rent.load(Ordering::Relaxed)
}
/// squash the parent's state up into this Bank,
/// this Bank becomes a root
pub fn squash(&self) {
@ -824,9 +836,11 @@ impl Bank {
self.process_transactions(&txs)[0].clone()?;
// Call this instead of commit_credits(), so that the credit-only locks hashmap on this
// bank isn't deleted
self.rc
.accounts
.commit_credits_unsafe(&self.ancestors, self.slot());
self.rc.accounts.commit_credits_and_rents_unsafe(
&self.rent_collector,
&self.ancestors,
self.slot(),
);
tx.signatures
.get(0)
.map_or(Ok(()), |sig| self.get_signature_status(sig).unwrap())
@ -1198,6 +1212,7 @@ impl Bank {
write_time.stop();
debug!("store: {}us txs_len={}", write_time.as_us(), txs.len(),);
self.update_transaction_statuses(txs, iteration_order, &executed);
self.tally_credit_debit_rent(txs, iteration_order, &executed, loaded_accounts);
self.filter_program_errors_and_collect_fee(txs, iteration_order, executed)
}
@ -1566,10 +1581,60 @@ impl Bank {
);
}
fn commit_credits(&self) {
self.rc
.accounts
.commit_credits(&self.ancestors, self.slot());
fn commit_credits_and_rents(&self) -> u64 {
self.rc.accounts.commit_credits_and_rents(
&self.rent_collector,
&self.ancestors,
self.slot(),
)
}
fn tally_credit_debit_rent(
&self,
txs: &[Transaction],
iteration_order: Option<&[usize]>,
res: &[Result<()>],
loaded_accounts: &[Result<TransactionLoadResult>],
) {
let mut collected_rent = 0;
for (i, (raccs, tx)) in loaded_accounts
.iter()
.zip(OrderedIterator::new(txs, iteration_order))
.enumerate()
{
if res[i].is_err() || raccs.is_err() {
continue;
}
let message = &tx.message();
let acc = raccs.as_ref().unwrap();
for (_i, rent) in acc
.3
.iter()
.enumerate()
.filter(|(i, _rent)| message.is_debitable(*i))
{
collected_rent += rent;
}
}
self.tallied_credit_debit_rent
.fetch_add(collected_rent, Ordering::Relaxed);
}
fn distribute_rent(&self, credit_only_collected_rent: u64) {
let total_rent_collected =
credit_only_collected_rent + self.tallied_credit_debit_rent.load(Ordering::Relaxed);
if total_rent_collected != 0 {
let burned_portion = (total_rent_collected
* u64::from(self.rent_collector.rent_calculator.burn_percent))
/ 100;
let _rent_to_be_distributed = total_rent_collected - burned_portion;
// TODO: distribute remaining rent amount to validators
// self.capitalization.fetch_sub(burned_portion, Ordering::Relaxed);
}
}
}
@ -1592,6 +1657,7 @@ mod tests {
status_cache::MAX_CACHE_ENTRIES,
};
use bincode::{deserialize_from, serialize_into, serialized_size};
use solana_sdk::system_program::solana_system_program;
use solana_sdk::{
clock::DEFAULT_TICKS_PER_SLOT,
epoch_schedule::MINIMUM_SLOTS_PER_EPOCH,
@ -1753,6 +1819,349 @@ mod tests {
assert_eq!(bank.transaction_count(), 2);
}
#[test]
fn test_credit_debit_rent_no_side_effect_on_hash() {
let (mut genesis_block, _mint_keypair) = create_genesis_block(10);
let credit_only_key1 = Pubkey::new_rand();
let credit_only_key2 = Pubkey::new_rand();
let credit_debit_keypair1: Keypair = Keypair::new();
let credit_debit_keypair2: Keypair = Keypair::new();
let rent_overdue_credit_only_key1 = Pubkey::new_rand();
let rent_overdue_credit_only_key2 = Pubkey::new_rand();
let rent_overdue_credit_debit_keypair1 = Keypair::new();
let rent_overdue_credit_debit_keypair2 = Keypair::new();
genesis_block.rent_calculator = RentCalculator {
lamports_per_byte_year: 1,
exemption_threshold: 21.0,
burn_percent: 10,
};
let root_bank = Arc::new(Bank::new(&genesis_block));
let bank = Bank::new_from_parent(
&root_bank,
&Pubkey::default(),
2 * (SECONDS_PER_YEAR
// * (ns/s)/(ns/tick) / ticks/slot = 1/s/1/tick = ticks/s
*(1_000_000_000.0 / duration_as_ns(&genesis_block.poh_config.target_tick_duration) as f64)
// / ticks/slot
/ genesis_block.ticks_per_slot as f64) as u64,
);
let root_bank_2 = Arc::new(Bank::new(&genesis_block));
let bank_with_success_txs = Bank::new_from_parent(
&root_bank_2,
&Pubkey::default(),
2 * (SECONDS_PER_YEAR
// * (ns/s)/(ns/tick) / ticks/slot = 1/s/1/tick = ticks/s
*(1_000_000_000.0 / duration_as_ns(&genesis_block.poh_config.target_tick_duration) as f64)
// / ticks/slot
/ genesis_block.ticks_per_slot as f64) as u64,
);
assert_eq!(bank.last_blockhash(), genesis_block.hash());
// Initialize credit-debit and credit only accounts
let credit_debit_account1 = Account::new(20, 1, &Pubkey::default());
let credit_debit_account2 = Account::new(20, 1, &Pubkey::default());
let credit_only_account1 = Account::new(3, 1, &Pubkey::default());
let credit_only_account2 = Account::new(3, 1, &Pubkey::default());
bank.store_account(&credit_debit_keypair1.pubkey(), &credit_debit_account1);
bank.store_account(&credit_debit_keypair2.pubkey(), &credit_debit_account2);
bank.store_account(&credit_only_key1, &credit_only_account1);
bank.store_account(&credit_only_key2, &credit_only_account2);
bank_with_success_txs
.store_account(&credit_debit_keypair1.pubkey(), &credit_debit_account1);
bank_with_success_txs
.store_account(&credit_debit_keypair2.pubkey(), &credit_debit_account2);
bank_with_success_txs.store_account(&credit_only_key1, &credit_only_account1);
bank_with_success_txs.store_account(&credit_only_key2, &credit_only_account2);
let rent_overdue_credit_debit_account1 = Account::new(2, 1, &Pubkey::default());
let rent_overdue_credit_debit_account2 = Account::new(2, 1, &Pubkey::default());
let rent_overdue_credit_only_account1 = Account::new(1, 1, &Pubkey::default());
let rent_overdue_credit_only_account2 = Account::new(1, 1, &Pubkey::default());
bank.store_account(
&rent_overdue_credit_debit_keypair1.pubkey(),
&rent_overdue_credit_debit_account1,
);
bank.store_account(
&rent_overdue_credit_debit_keypair2.pubkey(),
&rent_overdue_credit_debit_account2,
);
bank.store_account(
&rent_overdue_credit_only_key1,
&rent_overdue_credit_only_account1,
);
bank.store_account(
&rent_overdue_credit_only_key2,
&rent_overdue_credit_only_account2,
);
bank_with_success_txs.store_account(
&rent_overdue_credit_debit_keypair1.pubkey(),
&rent_overdue_credit_debit_account1,
);
bank_with_success_txs.store_account(
&rent_overdue_credit_debit_keypair2.pubkey(),
&rent_overdue_credit_debit_account2,
);
bank_with_success_txs.store_account(
&rent_overdue_credit_only_key1,
&rent_overdue_credit_only_account1,
);
bank_with_success_txs.store_account(
&rent_overdue_credit_only_key2,
&rent_overdue_credit_only_account2,
);
// Make native instruction loader rent exempt
let system_program_id = solana_system_program().1;
let mut system_program_account = bank.get_account(&system_program_id).unwrap();
system_program_account.lamports =
bank.get_minimum_balance_for_rent_exemption(system_program_account.data.len());
bank.store_account(&system_program_id, &system_program_account);
bank_with_success_txs.store_account(&system_program_id, &system_program_account);
let t1 = system_transaction::transfer(
&credit_debit_keypair1,
&rent_overdue_credit_only_key1,
1,
genesis_block.hash(),
);
let t2 = system_transaction::transfer(
&rent_overdue_credit_debit_keypair1,
&credit_only_key1,
1,
genesis_block.hash(),
);
let t3 = system_transaction::transfer(
&credit_debit_keypair2,
&credit_only_key2,
1,
genesis_block.hash(),
);
let t4 = system_transaction::transfer(
&rent_overdue_credit_debit_keypair2,
&rent_overdue_credit_only_key2,
1,
genesis_block.hash(),
);
let res = bank.process_transactions(&vec![t1.clone(), t2.clone(), t3.clone(), t4.clone()]);
assert_eq!(res.len(), 4);
assert_eq!(res[0], Ok(()));
assert_eq!(res[1], Err(TransactionError::AccountNotFound));
assert_eq!(res[2], Ok(()));
assert_eq!(res[3], Err(TransactionError::AccountNotFound));
bank.freeze();
let rwlockguard_bank_hash = bank.hash.read().unwrap();
let bank_hash = rwlockguard_bank_hash.as_ref();
let res = bank_with_success_txs.process_transactions(&vec![t3.clone(), t1.clone()]);
assert_eq!(res.len(), 2);
assert_eq!(res[0], Ok(()));
assert_eq!(res[1], Ok(()));
bank_with_success_txs.freeze();
let rwlockguard_bank_with_success_txs_hash = bank_with_success_txs.hash.read().unwrap();
let bank_with_success_txs_hash = rwlockguard_bank_with_success_txs_hash.as_ref();
assert_eq!(bank_with_success_txs_hash, bank_hash);
}
#[test]
fn test_credit_debit_rent() {
let (mut genesis_block, _mint_keypair) = create_genesis_block(10);
let credit_only_key1 = Pubkey::new_rand();
let credit_only_key2 = Pubkey::new_rand();
let credit_debit_keypair1: Keypair = Keypair::new();
let credit_debit_keypair2: Keypair = Keypair::new();
let rent_overdue_credit_only_key1 = Pubkey::new_rand();
let rent_overdue_credit_only_key2 = Pubkey::new_rand();
let rent_overdue_credit_debit_keypair1 = Keypair::new();
let rent_overdue_credit_debit_keypair2 = Keypair::new();
genesis_block.rent_calculator = RentCalculator {
lamports_per_byte_year: 1,
exemption_threshold: 21.0,
burn_percent: 10,
};
let root_bank = Bank::new(&genesis_block);
let bank = Bank::new_from_parent(
&Arc::new(root_bank),
&Pubkey::default(),
2 * (SECONDS_PER_YEAR
// * (ns/s)/(ns/tick) / ticks/slot = 1/s/1/tick = ticks/s
*(1_000_000_000.0 / duration_as_ns(&genesis_block.poh_config.target_tick_duration) as f64)
// / ticks/slot
/ genesis_block.ticks_per_slot as f64) as u64,
);
assert_eq!(bank.last_blockhash(), genesis_block.hash());
// Initialize credit-debit and credit only accounts
let credit_debit_account1 = Account::new(20, 1, &Pubkey::default());
let credit_debit_account2 = Account::new(20, 1, &Pubkey::default());
let credit_only_account1 = Account::new(3, 1, &Pubkey::default());
let credit_only_account2 = Account::new(3, 1, &Pubkey::default());
bank.store_account(&credit_debit_keypair1.pubkey(), &credit_debit_account1);
bank.store_account(&credit_debit_keypair2.pubkey(), &credit_debit_account2);
bank.store_account(&credit_only_key1, &credit_only_account1);
bank.store_account(&credit_only_key2, &credit_only_account2);
let rent_overdue_credit_debit_account1 = Account::new(2, 1, &Pubkey::default());
let rent_overdue_credit_debit_account2 = Account::new(2, 1, &Pubkey::default());
let rent_overdue_credit_only_account1 = Account::new(1, 1, &Pubkey::default());
let rent_overdue_credit_only_account2 = Account::new(1, 1, &Pubkey::default());
bank.store_account(
&rent_overdue_credit_debit_keypair1.pubkey(),
&rent_overdue_credit_debit_account1,
);
bank.store_account(
&rent_overdue_credit_debit_keypair2.pubkey(),
&rent_overdue_credit_debit_account2,
);
bank.store_account(
&rent_overdue_credit_only_key1,
&rent_overdue_credit_only_account1,
);
bank.store_account(
&rent_overdue_credit_only_key2,
&rent_overdue_credit_only_account2,
);
// Make native instruction loader rent exempt
let system_program_id = solana_system_program().1;
let mut system_program_account = bank.get_account(&system_program_id).unwrap();
system_program_account.lamports =
bank.get_minimum_balance_for_rent_exemption(system_program_account.data.len());
bank.store_account(&system_program_id, &system_program_account);
let total_lamports_before_txs = system_program_account.lamports
+ rent_overdue_credit_debit_account1.lamports
+ rent_overdue_credit_debit_account2.lamports
+ rent_overdue_credit_only_account1.lamports
+ rent_overdue_credit_only_account2.lamports
+ credit_debit_account1.lamports
+ credit_debit_account2.lamports
+ credit_only_account1.lamports
+ credit_only_account2.lamports;
let t1 = system_transaction::transfer(
&credit_debit_keypair1,
&rent_overdue_credit_only_key1,
1,
genesis_block.hash(),
);
let t2 = system_transaction::transfer(
&rent_overdue_credit_debit_keypair1,
&credit_only_key1,
1,
genesis_block.hash(),
);
let t3 = system_transaction::transfer(
&credit_debit_keypair2,
&credit_only_key2,
1,
genesis_block.hash(),
);
let t4 = system_transaction::transfer(
&rent_overdue_credit_debit_keypair2,
&rent_overdue_credit_only_key2,
1,
genesis_block.hash(),
);
let res = bank.process_transactions(&vec![t1.clone(), t2.clone(), t3.clone(), t4.clone()]);
let mut total_lamports_after_txs = 0;
assert_eq!(res.len(), 4);
assert_eq!(res[0], Ok(()));
assert_eq!(res[1], Err(TransactionError::AccountNotFound));
assert_eq!(res[2], Ok(()));
assert_eq!(res[3], Err(TransactionError::AccountNotFound));
// We haven't yet made any changes to credit only accounts
assert_eq!(bank.get_balance(&credit_only_key1), 3);
assert_eq!(bank.get_balance(&credit_only_key2), 3);
assert_eq!(bank.get_balance(&rent_overdue_credit_only_key1), 1);
assert_eq!(bank.get_balance(&rent_overdue_credit_only_key2), 1);
assert_eq!(
bank.get_balance(&system_program_id),
system_program_account.lamports
);
total_lamports_after_txs += bank.get_balance(&system_program_id);
// Credit-debit account's rent is already deducted
// 20 - 1(Transferred) - 2(Rent)
assert_eq!(bank.get_balance(&credit_debit_keypair1.pubkey()), 17);
total_lamports_after_txs += bank.get_balance(&credit_debit_keypair1.pubkey());
assert_eq!(bank.get_balance(&credit_debit_keypair2.pubkey()), 17);
total_lamports_after_txs += bank.get_balance(&credit_debit_keypair2.pubkey());
// Since this credit-debit accounts are unable to pay rent, load_tx_account failed, as they are
// the signer account. No change was done.
assert_eq!(
bank.get_balance(&rent_overdue_credit_debit_keypair1.pubkey()),
2
);
total_lamports_after_txs += bank.get_balance(&rent_overdue_credit_debit_keypair1.pubkey());
assert_eq!(
bank.get_balance(&rent_overdue_credit_debit_keypair2.pubkey()),
2
);
total_lamports_after_txs += bank.get_balance(&rent_overdue_credit_debit_keypair2.pubkey());
// Credit-debit account's rent is stored in `tallied_credit_debit_rent`
// Rent deducted is: 2+2
assert_eq!(bank.get_tallied_credit_debit_rent(), 4);
total_lamports_after_txs += bank.get_tallied_credit_debit_rent();
// Rent deducted is: 2+1
let commited_credit_only_rent = bank.commit_credits_and_rents();
assert_eq!(commited_credit_only_rent, 3);
total_lamports_after_txs += commited_credit_only_rent;
// No rent deducted because tx failed
assert_eq!(bank.get_balance(&credit_only_key1), 3);
total_lamports_after_txs += bank.get_balance(&credit_only_key1);
// Now, we have credited credits and debited rent
// 3 + 1(Transferred) - 2(Rent)
assert_eq!(bank.get_balance(&credit_only_key2), 2);
total_lamports_after_txs += bank.get_balance(&credit_only_key2);
// Since we were unable to pay rent, the account was reset, rent got deducted.
// And credit went to that overwritten account
// Rent deducted: 1
assert_eq!(bank.get_balance(&rent_overdue_credit_only_key1), 1);
assert_eq!(
bank.get_account(&rent_overdue_credit_only_key1)
.unwrap()
.data
.len(),
0
);
total_lamports_after_txs += bank.get_balance(&rent_overdue_credit_only_key1);
// No rent got deducted as, we were unable to load accounts (load_tx_accounts errored out)
assert_eq!(bank.get_balance(&rent_overdue_credit_only_key2), 1);
total_lamports_after_txs += bank.get_balance(&rent_overdue_credit_only_key2);
// total lamports in circulation should be same
assert_eq!(total_lamports_after_txs, total_lamports_before_txs);
}
#[test]
fn test_one_source_two_tx_one_batch() {
let (genesis_block, mint_keypair) = create_genesis_block(1);
@ -1764,7 +2173,7 @@ mod tests {
let t1 = system_transaction::transfer(&mint_keypair, &key1, 1, genesis_block.hash());
let t2 = system_transaction::transfer(&mint_keypair, &key2, 1, genesis_block.hash());
let res = bank.process_transactions(&vec![t1.clone(), t2.clone()]);
bank.commit_credits();
bank.commit_credits_and_rents();
assert_eq!(res.len(), 2);
assert_eq!(res[0], Ok(()));
@ -2163,9 +2572,11 @@ mod tests {
system_transaction::transfer(&payer1, &recipient.pubkey(), 1, genesis_block.hash());
let txs = vec![tx0, tx1, tx2];
let results = bank.process_transactions(&txs);
bank.rc
.accounts
.commit_credits_unsafe(&bank.ancestors, bank.slot());
bank.rc.accounts.commit_credits_and_rents_unsafe(
&bank.rent_collector,
&bank.ancestors,
bank.slot(),
);
// If multiple transactions attempt to deposit into the same account, they should succeed,
// since System Transfer `To` accounts are given credit-only handling
@ -2184,9 +2595,11 @@ mod tests {
system_transaction::transfer(&recipient, &payer0.pubkey(), 1, genesis_block.hash());
let txs = vec![tx0, tx1];
let results = bank.process_transactions(&txs);
bank.rc
.accounts
.commit_credits_unsafe(&bank.ancestors, bank.slot());
bank.rc.accounts.commit_credits_and_rents_unsafe(
&bank.rent_collector,
&bank.ancestors,
bank.slot(),
);
// However, an account may not be locked as credit-only and credit-debit at the same time.
assert_eq!(results[0], Ok(()));
assert_eq!(results[1], Err(TransactionError::AccountInUse));

View File

@ -35,9 +35,9 @@ impl RentCollector {
// updates this account's lamports and status and returns
// the account rent collected, if any
//
pub fn update(&self, mut account: Account) -> Option<(Account, u64)> {
pub fn update(&self, account: &mut Account) -> u64 {
if account.data.is_empty() || account.rent_epoch > self.epoch {
Some((account, 0))
0
} else {
let slots_elapsed: u64 = (account.rent_epoch..=self.epoch)
.map(|epoch| self.epoch_schedule.get_slots_in_epoch(epoch + 1))
@ -53,13 +53,15 @@ impl RentCollector {
if account.lamports > rent_due {
account.rent_epoch = self.epoch + 1;
account.lamports -= rent_due;
Some((account, rent_due))
rent_due
} else {
None
let rent_charged = account.lamports;
*account = Account::default();
rent_charged
}
} else {
// maybe collect rent later, leave account alone
Some((account, 0))
0
}
}
}