Merge branch 'p2p-request-handler' into cuprated-startup

This commit is contained in:
Boog900 2024-10-06 20:08:16 +01:00
commit 49b8463d1b
No known key found for this signature in database
GPG key ID: 42AB1287CB0041C2
14 changed files with 610 additions and 49 deletions

1
Cargo.lock generated
View file

@ -550,6 +550,7 @@ version = "0.0.0"
dependencies = [
"bitflags 2.6.0",
"bytemuck",
"bytes",
"cuprate-constants",
"cuprate-database",
"cuprate-database-service",

View file

@ -17,7 +17,7 @@ use cuprate_consensus::transactions::new_tx_verification_data;
use cuprate_helper::cast::usize_to_u64;
use cuprate_types::{
blockchain::{BlockchainReadRequest, BlockchainResponse},
Chain,
Chain, TransactionVerificationData,
};
use crate::{
@ -59,7 +59,7 @@ pub enum IncomingBlockError {
/// - the block's parent is unknown
pub async fn handle_incoming_block(
block: Block,
given_txs: Vec<Transaction>,
given_txs: HashMap<[u8; 32], TransactionVerificationData>,
blockchain_read_handle: &mut BlockchainReadHandle,
) -> Result<IncomingBlockOk, IncomingBlockError> {
/// A [`HashSet`] of block hashes that the blockchain manager is currently handling.
@ -99,14 +99,6 @@ pub async fn handle_incoming_block(
}
// TODO: check we actually got given the right txs.
let prepped_txs = given_txs
.into_par_iter()
.map(|tx| {
let tx = new_tx_verification_data(tx)?;
Ok((tx.tx_hash, tx))
})
.collect::<Result<_, anyhow::Error>>()
.map_err(IncomingBlockError::InvalidBlock)?;
let Some(incoming_block_tx) = COMMAND_TX.get() else {
// We could still be starting up the blockchain manager.
@ -126,7 +118,7 @@ pub async fn handle_incoming_block(
incoming_block_tx
.send(BlockchainManagerCommand::AddBlock {
block,
prepped_txs,
prepped_txs: given_txs,
response_tx,
})
.await

View file

@ -1 +1,290 @@
use std::{
collections::HashSet,
future::{ready, Ready},
task::{Context, Poll},
};
use bytes::Bytes;
use futures::{future::BoxFuture, FutureExt};
use monero_serai::{block::Block, transaction::Transaction};
use tower::{Service, ServiceExt};
use cuprate_blockchain::service::BlockchainReadHandle;
use cuprate_consensus::{transactions::new_tx_verification_data, BlockChainContextService};
use cuprate_fixed_bytes::ByteArrayVec;
use cuprate_helper::{
asynch::rayon_spawn_async,
cast::usize_to_u64,
map::{combine_low_high_bits_to_u128, split_u128_into_low_high_bits},
};
use cuprate_p2p::constants::MAX_BLOCK_BATCH_LEN;
use cuprate_p2p_core::{client::PeerInformation, NetworkZone, ProtocolRequest, ProtocolResponse};
use cuprate_types::{
blockchain::{BlockchainReadRequest, BlockchainResponse},
BlockCompleteEntry, MissingTxsInBlock, TransactionBlobs,
};
use cuprate_wire::protocol::{
ChainRequest, ChainResponse, FluffyMissingTransactionsRequest, GetObjectsRequest,
GetObjectsResponse, NewFluffyBlock,
};
use crate::blockchain::interface::{self as blockchain_interface, IncomingBlockError};
/// The P2P protocol request handler [`MakeService`](tower::MakeService).
#[derive(Clone)]
pub struct P2pProtocolRequestHandlerMaker {
/// The [`BlockchainReadHandle`]
pub blockchain_read_handle: BlockchainReadHandle,
}
impl<N: NetworkZone> Service<PeerInformation<N>> for P2pProtocolRequestHandlerMaker {
type Response = P2pProtocolRequestHandler<N>;
type Error = tower::BoxError;
type Future = Ready<Result<Self::Response, Self::Error>>;
fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
Poll::Ready(Ok(()))
}
fn call(&mut self, peer_information: PeerInformation<N>) -> Self::Future {
// TODO: check sync info?
let blockchain_read_handle = self.blockchain_read_handle.clone();
ready(Ok(P2pProtocolRequestHandler {
peer_information,
blockchain_read_handle,
}))
}
}
/// The P2P protocol request handler.
#[derive(Clone)]
pub struct P2pProtocolRequestHandler<N: NetworkZone> {
/// The [`PeerInformation`] for this peer.
peer_information: PeerInformation<N>,
/// The [`BlockchainReadHandle`]
blockchain_read_handle: BlockchainReadHandle,
}
impl<Z: NetworkZone> Service<ProtocolRequest> for P2pProtocolRequestHandler<Z> {
type Response = ProtocolResponse;
type Error = anyhow::Error;
type Future = BoxFuture<'static, Result<Self::Response, Self::Error>>;
fn poll_ready(&mut self, _: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
Poll::Ready(Ok(()))
}
fn call(&mut self, request: ProtocolRequest) -> Self::Future {
match request {
ProtocolRequest::GetObjects(r) => {
get_objects(r, self.blockchain_read_handle.clone()).boxed()
}
ProtocolRequest::GetChain(r) => {
get_chain(r, self.blockchain_read_handle.clone()).boxed()
}
ProtocolRequest::FluffyMissingTxs(r) => {
fluffy_missing_txs(r, self.blockchain_read_handle.clone()).boxed()
}
ProtocolRequest::NewBlock(_) => ready(Err(anyhow::anyhow!(
"Peer sent a full block when we support fluffy blocks"
)))
.boxed(),
ProtocolRequest::NewFluffyBlock(r) => {
new_fluffy_block(r, self.blockchain_read_handle.clone()).boxed()
}
ProtocolRequest::GetTxPoolCompliment(_) | ProtocolRequest::NewTransactions(_) => {
ready(Ok(ProtocolResponse::NA)).boxed()
} // TODO: tx-pool
}
}
}
//---------------------------------------------------------------------------------------------------- Handler functions
/// [`ProtocolRequest::GetObjects`]
async fn get_objects(
request: GetObjectsRequest,
mut blockchain_read_handle: BlockchainReadHandle,
) -> anyhow::Result<ProtocolResponse> {
if request.blocks.len() > MAX_BLOCK_BATCH_LEN {
anyhow::bail!("Peer requested more blocks than allowed.")
}
let block_hashes: Vec<[u8; 32]> = (&request.blocks).into();
// deallocate the backing `Bytes`.
drop(request);
let BlockchainResponse::BlockCompleteEntries {
blocks,
missing_hashes,
blockchain_height,
} = blockchain_read_handle
.ready()
.await?
.call(BlockchainReadRequest::BlockCompleteEntries(block_hashes))
.await?
else {
panic!("blockchain returned wrong response!");
};
Ok(ProtocolResponse::GetObjects(GetObjectsResponse {
blocks,
missed_ids: ByteArrayVec::from(missing_hashes),
current_blockchain_height: usize_to_u64(blockchain_height),
}))
}
/// [`ProtocolRequest::GetChain`]
async fn get_chain(
request: ChainRequest,
mut blockchain_read_handle: BlockchainReadHandle,
) -> anyhow::Result<ProtocolResponse> {
if request.block_ids.len() > 25_000 {
anyhow::bail!("Peer sent too many block hashes in chain request.")
}
let block_hashes: Vec<[u8; 32]> = (&request.block_ids).into();
let want_pruned_data = request.prune;
// deallocate the backing `Bytes`.
drop(request);
let BlockchainResponse::NextChainEntry {
start_height,
chain_height,
block_ids,
block_weights,
cumulative_difficulty,
first_block_blob,
} = blockchain_read_handle
.ready()
.await?
.call(BlockchainReadRequest::NextChainEntry(block_hashes, 10_000))
.await?
else {
panic!("blockchain returned wrong response!");
};
if start_height == 0 {
anyhow::bail!("The peers chain has a different genesis block than ours.");
}
let (cumulative_difficulty_low64, cumulative_difficulty_top64) =
split_u128_into_low_high_bits(cumulative_difficulty);
Ok(ProtocolResponse::GetChain(ChainResponse {
start_height: usize_to_u64(start_height),
total_height: usize_to_u64(chain_height),
cumulative_difficulty_low64,
cumulative_difficulty_top64,
m_block_ids: ByteArrayVec::from(block_ids),
first_block: first_block_blob.map_or(Bytes::new(), Bytes::from),
// only needed when
m_block_weights: if want_pruned_data {
block_weights.into_iter().map(usize_to_u64).collect()
} else {
vec![]
},
}))
}
/// [`ProtocolRequest::FluffyMissingTxs`]
async fn fluffy_missing_txs(
mut request: FluffyMissingTransactionsRequest,
mut blockchain_read_handle: BlockchainReadHandle,
) -> anyhow::Result<ProtocolResponse> {
let tx_indexes = std::mem::take(&mut request.missing_tx_indices);
let block_hash: [u8; 32] = *request.block_hash;
let current_blockchain_height = request.current_blockchain_height;
// deallocate the backing `Bytes`.
drop(request);
let BlockchainResponse::MissingTxsInBlock(res) = blockchain_read_handle
.ready()
.await?
.call(BlockchainReadRequest::MissingTxsInBlock {
block_hash,
tx_indexes,
})
.await?
else {
panic!("blockchain returned wrong response!");
};
let Some(MissingTxsInBlock { block, txs }) = res else {
anyhow::bail!("The peer requested txs out of range.");
};
Ok(ProtocolResponse::NewFluffyBlock(NewFluffyBlock {
b: BlockCompleteEntry {
block: Bytes::from(block),
txs: TransactionBlobs::Normal(txs.into_iter().map(Bytes::from).collect()),
pruned: false,
// only needed for pruned blocks.
block_weight: 0,
},
current_blockchain_height,
}))
}
/// [`ProtocolRequest::NewFluffyBlock`]
async fn new_fluffy_block(
request: NewFluffyBlock,
mut blockchain_read_handle: BlockchainReadHandle,
) -> anyhow::Result<ProtocolResponse> {
let current_blockchain_height = request.current_blockchain_height;
let (block, txs) = rayon_spawn_async(move || -> Result<_, anyhow::Error> {
let block = Block::read(&mut request.b.block.as_ref())?;
let tx_blobs = request
.b
.txs
.take_normal()
.ok_or(anyhow::anyhow!("Peer sent pruned txs in fluffy block"))?;
let mut txs_in_block = block.transactions.iter().copied().collect::<HashSet<_>>();
// TODO: size check these tx blobs
let txs = tx_blobs
.into_iter()
.map(|tx_blob| {
let tx = Transaction::read(&mut tx_blob.as_ref())?;
let tx = new_tx_verification_data(tx)?;
if !txs_in_block.remove(&tx.tx_hash) {
anyhow::bail!("Peer sent tx in fluffy block that wasn't actually in block")
}
Ok((tx.tx_hash, tx))
})
.collect::<Result<_, anyhow::Error>>()?;
// The backing `Bytes` will be deallocated when this closure returns.
Ok((block, txs))
})
.await?;
let res =
blockchain_interface::handle_incoming_block(block, txs, &mut blockchain_read_handle).await;
match res {
Ok(_) => Ok(ProtocolResponse::NA),
Err(IncomingBlockError::UnknownTransactions(block_hash, missing_tx_indices)) => Ok(
ProtocolResponse::FluffyMissingTransactionsRequest(FluffyMissingTransactionsRequest {
block_hash: block_hash.into(),
current_blockchain_height,
missing_tx_indices,
}),
),
Err(IncomingBlockError::Orphan) => {
// Block's parent was unknown, could be syncing?
Ok(ProtocolResponse::NA)
}
Err(e) => Err(e.into()),
}
}

View file

@ -159,7 +159,7 @@ epee_object!(
current_blockchain_height: u64,
);
/// A request for Txs we are missing from our `TxPool`
/// A request for txs we are missing from an incoming block.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct FluffyMissingTransactionsRequest {
/// The Block we are missing the Txs in

View file

@ -116,6 +116,7 @@ pub enum ProtocolResponse {
GetChain(ChainResponse),
NewFluffyBlock(NewFluffyBlock),
NewTransactions(NewTransactions),
FluffyMissingTransactionsRequest(FluffyMissingTransactionsRequest),
NA,
}
@ -139,6 +140,9 @@ impl PeerResponse {
ProtocolResponse::GetChain(_) => MessageID::GetChain,
ProtocolResponse::NewFluffyBlock(_) => MessageID::NewBlock,
ProtocolResponse::NewTransactions(_) => MessageID::NewFluffyBlock,
ProtocolResponse::FluffyMissingTransactionsRequest(_) => {
MessageID::FluffyMissingTxs
}
ProtocolResponse::NA => return None,
},

View file

@ -71,6 +71,9 @@ impl TryFrom<ProtocolResponse> for ProtocolMessage {
ProtocolResponse::NewFluffyBlock(val) => Self::NewFluffyBlock(val),
ProtocolResponse::GetChain(val) => Self::ChainEntryResponse(val),
ProtocolResponse::GetObjects(val) => Self::GetObjectsResponse(val),
ProtocolResponse::FluffyMissingTransactionsRequest(val) => {
Self::FluffyMissingTransactionsRequest(val)
}
ProtocolResponse::NA => return Err(MessageConversionError),
})
}

View file

@ -52,7 +52,7 @@ pub(crate) const INITIAL_CHAIN_REQUESTS_TO_SEND: usize = 3;
/// The enforced maximum amount of blocks to request in a batch.
///
/// Requesting more than this will cause the peer to disconnect and potentially lead to bans.
pub(crate) const MAX_BLOCK_BATCH_LEN: usize = 100;
pub const MAX_BLOCK_BATCH_LEN: usize = 100;
/// The timeout that the block downloader will use for requests.
pub(crate) const BLOCK_DOWNLOADER_REQUEST_TIMEOUT: Duration = Duration::from_secs(30);

View file

@ -36,6 +36,7 @@ serde = { workspace = true, optional = true }
tower = { workspace = true }
thread_local = { workspace = true, optional = true }
rayon = { workspace = true, optional = true }
bytes = "1.7.2"
[dev-dependencies]
cuprate-constants = { path = "../../constants" }

View file

@ -2,21 +2,23 @@
//---------------------------------------------------------------------------------------------------- Import
use bytemuck::TransparentWrapper;
use bytes::Bytes;
use monero_serai::{
block::{Block, BlockHeader},
transaction::Transaction,
};
use cuprate_database::{
RuntimeError, StorableVec, {DatabaseRo, DatabaseRw},
RuntimeError, StorableVec, {DatabaseIter, DatabaseRo, DatabaseRw},
};
use cuprate_helper::cast::usize_to_u64;
use cuprate_helper::{
map::{combine_low_high_bits_to_u128, split_u128_into_low_high_bits},
tx::tx_fee,
};
use cuprate_types::{
AltBlockInformation, ChainId, ExtendedBlockHeader, HardFork, VerifiedBlockInformation,
VerifiedTransactionInformation,
AltBlockInformation, BlockCompleteEntry, ChainId, ExtendedBlockHeader, HardFork,
TransactionBlobs, VerifiedBlockInformation, VerifiedTransactionInformation,
};
use crate::{
@ -27,7 +29,7 @@ use crate::{
output::get_rct_num_outputs,
tx::{add_tx, remove_tx},
},
tables::{BlockHeights, BlockInfos, Tables, TablesMut},
tables::{BlockHeights, BlockInfos, Tables, TablesIter, TablesMut},
types::{BlockHash, BlockHeight, BlockInfo},
};
@ -224,6 +226,60 @@ pub fn pop_block(
Ok((block_height, block_info.block_hash, block))
}
//---------------------------------------------------------------------------------------------------- `get_block_blob_with_tx_indexes`
pub fn get_block_blob_with_tx_indexes(
block_height: &BlockHeight,
tables: &impl Tables,
) -> Result<(Vec<u8>, u64, usize), RuntimeError> {
use monero_serai::io::write_varint;
let block_info = tables.block_infos().get(block_height)?;
let miner_tx_idx = block_info.mining_tx_index;
let block_txs = tables.block_txs_hashes().get(block_height)?.0;
let numb_txs = block_txs.len();
// Get the block header
let mut block = tables.block_header_blobs().get(block_height)?.0;
// Add the miner tx to the blob.
let mut miner_tx_blob = tables.tx_blobs().get(&miner_tx_idx)?.0;
block.append(&mut miner_tx_blob);
// Add the blocks tx hashes.
write_varint(&block_txs.len(), &mut block)
.expect("The number of txs per block will not exceed u64::MAX");
for tx in block_txs {
block.extend_from_slice(&tx);
}
Ok((block, miner_tx_idx, numb_txs))
}
//---------------------------------------------------------------------------------------------------- `get_block_extended_header_*`
pub fn get_block_complete_entry(
block_hash: &BlockHash,
tables: &impl TablesIter,
) -> Result<BlockCompleteEntry, RuntimeError> {
let block_height = tables.block_heights().get(block_hash)?;
let (block_blob, miner_tx_idx, numb_non_miner_txs) =
get_block_blob_with_tx_indexes(&block_height, tables)?;
let first_tx_idx = miner_tx_idx + 1;
let tx_blobs = tables
.tx_blobs_iter()
.get_range(first_tx_idx..=usize_to_u64(numb_non_miner_txs))?
.map(|tx_blob| Ok(Bytes::from(tx_blob?.0)))
.collect::<Result<_, RuntimeError>>()?;
Ok(BlockCompleteEntry {
block: Bytes::from(block_blob),
txs: TransactionBlobs::Normal(tx_blobs),
pruned: false,
block_weight: 0,
})
}
//---------------------------------------------------------------------------------------------------- `get_block_extended_header_*`
/// Retrieve a [`ExtendedBlockHeader`] from the database.

View file

@ -3,6 +3,8 @@
//---------------------------------------------------------------------------------------------------- Import
use cuprate_database::{DatabaseRo, RuntimeError};
use crate::ops::block::block_exists;
use crate::types::BlockHash;
use crate::{
ops::macros::doc_error,
tables::{BlockHeights, BlockInfos},
@ -78,6 +80,45 @@ pub fn cumulative_generated_coins(
}
}
/// Find the split point between our chain and a list of [`BlockHash`]s from another chain.
///
/// This function can be used accepts chains in chronological and reverse chronological order, however
/// if the wrong order is specified the return value is meaningless.
///
/// For chronologically ordered chains this will return the index of the first unknown, for reverse
/// chronologically ordered chains this will return the index of the fist known.
///
/// If all blocks are known for chronologically ordered chains or unknown for reverse chronologically
/// ordered chains then the length of the chain will be returned.
#[doc = doc_error!()]
#[inline]
pub fn find_split_point(
block_ids: &[BlockHash],
chronological_order: bool,
table_block_heights: &impl DatabaseRo<BlockHeights>,
) -> Result<usize, RuntimeError> {
let mut err = None;
// Do a binary search to find the first unknown block in the batch.
let idx =
block_ids.partition_point(
|block_id| match block_exists(block_id, table_block_heights) {
Ok(exists) => exists & chronological_order,
Err(e) => {
err.get_or_insert(e);
// if this happens the search is scrapped, just return `false` back.
false
}
},
);
if let Some(e) = err {
return Err(e);
}
Ok(idx)
}
//---------------------------------------------------------------------------------------------------- Tests
#[cfg(test)]
mod test {

View file

@ -2,25 +2,18 @@
//---------------------------------------------------------------------------------------------------- Import
use std::{
cmp::min,
collections::{HashMap, HashSet},
sync::Arc,
};
use rayon::{
iter::{IntoParallelIterator, ParallelIterator},
iter::{Either, IntoParallelIterator, ParallelIterator},
prelude::*,
ThreadPool,
};
use thread_local::ThreadLocal;
use cuprate_database::{ConcreteEnv, DatabaseRo, Env, EnvInner, RuntimeError};
use cuprate_database_service::{init_thread_pool, DatabaseReadService, ReaderThreads};
use cuprate_helper::map::combine_low_high_bits_to_u128;
use cuprate_types::{
blockchain::{BlockchainReadRequest, BlockchainResponse},
Chain, ChainId, ExtendedBlockHeader, OutputOnChain,
};
use crate::{
ops::{
alt_block::{
@ -28,9 +21,10 @@ use crate::{
get_alt_chain_history_ranges,
},
block::{
block_exists, get_block_extended_header_from_height, get_block_height, get_block_info,
block_exists, get_block_blob_with_tx_indexes, get_block_complete_entry,
get_block_extended_header_from_height, get_block_height, get_block_info,
},
blockchain::{cumulative_generated_coins, top_block_height},
blockchain::{cumulative_generated_coins, find_split_point, top_block_height},
key_image::key_image_exists,
output::id_to_output_on_chain,
},
@ -38,11 +32,18 @@ use crate::{
free::{compact_history_genesis_not_included, compact_history_index_to_height_offset},
types::{BlockchainReadHandle, ResponseResult},
},
tables::{AltBlockHeights, BlockHeights, BlockInfos, OpenTables, Tables},
tables::{AltBlockHeights, BlockHeights, BlockInfos, OpenTables, Tables, TablesIter},
types::{
AltBlockHeight, Amount, AmountIndex, BlockHash, BlockHeight, KeyImage, PreRctOutputId,
},
};
use cuprate_database::{ConcreteEnv, DatabaseIter, DatabaseRo, Env, EnvInner, RuntimeError};
use cuprate_database_service::{init_thread_pool, DatabaseReadService, ReaderThreads};
use cuprate_helper::map::combine_low_high_bits_to_u128;
use cuprate_types::{
blockchain::{BlockchainReadRequest, BlockchainResponse},
Chain, ChainId, ExtendedBlockHeader, MissingTxsInBlock, OutputOnChain,
};
//---------------------------------------------------------------------------------------------------- init_read_service
/// Initialize the [`BlockchainReadHandle`] thread-pool backed by [`rayon`].
@ -92,6 +93,7 @@ fn map_request(
/* SOMEDAY: pre-request handling, run some code for each request? */
match request {
R::BlockCompleteEntries(block_hashes) => block_complete_entries(env, block_hashes),
R::BlockExtendedHeader(block) => block_extended_header(env, block),
R::BlockHash(block, chain) => block_hash(env, block, chain),
R::FindBlock(block_hash) => find_block(env, block_hash),
@ -105,7 +107,12 @@ fn map_request(
R::NumberOutputsWithAmount(vec) => number_outputs_with_amount(env, vec),
R::KeyImagesSpent(set) => key_images_spent(env, set),
R::CompactChainHistory => compact_chain_history(env),
R::NextChainEntry(block_hashes, amount) => next_chain_entry(env, &block_hashes, amount),
R::FindFirstUnknown(block_ids) => find_first_unknown(env, &block_ids),
R::MissingTxsInBlock {
block_hash,
tx_indexes,
} => missing_txs_in_block(env, block_hash, tx_indexes),
R::AltBlocksInChain(chain_id) => alt_blocks_in_chain(env, chain_id),
}
@ -182,6 +189,38 @@ macro_rules! get_tables {
// TODO: The overhead of parallelism may be too much for every request, perfomace test to find optimal
// amount of parallelism.
/// [`BlockchainReadRequest::BlockCompleteEntries`].
fn block_complete_entries(env: &ConcreteEnv, block_hashes: Vec<BlockHash>) -> ResponseResult {
// Prepare tx/tables in `ThreadLocal`.
let env_inner = env.env_inner();
let tx_ro = thread_local(env);
let tables = thread_local(env);
let (missing_hashes, blocks) = block_hashes
.into_par_iter()
.map(|block_hash| {
let tx_ro = tx_ro.get_or_try(|| env_inner.tx_ro())?;
let tables = get_tables!(env_inner, tx_ro, tables)?.as_ref();
match get_block_complete_entry(&block_hash, tables) {
Err(RuntimeError::KeyNotFound) => Ok(Either::Left(block_hash)),
res => res.map(Either::Right),
}
})
.collect::<Result<_, _>>()?;
let tx_ro = tx_ro.get_or_try(|| env_inner.tx_ro())?;
let tables = get_tables!(env_inner, tx_ro, tables)?.as_ref();
let blockchain_height = crate::ops::blockchain::chain_height(tables.block_heights())?;
Ok(BlockchainResponse::BlockCompleteEntries {
blocks,
missing_hashes,
blockchain_height,
})
}
/// [`BlockchainReadRequest::BlockExtendedHeader`].
#[inline]
fn block_extended_header(env: &ConcreteEnv, block_height: BlockHeight) -> ResponseResult {
@ -518,6 +557,76 @@ fn compact_chain_history(env: &ConcreteEnv) -> ResponseResult {
})
}
/// [`BlockchainReadRequest::NextChainEntry`]
///
/// # Invariant
/// `block_ids` must be sorted in reverse chronological block order, or else
/// the returned result is unspecified and meaningless, as this function
/// performs a binary search.
fn next_chain_entry(
env: &ConcreteEnv,
block_ids: &[BlockHash],
next_entry_size: usize,
) -> ResponseResult {
// Single-threaded, no `ThreadLocal` required.
let env_inner = env.env_inner();
let tx_ro = env_inner.tx_ro()?;
let tables = env_inner.open_tables(&tx_ro)?;
let table_block_heights = tables.block_heights();
let table_block_infos = tables.block_infos_iter();
let idx = find_split_point(block_ids, false, table_block_heights)?;
// This will happen if we have a different genesis block.
if idx == block_ids.len() {
return Ok(BlockchainResponse::NextChainEntry {
start_height: 0,
chain_height: 0,
block_ids: vec![],
block_weights: vec![],
cumulative_difficulty: 0,
first_block_blob: None,
});
}
// The returned chain entry must overlap with one of the blocks we were told about.
let first_known_block_hash = block_ids[idx];
let first_known_height = table_block_heights.get(&first_known_block_hash)?;
let chain_height = crate::ops::blockchain::chain_height(table_block_heights)?;
let last_height_in_chain_entry = min(first_known_height + next_entry_size, chain_height);
let (block_ids, block_weights) = table_block_infos
.get_range(first_known_height..last_height_in_chain_entry)?
.map(|block_info| {
let block_info = block_info?;
Ok((block_info.block_hash, block_info.weight))
})
.collect::<Result<(Vec<_>, Vec<_>), RuntimeError>>()?;
let top_block_info = table_block_infos.get(&(chain_height - 1))?;
let first_block_blob = if block_ids.len() >= 2 {
Some(get_block_blob_with_tx_indexes(&(first_known_height + 1), &tables)?.0)
} else {
None
};
Ok(BlockchainResponse::NextChainEntry {
start_height: first_known_height,
chain_height,
block_ids,
block_weights,
cumulative_difficulty: combine_low_high_bits_to_u128(
top_block_info.cumulative_difficulty_low,
top_block_info.cumulative_difficulty_high,
),
first_block_blob,
})
}
/// [`BlockchainReadRequest::FindFirstUnknown`]
///
/// # Invariant
@ -530,24 +639,7 @@ fn find_first_unknown(env: &ConcreteEnv, block_ids: &[BlockHash]) -> ResponseRes
let table_block_heights = env_inner.open_db_ro::<BlockHeights>(&tx_ro)?;
let mut err = None;
// Do a binary search to find the first unknown block in the batch.
let idx =
block_ids.partition_point(
|block_id| match block_exists(block_id, &table_block_heights) {
Ok(exists) => exists,
Err(e) => {
err.get_or_insert(e);
// if this happens the search is scrapped, just return `false` back.
false
}
},
);
if let Some(e) = err {
return Err(e);
}
let idx = find_split_point(block_ids, true, &table_block_heights)?;
Ok(if idx == block_ids.len() {
BlockchainResponse::FindFirstUnknown(None)
@ -560,6 +652,36 @@ fn find_first_unknown(env: &ConcreteEnv, block_ids: &[BlockHash]) -> ResponseRes
})
}
/// [`BlockchainReadRequest::MissingTxsInBlock`]
fn missing_txs_in_block(
env: &ConcreteEnv,
block_hash: [u8; 32],
missing_txs: Vec<u64>,
) -> ResponseResult {
// Single-threaded, no `ThreadLocal` required.
let env_inner = env.env_inner();
let tx_ro = env_inner.tx_ro()?;
let tables = env_inner.open_tables(&tx_ro)?;
let block_height = tables.block_heights().get(&block_hash)?;
let (block, miner_tx_index, numb_txs) = get_block_blob_with_tx_indexes(&block_height, &tables)?;
let first_tx_index = miner_tx_index + 1;
if numb_txs < missing_txs.len() {
return Ok(BlockchainResponse::MissingTxsInBlock(None));
}
let txs = missing_txs
.into_iter()
.map(|index_offset| Ok(tables.tx_blobs().get(&(first_tx_index + index_offset))?.0))
.collect::<Result<_, RuntimeError>>()?;
Ok(BlockchainResponse::MissingTxsInBlock(Some(
MissingTxsInBlock { block, txs },
)))
}
/// [`BlockchainReadRequest::AltBlocksInChain`]
fn alt_blocks_in_chain(env: &ConcreteEnv, chain_id: ChainId) -> ResponseResult {
// Prepare tx/tables in `ThreadLocal`.

View file

@ -8,9 +8,10 @@ use std::{
ops::Range,
};
use crate::types::MissingTxsInBlock;
use crate::{
types::{Chain, ExtendedBlockHeader, OutputOnChain, VerifiedBlockInformation},
AltBlockInformation, ChainId,
AltBlockInformation, BlockCompleteEntry, ChainId,
};
//---------------------------------------------------------------------------------------------------- ReadRequest
@ -24,6 +25,8 @@ use crate::{
/// See `Response` for the expected responses per `Request`.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum BlockchainReadRequest {
BlockCompleteEntries(Vec<[u8; 32]>),
/// Request a block's extended header.
///
/// The input is the block's height.
@ -93,6 +96,16 @@ pub enum BlockchainReadRequest {
/// A request for the compact chain history.
CompactChainHistory,
/// A request for the next chain entry.
///
/// Input is a list of block hashes and the amount of block hashes to return in the next chain entry.
///
/// # Invariant
/// The [`Vec`] containing the block IDs must be sorted in reverse chronological block
/// order, or else the returned response is unspecified and meaningless,
/// as this request performs a binary search
NextChainEntry(Vec<[u8; 32]>, usize),
/// A request to find the first unknown block ID in a list of block IDs.
///
/// # Invariant
@ -101,6 +114,16 @@ pub enum BlockchainReadRequest {
/// as this request performs a binary search.
FindFirstUnknown(Vec<[u8; 32]>),
/// A request for transactions from a specific block.
MissingTxsInBlock {
/// The block to get transactions from.
block_hash: [u8; 32],
/// The indexes of the transactions from the block.
/// This is not the global index of the txs, instead it is the local index as they appear in
/// the block/
tx_indexes: Vec<u64>,
},
/// A request for all alt blocks in the chain with the given [`ChainId`].
AltBlocksInChain(ChainId),
}
@ -149,6 +172,12 @@ pub enum BlockchainWriteRequest {
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum BlockchainResponse {
//------------------------------------------------------ Reads
BlockCompleteEntries {
blocks: Vec<BlockCompleteEntry>,
missing_hashes: Vec<[u8; 32]>,
blockchain_height: usize,
},
/// Response to [`BlockchainReadRequest::BlockExtendedHeader`].
///
/// Inner value is the extended headed of the requested block.
@ -215,6 +244,18 @@ pub enum BlockchainResponse {
cumulative_difficulty: u128,
},
/// Response to [`BlockchainReadRequest::NextChainEntry`].
///
/// If all blocks were unknown `start_height` will be `0`, the other fields will be meaningless.
NextChainEntry {
start_height: usize,
chain_height: usize,
block_ids: Vec<[u8; 32]>,
block_weights: Vec<usize>,
cumulative_difficulty: u128,
first_block_blob: Option<Vec<u8>>,
},
/// The response for [`BlockchainReadRequest::FindFirstUnknown`].
///
/// Contains the index of the first unknown block and its expected height.
@ -222,6 +263,11 @@ pub enum BlockchainResponse {
/// This will be [`None`] if all blocks were known.
FindFirstUnknown(Option<(usize, usize)>),
/// The response for [`BlockchainReadRequest::MissingTxsInBlock`].
///
/// Will return [`None`] if the request contained an index out of range.
MissingTxsInBlock(Option<MissingTxsInBlock>),
/// The response for [`BlockchainReadRequest::AltBlocksInChain`].
///
/// Contains all the alt blocks in the alt-chain in chronological order.

View file

@ -20,7 +20,7 @@ pub use transaction_verification_data::{
CachedVerificationState, TransactionVerificationData, TxVersion,
};
pub use types::{
AltBlockInformation, Chain, ChainId, ExtendedBlockHeader, OutputOnChain,
AltBlockInformation, Chain, ChainId, ExtendedBlockHeader, MissingTxsInBlock, OutputOnChain,
VerifiedBlockInformation, VerifiedTransactionInformation,
};

View file

@ -155,6 +155,12 @@ pub struct OutputOnChain {
pub commitment: EdwardsPoint,
}
#[derive(Clone, Debug, PartialEq, Eq)]
pub struct MissingTxsInBlock {
pub block: Vec<u8>,
pub txs: Vec<Vec<u8>>,
}
//---------------------------------------------------------------------------------------------------- Tests
#[cfg(test)]
mod test {