diff --git a/client/src/rpc_request.rs b/client/src/rpc_request.rs index 321858c1bc..43240b17b0 100644 --- a/client/src/rpc_request.rs +++ b/client/src/rpc_request.rs @@ -17,6 +17,7 @@ pub enum RpcRequest { GetEpochVoteAccounts, GetStorageBlockhash, GetStorageSlot, + GetSlotsPerSegment, GetStoragePubkeysForSlot, GetTransactionCount, RegisterNode, @@ -45,6 +46,7 @@ impl RpcRequest { RpcRequest::GetEpochVoteAccounts => "getEpochVoteAccounts", RpcRequest::GetStorageBlockhash => "getStorageBlockhash", RpcRequest::GetStorageSlot => "getStorageSlot", + RpcRequest::GetSlotsPerSegment => "getSlotsPerSegment", RpcRequest::GetStoragePubkeysForSlot => "getStoragePubkeysForSlot", RpcRequest::GetTransactionCount => "getTransactionCount", RpcRequest::RegisterNode => "registerNode", diff --git a/core/src/chacha.rs b/core/src/chacha.rs index eb782e95aa..7352334baa 100644 --- a/core/src/chacha.rs +++ b/core/src/chacha.rs @@ -1,5 +1,4 @@ use crate::blocktree::Blocktree; -use solana_storage_api::SLOTS_PER_SEGMENT; use std::fs::File; use std::io; use std::io::{BufWriter, Write}; @@ -14,6 +13,7 @@ pub const CHACHA_KEY_SIZE: usize = 32; pub fn chacha_cbc_encrypt_ledger( blocktree: &Arc, slice: u64, + slots_per_segment: u64, out_path: &Path, ivec: &mut [u8; CHACHA_BLOCK_SIZE], ) -> io::Result { @@ -28,7 +28,7 @@ pub fn chacha_cbc_encrypt_ledger( let mut entry = slice; loop { - match blocktree.read_blobs_bytes(0, SLOTS_PER_SEGMENT - total_entries, &mut buffer, entry) { + match blocktree.read_blobs_bytes(0, slots_per_segment - total_entries, &mut buffer, entry) { Ok((num_entries, entry_len)) => { debug!( "chacha: encrypting slice: {} num_entries: {} entry_len: {}", @@ -113,10 +113,11 @@ mod tests { let ledger_dir = "chacha_test_encrypt_file"; let ledger_path = get_tmp_ledger_path(ledger_dir); let ticks_per_slot = 16; + let slots_per_segment = 32; let blocktree = Arc::new(Blocktree::open(&ledger_path).unwrap()); let out_path = Path::new("test_chacha_encrypt_file_output.txt.enc"); - let entries = make_tiny_deterministic_test_entries(32); + let entries = make_tiny_deterministic_test_entries(slots_per_segment); blocktree .write_entries(0, 0, 0, ticks_per_slot, &entries) .unwrap(); @@ -125,7 +126,8 @@ mod tests { "abcd1234abcd1234abcd1234abcd1234 abcd1234abcd1234abcd1234abcd1234 abcd1234abcd1234abcd1234abcd1234 abcd1234abcd1234abcd1234abcd1234" ); - chacha_cbc_encrypt_ledger(&blocktree, 0, out_path, &mut key).unwrap(); + chacha_cbc_encrypt_ledger(&blocktree, 0, slots_per_segment as u64, out_path, &mut key) + .unwrap(); let mut out_file = File::open(out_path).unwrap(); let mut buf = vec![]; let size = out_file.read_to_end(&mut buf).unwrap(); @@ -133,7 +135,7 @@ mod tests { hasher.hash(&buf[..size]); // golden needs to be updated if blob stuff changes.... - let golden: Hash = "53P7UXH7JstJa994fZsfuyr7nrRDrDDpvS3WSPvMUs45" + let golden: Hash = "5FzYtpCqL7v6ZxZ1fW4wRkn8TK96NdiD8cLV59Rr7yav" .parse() .unwrap(); diff --git a/core/src/chacha_cuda.rs b/core/src/chacha_cuda.rs index cc6626cff6..731955fd36 100644 --- a/core/src/chacha_cuda.rs +++ b/core/src/chacha_cuda.rs @@ -7,7 +7,6 @@ use crate::sigverify::{ chacha_cbc_encrypt_many_sample, chacha_end_sha_state, chacha_init_sha_state, }; use solana_sdk::hash::Hash; -use solana_storage_api::SLOTS_PER_SEGMENT; use std::io; use std::mem::size_of; use std::sync::Arc; @@ -19,6 +18,7 @@ use std::sync::Arc; pub fn chacha_cbc_encrypt_file_many_keys( blocktree: &Arc, segment: u64, + slots_per_segment: u64, ivecs: &mut [u8], samples: &[u64], ) -> io::Result> { @@ -46,7 +46,7 @@ pub fn chacha_cbc_encrypt_file_many_keys( chacha_init_sha_state(int_sha_states.as_mut_ptr(), num_keys as u32); } loop { - match blocktree.read_blobs_bytes(entry, SLOTS_PER_SEGMENT - total_entries, &mut buffer, 0) { + match blocktree.read_blobs_bytes(entry, slots_per_segment - total_entries, &mut buffer, 0) { Ok((num_entries, entry_len)) => { debug!( "chacha_cuda: encrypting segment: {} num_entries: {} entry_len: {}", @@ -76,9 +76,9 @@ pub fn chacha_cbc_encrypt_file_many_keys( entry += num_entries; debug!( "total entries: {} entry: {} segment: {} entries_per_segment: {}", - total_entries, entry, segment, SLOTS_PER_SEGMENT + total_entries, entry, segment, slots_per_segment ); - if (entry - segment) >= SLOTS_PER_SEGMENT { + if (entry - segment) >= slots_per_segment { break; } } @@ -113,6 +113,7 @@ mod tests { use crate::entry::make_tiny_test_entries; use crate::replicator::sample_file; use solana_sdk::hash::Hash; + use solana_sdk::timing::DEFAULT_SLOTS_PER_SEGMENT; use std::fs::{remove_dir_all, remove_file}; use std::path::Path; use std::sync::Arc; @@ -121,7 +122,8 @@ mod tests { fn test_encrypt_file_many_keys_single() { solana_logger::setup(); - let entries = make_tiny_test_entries(32); + let slots_per_segment = 32; + let entries = make_tiny_test_entries(slots_per_segment); let ledger_dir = "test_encrypt_file_many_keys_single"; let ledger_path = get_tmp_ledger_path(ledger_dir); let ticks_per_slot = 16; @@ -140,12 +142,25 @@ mod tests { ); let mut cpu_iv = ivecs.clone(); - chacha_cbc_encrypt_ledger(&blocktree, 0, out_path, &mut cpu_iv).unwrap(); + chacha_cbc_encrypt_ledger( + &blocktree, + 0, + slots_per_segment as u64, + out_path, + &mut cpu_iv, + ) + .unwrap(); let ref_hash = sample_file(&out_path, &samples).unwrap(); - let hashes = - chacha_cbc_encrypt_file_many_keys(&blocktree, 0, &mut ivecs, &samples).unwrap(); + let hashes = chacha_cbc_encrypt_file_many_keys( + &blocktree, + 0, + slots_per_segment as u64, + &mut ivecs, + &samples, + ) + .unwrap(); assert_eq!(hashes[0], ref_hash); @@ -178,7 +193,14 @@ mod tests { ); ivec[0] = i; ivecs.extend(ivec.clone().iter()); - chacha_cbc_encrypt_ledger(&blocktree.clone(), 0, out_path, &mut ivec).unwrap(); + chacha_cbc_encrypt_ledger( + &blocktree.clone(), + 0, + DEFAULT_SLOTS_PER_SEGMENT, + out_path, + &mut ivec, + ) + .unwrap(); ref_hashes.push(sample_file(&out_path, &samples).unwrap()); info!( @@ -189,8 +211,14 @@ mod tests { ); } - let hashes = - chacha_cbc_encrypt_file_many_keys(&blocktree, 0, &mut ivecs, &samples).unwrap(); + let hashes = chacha_cbc_encrypt_file_many_keys( + &blocktree, + 0, + DEFAULT_SLOTS_PER_SEGMENT, + &mut ivecs, + &samples, + ) + .unwrap(); assert_eq!(hashes, ref_hashes); @@ -205,6 +233,13 @@ mod tests { let ledger_path = get_tmp_ledger_path(ledger_dir); let samples = [0]; let blocktree = Arc::new(Blocktree::open(&ledger_path).unwrap()); - assert!(chacha_cbc_encrypt_file_many_keys(&blocktree, 0, &mut keys, &samples,).is_err()); + assert!(chacha_cbc_encrypt_file_many_keys( + &blocktree, + 0, + DEFAULT_SLOTS_PER_SEGMENT, + &mut keys, + &samples, + ) + .is_err()); } } diff --git a/core/src/replicator.rs b/core/src/replicator.rs index 6aad93209f..6229f73e53 100644 --- a/core/src/replicator.rs +++ b/core/src/replicator.rs @@ -28,11 +28,11 @@ use solana_sdk::client::{AsyncClient, SyncClient}; use solana_sdk::hash::{Hash, Hasher}; use solana_sdk::message::Message; use solana_sdk::signature::{Keypair, KeypairUtil, Signature}; -use solana_sdk::timing::timestamp; +use solana_sdk::timing::{get_segment_from_slot, timestamp}; use solana_sdk::transaction::Transaction; use solana_sdk::transport::TransportError; use solana_storage_api::storage_contract::StorageContract; -use solana_storage_api::{get_segment_from_slot, storage_instruction, SLOTS_PER_SEGMENT}; +use solana_storage_api::storage_instruction; use std::fs::File; use std::io::{self, BufReader, ErrorKind, Read, Seek, SeekFrom}; use std::mem::size_of; @@ -59,6 +59,7 @@ pub struct Replicator { #[derive(Default)] struct ReplicatorMeta { slot: u64, + slots_per_segment: u64, ledger_path: String, signature: Signature, ledger_data_file_encrypted: PathBuf, @@ -103,15 +104,19 @@ pub(crate) fn sample_file(in_path: &Path, sample_offsets: &[u64]) -> io::Result< Ok(hasher.result()) } -fn get_slot_from_blockhash(signature: &ed25519_dalek::Signature, storage_slot: u64) -> u64 { +fn get_slot_from_blockhash( + signature: &ed25519_dalek::Signature, + storage_turn: u64, + slots_per_segment: u64, +) -> u64 { let signature_vec = signature.to_bytes(); let mut segment_index = u64::from(signature_vec[0]) | (u64::from(signature_vec[1]) << 8) | (u64::from(signature_vec[1]) << 16) | (u64::from(signature_vec[2]) << 24); - let max_segment_index = get_segment_from_slot(storage_slot); + let max_segment_index = get_segment_from_slot(storage_turn, slots_per_segment); segment_index %= max_segment_index as u64; - segment_index * SLOTS_PER_SEGMENT + segment_index * slots_per_segment } fn create_request_processor( @@ -333,17 +338,20 @@ impl Replicator { // TODO make this a lot more frequent by picking a "new" blockhash instead of picking a storage blockhash // prep the next proof - let (storage_blockhash, _) = - match Self::poll_for_blockhash_and_slot(&cluster_info, &meta.blockhash) { - Ok(blockhash_and_slot) => blockhash_and_slot, - Err(e) => { - warn!( - "Error couldn't get a newer blockhash than {:?}. {:?}", - meta.blockhash, e - ); - break; - } - }; + let (storage_blockhash, _) = match Self::poll_for_blockhash_and_slot( + &cluster_info, + meta.slots_per_segment, + &meta.blockhash, + ) { + Ok(blockhash_and_slot) => blockhash_and_slot, + Err(e) => { + warn!( + "Error couldn't get a newer blockhash than {:?}. {:?}", + meta.blockhash, e + ); + break; + } + }; meta.blockhash = storage_blockhash; Self::redeem_rewards(&cluster_info, replicator_keypair, storage_keypair); } @@ -393,26 +401,39 @@ impl Replicator { blob_fetch_receiver: BlobReceiver, slot_sender: Sender, ) -> Result<(WindowService)> { - let (storage_blockhash, storage_slot) = - match Self::poll_for_blockhash_and_slot(&cluster_info, &Hash::default()) { - Ok(blockhash_and_slot) => blockhash_and_slot, - Err(e) => { - //shutdown services before exiting - exit.store(true, Ordering::Relaxed); - return Err(e); - } - }; - + 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 (storage_blockhash, storage_slot) = match Self::poll_for_blockhash_and_slot( + &cluster_info, + slots_per_segment, + &Hash::default(), + ) { + Ok(blockhash_and_slot) => blockhash_and_slot, + Err(e) => { + error!("unable to get turn status, exiting..."); + //shutdown services before exiting + exit.store(true, Ordering::Relaxed); + return Err(e); + } + }; let signature = storage_keypair.sign(storage_blockhash.as_ref()); - let slot = get_slot_from_blockhash(&signature, storage_slot); + let slot = get_slot_from_blockhash(&signature, storage_slot, slots_per_segment); info!("replicating slot: {}", slot); slot_sender.send(slot)?; meta.slot = slot; + meta.slots_per_segment = slots_per_segment; meta.signature = Signature::new(&signature.to_bytes()); meta.blockhash = storage_blockhash; let mut repair_slot_range = RepairSlotRange::default(); - repair_slot_range.end = slot + SLOTS_PER_SEGMENT; + repair_slot_range.end = slot + slots_per_segment; repair_slot_range.start = slot; let (retransmit_sender, _) = channel(); @@ -428,12 +449,20 @@ impl Replicator { |_, _, _| true, ); info!("waiting for ledger download"); - Self::wait_for_segment_download(slot, &blocktree, &exit, &node_info, cluster_info); + Self::wait_for_segment_download( + slot, + slots_per_segment, + &blocktree, + &exit, + &node_info, + cluster_info, + ); Ok(window_service) } fn wait_for_segment_download( start_slot: u64, + slots_per_segment: u64, blocktree: &Arc, exit: &Arc, node_info: &ContactInfo, @@ -448,7 +477,7 @@ impl Replicator { while blocktree.is_full(current_slot) { current_slot += 1; info!("current slot: {}", current_slot); - if current_slot >= start_slot + SLOTS_PER_SEGMENT { + if current_slot >= start_slot + slots_per_segment { break 'outer; } } @@ -481,6 +510,7 @@ impl Replicator { let num_encrypted_bytes = chacha_cbc_encrypt_ledger( blocktree, meta.slot, + meta.slots_per_segment, &meta.ledger_data_file_encrypted, &mut ivec, )?; @@ -593,7 +623,7 @@ impl Replicator { let instruction = storage_instruction::mining_proof( &storage_keypair.pubkey(), meta.sha_state, - get_segment_from_slot(meta.slot), + get_segment_from_slot(meta.slot, meta.slots_per_segment), Signature::new(&meta.signature.as_ref()), meta.blockhash, ); @@ -625,9 +655,37 @@ impl Replicator { } } + fn get_segment_config(cluster_info: &Arc>) -> result::Result { + let rpc_peers = { + let cluster_info = cluster_info.read().unwrap(); + cluster_info.rpc_peers() + }; + debug!("rpc peers: {:?}", rpc_peers); + if !rpc_peers.is_empty() { + let rpc_client = { + let node_index = thread_rng().gen_range(0, rpc_peers.len()); + RpcClient::new_socket(rpc_peers[node_index].rpc) + }; + Ok(rpc_client + .retry_make_rpc_request(&RpcRequest::GetSlotsPerSegment, None, 0) + .map_err(|err| { + warn!("Error while making rpc request {:?}", err); + Error::IO(io::Error::new(ErrorKind::Other, "rpc error")) + })? + .as_u64() + .unwrap()) + } else { + Err(io::Error::new( + io::ErrorKind::Other, + "No RPC peers...".to_string(), + ))? + } + } + /// Poll for a different blockhash and associated max_slot than `previous_blockhash` fn poll_for_blockhash_and_slot( cluster_info: &Arc>, + slots_per_segment: u64, previous_blockhash: &Hash, ) -> result::Result<(Hash, u64), Error> { for _ in 0..10 { @@ -673,7 +731,7 @@ impl Replicator { .as_u64() .unwrap(); info!("storage slot: {}", storage_slot); - if get_segment_from_slot(storage_slot) != 0 { + if get_segment_from_slot(storage_slot, slots_per_segment) != 0 { return Ok((storage_blockhash, storage_slot)); } } @@ -696,6 +754,7 @@ impl Replicator { cluster_info: &Arc>, replicator_info: &ContactInfo, blocktree: &Arc, + slots_per_segment: u64, ) -> Result<(u64)> { // Create a client which downloads from the replicator and see that it // can respond with blobs. @@ -714,7 +773,7 @@ impl Replicator { ); let repair_slot_range = RepairSlotRange { start: start_slot, - end: start_slot + SLOTS_PER_SEGMENT, + end: start_slot + slots_per_segment, }; // try for upto 180 seconds //TODO needs tuning if segments are huge for _ in 0..120 { @@ -761,7 +820,7 @@ impl Replicator { window_service::process_blobs(&blobs, blocktree)?; } // check if all the slots in the segment are complete - if Self::segment_complete(start_slot, blocktree) { + if Self::segment_complete(start_slot, slots_per_segment, blocktree) { break; } sleep(Duration::from_millis(500)); @@ -770,7 +829,7 @@ impl Replicator { t_receiver.join().unwrap(); // check if all the slots in the segment are complete - if !Self::segment_complete(start_slot, blocktree) { + if !Self::segment_complete(start_slot, slots_per_segment, blocktree) { Err(io::Error::new( ErrorKind::Other, "Unable to download the full segment", @@ -779,8 +838,12 @@ impl Replicator { Ok(start_slot) } - fn segment_complete(start_slot: u64, blocktree: &Arc) -> bool { - for slot in start_slot..(start_slot + SLOTS_PER_SEGMENT) { + fn segment_complete( + start_slot: u64, + slots_per_segment: u64, + blocktree: &Arc, + ) -> bool { + for slot in start_slot..(start_slot + slots_per_segment) { if !blocktree.is_full(slot) { return false; } diff --git a/core/src/rpc.rs b/core/src/rpc.rs index 18b25b0cac..7065b8d610 100644 --- a/core/src/rpc.rs +++ b/core/src/rpc.rs @@ -143,6 +143,10 @@ impl JsonRpcRequestProcessor { Ok(self.storage_state.get_slot()) } + fn get_slots_per_segment(&self) -> Result { + Ok(self.bank().slots_per_segment()) + } + fn get_storage_pubkeys_for_slot(&self, slot: u64) -> Result> { Ok(self .storage_state @@ -265,6 +269,9 @@ pub trait RpcSol { #[rpc(meta, name = "getStorageSlot")] fn get_storage_slot(&self, _: Self::Metadata) -> Result; + #[rpc(meta, name = "getSlotsPerSegment")] + fn get_slots_per_segment(&self, _: Self::Metadata) -> Result; + #[rpc(meta, name = "getStoragePubkeysForSlot")] fn get_storage_pubkeys_for_slot(&self, _: Self::Metadata, _: u64) -> Result>; @@ -531,6 +538,13 @@ impl RpcSol for RpcSolImpl { meta.request_processor.read().unwrap().get_storage_slot() } + fn get_slots_per_segment(&self, meta: Self::Metadata) -> Result { + meta.request_processor + .read() + .unwrap() + .get_slots_per_segment() + } + fn get_storage_pubkeys_for_slot(&self, meta: Self::Metadata, slot: u64) -> Result> { meta.request_processor .read() diff --git a/core/src/storage_stage.rs b/core/src/storage_stage.rs index 63ad0cd1d6..d4ba1ab9a3 100644 --- a/core/src/storage_stage.rs +++ b/core/src/storage_stage.rs @@ -20,10 +20,11 @@ use solana_sdk::instruction::Instruction; use solana_sdk::message::Message; use solana_sdk::pubkey::Pubkey; use solana_sdk::signature::{Keypair, KeypairUtil, Signature}; +use solana_sdk::timing::get_segment_from_slot; use solana_sdk::transaction::Transaction; use solana_storage_api::storage_contract::{Proof, ProofStatus, StorageContract}; +use solana_storage_api::storage_instruction; use solana_storage_api::storage_instruction::proof_validation; -use solana_storage_api::{get_segment_from_slot, storage_instruction}; use std::collections::HashMap; use std::mem::size_of; use std::net::UdpSocket; @@ -47,6 +48,7 @@ pub struct StorageStateInner { replicator_map: ReplicatorMap, storage_blockhash: Hash, slot: u64, + slots_per_segment: u64, } // Used to track root slots in storage stage @@ -86,7 +88,7 @@ fn get_identity_index_from_signature(key: &Signature) -> usize { } impl StorageState { - pub fn new(hash: &Hash) -> Self { + pub fn new(hash: &Hash, slots_per_segment: u64) -> Self { let storage_keys = vec![0u8; KEY_SIZE * NUM_IDENTITIES]; let storage_results = vec![Hash::default(); NUM_IDENTITIES]; let replicator_map = vec![]; @@ -96,6 +98,7 @@ impl StorageState { storage_results, replicator_map, slot: 0, + slots_per_segment, storage_blockhash: *hash, }; @@ -129,7 +132,8 @@ impl StorageState { ) -> Vec { // TODO: keep track of age? const MAX_PUBKEYS_TO_RETURN: usize = 5; - let index = get_segment_from_slot(slot) as usize; + let index = + get_segment_from_slot(slot, self.state.read().unwrap().slots_per_segment) as usize; let replicator_map = &self.state.read().unwrap().replicator_map; let working_bank = bank_forks.read().unwrap().working_bank(); let accounts = replicator_accounts(&working_bank); @@ -356,6 +360,7 @@ impl StorageStage { _blocktree: &Arc, blockhash: Hash, slot: u64, + slots_per_segment: u64, instruction_sender: &InstructionSender, ) -> Result<()> { let mut seed = [0u8; 32]; @@ -364,7 +369,7 @@ impl StorageStage { let ix = storage_instruction::advertise_recent_blockhash( &storage_keypair.pubkey(), blockhash, - slot, + get_segment_from_slot(slot, slots_per_segment), ); instruction_sender.send(ix)?; @@ -379,7 +384,7 @@ impl StorageStage { } // Regenerate the answers - let num_segments = get_segment_from_slot(slot) as usize; + let num_segments = get_segment_from_slot(slot, slots_per_segment) as usize; if num_segments == 0 { info!("Ledger has 0 segments!"); return Ok(()); @@ -412,6 +417,7 @@ impl StorageStage { match chacha_cbc_encrypt_file_many_keys( _blocktree, segment as u64, + statew.slots_per_segment, &mut statew.storage_keys, &samples, ) { @@ -430,6 +436,7 @@ impl StorageStage { fn collect_proofs( slot: u64, + slots_per_segment: u64, account_id: Pubkey, account: Account, storage_state: &Arc>, @@ -437,7 +444,7 @@ impl StorageStage { ) { if let Ok(StorageContract::ReplicatorStorage { proofs, .. }) = account.state() { //convert slot to segment - let segment = get_segment_from_slot(slot); + let segment = get_segment_from_slot(slot, slots_per_segment); if let Some(proofs) = proofs.get(&segment) { for proof in proofs.iter() { { @@ -454,10 +461,12 @@ impl StorageStage { } let mut statew = storage_state.write().unwrap(); - if statew.replicator_map.len() < segment { - statew.replicator_map.resize(segment, HashMap::new()); + if statew.replicator_map.len() < segment as usize { + statew + .replicator_map + .resize(segment as usize, HashMap::new()); } - let proof_segment_index = proof.segment_index; + let proof_segment_index = proof.segment_index as usize; if proof_segment_index < statew.replicator_map.len() { // TODO randomly select and verify the proof first // Copy the submitted proof @@ -503,6 +512,7 @@ impl StorageStage { for (account_id, account) in replicator_accounts.into_iter() { Self::collect_proofs( bank.slot(), + bank.slots_per_segment(), account_id, account, storage_state, @@ -517,10 +527,11 @@ impl StorageStage { &blocktree, bank.last_blockhash(), bank.slot(), + bank.slots_per_segment(), instruction_sender, ); Self::submit_verifications( - get_segment_from_slot(bank.slot()), + get_segment_from_slot(bank.slot(), bank.slots_per_segment()), &storage_state, &storage_keypair, instruction_sender, @@ -532,7 +543,7 @@ impl StorageStage { } fn submit_verifications( - current_segment: usize, + current_segment: u64, storage_state: &Arc>, storage_keypair: &Arc, ix_sender: &Sender, @@ -578,7 +589,7 @@ impl StorageStage { .map(|checked_proofs| { proof_validation( &storage_keypair.pubkey(), - current_segment as u64, + current_segment, checked_proofs.to_vec(), ) }) @@ -626,7 +637,6 @@ mod tests { use solana_sdk::pubkey::Pubkey; use solana_sdk::signature::{Keypair, KeypairUtil}; use solana_sdk::timing::DEFAULT_TICKS_PER_SLOT; - use solana_storage_api::SLOTS_PER_SEGMENT; use std::cmp::{max, min}; use std::fs::remove_dir_all; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; @@ -646,7 +656,7 @@ mod tests { let bank = Arc::new(Bank::new(&genesis_block)); let bank_forks = Arc::new(RwLock::new(BankForks::new_from_banks(&[bank.clone()], 0))); let (_slot_sender, slot_receiver) = channel(); - let storage_state = StorageState::new(&bank.last_blockhash()); + let storage_state = StorageState::new(&bank.last_blockhash(), bank.slots_per_segment()); let storage_stage = StorageStage::new( &storage_state, slot_receiver, @@ -685,7 +695,7 @@ mod tests { let cluster_info = test_cluster_info(&keypair.pubkey()); let (bank_sender, bank_receiver) = channel(); - let storage_state = StorageState::new(&bank.last_blockhash()); + let storage_state = StorageState::new(&bank.last_blockhash(), bank.slots_per_segment()); let storage_stage = StorageStage::new( &storage_state, bank_receiver, @@ -710,7 +720,7 @@ mod tests { assert_eq!(result, Hash::default()); let mut last_bank = bank; - let rooted_banks = (slot..slot + SLOTS_PER_SEGMENT + 1) + let rooted_banks = (slot..slot + last_bank.slots_per_segment() + 1) .map(|i| { let bank = Bank::new_from_parent(&last_bank, &keypair.pubkey(), i); blocktree_processor::process_entries( @@ -774,7 +784,7 @@ mod tests { let cluster_info = test_cluster_info(&keypair.pubkey()); let (bank_sender, bank_receiver) = channel(); - let storage_state = StorageState::new(&bank.last_blockhash()); + let storage_state = StorageState::new(&bank.last_blockhash(), bank.slots_per_segment()); let storage_stage = StorageStage::new( &storage_state, bank_receiver, @@ -827,7 +837,7 @@ mod tests { blocktree_processor::process_entries( &next_bank, &entry::create_ticks( - DEFAULT_TICKS_PER_SLOT * SLOTS_PER_SEGMENT + 1, + DEFAULT_TICKS_PER_SLOT * next_bank.slots_per_segment() + 1, bank.last_blockhash(), ), ) diff --git a/core/src/validator.rs b/core/src/validator.rs index ac15871871..b46f917683 100644 --- a/core/src/validator.rs +++ b/core/src/validator.rs @@ -23,8 +23,7 @@ use solana_sdk::genesis_block::GenesisBlock; use solana_sdk::poh_config::PohConfig; use solana_sdk::pubkey::Pubkey; use solana_sdk::signature::{Keypair, KeypairUtil}; -use solana_sdk::timing::timestamp; -use solana_storage_api::SLOTS_PER_SEGMENT; +use solana_sdk::timing::{timestamp, DEFAULT_SLOTS_PER_SEGMENT}; use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::mpsc::Receiver; @@ -48,7 +47,7 @@ impl Default for ValidatorConfig { // TODO: remove this, temporary parameter to configure // storage amount differently for test configurations // so tests don't take forever to run. - const NUM_HASHES_FOR_STORAGE_ROTATE: u64 = SLOTS_PER_SEGMENT; + const NUM_HASHES_FOR_STORAGE_ROTATE: u64 = DEFAULT_SLOTS_PER_SEGMENT; Self { sigverify_disabled: false, voting_disabled: false, @@ -158,7 +157,7 @@ impl Validator { keypair.clone(), ))); - let storage_state = StorageState::new(&bank.last_blockhash()); + let storage_state = StorageState::new(&bank.last_blockhash(), bank.slots_per_segment()); let rpc_service = if node.info.rpc.port() == 0 { None diff --git a/core/tests/replicator.rs b/core/tests/replicator.rs index 6a058af8d0..0964467fa4 100644 --- a/core/tests/replicator.rs +++ b/core/tests/replicator.rs @@ -16,6 +16,7 @@ use solana::validator::ValidatorConfig; use solana_client::thin_client::create_client; use solana_sdk::genesis_block::create_genesis_block; use solana_sdk::signature::{Keypair, KeypairUtil}; +use solana_sdk::timing::DEFAULT_SLOTS_PER_SEGMENT; use std::fs::remove_dir_all; use std::sync::{Arc, RwLock}; @@ -28,7 +29,7 @@ fn run_replicator_startup_basic(num_nodes: usize, num_replicators: usize) { let mut validator_config = ValidatorConfig::default(); validator_config.storage_rotate_count = STORAGE_ROTATE_TEST_COUNT; let config = ClusterConfig { - validator_configs: vec![ValidatorConfig::default(); num_nodes], + validator_configs: vec![validator_config; num_nodes], num_replicators, node_stakes: vec![100; num_nodes], cluster_lamports: 10_000, @@ -62,7 +63,13 @@ fn run_replicator_startup_basic(num_nodes: usize, num_replicators: usize) { let path = get_tmp_ledger_path("test"); let blocktree = Arc::new(Blocktree::open(&path).unwrap()); assert_eq!( - Replicator::download_from_replicator(&cluster_info, &replicator_info, &blocktree).unwrap(), + Replicator::download_from_replicator( + &cluster_info, + &replicator_info, + &blocktree, + DEFAULT_SLOTS_PER_SEGMENT, + ) + .unwrap(), 0 ); } diff --git a/programs/stake_api/src/stake_instruction.rs b/programs/stake_api/src/stake_instruction.rs index 7e17d44550..a496149ca8 100644 --- a/programs/stake_api/src/stake_instruction.rs +++ b/programs/stake_api/src/stake_instruction.rs @@ -194,7 +194,7 @@ mod tests { .iter() .map(|meta| { if syscall::current::check_id(&meta.pubkey) { - syscall::current::create_account(1, 0, 0, 0) + syscall::current::create_account(1, 0, 0, 0, 0) } else if syscall::rewards::check_id(&meta.pubkey) { syscall::rewards::create_account(1, 0.0, 0.0) } else { @@ -288,7 +288,7 @@ mod tests { KeyedAccount::new( &syscall::current::id(), false, - &mut syscall::current::create_account(1, 0, 0, 0) + &mut syscall::current::create_account(1, 0, 0, 0, 0) ), ], &serialize(&StakeInstruction::DelegateStake(0)).unwrap(), diff --git a/programs/storage_api/src/lib.rs b/programs/storage_api/src/lib.rs index 187793e2ba..d7f8bb7efb 100644 --- a/programs/storage_api/src/lib.rs +++ b/programs/storage_api/src/lib.rs @@ -3,12 +3,6 @@ pub mod storage_contract; pub mod storage_instruction; pub mod storage_processor; -pub const SLOTS_PER_SEGMENT: u64 = 16; - -pub fn get_segment_from_slot(slot: u64) -> usize { - ((slot + (SLOTS_PER_SEGMENT - 1)) / SLOTS_PER_SEGMENT) as usize -} - const STORAGE_PROGRAM_ID: [u8; 32] = [ 6, 162, 25, 123, 127, 68, 233, 59, 131, 151, 21, 152, 162, 120, 90, 37, 154, 88, 86, 5, 156, 221, 182, 201, 142, 103, 151, 112, 0, 0, 0, 0, diff --git a/programs/storage_api/src/storage_contract.rs b/programs/storage_api/src/storage_contract.rs index 5ebee79d29..ebca9172b0 100644 --- a/programs/storage_api/src/storage_contract.rs +++ b/programs/storage_api/src/storage_contract.rs @@ -1,4 +1,3 @@ -use crate::get_segment_from_slot; use log::*; use num_derive::FromPrimitive; use serde_derive::{Deserialize, Serialize}; @@ -68,8 +67,8 @@ pub struct Proof { pub blockhash: Hash, /// The resulting sampled state pub sha_state: Hash, - /// The start index of the segment proof is for - pub segment_index: usize, + /// The segment this proof is for + pub segment_index: u64, } #[derive(Debug, Serialize, Deserialize)] @@ -78,13 +77,13 @@ pub enum StorageContract { ValidatorStorage { owner: Pubkey, - // Most recently advertised slot - slot: u64, + // Most recently advertised segment + segment: u64, // Most recently advertised blockhash hash: Hash, // Lockouts and Rewards are per segment per replicator. It needs to remain this way until // the challenge stage is added. - lockout_validations: BTreeMap>>, + lockout_validations: BTreeMap>>, // Used to keep track of ongoing credits credits: Credits, }, @@ -93,10 +92,10 @@ pub enum StorageContract { owner: Pubkey, // TODO what to do about duplicate proofs across segments? - Check the blockhashes // Map of Proofs per segment, in a Vec - proofs: BTreeMap>, + proofs: BTreeMap>, // Map of Rewards per segment, in a BTreeMap based on the validator account that verified // the proof. This can be used for challenge stage when its added - validations: BTreeMap>>, + validations: BTreeMap>>, // Used to keep track of ongoing credits credits: Credits, }, @@ -111,7 +110,7 @@ pub fn create_validator_storage_account(owner: Pubkey, lamports: u64) -> Account storage_account .set_state(&StorageContract::ValidatorStorage { owner, - slot: 0, + segment: 0, hash: Hash::default(), lockout_validations: BTreeMap::new(), credits: Credits::default(), @@ -151,7 +150,7 @@ impl<'a> StorageAccount<'a> { if let StorageContract::Uninitialized = storage_contract { *storage_contract = StorageContract::ValidatorStorage { owner, - slot: 0, + segment: 0, hash: Hash::default(), lockout_validations: BTreeMap::new(), credits: Credits::default(), @@ -165,7 +164,7 @@ impl<'a> StorageAccount<'a> { pub fn submit_mining_proof( &mut self, sha_state: Hash, - segment_index: usize, + segment_index: u64, signature: Signature, blockhash: Hash, current: syscall::current::Current, @@ -178,7 +177,7 @@ impl<'a> StorageAccount<'a> { .. } = &mut storage_contract { - let current_segment = get_segment_from_slot(current.slot); + let current_segment = current.segment; // clean up the account // TODO check for time correctness - storage seems to run at a delay of about 3 @@ -242,32 +241,29 @@ impl<'a> StorageAccount<'a> { pub fn advertise_storage_recent_blockhash( &mut self, hash: Hash, - slot: u64, + segment: u64, current: syscall::current::Current, ) -> Result<(), InstructionError> { let mut storage_contract = &mut self.account.state()?; if let StorageContract::ValidatorStorage { - slot: state_slot, + segment: state_segment, hash: state_hash, lockout_validations, credits, .. } = &mut storage_contract { - let current_segment = get_segment_from_slot(current.slot); - let original_segment = get_segment_from_slot(*state_slot); - let segment = get_segment_from_slot(slot); debug!( "advertise new segment: {} orig: {}", - segment, current_segment + segment, current.segment ); - if segment < original_segment || segment >= current_segment { + if segment < *state_segment || segment > current.segment { return Err(InstructionError::CustomError( StorageError::InvalidSegment as u32, )); } - *state_slot = slot; + *state_segment = segment; *state_hash = hash; // storage epoch updated, move the lockout_validations to credits @@ -285,21 +281,18 @@ impl<'a> StorageAccount<'a> { &mut self, me: &Pubkey, current: syscall::current::Current, - segment: u64, + segment_index: u64, proofs_per_account: Vec>, replicator_accounts: &mut [StorageAccount], ) -> Result<(), InstructionError> { let mut storage_contract = &mut self.account.state()?; if let StorageContract::ValidatorStorage { - slot: state_slot, + segment: state_segment, lockout_validations, .. } = &mut storage_contract { - let segment_index = segment as usize; - let state_segment = get_segment_from_slot(*state_slot); - - if segment_index > state_segment { + if segment_index > *state_segment { return Err(InstructionError::CustomError( StorageError::InvalidSegment as u32, )); @@ -460,7 +453,7 @@ fn store_validation_result( me: &Pubkey, current: &syscall::current::Current, storage_account: &mut StorageAccount, - segment: usize, + segment: u64, proof_mask: &[ProofStatus], ) -> Result<(), InstructionError> { let mut storage_contract = storage_account.account.state()?; @@ -494,7 +487,7 @@ fn store_validation_result( } fn count_valid_proofs( - validations: &BTreeMap>>, + validations: &BTreeMap>>, ) -> (u64, u64) { let proofs = validations .iter() @@ -537,7 +530,7 @@ mod tests { contract = StorageContract::ValidatorStorage { owner: Pubkey::default(), - slot: 0, + segment: 0, hash: Hash::default(), lockout_validations: BTreeMap::new(), credits: Credits::default(), @@ -569,7 +562,7 @@ mod tests { executable: false, }, }; - let segment_index = 0_usize; + let segment_index = 0; let proof = Proof { segment_index, ..Proof::default() diff --git a/programs/storage_api/src/storage_instruction.rs b/programs/storage_api/src/storage_instruction.rs index b3bd0eaad3..292882c38c 100644 --- a/programs/storage_api/src/storage_instruction.rs +++ b/programs/storage_api/src/storage_instruction.rs @@ -23,13 +23,13 @@ pub enum StorageInstruction { SubmitMiningProof { sha_state: Hash, - segment_index: usize, + segment_index: u64, signature: Signature, blockhash: Hash, }, AdvertiseStorageRecentBlockhash { hash: Hash, - slot: u64, + segment: u64, }, /// Redeem storage reward credits /// @@ -132,7 +132,7 @@ pub fn create_replicator_storage_account( pub fn mining_proof( storage_pubkey: &Pubkey, sha_state: Hash, - segment_index: usize, + segment_index: u64, signature: Signature, blockhash: Hash, ) -> Instruction { @@ -152,11 +152,11 @@ pub fn mining_proof( pub fn advertise_recent_blockhash( storage_pubkey: &Pubkey, storage_hash: Hash, - slot: u64, + segment: u64, ) -> Instruction { let storage_instruction = StorageInstruction::AdvertiseStorageRecentBlockhash { hash: storage_hash, - slot, + segment, }; let account_metas = vec![ AccountMeta::new(*storage_pubkey, true), diff --git a/programs/storage_api/src/storage_processor.rs b/programs/storage_api/src/storage_processor.rs index e680df4f46..44e4be4aff 100644 --- a/programs/storage_api/src/storage_processor.rs +++ b/programs/storage_api/src/storage_processor.rs @@ -51,13 +51,13 @@ pub fn process_instruction( current, ) } - StorageInstruction::AdvertiseStorageRecentBlockhash { hash, slot } => { + StorageInstruction::AdvertiseStorageRecentBlockhash { hash, segment } => { if me_unsigned || rest.len() != 1 { // This instruction must be signed by `me` Err(InstructionError::InvalidArgument)?; } let current = syscall::current::from_keyed_account(&rest[0])?; - storage_account.advertise_storage_recent_blockhash(hash, slot, current) + storage_account.advertise_storage_recent_blockhash(hash, segment, current) } StorageInstruction::ClaimStorageReward => { if rest.len() != 4 { diff --git a/programs/storage_program/tests/storage_processor.rs b/programs/storage_program/tests/storage_processor.rs index 3166a6a23e..5b03222431 100644 --- a/programs/storage_program/tests/storage_processor.rs +++ b/programs/storage_program/tests/storage_processor.rs @@ -16,17 +16,18 @@ use solana_sdk::syscall::current::Current; use solana_sdk::syscall::rewards::Rewards; use solana_sdk::syscall::{current, rewards}; use solana_sdk::system_instruction; -use solana_sdk::timing::DEFAULT_TICKS_PER_SLOT; +use solana_sdk::timing::{ + get_segment_from_slot, DEFAULT_SLOTS_PER_SEGMENT, DEFAULT_TICKS_PER_SLOT, +}; +use solana_storage_api::id; use solana_storage_api::storage_contract::StorageAccount; use solana_storage_api::storage_contract::{ProofStatus, StorageContract, STORAGE_ACCOUNT_SPACE}; use solana_storage_api::storage_instruction; use solana_storage_api::storage_processor::process_instruction; -use solana_storage_api::SLOTS_PER_SEGMENT; -use solana_storage_api::{get_segment_from_slot, id}; use std::collections::HashMap; use std::sync::Arc; -const TICKS_IN_SEGMENT: u64 = SLOTS_PER_SEGMENT * DEFAULT_TICKS_PER_SLOT; +const TICKS_IN_SEGMENT: u64 = DEFAULT_SLOTS_PER_SEGMENT * DEFAULT_TICKS_PER_SLOT; fn test_instruction( ix: &Instruction, @@ -125,10 +126,11 @@ fn test_proof_bounds() { Hash::default(), ); // the proof is for segment 0, need to move the slot into segment 2 - let mut current_account = current::create_account(1, 0, 0, 0); + let mut current_account = current::create_account(1, 0, 0, 0, 0); Current::to( &Current { - slot: SLOTS_PER_SEGMENT * 2, + slot: DEFAULT_SLOTS_PER_SEGMENT * 2, + segment: 2, epoch: 0, stakers_epoch: 0, }, @@ -155,15 +157,11 @@ fn test_serialize_overflow() { let current_id = current::id(); let mut keyed_accounts = Vec::new(); let mut user_account = Account::default(); - let mut current_account = current::create_account(1, 0, 0, 0); + let mut current_account = current::create_account(1, 0, 0, 0, 0); keyed_accounts.push(KeyedAccount::new(&pubkey, true, &mut user_account)); keyed_accounts.push(KeyedAccount::new(¤t_id, false, &mut current_account)); - let ix = storage_instruction::advertise_recent_blockhash( - &pubkey, - Hash::default(), - SLOTS_PER_SEGMENT, - ); + let ix = storage_instruction::advertise_recent_blockhash(&pubkey, Hash::default(), 1); assert_eq!( process_instruction(&id(), &mut keyed_accounts, &ix.data), @@ -184,10 +182,11 @@ fn test_invalid_accounts_len() { Hash::default(), ); // move tick height into segment 1 - let mut current_account = current::create_account(1, 0, 0, 0); + let mut current_account = current::create_account(1, 0, 0, 0, 0); Current::to( &Current { slot: 16, + segment: 1, epoch: 0, stakers_epoch: 0, }, @@ -243,10 +242,11 @@ fn test_submit_mining_ok() { Hash::default(), ); // move slot into segment 1 - let mut current_account = current::create_account(1, 0, 0, 0); + let mut current_account = current::create_account(1, 0, 0, 0, 0); Current::to( &Current { - slot: SLOTS_PER_SEGMENT, + slot: DEFAULT_SLOTS_PER_SEGMENT, + segment: 1, epoch: 0, stakers_epoch: 0, }, @@ -300,7 +300,7 @@ fn test_validate_mining() { let bank = Arc::new(Bank::new_from_parent( &bank, &Pubkey::default(), - SLOTS_PER_SEGMENT * 2, + DEFAULT_SLOTS_PER_SEGMENT * 2, )); let bank_client = BankClient::new_shared(&bank); @@ -309,7 +309,7 @@ fn test_validate_mining() { vec![storage_instruction::advertise_recent_blockhash( &validator_storage_id, Hash::default(), - SLOTS_PER_SEGMENT, + 1, )], Some(&mint_pubkey), ); @@ -344,17 +344,17 @@ fn test_validate_mining() { vec![storage_instruction::advertise_recent_blockhash( &validator_storage_id, Hash::default(), - SLOTS_PER_SEGMENT * 2, + 2, )], Some(&mint_pubkey), ); // move banks into the next segment - let proof_segment = get_segment_from_slot(bank.slot()); + let proof_segment = get_segment_from_slot(bank.slot(), bank.slots_per_segment()); let bank = Arc::new(Bank::new_from_parent( &bank, &Pubkey::default(), - SLOTS_PER_SEGMENT + bank.slot(), + DEFAULT_SLOTS_PER_SEGMENT + bank.slot(), )); let bank_client = BankClient::new_shared(&bank); @@ -381,7 +381,7 @@ fn test_validate_mining() { vec![storage_instruction::advertise_recent_blockhash( &validator_storage_id, Hash::default(), - SLOTS_PER_SEGMENT * 3, + 3, )], Some(&mint_pubkey), ); @@ -390,7 +390,7 @@ fn test_validate_mining() { let bank = Arc::new(Bank::new_from_parent( &bank, &Pubkey::default(), - SLOTS_PER_SEGMENT + bank.slot(), + DEFAULT_SLOTS_PER_SEGMENT + bank.slot(), )); let bank_client = BankClient::new_shared(&bank); @@ -505,21 +505,21 @@ fn init_storage_accounts( client.send_message(&[mint], message).unwrap(); } -fn get_storage_slot(client: &C, account: &Pubkey) -> u64 { +fn get_storage_segment(client: &C, account: &Pubkey) -> u64 { match client.get_account_data(&account).unwrap() { Some(storage_system_account_data) => { let contract = deserialize(&storage_system_account_data); if let Ok(contract) = contract { match contract { - StorageContract::ValidatorStorage { slot, .. } => { - return slot; + StorageContract::ValidatorStorage { segment, .. } => { + return segment; } - _ => info!("error in reading slot"), + _ => info!("error in reading segment"), } } } None => { - info!("error in reading slot"); + info!("error in reading segment"); } } 0 @@ -536,7 +536,7 @@ fn submit_proof( vec![storage_instruction::mining_proof( &storage_keypair.pubkey(), sha_state, - segment_index as usize, + segment_index, Signature::default(), bank_client.get_recent_blockhash().unwrap().0, )], @@ -584,7 +584,11 @@ fn test_bank_storage() { let bank = Bank::new(&genesis_block); // tick the bank up until it's moved into storage segment 2 // create a new bank in storage segment 2 - let bank = Bank::new_from_parent(&Arc::new(bank), &Pubkey::new_rand(), SLOTS_PER_SEGMENT * 2); + let bank = Bank::new_from_parent( + &Arc::new(bank), + &Pubkey::new_rand(), + DEFAULT_SLOTS_PER_SEGMENT * 2, + ); let bank_client = BankClient::new(bank); let x = 42; @@ -615,7 +619,7 @@ fn test_bank_storage() { vec![storage_instruction::advertise_recent_blockhash( &validator_pubkey, storage_blockhash, - SLOTS_PER_SEGMENT as u64, + 1, )], Some(&mint_pubkey), ); @@ -641,10 +645,7 @@ fn test_bank_storage() { Ok(_) ); - assert_eq!( - get_storage_slot(&bank_client, &validator_pubkey), - SLOTS_PER_SEGMENT - ); + assert_eq!(get_storage_segment(&bank_client, &validator_pubkey), 1); assert_eq!( get_storage_blockhash(&bank_client, &validator_pubkey), storage_blockhash diff --git a/programs/vote_api/src/vote_instruction.rs b/programs/vote_api/src/vote_instruction.rs index 209a904234..969449dfa5 100644 --- a/programs/vote_api/src/vote_instruction.rs +++ b/programs/vote_api/src/vote_instruction.rs @@ -168,7 +168,7 @@ mod tests { .iter() .map(|meta| { if syscall::current::check_id(&meta.pubkey) { - syscall::current::create_account(1, 0, 0, 0) + syscall::current::create_account(1, 0, 0, 0, 0) } else if syscall::slot_hashes::check_id(&meta.pubkey) { syscall::slot_hashes::create_account(1, &[]) } else { diff --git a/runtime/src/bank.rs b/runtime/src/bank.rs index 36380c5f08..b077daf574 100644 --- a/runtime/src/bank.rs +++ b/runtime/src/bank.rs @@ -38,7 +38,7 @@ use solana_sdk::syscall::{ slot_hashes::{self, SlotHashes}, }; use solana_sdk::system_transaction; -use solana_sdk::timing::{duration_as_ns, MAX_RECENT_BLOCKHASHES}; +use solana_sdk::timing::{duration_as_ns, get_segment_from_slot, MAX_RECENT_BLOCKHASHES}; use solana_sdk::transaction::{Result, Transaction, TransactionError}; use std::cmp; use std::collections::HashMap; @@ -219,6 +219,9 @@ pub struct Bank { /// The number of slots per year, used for inflation slots_per_year: f64, + /// The number of slots per Storage segment + slots_per_segment: u64, + /// Bank fork (i.e. slot, i.e. block) slot: u64, @@ -307,6 +310,7 @@ impl Bank { // TODO: clean this up, soo much special-case copying... self.ticks_per_slot = parent.ticks_per_slot; + self.slots_per_segment = parent.slots_per_segment; self.slots_per_year = parent.slots_per_year; self.epoch_schedule = parent.epoch_schedule; @@ -410,6 +414,7 @@ impl Bank { ¤t::create_account( 1, self.slot, + get_segment_from_slot(self.slot, self.slots_per_segment), self.epoch_schedule.get_epoch(self.slot), self.epoch_schedule.get_stakers_epoch(self.slot), ), @@ -590,6 +595,7 @@ impl Bank { .genesis_hash(&genesis_block.hash(), &self.fee_calculator); self.ticks_per_slot = genesis_block.ticks_per_slot; + self.slots_per_segment = genesis_block.slots_per_segment; self.max_tick_height = (self.slot + 1) * self.ticks_per_slot - 1; // ticks/year = seconds/year ... self.slots_per_year = SECONDS_PER_YEAR @@ -1264,6 +1270,11 @@ impl Bank { self.ticks_per_slot } + /// Return the number of slots per segment + pub fn slots_per_segment(&self) -> u64 { + self.slots_per_segment + } + /// Return the number of ticks since genesis. pub fn tick_height(&self) -> u64 { // tick_height is using an AtomicUSize because AtomicU64 is not yet a stable API. diff --git a/sdk/src/genesis_block.rs b/sdk/src/genesis_block.rs index 783aafe826..4864c4b986 100644 --- a/sdk/src/genesis_block.rs +++ b/sdk/src/genesis_block.rs @@ -8,7 +8,7 @@ use crate::poh_config::PohConfig; use crate::pubkey::Pubkey; use crate::signature::{Keypair, KeypairUtil}; use crate::system_program; -use crate::timing::{DEFAULT_SLOTS_PER_EPOCH, DEFAULT_TICKS_PER_SLOT}; +use crate::timing::{DEFAULT_SLOTS_PER_EPOCH, DEFAULT_SLOTS_PER_SEGMENT, DEFAULT_TICKS_PER_SLOT}; use bincode::{deserialize, serialize}; use memmap::Mmap; use std::fs::{File, OpenOptions}; @@ -24,6 +24,7 @@ pub struct GenesisBlock { pub stakers_slot_offset: u64, pub epoch_warmup: bool, pub ticks_per_slot: u64, + pub slots_per_segment: u64, pub poh_config: PohConfig, pub fee_calculator: FeeCalculator, pub inflation: Inflation, @@ -54,6 +55,7 @@ impl Default for GenesisBlock { slots_per_epoch: DEFAULT_SLOTS_PER_EPOCH, stakers_slot_offset: DEFAULT_SLOTS_PER_EPOCH, ticks_per_slot: DEFAULT_TICKS_PER_SLOT, + slots_per_segment: DEFAULT_SLOTS_PER_SEGMENT, poh_config: PohConfig::default(), inflation: Inflation::default(), fee_calculator: FeeCalculator::default(), diff --git a/sdk/src/syscall/current.rs b/sdk/src/syscall/current.rs index c5c079815f..ad46d1b059 100644 --- a/sdk/src/syscall/current.rs +++ b/sdk/src/syscall/current.rs @@ -17,6 +17,7 @@ const ID: [u8; 32] = [ #[derive(Serialize, Deserialize, Debug, Default, PartialEq)] pub struct Current { pub slot: Slot, + pub segment: Segment, pub epoch: Epoch, pub stakers_epoch: Epoch, } @@ -34,11 +35,18 @@ impl Current { } } -pub fn create_account(lamports: u64, slot: Slot, epoch: Epoch, stakers_epoch: Epoch) -> Account { +pub fn create_account( + lamports: u64, + slot: Slot, + segment: Segment, + epoch: Epoch, + stakers_epoch: Epoch, +) -> Account { Account::new_data( lamports, &Current { slot, + segment, epoch, stakers_epoch, }, @@ -49,6 +57,8 @@ pub fn create_account(lamports: u64, slot: Slot, epoch: Epoch, stakers_epoch: Ep use crate::account::KeyedAccount; use crate::instruction::InstructionError; +use crate::timing::Segment; + pub fn from_keyed_account(account: &KeyedAccount) -> Result { if !check_id(account.unsigned_key()) { return Err(InstructionError::InvalidArgument); @@ -62,7 +72,7 @@ mod tests { #[test] fn test_create_account() { - let account = create_account(1, 0, 0, 0); + let account = create_account(1, 0, 0, 0, 0); let current = Current::from(&account).unwrap(); assert_eq!(current, Current::default()); } diff --git a/sdk/src/timing.rs b/sdk/src/timing.rs index fa407e9e74..4d53c74953 100644 --- a/sdk/src/timing.rs +++ b/sdk/src/timing.rs @@ -13,6 +13,9 @@ pub const DEFAULT_TICKS_PER_SLOT: u64 = 4; // 1 Epoch = 400 * 8192 ms ~= 55 minutes pub const DEFAULT_SLOTS_PER_EPOCH: u64 = 8192; +// Storage segment configuration +pub const DEFAULT_SLOTS_PER_SEGMENT: u64 = 16; + pub const NUM_CONSECUTIVE_LEADER_SLOTS: u64 = 4; /// The time window of recent block hash values that the bank will track the signatures @@ -61,10 +64,17 @@ pub fn timestamp() -> u64 { duration_as_ms(&now) } +pub fn get_segment_from_slot(slot: Slot, slots_per_segment: u64) -> Segment { + ((slot + (slots_per_segment - 1)) / slots_per_segment) +} + /// Slot is a unit of time given to a leader for encoding, /// is some some number of Ticks long. Use a u64 to count them. pub type Slot = u64; +/// A segment is some number of slots stored by replicators +pub type Segment = u64; + /// Epoch is a unit of time a given leader schedule is honored, /// some number of Slots. Use a u64 to count them. pub type Epoch = u64;