Port instructions sysvar and secp256k1 program activation to FeatureSet

This commit is contained in:
Michael Vines
2020-09-21 22:36:23 -07:00
parent 35f5f9fc7b
commit c10da16d7b
18 changed files with 212 additions and 241 deletions

View File

@ -67,7 +67,7 @@ pub fn check_account_for_multiple_fees_with_commitment(
pub fn calculate_fee(fee_calculator: &FeeCalculator, messages: &[&Message]) -> u64 { pub fn calculate_fee(fee_calculator: &FeeCalculator, messages: &[&Message]) -> u64 {
messages messages
.iter() .iter()
.map(|message| fee_calculator.calculate_fee(message, None)) .map(|message| fee_calculator.calculate_fee(message))
.sum() .sum()
} }

View File

@ -30,10 +30,9 @@ use solana_runtime::{
}; };
use solana_sdk::{ use solana_sdk::{
clock::{ clock::{
Epoch, Slot, DEFAULT_TICKS_PER_SLOT, MAX_PROCESSING_AGE, MAX_TRANSACTION_FORWARDING_DELAY, Slot, DEFAULT_TICKS_PER_SLOT, MAX_PROCESSING_AGE, MAX_TRANSACTION_FORWARDING_DELAY,
MAX_TRANSACTION_FORWARDING_DELAY_GPU, MAX_TRANSACTION_FORWARDING_DELAY_GPU,
}, },
genesis_config::ClusterType,
poh_config::PohConfig, poh_config::PohConfig,
pubkey::Pubkey, pubkey::Pubkey,
timing::{duration_as_ms, timestamp}, timing::{duration_as_ms, timestamp},
@ -737,8 +736,7 @@ impl BankingStage {
fn transactions_from_packets( fn transactions_from_packets(
msgs: &Packets, msgs: &Packets,
transaction_indexes: &[usize], transaction_indexes: &[usize],
cluster_type: ClusterType, secp256k1_program_enabled: bool,
epoch: Epoch,
) -> (Vec<Transaction>, Vec<usize>) { ) -> (Vec<Transaction>, Vec<usize>) {
let packets = Packets::new( let packets = Packets::new(
transaction_indexes transaction_indexes
@ -748,8 +746,7 @@ impl BankingStage {
); );
let transactions = Self::deserialize_transactions(&packets); let transactions = Self::deserialize_transactions(&packets);
let maybe_secp_verified_transactions: Vec<_> = let maybe_secp_verified_transactions: Vec<_> = if secp256k1_program_enabled {
if solana_sdk::secp256k1::is_enabled(cluster_type, epoch) {
transactions transactions
.into_iter() .into_iter()
.map(|tx| { .map(|tx| {
@ -820,8 +817,7 @@ impl BankingStage {
let (transactions, transaction_to_packet_indexes) = Self::transactions_from_packets( let (transactions, transaction_to_packet_indexes) = Self::transactions_from_packets(
msgs, msgs,
&packet_indexes, &packet_indexes,
bank.cluster_type(), bank.secp256k1_program_enabled(),
bank.epoch(),
); );
debug!( debug!(
"bank: {} filtered transactions {}", "bank: {} filtered transactions {}",
@ -874,8 +870,7 @@ impl BankingStage {
let (transactions, transaction_to_packet_indexes) = Self::transactions_from_packets( let (transactions, transaction_to_packet_indexes) = Self::transactions_from_packets(
msgs, msgs,
&transaction_indexes, &transaction_indexes,
bank.cluster_type(), bank.secp256k1_program_enabled(),
bank.epoch(),
); );
let tx_count = transaction_to_packet_indexes.len(); let tx_count = transaction_to_packet_indexes.len();

View File

@ -480,11 +480,11 @@ mod tests {
assert_eq!(None, process_rest(&bank_forks, "not-a-supported-rest-api")); assert_eq!(None, process_rest(&bank_forks, "not-a-supported-rest-api"));
assert_eq!( assert_eq!(
Some("0.000010127".to_string()), Some("0.000010129".to_string()),
process_rest(&bank_forks, "/v0/circulating-supply") process_rest(&bank_forks, "/v0/circulating-supply")
); );
assert_eq!( assert_eq!(
Some("0.000010127".to_string()), Some("0.000010129".to_string()),
process_rest(&bank_forks, "/v0/total-supply") process_rest(&bank_forks, "/v0/total-supply")
); );
} }

View File

@ -80,10 +80,7 @@ impl TransactionStatusService {
_ => bank.get_fee_calculator(&transaction.message().recent_blockhash), _ => bank.get_fee_calculator(&transaction.message().recent_blockhash),
} }
.expect("FeeCalculator must exist"); .expect("FeeCalculator must exist");
let fee = fee_calculator.calculate_fee( let fee = fee_calculator.calculate_fee(transaction.message());
transaction.message(),
solana_sdk::secp256k1::get_fee_config(bank.cluster_type(), bank.epoch()),
);
let (writable_keys, readonly_keys) = let (writable_keys, readonly_keys) =
transaction.message.get_account_keys_by_lock_type(); transaction.message.get_account_keys_by_lock_type();

View File

@ -681,8 +681,7 @@ pub fn confirm_slot(
let entry_state = entries.start_verify( let entry_state = entries.start_verify(
&progress.last_entry, &progress.last_entry,
recyclers.clone(), recyclers.clone(),
bank.cluster_type(), bank.secp256k1_program_enabled(),
bank.epoch(),
); );
if entry_state.status() == EntryVerificationStatus::Failure { if entry_state.status() == EntryVerificationStatus::Failure {
warn!("Ledger proof of history failed at slot: {}", slot); warn!("Ledger proof of history failed at slot: {}", slot);

View File

@ -17,8 +17,6 @@ use solana_perf::cuda_runtime::PinnedVec;
use solana_perf::perf_libs; use solana_perf::perf_libs;
use solana_perf::recycler::Recycler; use solana_perf::recycler::Recycler;
use solana_rayon_threadlimit::get_thread_count; use solana_rayon_threadlimit::get_thread_count;
use solana_sdk::clock::Epoch;
use solana_sdk::genesis_config::ClusterType;
use solana_sdk::hash::Hash; use solana_sdk::hash::Hash;
use solana_sdk::timing; use solana_sdk::timing;
use solana_sdk::transaction::Transaction; use solana_sdk::transaction::Transaction;
@ -329,8 +327,7 @@ pub trait EntrySlice {
&self, &self,
start_hash: &Hash, start_hash: &Hash,
recyclers: VerifyRecyclers, recyclers: VerifyRecyclers,
cluster_type: ClusterType, secp256k1_program_enabled: bool,
epoch: Epoch,
) -> EntryVerificationState; ) -> EntryVerificationState;
fn verify(&self, start_hash: &Hash) -> bool; fn verify(&self, start_hash: &Hash) -> bool;
/// Checks that each entry tick has the correct number of hashes. Entry slices do not /// Checks that each entry tick has the correct number of hashes. Entry slices do not
@ -339,17 +336,12 @@ pub trait EntrySlice {
fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool; fn verify_tick_hash_count(&self, tick_hash_count: &mut u64, hashes_per_tick: u64) -> bool;
/// Counts tick entries /// Counts tick entries
fn tick_count(&self) -> u64; fn tick_count(&self) -> u64;
fn verify_transaction_signatures(&self, cluster_type: ClusterType, epoch: Epoch) -> bool; fn verify_transaction_signatures(&self, secp256k1_program_enabled: bool) -> bool;
} }
impl EntrySlice for [Entry] { impl EntrySlice for [Entry] {
fn verify(&self, start_hash: &Hash) -> bool { fn verify(&self, start_hash: &Hash) -> bool {
self.start_verify( self.start_verify(start_hash, VerifyRecyclers::default(), true)
start_hash,
VerifyRecyclers::default(),
ClusterType::Development,
0,
)
.finish_verify(self) .finish_verify(self)
} }
@ -497,14 +489,14 @@ impl EntrySlice for [Entry] {
} }
} }
fn verify_transaction_signatures(&self, cluster_type: ClusterType, epoch: Epoch) -> bool { fn verify_transaction_signatures(&self, secp256k1_program_enabled: bool) -> bool {
PAR_THREAD_POOL.with(|thread_pool| { PAR_THREAD_POOL.with(|thread_pool| {
thread_pool.borrow().install(|| { thread_pool.borrow().install(|| {
self.par_iter().all(|e| { self.par_iter().all(|e| {
e.transactions.par_iter().all(|transaction| { e.transactions.par_iter().all(|transaction| {
let sig_verify = transaction.verify().is_ok(); let sig_verify = transaction.verify().is_ok();
if sig_verify if sig_verify
&& solana_sdk::secp256k1::is_enabled(cluster_type, epoch) && secp256k1_program_enabled
&& transaction.verify_precompiles().is_err() && transaction.verify_precompiles().is_err()
{ {
return false; return false;
@ -520,11 +512,10 @@ impl EntrySlice for [Entry] {
&self, &self,
start_hash: &Hash, start_hash: &Hash,
recyclers: VerifyRecyclers, recyclers: VerifyRecyclers,
cluster_type: ClusterType, secp256k1_program_enabled: bool,
epoch: Epoch,
) -> EntryVerificationState { ) -> EntryVerificationState {
let start = Instant::now(); let start = Instant::now();
let res = self.verify_transaction_signatures(cluster_type, epoch); let res = self.verify_transaction_signatures(secp256k1_program_enabled);
let transaction_duration_us = timing::duration_as_us(&start.elapsed()); let transaction_duration_us = timing::duration_as_us(&start.elapsed());
if !res { if !res {
return EntryVerificationState { return EntryVerificationState {

View File

@ -2,7 +2,7 @@ use clap::{crate_description, crate_name, value_t, App, Arg};
use solana_ledger::entry::{self, create_ticks, init_poh, EntrySlice, VerifyRecyclers}; use solana_ledger::entry::{self, create_ticks, init_poh, EntrySlice, VerifyRecyclers};
use solana_measure::measure::Measure; use solana_measure::measure::Measure;
use solana_perf::perf_libs; use solana_perf::perf_libs;
use solana_sdk::{genesis_config::ClusterType, hash::hash}; use solana_sdk::hash::hash;
fn main() { fn main() {
solana_logger::setup(); solana_logger::setup();
@ -118,7 +118,7 @@ fn main() {
let recyclers = VerifyRecyclers::default(); let recyclers = VerifyRecyclers::default();
for _ in 0..iterations { for _ in 0..iterations {
assert!(ticks[..num_entries] assert!(ticks[..num_entries]
.start_verify(&start_hash, recyclers.clone(), ClusterType::Development, 0) .start_verify(&start_hash, recyclers.clone(), true)
.finish_verify(&ticks[..num_entries])); .finish_verify(&ticks[..num_entries]));
} }
time.stop(); time.stop();

View File

@ -6,6 +6,7 @@ use crate::{
append_vec::StoredAccount, append_vec::StoredAccount,
bank::{HashAgeKind, TransactionProcessResult}, bank::{HashAgeKind, TransactionProcessResult},
blockhash_queue::BlockhashQueue, blockhash_queue::BlockhashQueue,
feature_set::{self, FeatureSet},
nonce_utils, nonce_utils,
rent_collector::RentCollector, rent_collector::RentCollector,
system_instruction_processor::{get_system_account_kind, SystemAccountKind}, system_instruction_processor::{get_system_account_kind, SystemAccountKind},
@ -17,7 +18,7 @@ use rayon::slice::ParallelSliceMut;
use solana_sdk::{ use solana_sdk::{
account::Account, account::Account,
clock::{Epoch, Slot}, clock::{Epoch, Slot},
fee_calculator::FeeCalculator, fee_calculator::{FeeCalculator, FeeConfig},
genesis_config::ClusterType, genesis_config::ClusterType,
hash::Hash, hash::Hash,
message::Message, message::Message,
@ -72,11 +73,10 @@ pub enum AccountAddressFilter {
impl Accounts { impl Accounts {
pub fn new(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self { pub fn new(paths: Vec<PathBuf>, cluster_type: &ClusterType) -> Self {
Self { Self {
slot: 0,
epoch: 0,
accounts_db: Arc::new(AccountsDB::new(paths, cluster_type)), accounts_db: Arc::new(AccountsDB::new(paths, cluster_type)),
account_locks: Mutex::new(HashSet::new()), account_locks: Mutex::new(HashSet::new()),
readonly_locks: Arc::new(RwLock::new(Some(HashMap::new()))), readonly_locks: Arc::new(RwLock::new(Some(HashMap::new()))),
..Self::default()
} }
} }
@ -94,11 +94,10 @@ impl Accounts {
pub(crate) fn new_empty(accounts_db: AccountsDB) -> Self { pub(crate) fn new_empty(accounts_db: AccountsDB) -> Self {
Self { Self {
slot: 0,
epoch: 0,
accounts_db: Arc::new(accounts_db), accounts_db: Arc::new(accounts_db),
account_locks: Mutex::new(HashSet::new()), account_locks: Mutex::new(HashSet::new()),
readonly_locks: Arc::new(RwLock::new(Some(HashMap::new()))), readonly_locks: Arc::new(RwLock::new(Some(HashMap::new()))),
..Self::default()
} }
} }
@ -133,6 +132,7 @@ impl Accounts {
fee: u64, fee: u64,
error_counters: &mut ErrorCounters, error_counters: &mut ErrorCounters,
rent_collector: &RentCollector, rent_collector: &RentCollector,
feature_set: &FeatureSet,
) -> Result<(TransactionAccounts, TransactionRent)> { ) -> Result<(TransactionAccounts, TransactionRent)> {
// Copy all the accounts // Copy all the accounts
let message = tx.message(); let message = tx.message();
@ -150,10 +150,8 @@ impl Accounts {
payer_index = Some(i); payer_index = Some(i);
} }
if solana_sdk::sysvar::instructions::is_enabled( if solana_sdk::sysvar::instructions::check_id(key)
self.epoch, && feature_set.active(&feature_set::instructions_sysvar_enabled::id())
self.accounts_db.cluster_type.unwrap(),
) && solana_sdk::sysvar::instructions::check_id(key)
{ {
if message.is_writable(i) { if message.is_writable(i) {
return Err(TransactionError::InvalidAccountIndex); return Err(TransactionError::InvalidAccountIndex);
@ -300,11 +298,17 @@ impl Accounts {
hash_queue: &BlockhashQueue, hash_queue: &BlockhashQueue,
error_counters: &mut ErrorCounters, error_counters: &mut ErrorCounters,
rent_collector: &RentCollector, rent_collector: &RentCollector,
feature_set: &FeatureSet,
) -> Vec<(Result<TransactionLoadResult>, Option<HashAgeKind>)> { ) -> Vec<(Result<TransactionLoadResult>, Option<HashAgeKind>)> {
//PERF: hold the lock to scan for the references, but not to clone the accounts //PERF: hold the lock to scan for the references, but not to clone the accounts
//TODO: two locks usually leads to deadlocks, should this be one structure? //TODO: two locks usually leads to deadlocks, should this be one structure?
let accounts_index = self.accounts_db.accounts_index.read().unwrap(); let accounts_index = self.accounts_db.accounts_index.read().unwrap();
let storage = self.accounts_db.storage.read().unwrap(); let storage = self.accounts_db.storage.read().unwrap();
let fee_config = FeeConfig {
secp256k1_program_enabled: feature_set
.active(&feature_set::secp256k1_program_enabled::id()),
};
OrderedIterator::new(txs, txs_iteration_order) OrderedIterator::new(txs, txs_iteration_order)
.zip(lock_results.into_iter()) .zip(lock_results.into_iter())
.map(|etx| match etx { .map(|etx| match etx {
@ -318,13 +322,7 @@ impl Accounts {
.cloned(), .cloned(),
}; };
let fee = if let Some(fee_calculator) = fee_calculator { let fee = if let Some(fee_calculator) = fee_calculator {
fee_calculator.calculate_fee( fee_calculator.calculate_fee_with_config(tx.message(), &fee_config)
tx.message(),
solana_sdk::secp256k1::get_fee_config(
self.accounts_db.cluster_type.unwrap(),
self.epoch,
),
)
} else { } else {
return (Err(TransactionError::BlockhashNotFound), hash_age_kind); return (Err(TransactionError::BlockhashNotFound), hash_age_kind);
}; };
@ -337,6 +335,7 @@ impl Accounts {
fee, fee,
error_counters, error_counters,
rent_collector, rent_collector,
feature_set,
); );
let (accounts, rents) = match load_res { let (accounts, rents) = match load_res {
Ok((a, r)) => (a, r), Ok((a, r)) => (a, r),
@ -888,6 +887,7 @@ mod tests {
&hash_queue, &hash_queue,
error_counters, error_counters,
rent_collector, rent_collector,
&FeatureSet::default(),
) )
} }
@ -1024,7 +1024,7 @@ mod tests {
); );
let fee_calculator = FeeCalculator::new(10); let fee_calculator = FeeCalculator::new(10);
assert_eq!(fee_calculator.calculate_fee(tx.message(), None), 10); assert_eq!(fee_calculator.calculate_fee(tx.message()), 10);
let loaded_accounts = let loaded_accounts =
load_accounts_with_fee(tx, &accounts, &fee_calculator, &mut error_counters); load_accounts_with_fee(tx, &accounts, &fee_calculator, &mut error_counters);
@ -1832,6 +1832,7 @@ mod tests {
&hash_queue, &hash_queue,
&mut error_counters, &mut error_counters,
&rent_collector, &rent_collector,
&FeatureSet::default(),
) )
} }

View File

@ -10,11 +10,11 @@ use crate::{
accounts_db::{ErrorCounters, SnapshotStorages}, accounts_db::{ErrorCounters, SnapshotStorages},
accounts_index::Ancestors, accounts_index::Ancestors,
blockhash_queue::BlockhashQueue, blockhash_queue::BlockhashQueue,
builtins::get_builtins, builtins::*,
epoch_stakes::{EpochStakes, NodeVoteAccounts}, epoch_stakes::{EpochStakes, NodeVoteAccounts},
instruction_recorder::InstructionRecorder, instruction_recorder::InstructionRecorder,
feature::Feature, feature::Feature,
feature_set::{FeatureSet}, feature_set::{self, FeatureSet},
log_collector::LogCollector, log_collector::LogCollector,
message_processor::{Executors, MessageProcessor}, message_processor::{Executors, MessageProcessor},
nonce_utils, nonce_utils,
@ -43,7 +43,7 @@ use solana_sdk::{
}, },
epoch_info::EpochInfo, epoch_info::EpochInfo,
epoch_schedule::EpochSchedule, epoch_schedule::EpochSchedule,
fee_calculator::{FeeCalculator, FeeRateGovernor}, fee_calculator::{FeeCalculator, FeeConfig, FeeRateGovernor},
genesis_config::{ClusterType, GenesisConfig}, genesis_config::{ClusterType, GenesisConfig},
hard_forks::HardForks, hard_forks::HardForks,
hash::{extend_and_hash, hashv, Hash}, hash::{extend_and_hash, hashv, Hash},
@ -1594,6 +1594,7 @@ impl Bank {
&self.blockhash_queue.read().unwrap(), &self.blockhash_queue.read().unwrap(),
error_counters, error_counters,
&self.rent_collector, &self.rent_collector,
&self.feature_set,
) )
} }
fn check_age( fn check_age(
@ -2032,8 +2033,7 @@ impl Bank {
log_collector.clone(), log_collector.clone(),
executors.clone(), executors.clone(),
instruction_recorders.as_deref(), instruction_recorders.as_deref(),
self.cluster_type(), &self.feature_set,
self.epoch(),
); );
Self::compile_recorded_instructions( Self::compile_recorded_instructions(
@ -2114,6 +2114,11 @@ impl Bank {
) -> Vec<Result<()>> { ) -> Vec<Result<()>> {
let hash_queue = self.blockhash_queue.read().unwrap(); let hash_queue = self.blockhash_queue.read().unwrap();
let mut fees = 0; let mut fees = 0;
let fee_config = FeeConfig {
secp256k1_program_enabled: self.secp256k1_program_enabled(),
};
let results = OrderedIterator::new(txs, iteration_order) let results = OrderedIterator::new(txs, iteration_order)
.zip(executed.iter()) .zip(executed.iter())
.map(|((_, tx), (res, hash_age_kind))| { .map(|((_, tx), (res, hash_age_kind))| {
@ -2130,10 +2135,7 @@ impl Bank {
}; };
let fee_calculator = fee_calculator.ok_or(TransactionError::BlockhashNotFound)?; let fee_calculator = fee_calculator.ok_or(TransactionError::BlockhashNotFound)?;
let fee = fee_calculator.calculate_fee( let fee = fee_calculator.calculate_fee_with_config(tx.message(), &fee_config);
tx.message(),
solana_sdk::secp256k1::get_fee_config(self.cluster_type(), self.epoch()),
);
let message = tx.message(); let message = tx.message();
match *res { match *res {
@ -3461,15 +3463,16 @@ impl Bank {
consumed_budget.saturating_sub(budget_recovery_delta) consumed_budget.saturating_sub(budget_recovery_delta)
} }
pub fn secp256k1_program_enabled(&self) -> bool {
self.feature_set
.active(&feature_set::secp256k1_program_enabled::id())
}
// This is called from snapshot restore AND for each epoch boundary // This is called from snapshot restore AND for each epoch boundary
// The entire code path herein must be idempotent // The entire code path herein must be idempotent
fn apply_feature_activations(&mut self, init_finish_or_warp: bool, initiate_callback: bool) { fn apply_feature_activations(&mut self, init_finish_or_warp: bool, initiate_callback: bool) {
let new_feature_activations = self.compute_active_feature_set(); let new_feature_activations = self.compute_active_feature_set(!init_finish_or_warp);
for feature_id in new_feature_activations { self.ensure_builtins(init_finish_or_warp, &new_feature_activations);
info!("New feature activated: {}", feature_id);
}
self.ensure_builtins(init_finish_or_warp);
self.reinvoke_entered_epoch_callback(initiate_callback); self.reinvoke_entered_epoch_callback(initiate_callback);
self.recheck_cross_program_support(); self.recheck_cross_program_support();
self.recheck_compute_budget(); self.recheck_compute_budget();
@ -3478,7 +3481,7 @@ impl Bank {
} }
// Compute the active feature set based on the current bank state, and return the set of newly activated features // Compute the active feature set based on the current bank state, and return the set of newly activated features
fn compute_active_feature_set(&mut self) -> HashSet<Pubkey> { fn compute_active_feature_set(&mut self, allow_new_activations: bool) -> HashSet<Pubkey> {
let mut active = self.feature_set.active.clone(); let mut active = self.feature_set.active.clone();
let mut inactive = HashSet::new(); let mut inactive = HashSet::new();
let mut newly_activated = HashSet::new(); let mut newly_activated = HashSet::new();
@ -3489,6 +3492,7 @@ impl Bank {
if let Some(mut feature) = Feature::from_account(&account) { if let Some(mut feature) = Feature::from_account(&account) {
match feature.activated_at { match feature.activated_at {
None => { None => {
if allow_new_activations {
// Feature has been requested, activate it now // Feature has been requested, activate it now
feature.activated_at = Some(slot); feature.activated_at = Some(slot);
if feature.to_account(&mut account).is_some() { if feature.to_account(&mut account).is_some() {
@ -3496,8 +3500,10 @@ impl Bank {
} }
newly_activated.insert(*feature_id); newly_activated.insert(*feature_id);
active.insert(*feature_id); active.insert(*feature_id);
info!("Feature {} activated at slot {}", feature_id, slot);
continue; continue;
} }
}
Some(activation_slot) => { Some(activation_slot) => {
if slot >= activation_slot { if slot >= activation_slot {
// Feature is already active // Feature is already active
@ -3519,14 +3525,22 @@ impl Bank {
newly_activated newly_activated
} }
fn ensure_builtins(&mut self, init_or_warp: bool) { fn ensure_builtins(&mut self, init_or_warp: bool, new_feature_activations: &HashSet<Pubkey>) {
for (program, start_epoch) in get_builtins(self.cluster_type()) { for (program, start_epoch) in get_cluster_builtins(self.cluster_type()) {
let should_populate = init_or_warp && self.epoch() >= start_epoch let should_populate = init_or_warp && self.epoch() >= start_epoch
|| !init_or_warp && self.epoch() == start_epoch; || !init_or_warp && self.epoch() == start_epoch;
if should_populate { if should_populate {
self.add_builtin(&program.name, program.id, program.entrypoint); self.add_builtin(&program.name, program.id, program.entrypoint);
} }
} }
for (program, feature) in get_feature_builtins() {
let should_populate = init_or_warp && self.feature_set.active(&feature)
|| !init_or_warp && new_feature_activations.contains(&feature);
if should_populate {
self.add_builtin(&program.name, program.id, program.entrypoint);
}
}
} }
fn reinvoke_entered_epoch_callback(&mut self, initiate: bool) { fn reinvoke_entered_epoch_callback(&mut self, initiate: bool) {
@ -8518,7 +8532,7 @@ mod tests {
.collect::<Vec<_>>(); .collect::<Vec<_>>();
consumed_budgets.sort(); consumed_budgets.sort();
// consumed_budgets represents the count of alive accounts in the three slots 0,1,2 // consumed_budgets represents the count of alive accounts in the three slots 0,1,2
assert_eq!(consumed_budgets, vec![0, 1, 10]); assert_eq!(consumed_budgets, vec![0, 1, 9]);
} }
#[test] #[test]

View File

@ -1,18 +1,21 @@
use crate::{ use crate::{
bank::{Builtin, Entrypoint}, bank::{Builtin, Entrypoint},
system_instruction_processor, feature_set, system_instruction_processor,
}; };
use solana_sdk::{ use solana_sdk::{
clock::{Epoch, GENESIS_EPOCH}, clock::{Epoch, GENESIS_EPOCH},
genesis_config::ClusterType, genesis_config::ClusterType,
pubkey::Pubkey,
system_program, system_program,
}; };
use log::*; use log::*;
/// The entire set of available builtin programs that should be active at the given cluster_type /// Builtin programs that should be active for the given cluster_type
pub fn get_builtins(cluster_type: ClusterType) -> Vec<(Builtin, Epoch)> { ///
trace!("get_builtins: {:?}", cluster_type); /// Old style. Use `get_feature_builtins()` instead
pub fn get_cluster_builtins(cluster_type: ClusterType) -> Vec<(Builtin, Epoch)> {
trace!("get_cluster_builtins: {:?}", cluster_type);
let mut builtins = vec![]; let mut builtins = vec![];
builtins.extend( builtins.extend(
@ -46,8 +49,8 @@ pub fn get_builtins(cluster_type: ClusterType) -> Vec<(Builtin, Epoch)> {
// repurpose Testnet for test_get_builtins because the Development is overloaded... // repurpose Testnet for test_get_builtins because the Development is overloaded...
#[cfg(test)] #[cfg(test)]
if cluster_type == ClusterType::Testnet { if cluster_type == ClusterType::Testnet {
use solana_sdk::account::KeyedAccount;
use solana_sdk::instruction::InstructionError; use solana_sdk::instruction::InstructionError;
use solana_sdk::{account::KeyedAccount, pubkey::Pubkey};
use std::str::FromStr; use std::str::FromStr;
fn mock_ix_processor( fn mock_ix_processor(
_pubkey: &Pubkey, _pubkey: &Pubkey,
@ -57,35 +60,33 @@ pub fn get_builtins(cluster_type: ClusterType) -> Vec<(Builtin, Epoch)> {
Err(InstructionError::Custom(42)) Err(InstructionError::Custom(42))
} }
let program_id = Pubkey::from_str("7saCc6X5a2syoYANA5oUUnPZLcLMfKoSjiDhFU5fbpoK").unwrap(); let program_id = Pubkey::from_str("7saCc6X5a2syoYANA5oUUnPZLcLMfKoSjiDhFU5fbpoK").unwrap();
builtins.extend(vec![( builtins.push((
Builtin::new("mock", program_id, Entrypoint::Program(mock_ix_processor)), Builtin::new("mock", program_id, Entrypoint::Program(mock_ix_processor)),
2, 2,
)]); ));
} }
let secp256k1_builtin = Builtin::new( builtins
}
/// Builtin programs that are activated dynamically by feature
pub fn get_feature_builtins() -> Vec<(Builtin, Pubkey)> {
vec![(
Builtin::new(
"secp256k1_program", "secp256k1_program",
solana_sdk::secp256k1_program::id(), solana_sdk::secp256k1_program::id(),
Entrypoint::Program(solana_secp256k1_program::process_instruction), Entrypoint::Program(solana_secp256k1_program::process_instruction),
); ),
let secp_epoch = solana_sdk::secp256k1::is_enabled_epoch(cluster_type); feature_set::secp256k1_program_enabled::id(),
builtins.push((secp256k1_builtin, secp_epoch)); )]
builtins
} }
#[cfg(test)] #[cfg(test)]
mod tests { mod tests {
use super::*; use super::*;
use crate::bank::Bank; use crate::bank::Bank;
use solana_sdk::{ use solana_sdk::genesis_config::create_genesis_config;
genesis_config::{create_genesis_config, ClusterType}, use std::{collections::HashSet, str::FromStr, sync::Arc};
pubkey::Pubkey,
};
use std::collections::HashSet;
use std::str::FromStr;
use std::sync::Arc;
fn do_test_uniqueness(builtins: Vec<(Builtin, Epoch)>) { fn do_test_uniqueness(builtins: Vec<(Builtin, Epoch)>) {
let mut unique_ids = HashSet::new(); let mut unique_ids = HashSet::new();
@ -101,10 +102,10 @@ mod tests {
#[test] #[test]
fn test_uniqueness() { fn test_uniqueness() {
do_test_uniqueness(get_builtins(ClusterType::Development)); do_test_uniqueness(get_cluster_builtins(ClusterType::Development));
do_test_uniqueness(get_builtins(ClusterType::Devnet)); do_test_uniqueness(get_cluster_builtins(ClusterType::Devnet));
do_test_uniqueness(get_builtins(ClusterType::Testnet)); do_test_uniqueness(get_cluster_builtins(ClusterType::Testnet));
do_test_uniqueness(get_builtins(ClusterType::MainnetBeta)); do_test_uniqueness(get_cluster_builtins(ClusterType::MainnetBeta));
} }
#[test] #[test]

View File

@ -17,14 +17,17 @@ solana_sdk::declare_id!("Feature111111111111111111111111111111111111");
/// 2. When the next epoch is entered the runtime will check for new activation requests and /// 2. When the next epoch is entered the runtime will check for new activation requests and
/// active them. When this occurs, the activation slot is recorded in the feature account /// active them. When this occurs, the activation slot is recorded in the feature account
/// ///
#[derive(Default, Serialize, Deserialize)] #[derive(Default, Debug, Serialize, Deserialize)]
pub struct Feature { pub struct Feature {
pub activated_at: Option<Slot>, pub activated_at: Option<Slot>,
} }
impl Feature { impl Feature {
pub fn size_of() -> usize { pub fn size_of() -> usize {
bincode::serialized_size(&Self::default()).unwrap() as usize bincode::serialized_size(&Self {
activated_at: Some(Slot::MAX),
})
.unwrap() as usize
} }
pub fn from_account(account: &Account) -> Option<Self> { pub fn from_account(account: &Account) -> Option<Self> {
if account.owner != id() { if account.owner != id() {

View File

@ -1,8 +1,38 @@
use lazy_static::lazy_static;
use solana_sdk::{ use solana_sdk::{
hash::{Hash, Hasher}, hash::{Hash, Hasher},
pubkey::Pubkey, pubkey::Pubkey,
}; };
use std::collections::HashSet; use std::collections::{HashMap, HashSet};
pub mod instructions_sysvar_enabled {
solana_sdk::declare_id!("EnvhHCLvg55P7PDtbvR1NwuTuAeodqpusV3MR5QEK8gs");
}
pub mod secp256k1_program_enabled {
solana_sdk::declare_id!("E3PHP7w8kB7np3CTQ1qQ2tW3KCtjRSXBQgW9vM2mWv2Y");
}
lazy_static! {
pub static ref FEATURE_NAMES: HashMap<Pubkey, &'static str> = [
(instructions_sysvar_enabled::id(), "instructions sysvar"),
(secp256k1_program_enabled::id(), "secp256k1 program")
/*************** ADD NEW FEATURES HERE ***************/
]
.iter()
.cloned()
.collect();
static ref ID: Hash = {
let mut hasher = Hasher::default();
let mut feature_ids = FEATURE_NAMES.keys().collect::<Vec<_>>();
feature_ids.sort();
for feature in feature_ids {
hasher.hash(feature.as_ref());
}
hasher.result()
};
}
/// The `FeatureSet` struct tracks the set of available and currently active runtime features /// The `FeatureSet` struct tracks the set of available and currently active runtime features
#[derive(AbiExample)] #[derive(AbiExample)]
@ -26,34 +56,19 @@ impl FeatureSet {
impl Default for FeatureSet { impl Default for FeatureSet {
// By default all features are disabled // By default all features are disabled
fn default() -> Self { fn default() -> Self {
let features: [Pubkey; 0] = [];
Self { Self {
id: { id: *ID,
let mut hasher = Hasher::default();
for feature in features.iter() {
hasher.hash(feature.as_ref());
}
hasher.result()
},
active: HashSet::new(), active: HashSet::new(),
inactive: features.iter().cloned().collect(), inactive: FEATURE_NAMES.keys().cloned().collect(),
} }
} }
} }
impl FeatureSet { impl FeatureSet {
// New `FeatureSet` with all features enabled pub fn enabled() -> Self {
pub fn new_enabled() -> Self {
let default = Self::default();
Self { Self {
id: default.id, id: *ID,
active: default active: FEATURE_NAMES.keys().cloned().collect(),
.active
.intersection(&default.inactive)
.cloned()
.collect::<HashSet<_>>(),
inactive: HashSet::new(), inactive: HashSet::new(),
} }
} }

View File

@ -109,18 +109,18 @@ pub fn create_genesis_config_with_leader(
} }
pub fn add_feature_accounts(genesis_config: &mut GenesisConfig) { pub fn add_feature_accounts(genesis_config: &mut GenesisConfig) {
// Activate all features at genesis in development mode
if genesis_config.cluster_type == ClusterType::Development { if genesis_config.cluster_type == ClusterType::Development {
let feature_set = FeatureSet::new_enabled(); // Activate all features at genesis in development mode
for feature_id in FeatureSet::default().inactive {
for feature_id in feature_set.active {
let feature = Feature { let feature = Feature {
activated_at: Some(0), activated_at: Some(0),
}; };
genesis_config.accounts.insert( genesis_config.accounts.insert(
feature_id, feature_id,
feature.create_account(genesis_config.rent.minimum_balance(Feature::size_of())), feature.create_account(std::cmp::max(
genesis_config.rent.minimum_balance(Feature::size_of()),
1,
)),
); );
} }
} }

View File

@ -1,6 +1,7 @@
use crate::{ use crate::{
instruction_recorder::InstructionRecorder, log_collector::LogCollector, instruction_recorder::InstructionRecorder, log_collector::LogCollector,
native_loader::NativeLoader, rent_collector::RentCollector, native_loader::NativeLoader, rent_collector::RentCollector,
feature_set::{self, FeatureSet},
}; };
use log::*; use log::*;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
@ -11,7 +12,6 @@ use solana_sdk::{
ComputeBudget, ComputeMeter, ErasedProcessInstruction, ErasedProcessInstructionWithContext, ComputeBudget, ComputeMeter, ErasedProcessInstruction, ErasedProcessInstructionWithContext,
Executor, InvokeContext, Logger, ProcessInstruction, ProcessInstructionWithContext, Executor, InvokeContext, Logger, ProcessInstruction, ProcessInstructionWithContext,
}, },
genesis_config::ClusterType,
instruction::{CompiledInstruction, Instruction, InstructionError}, instruction::{CompiledInstruction, Instruction, InstructionError},
message::Message, message::Message,
native_loader, native_loader,
@ -679,12 +679,11 @@ impl MessageProcessor {
executors: Rc<RefCell<Executors>>, executors: Rc<RefCell<Executors>>,
instruction_recorder: Option<InstructionRecorder>, instruction_recorder: Option<InstructionRecorder>,
instruction_index: usize, instruction_index: usize,
cluster_type: ClusterType, feature_set: &FeatureSet,
epoch: Epoch,
) -> Result<(), InstructionError> { ) -> Result<(), InstructionError> {
// Fixup the special instructions key if present // Fixup the special instructions key if present
// before the account pre-values are taken care of // before the account pre-values are taken care of
if solana_sdk::sysvar::instructions::is_enabled(epoch, cluster_type) { if feature_set.active(&feature_set::instructions_sysvar_enabled::id()) {
for (i, key) in message.account_keys.iter().enumerate() { for (i, key) in message.account_keys.iter().enumerate() {
if solana_sdk::sysvar::instructions::check_id(key) { if solana_sdk::sysvar::instructions::check_id(key) {
let mut mut_account_ref = accounts[i].borrow_mut(); let mut mut_account_ref = accounts[i].borrow_mut();
@ -736,8 +735,7 @@ impl MessageProcessor {
log_collector: Option<Rc<LogCollector>>, log_collector: Option<Rc<LogCollector>>,
executors: Rc<RefCell<Executors>>, executors: Rc<RefCell<Executors>>,
instruction_recorders: Option<&[InstructionRecorder]>, instruction_recorders: Option<&[InstructionRecorder]>,
cluster_type: ClusterType, feature_set: &FeatureSet,
epoch: Epoch,
) -> Result<(), TransactionError> { ) -> Result<(), TransactionError> {
for (instruction_index, instruction) in message.instructions.iter().enumerate() { for (instruction_index, instruction) in message.instructions.iter().enumerate() {
let instruction_recorder = instruction_recorders let instruction_recorder = instruction_recorders
@ -753,8 +751,7 @@ impl MessageProcessor {
executors.clone(), executors.clone(),
instruction_recorder, instruction_recorder,
instruction_index, instruction_index,
cluster_type, feature_set,
epoch,
) )
.map_err(|err| TransactionError::InstructionError(instruction_index as u8, err))?; .map_err(|err| TransactionError::InstructionError(instruction_index as u8, err))?;
} }
@ -1349,8 +1346,7 @@ mod tests {
None, None,
executors.clone(), executors.clone(),
None, None,
ClusterType::Development, &FeatureSet::default(),
0,
); );
assert_eq!(result, Ok(())); assert_eq!(result, Ok(()));
assert_eq!(accounts[0].borrow().lamports, 100); assert_eq!(accounts[0].borrow().lamports, 100);
@ -1373,8 +1369,7 @@ mod tests {
None, None,
executors.clone(), executors.clone(),
None, None,
ClusterType::Development, &FeatureSet::default(),
0,
); );
assert_eq!( assert_eq!(
result, result,
@ -1401,8 +1396,7 @@ mod tests {
None, None,
executors, executors,
None, None,
ClusterType::Development, &FeatureSet::default(),
0,
); );
assert_eq!( assert_eq!(
result, result,
@ -1512,8 +1506,7 @@ mod tests {
None, None,
executors.clone(), executors.clone(),
None, None,
ClusterType::Development, &FeatureSet::default(),
0,
); );
assert_eq!( assert_eq!(
result, result,
@ -1540,8 +1533,7 @@ mod tests {
None, None,
executors.clone(), executors.clone(),
None, None,
ClusterType::Development, &FeatureSet::default(),
0,
); );
assert_eq!(result, Ok(())); assert_eq!(result, Ok(()));
@ -1565,8 +1557,7 @@ mod tests {
None, None,
executors, executors,
None, None,
ClusterType::Development, &FeatureSet::default(),
0,
); );
assert_eq!(result, Ok(())); assert_eq!(result, Ok(()));
assert_eq!(accounts[0].borrow().lamports, 80); assert_eq!(accounts[0].borrow().lamports, 80);

View File

@ -19,9 +19,16 @@ impl Default for FeeCalculator {
} }
} }
#[derive(Clone)]
pub struct FeeConfig { pub struct FeeConfig {
pub is_secp256k1_enabled: bool, pub secp256k1_program_enabled: bool,
}
impl Default for FeeConfig {
fn default() -> Self {
Self {
secp256k1_program_enabled: true,
}
}
} }
impl FeeCalculator { impl FeeCalculator {
@ -31,27 +38,27 @@ impl FeeCalculator {
} }
} }
// extra_config: None == everything enabled pub fn calculate_fee(&self, message: &Message) -> u64 {
pub fn calculate_fee(&self, message: &Message, extra_config: Option<FeeConfig>) -> u64 { self.calculate_fee_with_config(message, &FeeConfig::default())
let is_secp256k1_enabled = match extra_config { }
Some(config) => config.is_secp256k1_enabled,
None => true, pub fn calculate_fee_with_config(&self, message: &Message, fee_config: &FeeConfig) -> u64 {
}; let mut num_secp256k1_signatures: u64 = 0;
let mut num_secp_signatures: u64 = 0; if fee_config.secp256k1_program_enabled {
if is_secp256k1_enabled {
for instruction in &message.instructions { for instruction in &message.instructions {
let program_index = instruction.program_id_index as usize; let program_index = instruction.program_id_index as usize;
// Transaction may not be sanitized here // Transaction may not be sanitized here
if program_index < message.account_keys.len() { if program_index < message.account_keys.len() {
let id = message.account_keys[program_index]; let id = message.account_keys[program_index];
if secp256k1_program::check_id(&id) && !instruction.data.is_empty() { if secp256k1_program::check_id(&id) && !instruction.data.is_empty() {
num_secp_signatures += instruction.data[0] as u64; num_secp256k1_signatures += instruction.data[0] as u64;
} }
} }
} }
} }
self.lamports_per_signature self.lamports_per_signature
* (u64::from(message.header.num_required_signatures) + num_secp_signatures) * (u64::from(message.header.num_required_signatures) + num_secp256k1_signatures)
} }
} }
@ -182,9 +189,7 @@ impl FeeRateGovernor {
/// create a FeeCalculator based on current cluster signature throughput /// create a FeeCalculator based on current cluster signature throughput
pub fn create_fee_calculator(&self) -> FeeCalculator { pub fn create_fee_calculator(&self) -> FeeCalculator {
FeeCalculator { FeeCalculator::new(self.lamports_per_signature)
lamports_per_signature: self.lamports_per_signature,
}
} }
} }
@ -207,34 +212,25 @@ mod tests {
#[test] #[test]
fn test_fee_calculator_calculate_fee() { fn test_fee_calculator_calculate_fee() {
let fee_config = Some(FeeConfig {
is_secp256k1_enabled: true,
});
// Default: no fee. // Default: no fee.
let message = Message::default(); let message = Message::default();
assert_eq!( assert_eq!(FeeCalculator::default().calculate_fee(&message), 0);
FeeCalculator::default().calculate_fee(&message, fee_config.clone()),
0
);
// No signature, no fee. // No signature, no fee.
assert_eq!(FeeCalculator::new(1).calculate_fee(&message, fee_config), 0); assert_eq!(FeeCalculator::new(1).calculate_fee(&message), 0);
let fee_config = Some(FeeConfig {
is_secp256k1_enabled: false,
});
// One signature, a fee. // One signature, a fee.
let pubkey0 = Pubkey::new(&[0; 32]); let pubkey0 = Pubkey::new(&[0; 32]);
let pubkey1 = Pubkey::new(&[1; 32]); let pubkey1 = Pubkey::new(&[1; 32]);
let ix0 = system_instruction::transfer(&pubkey0, &pubkey1, 1); let ix0 = system_instruction::transfer(&pubkey0, &pubkey1, 1);
let message = Message::new(&[ix0], Some(&pubkey0)); let message = Message::new(&[ix0], Some(&pubkey0));
assert_eq!(FeeCalculator::new(2).calculate_fee(&message, fee_config), 2); assert_eq!(FeeCalculator::new(2).calculate_fee(&message), 2);
// Two signatures, double the fee. // Two signatures, double the fee.
let ix0 = system_instruction::transfer(&pubkey0, &pubkey1, 1); let ix0 = system_instruction::transfer(&pubkey0, &pubkey1, 1);
let ix1 = system_instruction::transfer(&pubkey1, &pubkey0, 1); let ix1 = system_instruction::transfer(&pubkey1, &pubkey0, 1);
let message = Message::new(&[ix0, ix1], Some(&pubkey0)); let message = Message::new(&[ix0, ix1], Some(&pubkey0));
assert_eq!(FeeCalculator::new(2).calculate_fee(&message, None), 4); assert_eq!(FeeCalculator::new(2).calculate_fee(&message), 4);
} }
#[test] #[test]
@ -262,21 +258,21 @@ mod tests {
], ],
Some(&pubkey0), Some(&pubkey0),
); );
let fee_config = Some(FeeConfig { assert_eq!(FeeCalculator::new(1).calculate_fee(&message), 2);
is_secp256k1_enabled: true,
});
assert_eq!( assert_eq!(
FeeCalculator::new(1).calculate_fee(&message, fee_config.clone()), FeeCalculator::new(1).calculate_fee_with_config(
2 &message,
&FeeConfig {
secp256k1_program_enabled: false
}
),
1
); );
secp_instruction.data = vec![0]; secp_instruction.data = vec![0];
secp_instruction2.data = vec![10]; secp_instruction2.data = vec![10];
let message = Message::new(&[ix0, secp_instruction, secp_instruction2], Some(&pubkey0)); let message = Message::new(&[ix0, secp_instruction, secp_instruction2], Some(&pubkey0));
assert_eq!( assert_eq!(FeeCalculator::new(1).calculate_fee(&message), 11);
FeeCalculator::new(1).calculate_fee(&message, fee_config),
11
);
} }
#[test] #[test]

View File

@ -1,28 +1,6 @@
use crate::clock::{Epoch, GENESIS_EPOCH};
use crate::fee_calculator::FeeConfig;
use crate::genesis_config::ClusterType;
use digest::Digest; use digest::Digest;
use serde_derive::{Deserialize, Serialize}; use serde_derive::{Deserialize, Serialize};
pub fn get_fee_config(cluster_type: ClusterType, epoch: Epoch) -> Option<FeeConfig> {
Some(FeeConfig {
is_secp256k1_enabled: is_enabled(cluster_type, epoch),
})
}
pub fn is_enabled_epoch(cluster_type: ClusterType) -> Epoch {
match cluster_type {
ClusterType::Development => GENESIS_EPOCH,
ClusterType::Testnet => u64::MAX,
ClusterType::MainnetBeta => u64::MAX,
ClusterType::Devnet => u64::MAX,
}
}
pub fn is_enabled(cluster_type: ClusterType, epoch: Epoch) -> bool {
epoch >= is_enabled_epoch(cluster_type)
}
#[derive(Debug)] #[derive(Debug)]
pub enum Secp256k1Error { pub enum Secp256k1Error {
InvalidSignature, InvalidSignature,

View File

@ -11,16 +11,6 @@ crate::declare_sysvar_id!("Sysvar1nstructions1111111111111111111111111", Instruc
impl Sysvar for Instructions {} impl Sysvar for Instructions {}
#[cfg(not(feature = "program"))]
use crate::clock::Epoch;
#[cfg(not(feature = "program"))]
use crate::genesis_config::ClusterType;
#[cfg(not(feature = "program"))]
pub fn is_enabled(_epoch: Epoch, cluster_type: ClusterType) -> bool {
cluster_type == ClusterType::Development
}
pub fn load_current_index(data: &[u8]) -> u16 { pub fn load_current_index(data: &[u8]) -> u16 {
let mut instr_fixed_data = [0u8; 2]; let mut instr_fixed_data = [0u8; 2];
let len = data.len(); let len = data.len();

View File

@ -430,7 +430,7 @@ fn transact(
info!("{} transactions to send", transactions.len()); info!("{} transactions to send", transactions.len());
let required_fee = transactions.iter().fold(0, |fee, (transaction, _)| { let required_fee = transactions.iter().fold(0, |fee, (transaction, _)| {
fee + fee_calculator.calculate_fee(&transaction.message, None) fee + fee_calculator.calculate_fee(&transaction.message)
}); });
info!("Required fee: {} SOL", lamports_to_sol(required_fee)); info!("Required fee: {} SOL", lamports_to_sol(required_fee));
if required_fee > authorized_staker_balance { if required_fee > authorized_staker_balance {