add MiningPools, fund validator MiningPools from inflation (#4676)

* add MiningPool fund validator MinigPools from inflation

* fixup

* finish rename of MINIMUM_SLOT_LENGTH to MINIMUM_SLOTS_PER_EPOCH

* deterministic miningpool location

* point_value, not credit_value... use f64
This commit is contained in:
Rob Walker 2019-06-14 11:38:37 -07:00 committed by GitHub
parent 589a9d3a72
commit 4c6a6d63bf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 366 additions and 89 deletions

View File

@ -9,7 +9,7 @@ use crate::entry::{Entry, EntrySlice};
use crate::gossip_service::discover_cluster;
use crate::locktower::VOTE_THRESHOLD_DEPTH;
use solana_client::thin_client::create_client;
use solana_runtime::epoch_schedule::MINIMUM_SLOT_LENGTH;
use solana_runtime::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH;
use solana_sdk::client::SyncClient;
use solana_sdk::hash::Hash;
use solana_sdk::poh_config::PohConfig;
@ -141,7 +141,7 @@ pub fn kill_entry_and_spend_and_verify_rest(
let (cluster_nodes, _) = discover_cluster(&entry_point_info.gossip, nodes).unwrap();
assert!(cluster_nodes.len() >= nodes);
let client = create_client(entry_point_info.client_facing_addr(), FULLNODE_PORT_RANGE);
let first_two_epoch_slots = MINIMUM_SLOT_LENGTH * 3;
let first_two_epoch_slots = MINIMUM_SLOTS_PER_EPOCH * 3;
for ingress_node in &cluster_nodes {
client

View File

@ -172,7 +172,7 @@ mod tests {
};
use crate::staking_utils::tests::setup_vote_and_stake_accounts;
use solana_runtime::bank::Bank;
use solana_runtime::epoch_schedule::{EpochSchedule, MINIMUM_SLOT_LENGTH};
use solana_runtime::epoch_schedule::{EpochSchedule, MINIMUM_SLOTS_PER_EPOCH};
use std::sync::mpsc::channel;
use std::sync::Arc;
use std::thread::Builder;
@ -221,7 +221,7 @@ mod tests {
}
fn run_thread_race() {
let slots_per_epoch = MINIMUM_SLOT_LENGTH as u64;
let slots_per_epoch = MINIMUM_SLOTS_PER_EPOCH as u64;
let epoch_schedule = EpochSchedule::new(slots_per_epoch, slots_per_epoch / 2, true);
let GenesisBlockInfo { genesis_block, .. } = create_genesis_block(2);
let bank = Arc::new(Bank::new(&genesis_block));

View File

@ -561,7 +561,7 @@ impl Drop for LocalCluster {
mod test {
use super::*;
use crate::storage_stage::STORAGE_ROTATE_TEST_COUNT;
use solana_runtime::epoch_schedule::MINIMUM_SLOT_LENGTH;
use solana_runtime::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH;
#[test]
fn test_local_cluster_start_and_exit() {
@ -586,7 +586,7 @@ mod test {
node_stakes: vec![3; NUM_NODES],
cluster_lamports: 100,
ticks_per_slot: 8,
slots_per_epoch: MINIMUM_SLOT_LENGTH as u64,
slots_per_epoch: MINIMUM_SLOTS_PER_EPOCH as u64,
..ClusterConfig::default()
};
let cluster = LocalCluster::new(&config);

View File

@ -280,7 +280,7 @@ mod test {
use crate::packet::index_blobs;
use crate::service::Service;
use crate::streamer::{blob_receiver, responder};
use solana_runtime::epoch_schedule::MINIMUM_SLOT_LENGTH;
use solana_runtime::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH;
use solana_sdk::hash::Hash;
use solana_sdk::signature::{Keypair, KeypairUtil};
use std::fs::remove_dir_all;
@ -348,7 +348,7 @@ mod test {
);
// with a Bank and no idea who leader is, blob gets thrown out
blob.set_slot(MINIMUM_SLOT_LENGTH as u64 * 3);
blob.set_slot(MINIMUM_SLOTS_PER_EPOCH as u64 * 3);
assert_eq!(
should_retransmit_and_persist(&blob, Some(bank), &cache, &me_id),
false

View File

@ -7,7 +7,7 @@ use solana::cluster_tests;
use solana::gossip_service::discover_cluster;
use solana::local_cluster::{ClusterConfig, LocalCluster};
use solana::validator::ValidatorConfig;
use solana_runtime::epoch_schedule::{EpochSchedule, MINIMUM_SLOT_LENGTH};
use solana_runtime::epoch_schedule::{EpochSchedule, MINIMUM_SLOTS_PER_EPOCH};
use solana_sdk::client::SyncClient;
use solana_sdk::poh_config::PohConfig;
use solana_sdk::timing;
@ -118,7 +118,7 @@ fn test_two_unbalanced_stakes() {
let mut validator_config = ValidatorConfig::default();
let num_ticks_per_second = 100;
let num_ticks_per_slot = 10;
let num_slots_per_epoch = MINIMUM_SLOT_LENGTH as u64;
let num_slots_per_epoch = MINIMUM_SLOTS_PER_EPOCH as u64;
validator_config.rpc_config.enable_fullnode_exit = true;
let mut cluster = LocalCluster::new(&ClusterConfig {
@ -172,7 +172,7 @@ fn test_forwarding() {
#[test]
fn test_restart_node() {
let validator_config = ValidatorConfig::default();
let slots_per_epoch = MINIMUM_SLOT_LENGTH as u64;
let slots_per_epoch = MINIMUM_SLOTS_PER_EPOCH as u64;
let ticks_per_slot = 16;
let mut cluster = LocalCluster::new(&ClusterConfig {
node_stakes: vec![3],
@ -221,10 +221,10 @@ fn run_repairman_catchup(num_repairmen: u64) {
let mut validator_config = ValidatorConfig::default();
let num_ticks_per_second = 100;
let num_ticks_per_slot = 40;
let num_slots_per_epoch = MINIMUM_SLOT_LENGTH as u64;
let num_slots_per_epoch = MINIMUM_SLOTS_PER_EPOCH as u64;
let num_root_buffer_slots = 10;
// Calculate the leader schedule num_root_buffer slots ahead. Otherwise, if stakers_slot_offset ==
// num_slots_per_epoch, and num_slots_per_epoch == MINIMUM_SLOT_LENGTH, then repairmen
// num_slots_per_epoch, and num_slots_per_epoch == MINIMUM_SLOTS_PER_EPOCH, then repairmen
// will stop sending repairs after the last slot in epoch 1 (0-indexed), because the root
// is at most in the first epoch.
//
@ -232,7 +232,7 @@ fn run_repairman_catchup(num_repairmen: u64) {
// Assume:
// 1) num_slots_per_epoch = 32
// 2) stakers_slot_offset = 32
// 3) MINIMUM_SLOT_LENGTH = 32
// 3) MINIMUM_SLOTS_PER_EPOCH = 32
//
// Then the last slot in epoch 1 is slot 63. After completing slots 0 to 63, the root on the
// repairee is at most 31. Because, the stakers_slot_offset == 32, then the max confirmed epoch

View File

@ -17,7 +17,7 @@ use solana::storage_stage::STORAGE_ROTATE_TEST_COUNT;
use solana::streamer;
use solana::tvu::{Sockets, Tvu};
use solana::validator;
use solana_runtime::epoch_schedule::MINIMUM_SLOT_LENGTH;
use solana_runtime::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH;
use solana_sdk::signature::Signable;
use solana_sdk::signature::{Keypair, KeypairUtil};
use solana_sdk::system_transaction;
@ -85,7 +85,7 @@ fn test_replay() {
..
} = create_genesis_block_with_leader(mint_balance, &leader.info.id, leader_balance);
genesis_block.ticks_per_slot = 160;
genesis_block.slots_per_epoch = MINIMUM_SLOT_LENGTH as u64;
genesis_block.slots_per_epoch = MINIMUM_SLOTS_PER_EPOCH as u64;
let (blocktree_path, blockhash) = create_new_tmp_ledger!(&genesis_block);
let tvu_addr = target1.info.tvu;

View File

@ -11,7 +11,7 @@ use solana_sdk::instruction::InstructionError;
use solana_sdk::pubkey::Pubkey;
use solana_vote_api::vote_state::VoteState;
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)]
#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)]
pub enum StakeState {
Uninitialized,
Stake {
@ -19,7 +19,13 @@ pub enum StakeState {
credits_observed: u64,
stake: u64,
},
MiningPool,
MiningPool {
/// epoch for which this Pool will redeem rewards
epoch: u64,
/// the number of lamports each point is worth
point_value: f64,
},
}
impl Default for StakeState {
@ -121,7 +127,10 @@ pub trait StakeAccount {
impl<'a> StakeAccount for KeyedAccount<'a> {
fn initialize_mining_pool(&mut self) -> Result<(), InstructionError> {
if let StakeState::Uninitialized = self.state()? {
self.set_state(&StakeState::MiningPool)
self.set_state(&StakeState::MiningPool {
epoch: 0,
point_value: 0.0,
})
} else {
Err(InstructionError::InvalidAccountData)
}
@ -167,7 +176,7 @@ impl<'a> StakeAccount for KeyedAccount<'a> {
vote_account: &mut KeyedAccount,
) -> Result<(), InstructionError> {
if let (
StakeState::MiningPool,
StakeState::MiningPool { .. },
StakeState::Stake {
voter_pubkey,
credits_observed,
@ -231,6 +240,17 @@ pub fn create_stake_account(
stake_account
}
// utility function, used by Bank, tests, genesis
pub fn create_mining_pool(lamports: u64, epoch: u64, point_value: f64) -> Account {
let mut mining_pool_account = Account::new(lamports, std::mem::size_of::<StakeState>(), &id());
mining_pool_account
.set_state(&StakeState::MiningPool { epoch, point_value })
.expect("set_state");
mining_pool_account
}
#[cfg(test)]
mod tests {
use super::*;
@ -301,7 +321,10 @@ mod tests {
}
);
let stake_state = StakeState::MiningPool;
let stake_state = StakeState::MiningPool {
epoch: 0,
point_value: 0.0,
};
stake_keyed_account.set_state(&stake_state).unwrap();
assert!(stake_keyed_account
.delegate_stake(&vote_keyed_account, 0)
@ -395,7 +418,10 @@ mod tests {
);
mining_pool_keyed_account
.set_state(&StakeState::MiningPool)
.set_state(&StakeState::MiningPool {
epoch: 0,
point_value: 0.0,
})
.unwrap();
// no movement in vote account, so no redemption needed
@ -459,7 +485,10 @@ mod tests {
let mut mining_pool_keyed_account =
KeyedAccount::new(&pubkey, true, &mut mining_pool_account);
mining_pool_keyed_account
.set_state(&StakeState::MiningPool)
.set_state(&StakeState::MiningPool {
epoch: 0,
point_value: 0.0,
})
.unwrap();
let mut vote_state = VoteState::default();

View File

@ -97,7 +97,12 @@ impl VoteState {
// utility function, used by Stakes, tests
pub fn from(account: &Account) -> Option<VoteState> {
account.state().ok()
Self::deserialize(&account.data).ok()
}
// utility function, used by Stakes, tests
pub fn to(&self, account: &mut Account) -> Option<()> {
Self::serialize(self, &mut account.data).ok()
}
pub fn deserialize(input: &[u8]) -> Result<Self, InstructionError> {
@ -111,6 +116,11 @@ impl VoteState {
})
}
// utility function, used by Stakes, tests
pub fn credits_from(account: &Account) -> Option<u64> {
Self::from(account).map(|state| state.credits())
}
/// returns commission split as (voter_portion, staker_portion, was_split) tuple
///
/// if commission calculation is 100% one way or other,

View File

@ -27,7 +27,8 @@ use solana_metrics::{
use solana_sdk::account::Account;
use solana_sdk::fee_calculator::FeeCalculator;
use solana_sdk::genesis_block::GenesisBlock;
use solana_sdk::hash::{extend_and_hash, Hash};
use solana_sdk::hash::{extend_and_hash, hashv, Hash};
use solana_sdk::inflation::Inflation;
use solana_sdk::native_loader;
use solana_sdk::pubkey::Pubkey;
use solana_sdk::signature::{Keypair, Signature};
@ -36,7 +37,7 @@ use solana_sdk::syscall::fees::{self, Fees};
use solana_sdk::syscall::slot_hashes::{self, SlotHashes};
use solana_sdk::syscall::tick_height::{self, TickHeight};
use solana_sdk::system_transaction;
use solana_sdk::timing::{duration_as_ms, duration_as_us, MAX_RECENT_BLOCKHASHES};
use solana_sdk::timing::{duration_as_ms, duration_as_ns, duration_as_us, MAX_RECENT_BLOCKHASHES};
use solana_sdk::transaction::{Result, Transaction, TransactionError};
use std::cmp;
use std::collections::HashMap;
@ -46,6 +47,8 @@ use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering};
use std::sync::{Arc, RwLock, RwLockReadGuard};
use std::time::Instant;
pub const SECONDS_PER_YEAR: f64 = (365.0 * 24.0 * 60.0 * 60.0);
type BankStatusCache = StatusCache<Result<()>>;
#[derive(Default)]
@ -223,6 +226,9 @@ pub struct Bank {
/// The number of ticks in each slot.
ticks_per_slot: u64,
/// The number of slots per year, used for inflation
slots_per_year: f64,
/// Bank fork (i.e. slot, i.e. block)
slot: u64,
@ -243,6 +249,9 @@ pub struct Bank {
/// initialized from genesis
epoch_schedule: EpochSchedule,
/// inflation specs
inflation: Inflation,
/// cache of vote_account and stake_account state for this fork
stakes: RwLock<Stakes>,
@ -293,61 +302,79 @@ impl Bank {
/// Create a new bank that points to an immutable checkpoint of another bank.
pub fn new_from_parent(parent: &Arc<Bank>, collector_id: &Pubkey, slot: u64) -> Self {
Self::default().init_from_parent(parent, collector_id, slot)
}
/// Create a new bank that points to an immutable checkpoint of another bank.
pub fn init_from_parent(
mut self,
parent: &Arc<Bank>,
collector_id: &Pubkey,
slot: u64,
) -> Self {
parent.freeze();
assert_ne!(slot, parent.slot());
let mut bank = Self::default();
bank.blockhash_queue = RwLock::new(parent.blockhash_queue.read().unwrap().clone());
bank.src.status_cache = parent.src.status_cache.clone();
bank.bank_height = parent.bank_height + 1;
bank.fee_calculator =
// TODO: clean this up, soo much special-case copying...
self.ticks_per_slot = parent.ticks_per_slot;
self.slots_per_year = parent.slots_per_year;
self.epoch_schedule = parent.epoch_schedule;
self.slot = slot;
self.max_tick_height = (self.slot + 1) * self.ticks_per_slot - 1;
self.blockhash_queue = RwLock::new(parent.blockhash_queue.read().unwrap().clone());
self.src.status_cache = parent.src.status_cache.clone();
self.bank_height = parent.bank_height + 1;
self.fee_calculator =
FeeCalculator::new_derived(&parent.fee_calculator, parent.signature_count());
bank.transaction_count
.store(parent.transaction_count() as usize, Ordering::Relaxed);
bank.stakes = RwLock::new(parent.stakes.read().unwrap().clone());
bank.storage_accounts = RwLock::new(parent.storage_accounts.read().unwrap().clone());
self.capitalization
.store(parent.capitalization() as usize, Ordering::Relaxed);
self.inflation = parent.inflation.clone();
bank.tick_height.store(
self.transaction_count
.store(parent.transaction_count() as usize, Ordering::Relaxed);
self.stakes = RwLock::new(parent.stakes.read().unwrap().clone());
self.storage_accounts = RwLock::new(parent.storage_accounts.read().unwrap().clone());
self.tick_height.store(
parent.tick_height.load(Ordering::Relaxed),
Ordering::Relaxed,
);
bank.ticks_per_slot = parent.ticks_per_slot;
bank.epoch_schedule = parent.epoch_schedule;
bank.slot = slot;
bank.max_tick_height = (bank.slot + 1) * bank.ticks_per_slot - 1;
datapoint_info!(
"bank-new_from_parent-heights",
("slot_height", slot, i64),
("bank_height", bank.bank_height, i64)
("bank_height", self.bank_height, i64)
);
bank.rc.parent = RwLock::new(Some(parent.clone()));
bank.parent_hash = parent.hash();
bank.collector_id = *collector_id;
self.rc.parent = RwLock::new(Some(parent.clone()));
self.parent_hash = parent.hash();
self.collector_id = *collector_id;
bank.rc.accounts = Arc::new(Accounts::new_from_parent(&parent.rc.accounts));
self.rc.accounts = Arc::new(Accounts::new_from_parent(&parent.rc.accounts));
bank.epoch_stakes = {
self.epoch_stakes = {
let mut epoch_stakes = parent.epoch_stakes.clone();
let epoch = bank.get_stakers_epoch(bank.slot);
let epoch = self.get_stakers_epoch(self.slot);
// update epoch_vote_states cache
// if my parent didn't populate for this epoch, we've
// crossed a boundary
if epoch_stakes.get(&epoch).is_none() {
epoch_stakes.insert(epoch, bank.stakes.read().unwrap().clone());
epoch_stakes.insert(epoch, self.stakes.read().unwrap().clone());
}
epoch_stakes
};
bank.ancestors.insert(bank.slot(), 0);
bank.parents().iter().enumerate().for_each(|(i, p)| {
bank.ancestors.insert(p.slot(), i + 1);
self.ancestors.insert(self.slot(), 0);
self.parents().iter().enumerate().for_each(|(i, p)| {
self.ancestors.insert(p.slot(), i + 1);
});
bank.update_current();
bank.update_fees();
bank
self.update_rewards(parent.epoch(), parent.last_blockhash());
self.update_current();
self.update_fees();
self
}
pub fn collector_id(&self) -> &Pubkey {
@ -358,6 +385,10 @@ impl Bank {
self.slot
}
pub fn epoch(&self) -> u64 {
self.epoch_schedule.get_epoch(self.slot)
}
pub fn freeze_lock(&self) -> RwLockReadGuard<Hash> {
self.hash.read().unwrap()
}
@ -416,15 +447,58 @@ impl Bank {
self.store_account(&tick_height::id(), &account);
}
// update reward for previous epoch
fn update_rewards(&mut self, epoch: u64, blockhash: Hash) {
if epoch == self.epoch() {
return;
}
// if I'm the first Bank in an epoch, count, claim, disburse rewards from Inflation
// TODO: on-chain wallclock?
// years_elapsed = slots_elapsed / slots/year
let year = (self.epoch_schedule.get_last_slot_in_epoch(epoch)) as f64 / self.slots_per_year;
// period: time that has passed as a fraction of a year, basically the length of
// an epoch as a fraction of a year
// years_elapsed = slots_elapsed / slots/year
let period = self.epoch_schedule.get_slots_in_epoch(epoch) as f64 / self.slots_per_year;
// validators
{
let validator_rewards =
(self.inflation.validator(year) * self.capitalization() as f64 * period) as u64;
// claim points and create a pool
let mining_pool = self
.stakes
.write()
.unwrap()
.create_mining_pool(epoch, validator_rewards);
self.store(
&Pubkey::new(
hashv(&[
blockhash.as_ref(),
"StakeMiningPool".as_ref(),
&serialize(&epoch).unwrap(),
])
.as_ref(),
),
&mining_pool,
);
self.capitalization
.fetch_add(validator_rewards as usize, Ordering::Relaxed);
}
}
fn set_hash(&self) -> bool {
let mut hash = self.hash.write().unwrap();
if *hash == Hash::default() {
let collector_fees = self.collector_fees.load(Ordering::Relaxed) as u64;
if collector_fees != 0 {
self.deposit(&self.collector_id, collector_fees);
}
// freeze is a one-way trip, idempotent
*hash = self.hash_internal_state();
true
@ -501,6 +575,12 @@ impl Bank {
self.ticks_per_slot = genesis_block.ticks_per_slot;
self.max_tick_height = (self.slot + 1) * self.ticks_per_slot - 1;
// ticks/year = seconds/year ...
self.slots_per_year = 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
/ self.ticks_per_slot as f64;
// make bank 0 votable
self.is_delta.store(true, Ordering::Relaxed);
@ -511,6 +591,8 @@ impl Bank {
genesis_block.epoch_warmup,
);
self.inflation = genesis_block.inflation.clone();
// Add native programs mandatory for the MessageProcessor to function
self.register_native_instruction_processor(
"solana_system_program",
@ -1311,7 +1393,7 @@ impl Drop for Bank {
#[cfg(test)]
mod tests {
use super::*;
use crate::epoch_schedule::MINIMUM_SLOT_LENGTH;
use crate::epoch_schedule::MINIMUM_SLOTS_PER_EPOCH;
use crate::genesis_utils::{
create_genesis_block_with_leader, GenesisBlockInfo, BOOTSTRAP_LEADER_LAMPORTS,
};
@ -1319,12 +1401,15 @@ mod tests {
use solana_sdk::genesis_block::create_genesis_block;
use solana_sdk::hash;
use solana_sdk::instruction::InstructionError;
use solana_sdk::poh_config::PohConfig;
use solana_sdk::signature::{Keypair, KeypairUtil};
use solana_sdk::system_instruction;
use solana_sdk::system_transaction;
use solana_sdk::timing::DEFAULT_TICKS_PER_SLOT;
use solana_vote_api::vote_instruction;
use solana_vote_api::vote_state::VoteState;
use solana_vote_api::vote_state::{VoteState, MAX_LOCKOUT_HISTORY};
use std::io::Cursor;
use std::time::Duration;
#[test]
fn test_bank_new() {
@ -1351,11 +1436,75 @@ mod tests {
#[test]
fn test_bank_capitalization() {
let bank = Bank::new(&GenesisBlock {
let bank = Arc::new(Bank::new(&GenesisBlock {
accounts: vec![(Pubkey::default(), Account::new(42, 0, &Pubkey::default()),); 42],
..GenesisBlock::default()
});
}));
assert_eq!(bank.capitalization(), 42 * 42);
let bank1 = Bank::new_from_parent(&bank, &Pubkey::default(), 1);
assert_eq!(bank1.capitalization(), 42 * 42);
}
#[test]
fn test_bank_update_rewards() {
// create a bank that ticks really slowly...
let bank = Arc::new(Bank::new(&GenesisBlock {
accounts: vec![
(
Pubkey::default(),
Account::new(1_000_000_000, 0, &Pubkey::default()),
);
42
],
// set it up so the first epoch is a full year long
poh_config: PohConfig {
target_tick_duration: Duration::from_secs(
SECONDS_PER_YEAR as u64
/ MINIMUM_SLOTS_PER_EPOCH as u64
/ DEFAULT_TICKS_PER_SLOT,
),
hashes_per_tick: None,
},
..GenesisBlock::default()
}));
assert_eq!(bank.capitalization(), 42 * 1_000_000_000);
let ((vote_id, mut vote_account), stake) =
crate::stakes::tests::create_staked_node_accounts(1_0000);
// set up stakes and vote accounts
bank.store(&stake.0, &stake.1);
// generate some rewards
let mut vote_state = VoteState::from(&vote_account).unwrap();
for i in 0..MAX_LOCKOUT_HISTORY + 42 {
vote_state.process_slot_vote_unchecked(i as u64);
vote_state.to(&mut vote_account).unwrap();
bank.store(&vote_id, &vote_account);
}
bank.store(&vote_id, &vote_account);
// put a child bank in epoch 1, which calls update_rewards()...
let bank1 = Bank::new_from_parent(
&bank,
&Pubkey::default(),
bank.get_slots_in_epoch(bank.epoch()) + 1,
);
// verify that there's inflation
assert_ne!(bank1.capitalization(), bank.capitalization());
// verify the inflation is in rewards pools
let inflation = bank1.capitalization() - bank.capitalization();
let validator_rewards: u64 = bank1
.stakes
.read()
.unwrap()
.mining_pools()
.map(|(_key, account)| account.lamports)
.sum();
assert_eq!(validator_rewards, inflation);
}
#[test]
@ -2098,7 +2247,7 @@ mod tests {
// set this up weird, forces future generation, odd mod(), etc.
// this says: "vote_accounts for epoch X should be generated at slot index 3 in epoch X-2...
const SLOTS_PER_EPOCH: u64 = MINIMUM_SLOT_LENGTH as u64;
const SLOTS_PER_EPOCH: u64 = MINIMUM_SLOTS_PER_EPOCH as u64;
const STAKERS_SLOT_OFFSET: u64 = SLOTS_PER_EPOCH * 3 - 3;
genesis_block.slots_per_epoch = SLOTS_PER_EPOCH;
genesis_block.stakers_slot_offset = STAKERS_SLOT_OFFSET;
@ -2180,8 +2329,11 @@ mod tests {
let bank = Bank::new(&genesis_block);
assert_eq!(bank.get_slots_in_epoch(0), MINIMUM_SLOT_LENGTH as u64);
assert_eq!(bank.get_slots_in_epoch(2), (MINIMUM_SLOT_LENGTH * 4) as u64);
assert_eq!(bank.get_slots_in_epoch(0), MINIMUM_SLOTS_PER_EPOCH as u64);
assert_eq!(
bank.get_slots_in_epoch(2),
(MINIMUM_SLOTS_PER_EPOCH * 4) as u64
);
assert_eq!(bank.get_slots_in_epoch(5000), genesis_block.slots_per_epoch);
}

View File

@ -1,6 +1,6 @@
use solana_vote_api::vote_state::MAX_LOCKOUT_HISTORY;
pub const MINIMUM_SLOT_LENGTH: usize = MAX_LOCKOUT_HISTORY + 1;
pub const MINIMUM_SLOTS_PER_EPOCH: u64 = (MAX_LOCKOUT_HISTORY + 1) as u64;
#[derive(Default, Debug, Clone, Copy, PartialEq, Deserialize, Serialize)]
pub struct EpochSchedule {
@ -19,16 +19,16 @@ pub struct EpochSchedule {
impl EpochSchedule {
pub fn new(slots_per_epoch: u64, stakers_slot_offset: u64, warmup: bool) -> Self {
assert!(slots_per_epoch >= MINIMUM_SLOT_LENGTH as u64);
assert!(slots_per_epoch >= MINIMUM_SLOTS_PER_EPOCH as u64);
let (first_normal_epoch, first_normal_slot) = if warmup {
let next_power_of_two = slots_per_epoch.next_power_of_two();
let log2_slots_per_epoch = next_power_of_two
.trailing_zeros()
.saturating_sub(MINIMUM_SLOT_LENGTH.trailing_zeros());
.saturating_sub(MINIMUM_SLOTS_PER_EPOCH.trailing_zeros());
(
u64::from(log2_slots_per_epoch),
next_power_of_two.saturating_sub(MINIMUM_SLOT_LENGTH as u64),
next_power_of_two.saturating_sub(MINIMUM_SLOTS_PER_EPOCH),
)
} else {
(0, 0)
@ -44,7 +44,7 @@ impl EpochSchedule {
/// get the length of the given epoch (in slots)
pub fn get_slots_in_epoch(&self, epoch: u64) -> u64 {
if epoch < self.first_normal_epoch {
2u64.pow(epoch as u32 + MINIMUM_SLOT_LENGTH.trailing_zeros() as u32)
2u64.pow(epoch as u32 + MINIMUM_SLOTS_PER_EPOCH.trailing_zeros() as u32)
} else {
self.slots_per_epoch
}
@ -70,17 +70,17 @@ impl EpochSchedule {
/// get epoch and offset into the epoch for the given slot
pub fn get_epoch_and_slot_index(&self, slot: u64) -> (u64, u64) {
if slot < self.first_normal_slot {
let epoch = (slot + MINIMUM_SLOT_LENGTH as u64 + 1)
let epoch = (slot + MINIMUM_SLOTS_PER_EPOCH + 1)
.next_power_of_two()
.trailing_zeros()
- MINIMUM_SLOT_LENGTH.trailing_zeros()
- MINIMUM_SLOTS_PER_EPOCH.trailing_zeros()
- 1;
let epoch_len = 2u64.pow(epoch + MINIMUM_SLOT_LENGTH.trailing_zeros());
let epoch_len = 2u64.pow(epoch + MINIMUM_SLOTS_PER_EPOCH.trailing_zeros());
(
u64::from(epoch),
slot - (epoch_len - MINIMUM_SLOT_LENGTH as u64),
slot - (epoch_len - MINIMUM_SLOTS_PER_EPOCH),
)
} else {
(
@ -92,7 +92,7 @@ impl EpochSchedule {
pub fn get_first_slot_in_epoch(&self, epoch: u64) -> u64 {
if epoch <= self.first_normal_epoch {
(2u64.pow(epoch as u32) - 1) * MINIMUM_SLOT_LENGTH as u64
(2u64.pow(epoch as u32) - 1) * MINIMUM_SLOTS_PER_EPOCH
} else {
(epoch - self.first_normal_epoch) * self.slots_per_epoch + self.first_normal_slot
}
@ -113,18 +113,18 @@ mod tests {
// (1 * 7 * 24 * 4500u64).next_power_of_two();
// test values between MINIMUM_SLOT_LEN and MINIMUM_SLOT_LEN * 16, should cover a good mix
for slots_per_epoch in MINIMUM_SLOT_LENGTH as u64..=MINIMUM_SLOT_LENGTH as u64 * 16 {
for slots_per_epoch in MINIMUM_SLOTS_PER_EPOCH..=MINIMUM_SLOTS_PER_EPOCH * 16 {
let epoch_schedule = EpochSchedule::new(slots_per_epoch, slots_per_epoch / 2, true);
assert_eq!(epoch_schedule.get_first_slot_in_epoch(0), 0);
assert_eq!(
epoch_schedule.get_last_slot_in_epoch(0),
MINIMUM_SLOT_LENGTH as u64 - 1
MINIMUM_SLOTS_PER_EPOCH - 1
);
let mut last_stakers = 0;
let mut last_epoch = 0;
let mut last_slots_in_epoch = MINIMUM_SLOT_LENGTH as u64;
let mut last_slots_in_epoch = MINIMUM_SLOTS_PER_EPOCH;
for slot in 0..(2 * slots_per_epoch) {
// verify that stakers_epoch is continuous over the warmup
// and into the first normal epoch

View File

@ -2,7 +2,7 @@
//! node stakes
use solana_sdk::account::Account;
use solana_sdk::pubkey::Pubkey;
use solana_stake_api::stake_state::StakeState;
use solana_stake_api::stake_state::{create_mining_pool, StakeState};
use solana_vote_api::vote_state::VoteState;
use std::collections::HashMap;
@ -13,6 +13,10 @@ pub struct Stakes {
/// stake_accounts
stake_accounts: HashMap<Pubkey, Account>,
/// unclaimed points.
// a point is a credit multiplied by the stake
points: u64,
}
impl Stakes {
@ -40,11 +44,18 @@ impl Stakes {
if account.lamports == 0 {
self.vote_accounts.remove(pubkey);
} else {
// update the stake of this entry
let stake = self
.vote_accounts
.get(pubkey)
.map_or_else(|| self.calculate_stake(pubkey), |v| v.0);
let old = self.vote_accounts.get(pubkey);
let stake = old.map_or_else(|| self.calculate_stake(pubkey), |v| v.0);
// count any increase in points, can only go forward
let old_credits = old
.and_then(|(_stake, old_account)| VoteState::credits_from(old_account))
.unwrap_or(0);
let credits = VoteState::credits_from(account).unwrap_or(old_credits);
self.points += credits.saturating_sub(old_credits) * stake;
self.vote_accounts.insert(*pubkey, (stake, account.clone()));
}
@ -86,6 +97,15 @@ impl Stakes {
&self.vote_accounts
}
pub fn mining_pools(&self) -> impl Iterator<Item = (&Pubkey, &Account)> {
self.stake_accounts
.iter()
.filter(|(_key, account)| match StakeState::from(account) {
Some(StakeState::MiningPool { .. }) => true,
_ => false,
})
}
pub fn highest_staked_node(&self) -> Option<Pubkey> {
self.vote_accounts
.iter()
@ -93,17 +113,37 @@ impl Stakes {
.and_then(|(_k, (_stake, account))| VoteState::from(account))
.map(|vote_state| vote_state.node_pubkey)
}
/// currently unclaimed points
pub fn points(&mut self) -> u64 {
self.points
}
/// "claims" points, resets points to 0
pub fn claim_points(&mut self) -> u64 {
let points = self.points;
self.points = 0;
points
}
/// claims points
/// makes a pool with the lamports and points spread over those points and
pub fn create_mining_pool(&mut self, epoch: u64, lamports: u64) -> Account {
let points = self.claim_points();
create_mining_pool(lamports, epoch, lamports as f64 / points as f64)
}
}
#[cfg(test)]
mod tests {
pub mod tests {
use super::*;
use solana_sdk::pubkey::Pubkey;
use solana_stake_api::stake_state;
use solana_vote_api::vote_state::{self, VoteState};
use solana_vote_api::vote_state::{self, VoteState, MAX_LOCKOUT_HISTORY};
// set up some dummies for a staked node (( vote ) ( stake ))
fn create_staked_node_accounts(stake: u64) -> ((Pubkey, Account), (Pubkey, Account)) {
// set up some dummies for a staked node (( vote ) ( stake ))
pub fn create_staked_node_accounts(stake: u64) -> ((Pubkey, Account), (Pubkey, Account)) {
let vote_pubkey = Pubkey::new_rand();
let vote_account = vote_state::create_account(&vote_pubkey, &Pubkey::new_rand(), 0, 1);
(
@ -113,7 +153,7 @@ mod tests {
}
// add stake to a vote_pubkey ( stake )
fn create_stake_account(stake: u64, vote_pubkey: &Pubkey) -> (Pubkey, Account) {
pub fn create_stake_account(stake: u64, vote_pubkey: &Pubkey) -> (Pubkey, Account) {
(
Pubkey::new_rand(),
stake_state::create_stake_account(&vote_pubkey, &VoteState::default(), stake),
@ -185,6 +225,50 @@ mod tests {
assert_eq!(stakes.highest_staked_node(), Some(vote11_node_pubkey))
}
#[test]
fn test_stakes_points() {
let mut stakes = Stakes::default();
let stake = 42;
assert_eq!(stakes.points(), 0);
assert_eq!(stakes.claim_points(), 0);
assert_eq!(stakes.claim_points(), 0);
let ((vote_pubkey, mut vote_account), (stake_pubkey, stake_account)) =
create_staked_node_accounts(stake);
stakes.store(&vote_pubkey, &vote_account);
stakes.store(&stake_pubkey, &stake_account);
assert_eq!(stakes.points(), 0);
assert_eq!(stakes.claim_points(), 0);
let mut vote_state = VoteState::from(&vote_account).unwrap();
for i in 0..MAX_LOCKOUT_HISTORY + 42 {
vote_state.process_slot_vote_unchecked(i as u64);
vote_state.to(&mut vote_account).unwrap();
stakes.store(&vote_pubkey, &vote_account);
assert_eq!(stakes.points(), vote_state.credits() * stake);
}
vote_account.lamports = 0;
stakes.store(&vote_pubkey, &vote_account);
assert_eq!(stakes.points(), vote_state.credits() * stake);
assert_eq!(stakes.claim_points(), vote_state.credits() * stake);
assert_eq!(stakes.claim_points(), 0);
assert_eq!(stakes.claim_points(), 0);
// points come out of nowhere, but don't care here ;)
vote_account.lamports = 1;
stakes.store(&vote_pubkey, &vote_account);
assert_eq!(stakes.points(), vote_state.credits() * stake);
// test going backwards, should never go backwards
let old_vote_state = vote_state;
let vote_account = vote_state::create_account(&vote_pubkey, &Pubkey::new_rand(), 0, 1);
stakes.store(&vote_pubkey, &vote_account);
assert_eq!(stakes.points(), old_vote_state.credits() * stake);
}
#[test]
fn test_stakes_vote_account_disappear_reappear() {
let mut stakes = Stakes::default();

View File

@ -670,8 +670,10 @@ fn process_show_stake_account(
println!("activated stake: {}", stake);
Ok("".to_string())
}
Ok(StakeState::MiningPool) => {
Ok(StakeState::MiningPool { epoch, point_value }) => {
println!("account lamports: {}", stake_account.lamports);
println!("epoch: {}", epoch);
println!("point_value: {}", point_value);
Ok("".to_string())
}
_ => Err(WalletError::RpcRequestError(