Use rooted bank by default in rpc bank selection (#6759)
* Name anonymous parameters for clarity * Add CommitmentConfig to select bank for rpc * Add commitment information to jsonrpc docs * Update send_and_confirm retries as per commitment defaults * Pass CommitmentConfig into client requests; also various 'use' cleanup * Use _with_commitment methods to speed local_cluster tests * Pass CommitmentConfig into Archiver in order to enable quick confirmations in local_cluster tests * Restore solana ping speed * Increase wallet-sanity timeout to account for longer confirmation time
This commit is contained in:
@ -29,6 +29,7 @@ use solana_sdk::{
|
||||
account_utils::State,
|
||||
client::{AsyncClient, SyncClient},
|
||||
clock::{get_complete_segment_from_slot, get_segment_from_slot, Slot},
|
||||
commitment_config::CommitmentConfig,
|
||||
hash::{Hash, Hasher},
|
||||
message::Message,
|
||||
signature::{Keypair, KeypairUtil, Signature},
|
||||
@ -78,6 +79,7 @@ struct ArchiverMeta {
|
||||
blockhash: Hash,
|
||||
sha_state: Hash,
|
||||
num_chacha_blocks: usize,
|
||||
client_commitment: CommitmentConfig,
|
||||
}
|
||||
|
||||
pub(crate) fn sample_file(in_path: &Path, sample_offsets: &[u64]) -> io::Result<Hash> {
|
||||
@ -208,6 +210,7 @@ impl Archiver {
|
||||
cluster_entrypoint: ContactInfo,
|
||||
keypair: Arc<Keypair>,
|
||||
storage_keypair: Arc<Keypair>,
|
||||
client_commitment: CommitmentConfig,
|
||||
) -> Result<Self> {
|
||||
let exit = Arc::new(AtomicBool::new(false));
|
||||
|
||||
@ -246,7 +249,12 @@ impl Archiver {
|
||||
let client = crate::gossip_service::get_client(&nodes);
|
||||
|
||||
info!("Setting up mining account...");
|
||||
if let Err(e) = Self::setup_mining_account(&client, &keypair, &storage_keypair) {
|
||||
if let Err(e) = Self::setup_mining_account(
|
||||
&client,
|
||||
&keypair,
|
||||
&storage_keypair,
|
||||
client_commitment.clone(),
|
||||
) {
|
||||
//shutdown services before exiting
|
||||
exit.store(true, Ordering::Relaxed);
|
||||
gossip_service.join()?;
|
||||
@ -279,6 +287,7 @@ impl Archiver {
|
||||
let node_info = node.info.clone();
|
||||
let mut meta = ArchiverMeta {
|
||||
ledger_path: ledger_path.to_path_buf(),
|
||||
client_commitment,
|
||||
..ArchiverMeta::default()
|
||||
};
|
||||
spawn(move || {
|
||||
@ -383,7 +392,12 @@ impl Archiver {
|
||||
}
|
||||
};
|
||||
meta.blockhash = storage_blockhash;
|
||||
Self::redeem_rewards(&cluster_info, archiver_keypair, storage_keypair);
|
||||
Self::redeem_rewards(
|
||||
&cluster_info,
|
||||
archiver_keypair,
|
||||
storage_keypair,
|
||||
meta.client_commitment.clone(),
|
||||
);
|
||||
}
|
||||
exit.store(true, Ordering::Relaxed);
|
||||
}
|
||||
@ -392,11 +406,14 @@ impl Archiver {
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
archiver_keypair: &Arc<Keypair>,
|
||||
storage_keypair: &Arc<Keypair>,
|
||||
client_commitment: CommitmentConfig,
|
||||
) {
|
||||
let nodes = cluster_info.read().unwrap().tvu_peers();
|
||||
let client = crate::gossip_service::get_client(&nodes);
|
||||
|
||||
if let Ok(Some(account)) = client.get_account(&storage_keypair.pubkey()) {
|
||||
if let Ok(Some(account)) =
|
||||
client.get_account_with_commitment(&storage_keypair.pubkey(), client_commitment.clone())
|
||||
{
|
||||
if let Ok(StorageContract::ArchiverStorage { validations, .. }) = account.state() {
|
||||
if !validations.is_empty() {
|
||||
let ix = storage_instruction::claim_reward(
|
||||
@ -410,7 +427,10 @@ impl Archiver {
|
||||
} else {
|
||||
info!(
|
||||
"collected mining rewards: Account balance {:?}",
|
||||
client.get_balance(&archiver_keypair.pubkey())
|
||||
client.get_balance_with_commitment(
|
||||
&archiver_keypair.pubkey(),
|
||||
client_commitment.clone()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
@ -432,15 +452,16 @@ impl Archiver {
|
||||
blob_fetch_receiver: PacketReceiver,
|
||||
slot_sender: Sender<u64>,
|
||||
) -> Result<(WindowService)> {
|
||||
let slots_per_segment = match Self::get_segment_config(&cluster_info) {
|
||||
Ok(slots_per_segment) => slots_per_segment,
|
||||
Err(e) => {
|
||||
error!("unable to get segment size configuration, exiting...");
|
||||
//shutdown services before exiting
|
||||
exit.store(true, Ordering::Relaxed);
|
||||
return Err(e);
|
||||
}
|
||||
};
|
||||
let slots_per_segment =
|
||||
match Self::get_segment_config(&cluster_info, meta.client_commitment.clone()) {
|
||||
Ok(slots_per_segment) => slots_per_segment,
|
||||
Err(e) => {
|
||||
error!("unable to get segment size configuration, exiting...");
|
||||
//shutdown services before exiting
|
||||
exit.store(true, Ordering::Relaxed);
|
||||
return Err(e);
|
||||
}
|
||||
};
|
||||
let (segment_blockhash, segment_slot) = match Self::poll_for_segment(
|
||||
&cluster_info,
|
||||
slots_per_segment,
|
||||
@ -588,13 +609,15 @@ impl Archiver {
|
||||
client: &ThinClient,
|
||||
keypair: &Keypair,
|
||||
storage_keypair: &Keypair,
|
||||
client_commitment: CommitmentConfig,
|
||||
) -> Result<()> {
|
||||
// make sure archiver has some balance
|
||||
info!("checking archiver keypair...");
|
||||
if client.poll_balance_with_timeout(
|
||||
if client.poll_balance_with_timeout_and_commitment(
|
||||
&keypair.pubkey(),
|
||||
&Duration::from_millis(100),
|
||||
&Duration::from_secs(5),
|
||||
client_commitment.clone(),
|
||||
)? == 0
|
||||
{
|
||||
return Err(
|
||||
@ -604,17 +627,19 @@ impl Archiver {
|
||||
|
||||
info!("checking storage account keypair...");
|
||||
// check if the storage account exists
|
||||
let balance = client.poll_get_balance(&storage_keypair.pubkey());
|
||||
let balance = client
|
||||
.poll_get_balance_with_commitment(&storage_keypair.pubkey(), client_commitment.clone());
|
||||
if balance.is_err() || balance.unwrap() == 0 {
|
||||
let blockhash = match client.get_recent_blockhash() {
|
||||
Ok((blockhash, _)) => blockhash,
|
||||
Err(_) => {
|
||||
return Err(Error::IO(<io::Error>::new(
|
||||
io::ErrorKind::Other,
|
||||
"unable to get recent blockhash, can't submit proof",
|
||||
)));
|
||||
}
|
||||
};
|
||||
let blockhash =
|
||||
match client.get_recent_blockhash_with_commitment(client_commitment.clone()) {
|
||||
Ok((blockhash, _)) => blockhash,
|
||||
Err(_) => {
|
||||
return Err(Error::IO(<io::Error>::new(
|
||||
io::ErrorKind::Other,
|
||||
"unable to get recent blockhash, can't submit proof",
|
||||
)));
|
||||
}
|
||||
};
|
||||
|
||||
let ix = storage_instruction::create_storage_account(
|
||||
&keypair.pubkey(),
|
||||
@ -626,7 +651,7 @@ impl Archiver {
|
||||
let tx = Transaction::new_signed_instructions(&[keypair], ix, blockhash);
|
||||
let signature = client.async_send_transaction(tx)?;
|
||||
client
|
||||
.poll_for_signature(&signature)
|
||||
.poll_for_signature_with_commitment(&signature, client_commitment.clone())
|
||||
.map_err(|err| match err {
|
||||
TransportError::IoError(e) => e,
|
||||
TransportError::TransactionError(_) => io::Error::new(
|
||||
@ -647,25 +672,32 @@ impl Archiver {
|
||||
// No point if we've got no storage account...
|
||||
let nodes = cluster_info.read().unwrap().tvu_peers();
|
||||
let client = crate::gossip_service::get_client(&nodes);
|
||||
let storage_balance = client.poll_get_balance(&storage_keypair.pubkey());
|
||||
let storage_balance = client.poll_get_balance_with_commitment(
|
||||
&storage_keypair.pubkey(),
|
||||
meta.client_commitment.clone(),
|
||||
);
|
||||
if storage_balance.is_err() || storage_balance.unwrap() == 0 {
|
||||
error!("Unable to submit mining proof, no storage account");
|
||||
return;
|
||||
}
|
||||
// ...or no lamports for fees
|
||||
let balance = client.poll_get_balance(&archiver_keypair.pubkey());
|
||||
let balance = client.poll_get_balance_with_commitment(
|
||||
&archiver_keypair.pubkey(),
|
||||
meta.client_commitment.clone(),
|
||||
);
|
||||
if balance.is_err() || balance.unwrap() == 0 {
|
||||
error!("Unable to submit mining proof, insufficient Archiver Account balance");
|
||||
return;
|
||||
}
|
||||
|
||||
let blockhash = match client.get_recent_blockhash() {
|
||||
Ok((blockhash, _)) => blockhash,
|
||||
Err(_) => {
|
||||
error!("unable to get recent blockhash, can't submit proof");
|
||||
return;
|
||||
}
|
||||
};
|
||||
let blockhash =
|
||||
match client.get_recent_blockhash_with_commitment(meta.client_commitment.clone()) {
|
||||
Ok((blockhash, _)) => blockhash,
|
||||
Err(_) => {
|
||||
error!("unable to get recent blockhash, can't submit proof");
|
||||
return;
|
||||
}
|
||||
};
|
||||
let instruction = storage_instruction::mining_proof(
|
||||
&storage_keypair.pubkey(),
|
||||
meta.sha_state,
|
||||
@ -700,7 +732,10 @@ impl Archiver {
|
||||
}
|
||||
}
|
||||
|
||||
fn get_segment_config(cluster_info: &Arc<RwLock<ClusterInfo>>) -> result::Result<u64, Error> {
|
||||
fn get_segment_config(
|
||||
cluster_info: &Arc<RwLock<ClusterInfo>>,
|
||||
client_commitment: CommitmentConfig,
|
||||
) -> result::Result<u64, Error> {
|
||||
let rpc_peers = {
|
||||
let cluster_info = cluster_info.read().unwrap();
|
||||
cluster_info.rpc_peers()
|
||||
@ -712,7 +747,12 @@ impl Archiver {
|
||||
RpcClient::new_socket(rpc_peers[node_index].rpc)
|
||||
};
|
||||
Ok(rpc_client
|
||||
.retry_make_rpc_request(&RpcRequest::GetSlotsPerSegment, None, 0)
|
||||
.retry_make_rpc_request(
|
||||
&RpcRequest::GetSlotsPerSegment,
|
||||
None,
|
||||
0,
|
||||
Some(client_commitment),
|
||||
)
|
||||
.map_err(|err| {
|
||||
warn!("Error while making rpc request {:?}", err);
|
||||
Error::IO(io::Error::new(ErrorKind::Other, "rpc error"))
|
||||
@ -764,7 +804,7 @@ impl Archiver {
|
||||
RpcClient::new_socket(rpc_peers[node_index].rpc)
|
||||
};
|
||||
let response = rpc_client
|
||||
.retry_make_rpc_request(&RpcRequest::GetStorageTurn, None, 0)
|
||||
.retry_make_rpc_request(&RpcRequest::GetStorageTurn, None, 0, None)
|
||||
.map_err(|err| {
|
||||
warn!("Error while making rpc request {:?}", err);
|
||||
Error::IO(io::Error::new(ErrorKind::Other, "rpc error"))
|
||||
|
441
core/src/rpc.rs
441
core/src/rpc.rs
@ -19,6 +19,7 @@ use solana_runtime::bank::Bank;
|
||||
use solana_sdk::{
|
||||
account::Account,
|
||||
clock::Slot,
|
||||
commitment_config::{CommitmentConfig, CommitmentLevel},
|
||||
epoch_schedule::EpochSchedule,
|
||||
fee_calculator::FeeCalculator,
|
||||
hash::Hash,
|
||||
@ -60,8 +61,18 @@ pub struct JsonRpcRequestProcessor {
|
||||
}
|
||||
|
||||
impl JsonRpcRequestProcessor {
|
||||
fn bank(&self) -> Arc<Bank> {
|
||||
self.bank_forks.read().unwrap().working_bank()
|
||||
fn bank(&self, commitment: Option<CommitmentConfig>) -> Arc<Bank> {
|
||||
debug!("RPC commitment_config: {:?}", commitment);
|
||||
let r_bank_forks = self.bank_forks.read().unwrap();
|
||||
if commitment.is_some() && commitment.unwrap().commitment == CommitmentLevel::Recent {
|
||||
let bank = r_bank_forks.working_bank();
|
||||
debug!("RPC using working_bank: {:?}", bank.slot());
|
||||
bank
|
||||
} else {
|
||||
let slot = r_bank_forks.root();
|
||||
debug!("RPC using block: {:?}", slot);
|
||||
r_bank_forks.get(slot).cloned().unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
pub fn new(
|
||||
@ -80,43 +91,62 @@ impl JsonRpcRequestProcessor {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_account_info(&self, pubkey: &Pubkey) -> Result<Account> {
|
||||
self.bank()
|
||||
pub fn get_account_info(
|
||||
&self,
|
||||
pubkey: &Pubkey,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Account> {
|
||||
self.bank(commitment)
|
||||
.get_account(&pubkey)
|
||||
.ok_or_else(Error::invalid_request)
|
||||
}
|
||||
|
||||
pub fn get_minimum_balance_for_rent_exemption(&self, data_len: usize) -> Result<u64> {
|
||||
Ok(self.bank().get_minimum_balance_for_rent_exemption(data_len))
|
||||
pub fn get_minimum_balance_for_rent_exemption(
|
||||
&self,
|
||||
data_len: usize,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64> {
|
||||
Ok(self
|
||||
.bank(commitment)
|
||||
.get_minimum_balance_for_rent_exemption(data_len))
|
||||
}
|
||||
|
||||
pub fn get_program_accounts(&self, program_id: &Pubkey) -> Result<Vec<(String, Account)>> {
|
||||
pub fn get_program_accounts(
|
||||
&self,
|
||||
program_id: &Pubkey,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Vec<(String, Account)>> {
|
||||
Ok(self
|
||||
.bank()
|
||||
.bank(commitment)
|
||||
.get_program_accounts(&program_id)
|
||||
.into_iter()
|
||||
.map(|(pubkey, account)| (pubkey.to_string(), account))
|
||||
.collect())
|
||||
}
|
||||
|
||||
pub fn get_inflation(&self) -> Result<Inflation> {
|
||||
Ok(self.bank().inflation())
|
||||
pub fn get_inflation(&self, commitment: Option<CommitmentConfig>) -> Result<Inflation> {
|
||||
Ok(self.bank(commitment).inflation())
|
||||
}
|
||||
|
||||
pub fn get_epoch_schedule(&self) -> Result<EpochSchedule> {
|
||||
Ok(*self.bank().epoch_schedule())
|
||||
// Since epoch schedule data comes from the genesis block, any commitment level should be
|
||||
// fine
|
||||
Ok(*self.bank(None).epoch_schedule())
|
||||
}
|
||||
|
||||
pub fn get_balance(&self, pubkey: &Pubkey) -> u64 {
|
||||
self.bank().get_balance(&pubkey)
|
||||
pub fn get_balance(&self, pubkey: &Pubkey, commitment: Option<CommitmentConfig>) -> u64 {
|
||||
self.bank(commitment).get_balance(&pubkey)
|
||||
}
|
||||
|
||||
fn get_recent_blockhash(&self) -> (String, FeeCalculator) {
|
||||
let (blockhash, fee_calculator) = self.bank().confirmed_last_blockhash();
|
||||
fn get_recent_blockhash(
|
||||
&self,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> (String, FeeCalculator) {
|
||||
let (blockhash, fee_calculator) = self.bank(commitment).confirmed_last_blockhash();
|
||||
(blockhash.to_string(), fee_calculator)
|
||||
}
|
||||
|
||||
fn get_block_commitment(&self, block: u64) -> (Option<BlockCommitment>, u64) {
|
||||
fn get_block_commitment(&self, block: Slot) -> (Option<BlockCommitment>, u64) {
|
||||
let r_block_commitment = self.block_commitment_cache.read().unwrap();
|
||||
(
|
||||
r_block_commitment.get_block_commitment(block).cloned(),
|
||||
@ -124,41 +154,36 @@ impl JsonRpcRequestProcessor {
|
||||
)
|
||||
}
|
||||
|
||||
pub fn get_signature_status(&self, signature: Signature) -> Option<transaction::Result<()>> {
|
||||
self.get_signature_confirmation_status(signature)
|
||||
.map(|x| x.1)
|
||||
}
|
||||
|
||||
pub fn get_signature_confirmations(&self, signature: Signature) -> Option<usize> {
|
||||
self.get_signature_confirmation_status(signature)
|
||||
.map(|x| x.0)
|
||||
}
|
||||
|
||||
pub fn get_signature_confirmation_status(
|
||||
&self,
|
||||
signature: Signature,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Option<(usize, transaction::Result<()>)> {
|
||||
self.bank().get_signature_confirmation_status(&signature)
|
||||
self.bank(commitment)
|
||||
.get_signature_confirmation_status(&signature)
|
||||
}
|
||||
|
||||
fn get_slot(&self) -> Result<u64> {
|
||||
Ok(self.bank().slot())
|
||||
fn get_slot(&self, commitment: Option<CommitmentConfig>) -> Result<u64> {
|
||||
Ok(self.bank(commitment).slot())
|
||||
}
|
||||
|
||||
fn get_slot_leader(&self) -> Result<String> {
|
||||
Ok(self.bank().collector_id().to_string())
|
||||
fn get_slot_leader(&self, commitment: Option<CommitmentConfig>) -> Result<String> {
|
||||
Ok(self.bank(commitment).collector_id().to_string())
|
||||
}
|
||||
|
||||
fn get_transaction_count(&self) -> Result<u64> {
|
||||
Ok(self.bank().transaction_count() as u64)
|
||||
fn get_transaction_count(&self, commitment: Option<CommitmentConfig>) -> Result<u64> {
|
||||
Ok(self.bank(commitment).transaction_count() as u64)
|
||||
}
|
||||
|
||||
fn get_total_supply(&self) -> Result<u64> {
|
||||
Ok(self.bank().capitalization())
|
||||
fn get_total_supply(&self, commitment: Option<CommitmentConfig>) -> Result<u64> {
|
||||
Ok(self.bank(commitment).capitalization())
|
||||
}
|
||||
|
||||
fn get_vote_accounts(&self) -> Result<RpcVoteAccountStatus> {
|
||||
let bank = self.bank();
|
||||
fn get_vote_accounts(
|
||||
&self,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<RpcVoteAccountStatus> {
|
||||
let bank = self.bank(commitment);
|
||||
let vote_accounts = bank.vote_accounts();
|
||||
let epoch_vote_accounts = bank
|
||||
.epoch_vote_accounts(bank.get_epoch_and_slot_index(bank.slot()).0)
|
||||
@ -212,8 +237,8 @@ impl JsonRpcRequestProcessor {
|
||||
))
|
||||
}
|
||||
|
||||
fn get_slots_per_segment(&self) -> Result<u64> {
|
||||
Ok(self.bank().slots_per_segment())
|
||||
fn get_slots_per_segment(&self, commitment: Option<CommitmentConfig>) -> Result<u64> {
|
||||
Ok(self.bank(commitment).slots_per_segment())
|
||||
}
|
||||
|
||||
fn get_storage_pubkeys_for_slot(&self, slot: Slot) -> Result<Vec<Pubkey>> {
|
||||
@ -281,139 +306,224 @@ pub trait RpcSol {
|
||||
type Metadata;
|
||||
|
||||
#[rpc(meta, name = "confirmTransaction")]
|
||||
fn confirm_transaction(&self, _: Self::Metadata, _: String) -> Result<bool>;
|
||||
fn confirm_transaction(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<bool>;
|
||||
|
||||
#[rpc(meta, name = "getAccountInfo")]
|
||||
fn get_account_info(&self, _: Self::Metadata, _: String) -> Result<Account>;
|
||||
fn get_account_info(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
pubkey_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Account>;
|
||||
|
||||
#[rpc(meta, name = "getProgramAccounts")]
|
||||
fn get_program_accounts(&self, _: Self::Metadata, _: String) -> Result<Vec<(String, Account)>>;
|
||||
fn get_program_accounts(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
program_id_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Vec<(String, Account)>>;
|
||||
|
||||
#[rpc(meta, name = "getMinimumBalanceForRentExemption")]
|
||||
fn get_minimum_balance_for_rent_exemption(&self, _: Self::Metadata, _: usize) -> Result<u64>;
|
||||
fn get_minimum_balance_for_rent_exemption(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
data_len: usize,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getInflation")]
|
||||
fn get_inflation(&self, _: Self::Metadata) -> Result<Inflation>;
|
||||
fn get_inflation(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Inflation>;
|
||||
|
||||
#[rpc(meta, name = "getEpochSchedule")]
|
||||
fn get_epoch_schedule(&self, _: Self::Metadata) -> Result<EpochSchedule>;
|
||||
fn get_epoch_schedule(&self, meta: Self::Metadata) -> Result<EpochSchedule>;
|
||||
|
||||
#[rpc(meta, name = "getBalance")]
|
||||
fn get_balance(&self, _: Self::Metadata, _: String) -> Result<u64>;
|
||||
fn get_balance(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
pubkey_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getClusterNodes")]
|
||||
fn get_cluster_nodes(&self, _: Self::Metadata) -> Result<Vec<RpcContactInfo>>;
|
||||
fn get_cluster_nodes(&self, meta: Self::Metadata) -> Result<Vec<RpcContactInfo>>;
|
||||
|
||||
#[rpc(meta, name = "getEpochInfo")]
|
||||
fn get_epoch_info(&self, _: Self::Metadata) -> Result<RpcEpochInfo>;
|
||||
fn get_epoch_info(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<RpcEpochInfo>;
|
||||
|
||||
#[rpc(meta, name = "getBlockCommitment")]
|
||||
fn get_block_commitment(
|
||||
&self,
|
||||
_: Self::Metadata,
|
||||
_: u64,
|
||||
meta: Self::Metadata,
|
||||
block: u64,
|
||||
) -> Result<(Option<BlockCommitment>, u64)>;
|
||||
|
||||
#[rpc(meta, name = "getGenesisBlockhash")]
|
||||
fn get_genesis_blockhash(&self, _: Self::Metadata) -> Result<String>;
|
||||
fn get_genesis_blockhash(&self, meta: Self::Metadata) -> Result<String>;
|
||||
|
||||
#[rpc(meta, name = "getLeaderSchedule")]
|
||||
fn get_leader_schedule(&self, _: Self::Metadata) -> Result<Option<Vec<String>>>;
|
||||
fn get_leader_schedule(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<Vec<String>>>;
|
||||
|
||||
#[rpc(meta, name = "getRecentBlockhash")]
|
||||
fn get_recent_blockhash(&self, _: Self::Metadata) -> Result<(String, FeeCalculator)>;
|
||||
fn get_recent_blockhash(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<(String, FeeCalculator)>;
|
||||
|
||||
#[rpc(meta, name = "getSignatureStatus")]
|
||||
fn get_signature_status(
|
||||
&self,
|
||||
_: Self::Metadata,
|
||||
_: String,
|
||||
meta: Self::Metadata,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<transaction::Result<()>>>;
|
||||
|
||||
#[rpc(meta, name = "getSlot")]
|
||||
fn get_slot(&self, _: Self::Metadata) -> Result<u64>;
|
||||
fn get_slot(&self, meta: Self::Metadata, commitment: Option<CommitmentConfig>) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getTransactionCount")]
|
||||
fn get_transaction_count(&self, _: Self::Metadata) -> Result<u64>;
|
||||
fn get_transaction_count(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getTotalSupply")]
|
||||
fn get_total_supply(&self, _: Self::Metadata) -> Result<u64>;
|
||||
fn get_total_supply(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "requestAirdrop")]
|
||||
fn request_airdrop(&self, _: Self::Metadata, _: String, _: u64) -> Result<String>;
|
||||
fn request_airdrop(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
pubkey_str: String,
|
||||
lamports: u64,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<String>;
|
||||
|
||||
#[rpc(meta, name = "sendTransaction")]
|
||||
fn send_transaction(&self, _: Self::Metadata, _: Vec<u8>) -> Result<String>;
|
||||
fn send_transaction(&self, meta: Self::Metadata, data: Vec<u8>) -> Result<String>;
|
||||
|
||||
#[rpc(meta, name = "getSlotLeader")]
|
||||
fn get_slot_leader(&self, _: Self::Metadata) -> Result<String>;
|
||||
fn get_slot_leader(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<String>;
|
||||
|
||||
#[rpc(meta, name = "getVoteAccounts")]
|
||||
fn get_vote_accounts(&self, _: Self::Metadata) -> Result<RpcVoteAccountStatus>;
|
||||
fn get_vote_accounts(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<RpcVoteAccountStatus>;
|
||||
|
||||
#[rpc(meta, name = "getStorageTurnRate")]
|
||||
fn get_storage_turn_rate(&self, _: Self::Metadata) -> Result<u64>;
|
||||
fn get_storage_turn_rate(&self, meta: Self::Metadata) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getStorageTurn")]
|
||||
fn get_storage_turn(&self, _: Self::Metadata) -> Result<(String, u64)>;
|
||||
fn get_storage_turn(&self, meta: Self::Metadata) -> Result<(String, u64)>;
|
||||
|
||||
#[rpc(meta, name = "getSlotsPerSegment")]
|
||||
fn get_slots_per_segment(&self, _: Self::Metadata) -> Result<u64>;
|
||||
fn get_slots_per_segment(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64>;
|
||||
|
||||
#[rpc(meta, name = "getStoragePubkeysForSlot")]
|
||||
fn get_storage_pubkeys_for_slot(&self, _: Self::Metadata, _: u64) -> Result<Vec<Pubkey>>;
|
||||
fn get_storage_pubkeys_for_slot(&self, meta: Self::Metadata, slot: u64) -> Result<Vec<Pubkey>>;
|
||||
|
||||
#[rpc(meta, name = "validatorExit")]
|
||||
fn validator_exit(&self, _: Self::Metadata) -> Result<bool>;
|
||||
fn validator_exit(&self, meta: Self::Metadata) -> Result<bool>;
|
||||
|
||||
#[rpc(meta, name = "getNumBlocksSinceSignatureConfirmation")]
|
||||
fn get_num_blocks_since_signature_confirmation(
|
||||
&self,
|
||||
_: Self::Metadata,
|
||||
_: String,
|
||||
meta: Self::Metadata,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<usize>>;
|
||||
|
||||
#[rpc(meta, name = "getSignatureConfirmation")]
|
||||
fn get_signature_confirmation(
|
||||
&self,
|
||||
_: Self::Metadata,
|
||||
_: String,
|
||||
meta: Self::Metadata,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<(usize, transaction::Result<()>)>>;
|
||||
|
||||
#[rpc(meta, name = "getVersion")]
|
||||
fn get_version(&self, _: Self::Metadata) -> Result<RpcVersionInfo>;
|
||||
fn get_version(&self, meta: Self::Metadata) -> Result<RpcVersionInfo>;
|
||||
|
||||
#[rpc(meta, name = "setLogFilter")]
|
||||
fn set_log_filter(&self, _: Self::Metadata, _: String) -> Result<()>;
|
||||
fn set_log_filter(&self, _meta: Self::Metadata, filter: String) -> Result<()>;
|
||||
}
|
||||
|
||||
pub struct RpcSolImpl;
|
||||
impl RpcSol for RpcSolImpl {
|
||||
type Metadata = Meta;
|
||||
|
||||
fn confirm_transaction(&self, meta: Self::Metadata, id: String) -> Result<bool> {
|
||||
debug!("confirm_transaction rpc request received: {:?}", id);
|
||||
self.get_signature_status(meta, id).map(|status_option| {
|
||||
if status_option.is_none() {
|
||||
return false;
|
||||
}
|
||||
status_option.unwrap().is_ok()
|
||||
})
|
||||
fn confirm_transaction(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<bool> {
|
||||
debug!(
|
||||
"confirm_transaction rpc request received: {:?}",
|
||||
signature_str
|
||||
);
|
||||
self.get_signature_status(meta, signature_str, commitment)
|
||||
.map(|status_option| {
|
||||
if status_option.is_none() {
|
||||
return false;
|
||||
}
|
||||
status_option.unwrap().is_ok()
|
||||
})
|
||||
}
|
||||
|
||||
fn get_account_info(&self, meta: Self::Metadata, id: String) -> Result<Account> {
|
||||
debug!("get_account_info rpc request received: {:?}", id);
|
||||
let pubkey = verify_pubkey(id)?;
|
||||
fn get_account_info(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
pubkey_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Account> {
|
||||
debug!("get_account_info rpc request received: {:?}", pubkey_str);
|
||||
let pubkey = verify_pubkey(pubkey_str)?;
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_account_info(&pubkey)
|
||||
.get_account_info(&pubkey, commitment)
|
||||
}
|
||||
|
||||
fn get_minimum_balance_for_rent_exemption(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
data_len: usize,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64> {
|
||||
debug!(
|
||||
"get_minimum_balance_for_rent_exemption rpc request received: {:?}",
|
||||
@ -422,29 +532,37 @@ impl RpcSol for RpcSolImpl {
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_minimum_balance_for_rent_exemption(data_len)
|
||||
.get_minimum_balance_for_rent_exemption(data_len, commitment)
|
||||
}
|
||||
|
||||
fn get_program_accounts(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
id: String,
|
||||
program_id_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Vec<(String, Account)>> {
|
||||
debug!("get_program_accounts rpc request received: {:?}", id);
|
||||
let program_id = verify_pubkey(id)?;
|
||||
debug!(
|
||||
"get_program_accounts rpc request received: {:?}",
|
||||
program_id_str
|
||||
);
|
||||
let program_id = verify_pubkey(program_id_str)?;
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_program_accounts(&program_id)
|
||||
.get_program_accounts(&program_id, commitment)
|
||||
}
|
||||
|
||||
fn get_inflation(&self, meta: Self::Metadata) -> Result<Inflation> {
|
||||
fn get_inflation(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Inflation> {
|
||||
debug!("get_inflation rpc request received");
|
||||
Ok(meta
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_inflation()
|
||||
.get_inflation(commitment)
|
||||
.unwrap())
|
||||
}
|
||||
|
||||
@ -458,10 +576,19 @@ impl RpcSol for RpcSolImpl {
|
||||
.unwrap())
|
||||
}
|
||||
|
||||
fn get_balance(&self, meta: Self::Metadata, id: String) -> Result<u64> {
|
||||
debug!("get_balance rpc request received: {:?}", id);
|
||||
let pubkey = verify_pubkey(id)?;
|
||||
Ok(meta.request_processor.read().unwrap().get_balance(&pubkey))
|
||||
fn get_balance(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
pubkey_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64> {
|
||||
debug!("get_balance rpc request received: {:?}", pubkey_str);
|
||||
let pubkey = verify_pubkey(pubkey_str)?;
|
||||
Ok(meta
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_balance(&pubkey, commitment))
|
||||
}
|
||||
|
||||
fn get_cluster_nodes(&self, meta: Self::Metadata) -> Result<Vec<RpcContactInfo>> {
|
||||
@ -491,8 +618,12 @@ impl RpcSol for RpcSolImpl {
|
||||
.collect())
|
||||
}
|
||||
|
||||
fn get_epoch_info(&self, meta: Self::Metadata) -> Result<RpcEpochInfo> {
|
||||
let bank = meta.request_processor.read().unwrap().bank();
|
||||
fn get_epoch_info(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<RpcEpochInfo> {
|
||||
let bank = meta.request_processor.read().unwrap().bank(commitment);
|
||||
let epoch_schedule = bank.epoch_schedule();
|
||||
let (epoch, slot_index) = epoch_schedule.get_epoch_and_slot_index(bank.slot());
|
||||
let slot = bank.slot();
|
||||
@ -507,7 +638,7 @@ impl RpcSol for RpcSolImpl {
|
||||
fn get_block_commitment(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
block: u64,
|
||||
block: Slot,
|
||||
) -> Result<(Option<BlockCommitment>, u64)> {
|
||||
Ok(meta
|
||||
.request_processor
|
||||
@ -521,8 +652,12 @@ impl RpcSol for RpcSolImpl {
|
||||
Ok(meta.genesis_blockhash.to_string())
|
||||
}
|
||||
|
||||
fn get_leader_schedule(&self, meta: Self::Metadata) -> Result<Option<Vec<String>>> {
|
||||
let bank = meta.request_processor.read().unwrap().bank();
|
||||
fn get_leader_schedule(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<Vec<String>>> {
|
||||
let bank = meta.request_processor.read().unwrap().bank(commitment);
|
||||
Ok(
|
||||
solana_ledger::leader_schedule_utils::leader_schedule(bank.epoch(), &bank).map(
|
||||
|leader_schedule| {
|
||||
@ -536,66 +671,93 @@ impl RpcSol for RpcSolImpl {
|
||||
)
|
||||
}
|
||||
|
||||
fn get_recent_blockhash(&self, meta: Self::Metadata) -> Result<(String, FeeCalculator)> {
|
||||
fn get_recent_blockhash(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<(String, FeeCalculator)> {
|
||||
debug!("get_recent_blockhash rpc request received");
|
||||
Ok(meta
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_recent_blockhash())
|
||||
.get_recent_blockhash(commitment))
|
||||
}
|
||||
|
||||
fn get_signature_status(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
id: String,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<transaction::Result<()>>> {
|
||||
self.get_signature_confirmation(meta, id)
|
||||
self.get_signature_confirmation(meta, signature_str, commitment)
|
||||
.map(|res| res.map(|x| x.1))
|
||||
}
|
||||
|
||||
fn get_slot(&self, meta: Self::Metadata) -> Result<u64> {
|
||||
meta.request_processor.read().unwrap().get_slot()
|
||||
fn get_slot(&self, meta: Self::Metadata, commitment: Option<CommitmentConfig>) -> Result<u64> {
|
||||
meta.request_processor.read().unwrap().get_slot(commitment)
|
||||
}
|
||||
|
||||
fn get_num_blocks_since_signature_confirmation(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
id: String,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<usize>> {
|
||||
self.get_signature_confirmation(meta, id)
|
||||
self.get_signature_confirmation(meta, signature_str, commitment)
|
||||
.map(|res| res.map(|x| x.0))
|
||||
}
|
||||
|
||||
fn get_signature_confirmation(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
id: String,
|
||||
signature_str: String,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<Option<(usize, transaction::Result<()>)>> {
|
||||
debug!("get_signature_confirmation rpc request received: {:?}", id);
|
||||
let signature = verify_signature(&id)?;
|
||||
debug!(
|
||||
"get_signature_confirmation rpc request received: {:?}",
|
||||
signature_str
|
||||
);
|
||||
let signature = verify_signature(&signature_str)?;
|
||||
Ok(meta
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_signature_confirmation_status(signature))
|
||||
.get_signature_confirmation_status(signature, commitment))
|
||||
}
|
||||
|
||||
fn get_transaction_count(&self, meta: Self::Metadata) -> Result<u64> {
|
||||
fn get_transaction_count(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64> {
|
||||
debug!("get_transaction_count rpc request received");
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_transaction_count()
|
||||
.get_transaction_count(commitment)
|
||||
}
|
||||
|
||||
fn get_total_supply(&self, meta: Self::Metadata) -> Result<u64> {
|
||||
fn get_total_supply(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64> {
|
||||
debug!("get_total_supply rpc request received");
|
||||
meta.request_processor.read().unwrap().get_total_supply()
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_total_supply(commitment)
|
||||
}
|
||||
|
||||
fn request_airdrop(&self, meta: Self::Metadata, id: String, lamports: u64) -> Result<String> {
|
||||
trace!("request_airdrop id={} lamports={}", id, lamports);
|
||||
fn request_airdrop(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
pubkey_str: String,
|
||||
lamports: u64,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<String> {
|
||||
trace!("request_airdrop id={} lamports={}", pubkey_str, lamports);
|
||||
|
||||
let drone_addr = meta
|
||||
.request_processor
|
||||
@ -604,13 +766,13 @@ impl RpcSol for RpcSolImpl {
|
||||
.config
|
||||
.drone_addr
|
||||
.ok_or_else(Error::invalid_request)?;
|
||||
let pubkey = verify_pubkey(id)?;
|
||||
let pubkey = verify_pubkey(pubkey_str)?;
|
||||
|
||||
let blockhash = meta
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.bank()
|
||||
.bank(commitment.clone())
|
||||
.confirmed_last_blockhash()
|
||||
.0;
|
||||
let transaction = request_airdrop_transaction(&drone_addr, &pubkey, lamports, blockhash)
|
||||
@ -641,7 +803,8 @@ impl RpcSol for RpcSolImpl {
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_signature_status(signature);
|
||||
.get_signature_confirmation_status(signature, commitment.clone())
|
||||
.map(|x| x.1);
|
||||
|
||||
if signature_status == Some(Ok(())) {
|
||||
info!("airdrop signature ok");
|
||||
@ -689,12 +852,26 @@ impl RpcSol for RpcSolImpl {
|
||||
Ok(signature)
|
||||
}
|
||||
|
||||
fn get_slot_leader(&self, meta: Self::Metadata) -> Result<String> {
|
||||
meta.request_processor.read().unwrap().get_slot_leader()
|
||||
fn get_slot_leader(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<String> {
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_slot_leader(commitment)
|
||||
}
|
||||
|
||||
fn get_vote_accounts(&self, meta: Self::Metadata) -> Result<RpcVoteAccountStatus> {
|
||||
meta.request_processor.read().unwrap().get_vote_accounts()
|
||||
fn get_vote_accounts(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<RpcVoteAccountStatus> {
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_vote_accounts(commitment)
|
||||
}
|
||||
|
||||
fn get_storage_turn_rate(&self, meta: Self::Metadata) -> Result<u64> {
|
||||
@ -708,11 +885,15 @@ impl RpcSol for RpcSolImpl {
|
||||
meta.request_processor.read().unwrap().get_storage_turn()
|
||||
}
|
||||
|
||||
fn get_slots_per_segment(&self, meta: Self::Metadata) -> Result<u64> {
|
||||
fn get_slots_per_segment(
|
||||
&self,
|
||||
meta: Self::Metadata,
|
||||
commitment: Option<CommitmentConfig>,
|
||||
) -> Result<u64> {
|
||||
meta.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_slots_per_segment()
|
||||
.get_slots_per_segment(commitment)
|
||||
}
|
||||
|
||||
fn get_storage_pubkeys_for_slot(
|
||||
@ -736,7 +917,7 @@ impl RpcSol for RpcSolImpl {
|
||||
})
|
||||
}
|
||||
|
||||
fn set_log_filter(&self, _: Self::Metadata, filter: String) -> Result<()> {
|
||||
fn set_log_filter(&self, _meta: Self::Metadata, filter: String) -> Result<()> {
|
||||
solana_logger::setup_with_filter(&filter);
|
||||
Ok(())
|
||||
}
|
||||
@ -863,7 +1044,7 @@ pub mod tests {
|
||||
})
|
||||
.join()
|
||||
.unwrap();
|
||||
assert_eq!(request_processor.get_transaction_count().unwrap(), 1);
|
||||
assert_eq!(request_processor.get_transaction_count(None).unwrap(), 1);
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
@ -221,7 +221,7 @@ mod tests {
|
||||
.request_processor
|
||||
.read()
|
||||
.unwrap()
|
||||
.get_balance(&mint_keypair.pubkey())
|
||||
.get_balance(&mint_keypair.pubkey(), None)
|
||||
);
|
||||
rpc_service.exit();
|
||||
rpc_service.join().unwrap();
|
||||
|
@ -511,7 +511,7 @@ pub fn new_validator_for_tests() -> (Validator, ContactInfo, Keypair, PathBuf) {
|
||||
let GenesisBlockInfo {
|
||||
mut genesis_block,
|
||||
mint_keypair,
|
||||
..
|
||||
voting_keypair,
|
||||
} = create_genesis_block_with_leader(10_000, &contact_info.id, 42);
|
||||
genesis_block
|
||||
.native_instruction_processors
|
||||
@ -522,14 +522,14 @@ pub fn new_validator_for_tests() -> (Validator, ContactInfo, Keypair, PathBuf) {
|
||||
|
||||
let (ledger_path, _blockhash) = create_new_tmp_ledger!(&genesis_block);
|
||||
|
||||
let voting_keypair = Arc::new(Keypair::new());
|
||||
let leader_voting_keypair = Arc::new(voting_keypair);
|
||||
let storage_keypair = Arc::new(Keypair::new());
|
||||
let node = Validator::new(
|
||||
node,
|
||||
&node_keypair,
|
||||
&ledger_path,
|
||||
&voting_keypair.pubkey(),
|
||||
&voting_keypair,
|
||||
&leader_voting_keypair.pubkey(),
|
||||
&leader_voting_keypair,
|
||||
&storage_keypair,
|
||||
None,
|
||||
true,
|
||||
|
Reference in New Issue
Block a user