diff --git a/client/tests/integration/events/pipeline.rs b/client/tests/integration/events/pipeline.rs index 28afa87ac40..1e6a0f49317 100644 --- a/client/tests/integration/events/pipeline.rs +++ b/client/tests/integration/events/pipeline.rs @@ -1,4 +1,7 @@ -use std::thread::{self, JoinHandle}; +use std::{ + num::NonZeroUsize, + thread::{self, JoinHandle}, +}; use eyre::Result; use iroha::{ @@ -129,6 +132,8 @@ fn applied_block_must_be_available_in_kura() { .as_ref() .expect("Must be some") .kura() - .get_block_by_height(event.header().height()) + .get_block_by_height( + NonZeroUsize::new(event.header().height().try_into().unwrap()).unwrap(), + ) .expect("Block applied event was received earlier"); } diff --git a/core/src/block.rs b/core/src/block.rs index e4c1d30802d..9a692b8ed9a 100644 --- a/core/src/block.rs +++ b/core/src/block.rs @@ -48,9 +48,9 @@ pub enum BlockValidationError { /// Mismatch between the actual and expected height of the latest block. Expected: {expected}, actual: {actual} LatestBlockHeightMismatch { /// Expected value - expected: u64, + expected: usize, /// Actual value - actual: u64, + actual: usize, }, /// Mismatch between the actual and expected hashes of the current block. Expected: {expected:?}, actual: {actual:?} IncorrectHash { @@ -145,13 +145,13 @@ mod pending { } fn make_header( - previous_height: u64, + previous_height: usize, prev_block_hash: Option>, - view_change_index: u64, + view_change_index: usize, transactions: &[CommittedTransaction], ) -> BlockHeader { BlockHeader { - height: previous_height + 1, + height: (previous_height + 1) as u64, previous_block_hash: prev_block_hash, transactions_hash: transactions .iter() @@ -164,7 +164,9 @@ mod pending { .as_millis() .try_into() .expect("Time should fit into u64"), - view_change_index, + view_change_index: view_change_index + .try_into() + .expect("View change index should fit into u32"), consensus_estimation_ms: DEFAULT_CONSENSUS_ESTIMATION .as_millis() .try_into() @@ -205,7 +207,7 @@ mod pending { /// Upon executing this method current timestamp is stored in the block header. pub fn chain( self, - view_change_index: u64, + view_change_index: usize, state: &mut StateBlock<'_>, ) -> BlockBuilder { let transactions = Self::categorize_transactions(self.0.transactions, state); @@ -273,7 +275,11 @@ mod valid { state_block: &mut StateBlock<'_>, ) -> WithEvents> { let expected_block_height = state_block.height() + 1; - let actual_height = block.header().height; + let actual_height = block + .header() + .height + .try_into() + .expect("Block height should fit into usize"); if expected_block_height != actual_height { return WithEvents::new(Err(( diff --git a/core/src/block_sync.rs b/core/src/block_sync.rs index ef7f5b8c10a..bb84564537f 100644 --- a/core/src/block_sync.rs +++ b/core/src/block_sync.rs @@ -127,6 +127,8 @@ impl BlockSynchronizer { pub mod message { //! Module containing messages for [`BlockSynchronizer`](super::BlockSynchronizer). + use std::num::NonZeroUsize; + use super::*; /// Get blocks after some block @@ -204,14 +206,17 @@ pub mod message { error!(?prev_hash, "Block hash not found"); return; } - Some(height) => height + 1, // It's get blocks *after*, so we add 1. + Some(height) => height.saturating_add(1), // It's get blocks *after*, so we add 1. }, - None => 1, + None => nonzero_ext::nonzero!(1_usize), }; - let blocks = (start_height..) - .take(1 + block_sync.gossip_max_size.get() as usize) - .map_while(|height| block_sync.kura.get_block_by_height(height)) + let blocks = (start_height.get()..) + .take(block_sync.gossip_max_size.get() as usize + 1) + .map_while(|height| { + NonZeroUsize::new(height) + .and_then(|height| block_sync.kura.get_block_by_height(height)) + }) .skip_while(|block| Some(block.hash()) == *latest_hash) .map(|block| (*block).clone()) .collect::>(); diff --git a/core/src/kura.rs b/core/src/kura.rs index b9d37c8d104..0e2d6a4840b 100644 --- a/core/src/kura.rs +++ b/core/src/kura.rs @@ -6,6 +6,7 @@ use std::{ fmt::Debug, fs, io::{BufWriter, Read, Seek, SeekFrom, Write}, + num::NonZeroUsize, path::{Path, PathBuf}, sync::Arc, }; @@ -273,45 +274,46 @@ impl Kura { } /// Get the hash of the block at the provided height. - pub fn get_block_hash(&self, block_height: u64) -> Option> { + pub fn get_block_hash(&self, block_height: NonZeroUsize) -> Option> { let hash_data_guard = self.block_data.lock(); - if block_height == 0 || block_height > hash_data_guard.len() as u64 { + + let block_height = block_height.get(); + if hash_data_guard.len() < block_height { return None; } - let index: usize = (block_height - 1) - .try_into() - .expect("block_height fits in 32 bits or we are running on a 64 bit machine"); - Some(hash_data_guard[index].0) + + let block_index = block_height - 1; + Some(hash_data_guard[block_index].0) } /// Search through blocks for the height of the block with the given hash. - pub fn get_block_height_by_hash(&self, hash: &HashOf) -> Option { + pub fn get_block_height_by_hash(&self, hash: &HashOf) -> Option { self.block_data .lock() .iter() .position(|(block_hash, _block_arc)| block_hash == hash) - .map(|index| index as u64 + 1) + .and_then(NonZeroUsize::new) + .and_then(|index| index.checked_add(1)) } /// Get a reference to block by height, loading it from disk if needed. // The below lint suggests changing the code into something that does not compile due // to the borrow checker. - pub fn get_block_by_height(&self, block_height: u64) -> Option> { + pub fn get_block_by_height(&self, block_height: NonZeroUsize) -> Option> { let mut data_array_guard = self.block_data.lock(); - if block_height == 0 || block_height > data_array_guard.len() as u64 { + + if data_array_guard.len() < block_height.get() { return None; } - let block_number: usize = (block_height - 1) - .try_into() - .expect("Failed to cast to u32."); - if let Some(block_arc) = data_array_guard[block_number].1.as_ref() { + let block_index = block_height.get() - 1; + if let Some(block_arc) = data_array_guard[block_index].1.as_ref() { return Some(Arc::clone(block_arc)); }; let block_store = self.block_store.lock(); let BlockIndex { start, length } = block_store - .read_block_index(block_number as u64) + .read_block_index(block_index as u64) .expect("Failed to read block index from disk."); let mut block_buf = @@ -322,7 +324,7 @@ impl Kura { let block = SignedBlock::decode_all_versioned(&block_buf).expect("Failed to decode block"); let block_arc = Arc::new(block); - data_array_guard[block_number].1 = Some(Arc::clone(&block_arc)); + data_array_guard[block_index].1 = Some(Arc::clone(&block_arc)); Some(block_arc) } @@ -332,13 +334,13 @@ impl Kura { /// then calls `get_block_by_height`. If you know the height of the block, /// call `get_block_by_height` directly. pub fn get_block_by_hash(&self, block_hash: &HashOf) -> Option> { - let index = self - .block_data + self.block_data .lock() .iter() - .position(|(hash, _arc)| hash == block_hash); - - index.and_then(|index| self.get_block_by_height(index as u64 + 1)) + .position(|(hash, _arc)| hash == block_hash) + .and_then(NonZeroUsize::new) + .and_then(|index| index.checked_add(1)) + .and_then(|index| self.get_block_by_height(index)) } /// Put a block in kura's in memory block store. diff --git a/core/src/metrics.rs b/core/src/metrics.rs index f5adbb9f6ed..32a19258f15 100644 --- a/core/src/metrics.rs +++ b/core/src/metrics.rs @@ -1,6 +1,6 @@ //! Metrics and status reporting -use std::{sync::Arc, time::SystemTime}; +use std::{num::NonZeroUsize, sync::Arc, time::SystemTime}; use eyre::{Result, WrapErr as _}; use iroha_telemetry::metrics::Metrics; @@ -23,7 +23,7 @@ pub struct MetricsReporter { queue: Arc, metrics: Metrics, /// Latest observed and processed height by metrics reporter - latest_block_height: Arc>, + latest_block_height: Arc>, } impl MetricsReporter { @@ -53,14 +53,10 @@ impl MetricsReporter { /// - If either mutex is poisoned #[allow(clippy::cast_precision_loss)] pub fn update_metrics(&self) -> Result<()> { - let online_peers_count: u64 = self - .network - .online_peers( - #[allow(clippy::disallowed_types)] - std::collections::HashSet::len, - ) - .try_into() - .expect("casting usize to u64"); + let online_peers_count: usize = self.network.online_peers( + #[allow(clippy::disallowed_types)] + std::collections::HashSet::len, + ); let state_view = self.state.view(); @@ -70,7 +66,9 @@ impl MetricsReporter { { let mut block_index = start_index; while block_index < state_view.height() { - let Some(block) = self.kura.get_block_by_height(block_index + 1) else { + let Some(block) = NonZeroUsize::new(block_index) + .and_then(|index| self.kura.get_block_by_height(index.saturating_add(1))) + else { break; }; block_index += 1; @@ -126,7 +124,7 @@ impl MetricsReporter { ) }; - self.metrics.connected_peers.set(online_peers_count); + self.metrics.connected_peers.set(online_peers_count as u64); self.metrics .domains diff --git a/core/src/smartcontracts/isi/query.rs b/core/src/smartcontracts/isi/query.rs index 26b6d603808..c14c2119ad7 100644 --- a/core/src/smartcontracts/isi/query.rs +++ b/core/src/smartcontracts/isi/query.rs @@ -41,6 +41,11 @@ impl LazyQueryOutput<'_> { /// - sorting /// - pagination /// - batching + /// + /// # Errors + /// + /// - if fetch size is too big + /// - defined pagination parameter for a query that returns singular result pub fn apply_postprocessing( self, filter: &PredicateBox, diff --git a/core/src/smartcontracts/wasm.rs b/core/src/smartcontracts/wasm.rs index 1f6d2474193..641abd81895 100644 --- a/core/src/smartcontracts/wasm.rs +++ b/core/src/smartcontracts/wasm.rs @@ -1151,7 +1151,7 @@ impl<'wrld, 'block, 'state> ) -> Validate { Validate { authority: state.authority.clone(), - block_height: state.state.0.height(), + block_height: state.state.0.height() as u64, target: state.specific_state.to_validate.clone(), } } @@ -1238,7 +1238,7 @@ impl<'wrld, 'block, 'state> ) -> Validate { Validate { authority: state.authority.clone(), - block_height: state.state.0.height(), + block_height: state.state.0.height() as u64, target: state.specific_state.to_validate.clone(), } } @@ -1341,7 +1341,7 @@ impl<'wrld, S: StateReadOnly> ) -> Validate { Validate { authority: state.authority.clone(), - block_height: state.state.0.height(), + block_height: state.state.0.height() as u64, target: state.specific_state.to_validate.clone(), } } @@ -1419,7 +1419,7 @@ impl<'wrld, 'block, 'state> state: &state::executor::Migrate<'wrld, 'block, 'state>, ) -> payloads::Migrate { payloads::Migrate { - block_height: state.state.0.height(), + block_height: state.state.0.height() as u64, } } diff --git a/core/src/snapshot.rs b/core/src/snapshot.rs index 572d0810c9b..f7a6bbf0779 100644 --- a/core/src/snapshot.rs +++ b/core/src/snapshot.rs @@ -1,6 +1,7 @@ //! This module contains [`State`] snapshot actor service. use std::{ io::Read, + num::NonZeroUsize, path::{Path, PathBuf}, sync::Arc, time::Duration, @@ -169,8 +170,8 @@ pub fn try_read_snapshot( }); } for height in 1..snapshot_height { - let kura_block_hash = kura - .get_block_hash(height as u64) + let kura_block_hash = NonZeroUsize::new(height) + .and_then(|height| kura.get_block_hash(height)) .expect("Kura has height at least as large as state height"); let snapshot_block_hash = state_view.block_hashes[height - 1]; if kura_block_hash != snapshot_block_hash { @@ -286,7 +287,7 @@ mod tests { &store_dir, &Kura::blank_kura_for_testing(), LiveQueryStore::test().start(), - BlockCount(usize::try_from(state.view().height()).unwrap()), + BlockCount(state.view().height()), ) .unwrap(); } diff --git a/core/src/state.rs b/core/src/state.rs index 38a74462a71..1b348a824c8 100644 --- a/core/src/state.rs +++ b/core/src/state.rs @@ -1,5 +1,8 @@ //! This module provides the [`State`] — an in-memory representation of the current blockchain state. -use std::{borrow::Borrow, collections::BTreeSet, marker::PhantomData, sync::Arc, time::Duration}; +use std::{ + borrow::Borrow, collections::BTreeSet, marker::PhantomData, num::NonZeroUsize, sync::Arc, + time::Duration, +}; use eyre::Result; use iroha_config::parameters::actual::ChainWide as Config; @@ -171,7 +174,7 @@ pub struct State { // TODO: Cell is redundant here since block_hashes is very easy to rollback by just popping the last element pub block_hashes: Cell>>, /// Hashes of transactions mapped onto block height where they stored - pub transactions: Storage, u64>, + pub transactions: Storage, usize>, /// Engine for WASM [`Runtime`](wasm::Runtime) to execute triggers. #[serde(skip)] pub engine: wasmtime::Engine, @@ -197,7 +200,7 @@ pub struct StateBlock<'state> { /// Blockchain. pub block_hashes: CellBlock<'state, Vec>>, /// Hashes of transactions mapped onto block height where they stored - pub transactions: StorageBlock<'state, HashOf, u64>, + pub transactions: StorageBlock<'state, HashOf, usize>, /// Engine for WASM [`Runtime`](wasm::Runtime) to execute triggers. pub engine: &'state wasmtime::Engine, @@ -219,7 +222,7 @@ pub struct StateTransaction<'block, 'state> { /// Blockchain. pub block_hashes: CellTransaction<'block, 'state, Vec>>, /// Hashes of transactions mapped onto block height where they stored - pub transactions: StorageTransaction<'block, 'state, HashOf, u64>, + pub transactions: StorageTransaction<'block, 'state, HashOf, usize>, /// Engine for WASM [`Runtime`](wasm::Runtime) to execute triggers. pub engine: &'state wasmtime::Engine, @@ -241,7 +244,7 @@ pub struct StateView<'state> { /// Blockchain. pub block_hashes: CellView<'state, Vec>>, /// Hashes of transactions mapped onto block height where they stored - pub transactions: StorageView<'state, HashOf, u64>, + pub transactions: StorageView<'state, HashOf, usize>, /// Engine for WASM [`Runtime`](wasm::Runtime) to execute triggers. pub engine: &'state wasmtime::Engine, @@ -990,7 +993,7 @@ pub trait StateReadOnly { fn world(&self) -> &impl WorldReadOnly; fn config(&self) -> &Config; fn block_hashes(&self) -> &[HashOf]; - fn transactions(&self) -> &impl StorageReadOnly, u64>; + fn transactions(&self) -> &impl StorageReadOnly, usize>; fn engine(&self) -> &wasmtime::Engine; fn kura(&self) -> &Kura; fn query_handle(&self) -> &LiveQueryStoreHandle; @@ -1001,8 +1004,7 @@ pub trait StateReadOnly { /// Get a reference to the latest block. Returns none if genesis is not committed. #[inline] fn latest_block_ref(&self) -> Option> { - self.kura() - .get_block_by_height(self.block_hashes().len() as u64) + NonZeroUsize::new(self.height()).and_then(|height| self.kura().get_block_by_height(height)) } /// Return the hash of the latest block @@ -1011,10 +1013,10 @@ pub trait StateReadOnly { } /// Return the view change index of the latest block - fn latest_block_view_change_index(&self) -> u64 { - self.kura() - .get_block_by_height(self.height()) - .map_or(0, |block| block.header().view_change_index) + fn latest_block_view_change_index(&self) -> usize { + NonZeroUsize::new(self.height()) + .and_then(|height| self.kura().get_block_by_height(height)) + .map_or(0, |block| block.header().view_change_index as usize) } /// Return the hash of the block one before the latest block @@ -1024,10 +1026,9 @@ pub trait StateReadOnly { /// Load all blocks in the block chain from disc fn all_blocks(&self) -> impl DoubleEndedIterator> + '_ { - let block_count = self.block_hashes().len() as u64; - (1..=block_count).map(|height| { - self.kura() - .get_block_by_height(height) + (1..=self.height()).map(|height| { + NonZeroUsize::new(height) + .and_then(|height| self.kura().get_block_by_height(height)) .expect("Failed to load block.") }) } @@ -1061,14 +1062,16 @@ pub trait StateReadOnly { /// Height of blockchain #[inline] - fn height(&self) -> u64 { - self.block_hashes().len() as u64 + fn height(&self) -> usize { + self.block_hashes().len() } /// Find a [`SignedBlock`] by hash. fn block_with_tx(&self, hash: &HashOf) -> Option> { - let height = *self.transactions().get(hash)?; - self.kura().get_block_by_height(height) + self.transactions() + .get(hash) + .and_then(|&height| NonZeroUsize::new(height)) + .and_then(|height| self.kura().get_block_by_height(height)) } /// Returns [`Some`] milliseconds since the genesis block was @@ -1080,12 +1083,13 @@ pub trait StateReadOnly { } else { let opt = self .kura() - .get_block_by_height(1) + .get_block_by_height(nonzero_ext::nonzero!(1_usize)) .map(|genesis_block| genesis_block.header().timestamp()); if opt.is_none() { error!("Failed to get genesis block from Kura."); } + opt } } @@ -1114,7 +1118,7 @@ macro_rules! impl_state_ro { fn block_hashes(&self) -> &[HashOf] { &self.block_hashes } - fn transactions(&self) -> &impl StorageReadOnly, u64> { + fn transactions(&self) -> &impl StorageReadOnly, usize> { &self.transactions } fn engine(&self) -> &wasmtime::Engine { @@ -1216,7 +1220,12 @@ impl<'state> StateBlock<'state> { let time_event = self.create_time_event(block); self.world.events_buffer.push(time_event.into()); - let block_height = block.as_ref().header().height; + let block_height = block + .as_ref() + .header() + .height + .try_into() + .expect("Block height should fit into usize"); block .as_ref() .transactions() diff --git a/core/src/sumeragi/main_loop.rs b/core/src/sumeragi/main_loop.rs index 74a433c00dc..787e670599e 100644 --- a/core/src/sumeragi/main_loop.rs +++ b/core/src/sumeragi/main_loop.rs @@ -168,7 +168,7 @@ impl Sumeragi { &self.current_topology.ordered_peers, self.current_topology.max_faults(), state_view.latest_block_hash(), - ) as u64; + ); loop { let block_msg = self @@ -182,8 +182,10 @@ impl Sumeragi { }) .ok()?; - let block_vc_index: Option = match &block_msg { - BlockMessage::BlockCreated(bc) => Some(bc.block.header().view_change_index), + let block_vc_index: Option = match &block_msg { + BlockMessage::BlockCreated(bc) => { + Some(bc.block.header().view_change_index as usize) + } // Signed and Committed contain no block. // Block sync updates are exempt from early pruning. BlockMessage::BlockSigned(_) @@ -415,13 +417,13 @@ impl Sumeragi { &self, state_view: &StateView<'_>, view_change_proof_chain: &mut ProofChain, - ) -> u64 { + ) -> usize { view_change_proof_chain.prune(state_view.latest_block_hash()); view_change_proof_chain.verify_with_state( &self.current_topology.ordered_peers, self.current_topology.max_faults(), state_view.latest_block_hash(), - ) as u64 + ) } #[allow(clippy::too_many_lines)] @@ -430,7 +432,7 @@ impl Sumeragi { message: BlockMessage, state: &'state State, voting_block: &mut Option>, - current_view_change_index: u64, + current_view_change_index: usize, genesis_public_key: &PublicKey, voting_signatures: &mut Vec>, ) { @@ -653,7 +655,7 @@ impl Sumeragi { &mut self, state: &'state State, voting_block: &mut Option>, - current_view_change_index: u64, + current_view_change_index: usize, round_start_time: &Instant, #[cfg_attr(not(debug_assertions), allow(unused_variables))] is_genesis_peer: bool, ) { @@ -757,8 +759,8 @@ impl Sumeragi { fn reset_state( peer_id: &PeerId, pipeline_time: Duration, - current_view_change_index: u64, - old_view_change_index: &mut u64, + current_view_change_index: usize, + old_view_change_index: &mut usize, old_latest_block_hash: &mut HashOf, latest_block: &SignedBlock, // below is the state that gets reset. @@ -922,7 +924,9 @@ pub(crate) fn run( &mut last_view_change_time, &mut view_change_time, ); - sumeragi.view_changes_metric.set(old_view_change_index); + sumeragi + .view_changes_metric + .set(old_view_change_index as u64); if let Some(message) = { let (msg, sleep) = @@ -1003,7 +1007,9 @@ pub(crate) fn run( &mut last_view_change_time, &mut view_change_time, ); - sumeragi.view_changes_metric.set(old_view_change_index); + sumeragi + .view_changes_metric + .set(old_view_change_index as u64); sumeragi.process_message_independent( &state, @@ -1100,12 +1106,12 @@ enum BlockSyncError { BlockNotValid(BlockValidationError), SoftForkBlockNotValid(BlockValidationError), SoftForkBlockSmallViewChangeIndex { - peer_view_change_index: u64, - block_view_change_index: u64, + peer_view_change_index: usize, + block_view_change_index: usize, }, BlockNotProperHeight { - peer_height: u64, - block_height: u64, + peer_height: usize, + block_height: usize, }, } @@ -1116,7 +1122,12 @@ fn handle_block_sync<'state, F: Fn(PipelineEventBox)>( state: &'state State, handle_events: &F, ) -> Result, (SignedBlock, BlockSyncError)> { - let block_height = block.header().height; + let block_height = block + .header() + .height + .try_into() + .expect("Block height should fit into usize"); + let state_height = state.view().height(); if state_height + 1 == block_height { // Normal branch for adding new block on top of current @@ -1126,7 +1137,7 @@ fn handle_block_sync<'state, F: Fn(PipelineEventBox)>( .latest_block_ref() .expect("Not in genesis round so must have at least genesis block"); let new_peers = state_block.world.peers().cloned().collect(); - let view_change_index = block.header().view_change_index; + let view_change_index = block.header().view_change_index as usize; Topology::recreate_topology(&last_committed_block, view_change_index, new_peers) }; ValidBlock::validate( @@ -1150,7 +1161,7 @@ fn handle_block_sync<'state, F: Fn(PipelineEventBox)>( // Soft fork branch for replacing current block with valid one let peer_view_change_index = state.view().latest_block_view_change_index(); - let block_view_change_index = block.header().view_change_index; + let block_view_change_index = block.header().view_change_index as usize; if peer_view_change_index >= block_view_change_index { return Err(( block, @@ -1167,7 +1178,7 @@ fn handle_block_sync<'state, F: Fn(PipelineEventBox)>( .latest_block_ref() .expect("Not in genesis round so must have at least genesis block"); let new_peers = state_block.world.peers().cloned().collect(); - let view_change_index = block.header().view_change_index; + let view_change_index = block.header().view_change_index as usize; Topology::recreate_topology(&last_committed_block, view_change_index, new_peers) }; ValidBlock::validate( diff --git a/core/src/sumeragi/mod.rs b/core/src/sumeragi/mod.rs index a5b7175a645..94bbf160333 100644 --- a/core/src/sumeragi/mod.rs +++ b/core/src/sumeragi/mod.rs @@ -3,6 +3,7 @@ //! `Consensus` trait is now implemented only by `Sumeragi` for now. use std::{ fmt::{self, Debug, Formatter}, + num::NonZeroUsize, sync::{mpsc, Arc}, time::{Duration, Instant}, }; @@ -75,7 +76,7 @@ impl SumeragiHandle { recreate_topology: RecreateTopologyByViewChangeIndex, ) -> RecreateTopologyByViewChangeIndex { // NOTE: topology need to be updated up to block's view_change_index - let current_topology = recreate_topology(block.header().view_change_index); + let current_topology = recreate_topology(block.header().view_change_index as usize); let block = ValidBlock::validate( block.clone(), @@ -144,7 +145,7 @@ impl SumeragiHandle { let state_view = state.view(); let skip_block_count = state_view.block_hashes.len(); blocks_iter = (skip_block_count + 1..=block_count).map(|block_height| { - kura.get_block_by_height(block_height as u64).expect( + NonZeroUsize::new(block_height).and_then(|height| kura.get_block_by_height(height)).expect( "Sumeragi should be able to load the block that was reported as presented. \ If not, the block storage was probably disconnected.", ) @@ -161,10 +162,12 @@ impl SumeragiHandle { Box::new(move |_view_change_index| Topology::new(peers)) } height => { - let block_ref = kura.get_block_by_height(height).expect( - "Sumeragi could not load block that was reported as present. \ - Please check that the block storage was not disconnected.", - ); + let block_ref = NonZeroUsize::new(height) + .and_then(|height| kura.get_block_by_height(height)) + .expect( + "Sumeragi could not load block that was reported as present. \ + Please check that the block storage was not disconnected.", + ); let peers = state_view.world.peers_ids().iter().cloned().collect(); Box::new(move |view_change_index| { Topology::recreate_topology(&block_ref, view_change_index, peers) @@ -247,7 +250,7 @@ impl SumeragiHandle { } /// Closure to get topology recreated at certain view change index -type RecreateTopologyByViewChangeIndex = Box Topology>; +type RecreateTopologyByViewChangeIndex = Box Topology>; /// The interval at which sumeragi checks if there are tx in the /// `queue`. And will create a block if is leader and the voting is diff --git a/core/src/sumeragi/network_topology.rs b/core/src/sumeragi/network_topology.rs index dfa22fd9cc5..2fdac11578f 100644 --- a/core/src/sumeragi/network_topology.rs +++ b/core/src/sumeragi/network_topology.rs @@ -138,17 +138,9 @@ impl Topology { } /// Rotate peers n times where n is a number of failed attempt to create a block. - pub fn rotate_all_n(&mut self, n: u64) { - let len = self - .ordered_peers - .len() - .try_into() - .expect("`usize` should fit into `u64`"); + pub fn rotate_all_n(&mut self, n: usize) { + let len = self.ordered_peers.len(); if let Some(rem) = n.checked_rem(len) { - let rem = rem.try_into().expect( - "`rem` is smaller than `usize::MAX`, because remainder is always smaller than divisor", - ); - self.modify_peers_directly(|peers| peers.rotate_left(rem)); } } @@ -178,7 +170,7 @@ impl Topology { /// Recreate topology for given block and view change index pub fn recreate_topology( block: &SignedBlock, - view_change_index: u64, + view_change_index: usize, new_peers: UniqueVec, ) -> Self { let mut topology = Topology::new(block.commit_topology().clone()); @@ -198,10 +190,7 @@ impl Topology { // FIXME: This is a hack to prevent consensus from running amock due to // a bug in the implementation by reverting to predictable ordering - let view_change_limit: usize = view_change_index - .saturating_sub(10) - .try_into() - .expect("u64 must fit into usize"); + let view_change_limit: usize = view_change_index.saturating_sub(10); if view_change_limit > 1 { iroha_logger::error!("Restarting consensus(internal bug). Report to developers"); diff --git a/core/src/sumeragi/view_change.rs b/core/src/sumeragi/view_change.rs index 9a24f0ece33..fca76fbee67 100644 --- a/core/src/sumeragi/view_change.rs +++ b/core/src/sumeragi/view_change.rs @@ -24,7 +24,7 @@ struct ProofPayload { /// Hash of the latest committed block. latest_block_hash: Option>, /// Within a round, what is the index of the view change this proof is trying to prove. - view_change_index: u64, + view_change_index: u32, } /// The proof of a view change. It needs to be signed by f+1 peers for proof to be valid and view change to happen. @@ -41,7 +41,11 @@ pub struct ProofBuilder(SignedProof); impl ProofBuilder { /// Constructor from index. - pub fn new(latest_block_hash: Option>, view_change_index: u64) -> Self { + pub fn new(latest_block_hash: Option>, view_change_index: usize) -> Self { + let view_change_index = view_change_index + .try_into() + .expect("View change index should fit into usize"); + let proof = SignedProof { payload: ProofPayload { latest_block_hash, @@ -106,9 +110,9 @@ impl ProofChain { ) -> usize { self.iter() .enumerate() - .take_while(|(i, proof)| { + .take_while(|&(i, proof)| { proof.payload.latest_block_hash == latest_block_hash - && proof.payload.view_change_index == (*i as u64) + && proof.payload.view_change_index as usize == i && proof.verify(peers, max_faults) }) .count() @@ -119,9 +123,9 @@ impl ProofChain { let valid_count = self .iter() .enumerate() - .take_while(|(i, proof)| { + .take_while(|&(i, proof)| { proof.payload.latest_block_hash == latest_block_hash - && proof.payload.view_change_index == (*i as u64) + && proof.payload.view_change_index as usize == i }) .count(); self.truncate(valid_count); @@ -144,7 +148,7 @@ impl ProofChain { } let next_unfinished_view_change = self.verify_with_state(peers, max_faults, latest_block_hash); - if new_proof.payload.view_change_index != (next_unfinished_view_change as u64) { + if new_proof.payload.view_change_index as usize != next_unfinished_view_change { return Err(Error::ViewChangeNotFound); // We only care about the current view change that may or may not happen. } diff --git a/data_model/src/block.rs b/data_model/src/block.rs index 2655beeaca2..ad20a2cbe6e 100644 --- a/data_model/src/block.rs +++ b/data_model/src/block.rs @@ -65,7 +65,7 @@ mod model { pub timestamp_ms: u64, /// Value of view change index. Used to resolve soft forks. #[getset(skip)] - pub view_change_index: u64, + pub view_change_index: u32, /// Estimation of consensus duration (in milliseconds). pub consensus_estimation_ms: u64, } diff --git a/docs/source/references/schema.json b/docs/source/references/schema.json index 1f11b12abf7..cd36d2eff68 100644 --- a/docs/source/references/schema.json +++ b/docs/source/references/schema.json @@ -620,7 +620,7 @@ }, { "name": "view_change_index", - "type": "u64" + "type": "u32" }, { "name": "consensus_estimation_ms", diff --git a/telemetry/src/metrics.rs b/telemetry/src/metrics.rs index 5146b6016d2..2b8d3077c10 100644 --- a/telemetry/src/metrics.rs +++ b/telemetry/src/metrics.rs @@ -64,7 +64,7 @@ pub struct Status { pub uptime: Uptime, /// Number of view changes in the current round #[codec(compact)] - pub view_changes: u64, + pub view_changes: u32, /// Number of the transactions in the queue #[codec(compact)] pub queue_size: u64, @@ -79,7 +79,7 @@ impl> From<&T> for Status { txs_accepted: val.txs.with_label_values(&["accepted"]).get(), txs_rejected: val.txs.with_label_values(&["rejected"]).get(), uptime: Uptime(Duration::from_millis(val.uptime_since_genesis_ms.get())), - view_changes: val.view_changes.get(), + view_changes: val.view_changes.get().try_into().expect("Insane value"), queue_size: val.queue_size.get(), } } diff --git a/torii/src/routing.rs b/torii/src/routing.rs index 5ed4e40240e..f908b506f09 100644 --- a/torii/src/routing.rs +++ b/torii/src/routing.rs @@ -153,7 +153,7 @@ pub async fn handle_blocks_stream(kura: Arc, mut stream: WebSocket) -> eyr } // This branch sends blocks _ = interval.tick() => { - if let Some(block) = kura.get_block_by_height(from_height.get()) { + if let Some(block) = kura.get_block_by_height(from_height.try_into().expect("Number of blocks exceeds usize::MAX")) { stream // TODO: to avoid clone `BlockMessage` could be split into sending and receiving parts .send(BlockMessage(SignedBlock::clone(&block)))