integrate ringCT rules into the RPC scanner.

This commit is contained in:
Boog900 2023-12-27 23:50:18 +00:00
parent f037532610
commit 5d6fb3f6b9
No known key found for this signature in database
GPG key ID: 5401367FB7302004
12 changed files with 357 additions and 151 deletions

View file

@ -5,7 +5,6 @@ mod decomposed_amount;
pub mod genesis;
pub mod hard_forks;
pub mod miner_tx;
pub mod signatures;
pub mod transactions;
pub use decomposed_amount::is_decomposed_amount;
@ -18,8 +17,6 @@ pub enum ConsensusError {
Block(#[from] blocks::BlockError),
#[error("Transaction error: {0}")]
Transaction(#[from] transactions::TransactionError),
#[error("Signatures error: {0}")]
Signatures(#[from] signatures::SignatureError),
}
/// Checks that a point is canonically encoded.

View file

@ -1,25 +0,0 @@
use monero_serai::transaction::Transaction;
use crate::transactions::Rings;
mod ring_signatures;
#[derive(Debug, Clone, Copy, PartialEq, Eq, thiserror::Error)]
pub enum SignatureError {
#[error("Number of signatures is different to the amount required.")]
MismatchSignatureSize,
#[error("The signature is incorrect.")]
IncorrectSignature,
}
pub fn verify_contextual_signatures(tx: &Transaction, rings: &Rings) -> Result<(), SignatureError> {
match rings {
Rings::Legacy(_) => ring_signatures::verify_inputs_signatures(
&tx.prefix.inputs,
&tx.signatures,
rings,
&tx.signature_hash(),
),
_ => panic!("TODO: RCT"),
}
}

View file

@ -1,12 +1,16 @@
use std::{cmp::Ordering, collections::HashSet, sync::Arc};
use monero_serai::transaction::{Input, Output, Timelock};
use monero_serai::transaction::{Input, Output, Timelock, Transaction};
use multiexp::BatchVerifier;
use crate::{check_point_canonically_encoded, is_decomposed_amount, HardFork};
mod contextual_data;
mod ring_ct;
mod ring_signatures;
pub use contextual_data::*;
pub use ring_ct::RingCTError;
#[derive(Debug, Clone, Copy, PartialEq, Eq, thiserror::Error)]
pub enum TransactionError {
@ -48,6 +52,12 @@ pub enum TransactionError {
NoInputs,
#[error("Ring member not in database")]
RingMemberNotFound,
//-------------------------------------------------------- Ring Signatures
#[error("Ring signature incorrect.")]
RingSignatureIncorrect,
//-------------------------------------------------------- RingCT
#[error("RingCT Error: {0}.")]
RingCTError(#[from] ring_ct::RingCTError),
}
#[derive(Debug, Copy, Clone, Eq, PartialEq, Ord, PartialOrd)]
@ -148,7 +158,7 @@ fn sum_outputs_v1(outputs: &[Output], hf: &HardFork) -> Result<u64, TransactionE
}
/// Checks the outputs against all output consensus rules, returning the sum of the output amounts.
pub fn check_outputs(
fn check_outputs_semantics(
outputs: &[Output],
hf: &HardFork,
tx_version: &TxVersion,
@ -158,37 +168,12 @@ pub fn check_outputs(
match tx_version {
TxVersion::RingSignatures => sum_outputs_v1(outputs, hf),
_ => todo!("RingCT"),
TxVersion::RingCT => Ok(0), // RCT outputs are checked to be zero in RCT checks.
}
}
//----------------------------------------------------------------------------------------------------------- TIME LOCKS
/// Checks all the time locks are unlocked.
///
/// `current_time_lock_timestamp` must be: https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#getting-the-current-time
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#unlock-time
pub fn check_all_time_locks(
time_locks: &[Timelock],
current_chain_height: u64,
current_time_lock_timestamp: u64,
hf: &HardFork,
) -> Result<(), TransactionError> {
time_locks.iter().try_for_each(|time_lock| {
if !output_unlocked(
time_lock,
current_chain_height,
current_time_lock_timestamp,
hf,
) {
Err(TransactionError::OneOrMoreDecoysLocked)
} else {
Ok(())
}
})
}
/// Checks if an outputs unlock time has passed.
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#unlock-time
@ -229,6 +214,31 @@ fn check_timestamp_time_lock(
current_time_lock_timestamp + hf.block_time().as_secs() >= unlock_timestamp
}
/// Checks all the time locks are unlocked.
///
/// `current_time_lock_timestamp` must be: https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#getting-the-current-time
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#unlock-time
fn check_all_time_locks(
time_locks: &[Timelock],
current_chain_height: u64,
current_time_lock_timestamp: u64,
hf: &HardFork,
) -> Result<(), TransactionError> {
time_locks.iter().try_for_each(|time_lock| {
if !output_unlocked(
time_lock,
current_chain_height,
current_time_lock_timestamp,
hf,
) {
Err(TransactionError::OneOrMoreDecoysLocked)
} else {
Ok(())
}
})
}
//----------------------------------------------------------------------------------------------------------- INPUTS
/// Checks the decoys are allowed.
@ -273,7 +283,7 @@ fn check_decoy_info(decoy_info: &DecoyInfo, hf: &HardFork) -> Result<(), Transac
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#unique-key-image
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#torsion-free-key-image
pub(crate) fn check_key_images(
fn check_key_images(
input: &Input,
spent_kis: &mut HashSet<[u8; 32]>,
) -> Result<(), TransactionError> {
@ -351,7 +361,7 @@ fn check_inputs_sorted(inputs: &[Input], hf: &HardFork) -> Result<(), Transactio
if hf >= &HardFork::V7 {
for inps in inputs.windows(2) {
match get_ki(&inps[0])?.cmp(&get_ki(&inps[1])?) {
Ordering::Less => (),
Ordering::Greater => (),
_ => return Err(TransactionError::InputsAreNotOrdered),
}
}
@ -383,7 +393,7 @@ fn check_10_block_lock(
/// Sums the inputs checking for overflow.
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#inputs-and-outputs-must-not-overflow
fn sum_inputs_v1(inputs: &[Input]) -> Result<u64, TransactionError> {
fn sum_inputs_check_overflow(inputs: &[Input]) -> Result<u64, TransactionError> {
let mut sum: u64 = 0;
for inp in inputs {
match inp {
@ -399,22 +409,30 @@ fn sum_inputs_v1(inputs: &[Input]) -> Result<u64, TransactionError> {
Ok(sum)
}
/// Checks all input consensus rules.
///
/// TODO: list rules.
///
pub fn check_inputs(
inputs: &[Input],
tx_ring_members_info: &TxRingMembersInfo,
current_chain_height: u64,
hf: &HardFork,
tx_version: &TxVersion,
spent_kis: Arc<std::sync::Mutex<HashSet<[u8; 32]>>>,
) -> Result<u64, TransactionError> {
fn check_inputs_semantics(inputs: &[Input], hf: &HardFork) -> Result<u64, TransactionError> {
if inputs.is_empty() {
return Err(TransactionError::NoInputs);
}
for input in inputs {
check_input_type(input)?;
check_input_has_decoys(input)?;
check_ring_members_unique(input, hf)?;
}
check_inputs_sorted(inputs, hf)?;
sum_inputs_check_overflow(inputs)
}
fn check_inputs_contextual(
inputs: &[Input],
tx_ring_members_info: &TxRingMembersInfo,
current_chain_height: u64,
hf: &HardFork,
spent_kis: Arc<std::sync::Mutex<HashSet<[u8; 32]>>>,
) -> Result<(), TransactionError> {
check_10_block_lock(
tx_ring_members_info.youngest_used_out_height,
current_chain_height,
@ -427,32 +445,20 @@ pub fn check_inputs(
assert_eq!(hf, &HardFork::V1);
}
let mut spent_kis_lock = spent_kis.lock().unwrap();
for input in inputs {
check_input_type(input)?;
check_input_has_decoys(input)?;
check_ring_members_unique(input, hf)?;
let mut spent_kis_lock = spent_kis.lock().unwrap();
check_key_images(input, &mut spent_kis_lock)?;
// Adding this here for clarity so we don't add more work here while the mutex guard is still
// in scope.
drop(spent_kis_lock);
}
drop(spent_kis_lock);
check_inputs_sorted(inputs, hf)?;
match tx_version {
TxVersion::RingSignatures => sum_inputs_v1(inputs),
_ => panic!("TODO: RCT"),
}
Ok(())
}
/// Checks the version is in the allowed range.
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#version
pub fn check_tx_version(
decoy_info: &Option<contextual_data::DecoyInfo>,
fn check_tx_version(
decoy_info: &Option<DecoyInfo>,
version: &TxVersion,
hf: &HardFork,
) -> Result<(), TransactionError> {
@ -492,3 +498,75 @@ fn min_tx_version(hf: &HardFork) -> TxVersion {
TxVersion::RingSignatures
}
}
pub fn check_transaction_semantic(
tx: &Transaction,
tx_hash: &[u8; 32],
hf: &HardFork,
verifier: &mut BatchVerifier<(), dalek_ff_group::EdwardsPoint>,
) -> Result<u64, TransactionError> {
let tx_version = TxVersion::from_raw(tx.prefix.version)
.ok_or(TransactionError::TransactionVersionInvalid)?;
let outputs_sum = check_outputs_semantics(&tx.prefix.outputs, hf, &tx_version)?;
let inputs_sum = check_inputs_semantics(&tx.prefix.inputs, hf)?;
let fee = match tx_version {
TxVersion::RingSignatures => {
if outputs_sum >= inputs_sum {
Err(TransactionError::OutputsTooHigh)?;
}
inputs_sum - outputs_sum
}
TxVersion::RingCT => {
ring_ct::ring_ct_semantic_checks(tx, tx_hash, verifier, hf)?;
tx.rct_signatures.base.fee
}
};
Ok(fee)
}
pub fn check_transaction_contextual(
tx: &Transaction,
tx_ring_members_info: &TxRingMembersInfo,
current_chain_height: u64,
current_time_lock_timestamp: u64,
hf: &HardFork,
spent_kis: Arc<std::sync::Mutex<HashSet<[u8; 32]>>>,
) -> Result<(), TransactionError> {
let tx_version = TxVersion::from_raw(tx.prefix.version)
.ok_or(TransactionError::TransactionVersionInvalid)?;
check_inputs_contextual(
&tx.prefix.inputs,
tx_ring_members_info,
current_chain_height,
hf,
spent_kis,
)?;
check_tx_version(&tx_ring_members_info.decoy_info, &tx_version, hf)?;
check_all_time_locks(
&tx_ring_members_info.time_locked_outs,
current_chain_height,
current_time_lock_timestamp,
hf,
)?;
match tx_version {
TxVersion::RingSignatures => ring_signatures::verify_inputs_signatures(
&tx.prefix.inputs,
&tx.signatures,
&tx_ring_members_info.rings,
&tx.signature_hash(),
),
TxVersion::RingCT => Ok(ring_ct::check_input_signatures(
&tx.signature_hash(),
&tx.prefix.inputs,
&tx.rct_signatures,
&tx_ring_members_info.rings,
)?),
}
}

View file

@ -278,7 +278,7 @@ impl DecoyInfo {
/// **There are exceptions to this always being the minimum decoys**
///
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/decoys.html#minimum-amount-of-decoys
pub fn minimum_decoys(hf: &HardFork) -> usize {
pub(crate) fn minimum_decoys(hf: &HardFork) -> usize {
use HardFork as HF;
match hf {
HF::V1 => panic!("hard-fork 1 does not use these rules!"),

View file

@ -1,5 +1,3 @@
#![allow(dead_code)]
use curve25519_dalek::{EdwardsPoint, Scalar};
use hex_literal::hex;
use monero_serai::{
@ -8,7 +6,7 @@ use monero_serai::{
mlsag::{AggregateRingMatrixBuilder, MlsagError, RingMatrix},
RctPrunable, RctSignatures, RctType,
},
transaction::{Input, Output},
transaction::{Input, Output, Transaction},
H,
};
use multiexp::BatchVerifier;
@ -135,11 +133,28 @@ fn check_output_range_proofs(
}
}
pub fn ring_ct_semantic_checks(
tx: &Transaction,
tx_hash: &[u8; 32],
verifier: &mut BatchVerifier<(), dalek_ff_group::EdwardsPoint>,
hf: &HardFork,
) -> Result<(), RingCTError> {
check_output_amount(&tx.prefix.outputs)?;
check_rct_type(&tx.rct_signatures.rct_type(), *hf, tx_hash)?;
check_output_range_proofs(&tx.rct_signatures, verifier)?;
if tx.rct_signatures.rct_type() != RctType::MlsagAggregate {
simple_type_balances(&tx.rct_signatures)?;
}
Ok(())
}
/// Check the input signatures, MLSAG, CLSAG.
///
/// https://monero-book.cuprate.org/consensus_rules/ring_ct/mlsag.html
/// https://monero-book.cuprate.org/consensus_rules/ring_ct/clsag.html
fn check_input_signatures(
pub fn check_input_signatures(
msg: &[u8; 32],
inputs: &[Input],
rct_sig: &RctSignatures,

View file

@ -11,7 +11,7 @@ use monero_serai::{ring_signatures::RingSignature, transaction::Input};
#[cfg(feature = "rayon")]
use rayon::prelude::*;
use super::{Rings, SignatureError};
use super::{Rings, TransactionError};
use crate::try_par_iter;
/// Verifies the ring signature.
@ -23,12 +23,12 @@ pub fn verify_inputs_signatures(
signatures: &[RingSignature],
rings: &Rings,
tx_sig_hash: &[u8; 32],
) -> Result<(), SignatureError> {
) -> Result<(), TransactionError> {
match rings {
Rings::Legacy(rings) => {
// rings.len() != inputs.len() can't happen but check any way.
if signatures.len() != inputs.len() || rings.len() != inputs.len() {
return Err(SignatureError::MismatchSignatureSize);
return Err(TransactionError::RingSignatureIncorrect);
}
try_par_iter(inputs)
@ -40,7 +40,7 @@ pub fn verify_inputs_signatures(
};
if !sig.verify(tx_sig_hash, ring, key_image) {
return Err(SignatureError::IncorrectSignature);
return Err(TransactionError::RingSignatureIncorrect);
}
Ok(())
})?;

View file

@ -8,7 +8,7 @@ use crate::ConsensusError;
/// A multi threaded batch verifier.
pub struct MultiThreadedBatchVerifier {
internal: ThreadLocal<UnsafeCell<InternalBatchVerifier<usize, dalek_ff_group::EdwardsPoint>>>,
internal: ThreadLocal<UnsafeCell<InternalBatchVerifier<(), dalek_ff_group::EdwardsPoint>>>,
}
impl MultiThreadedBatchVerifier {
@ -19,12 +19,12 @@ impl MultiThreadedBatchVerifier {
}
}
pub fn queue_statement(
pub fn queue_statement<R>(
&self,
stmt: impl FnOnce(
&mut InternalBatchVerifier<usize, dalek_ff_group::EdwardsPoint>,
) -> Result<(), ConsensusError>,
) -> Result<(), ConsensusError> {
&mut InternalBatchVerifier<(), dalek_ff_group::EdwardsPoint>,
) -> Result<R, ConsensusError>,
) -> Result<R, ConsensusError> {
let verifier_cell = self
.internal
.get_or(|| UnsafeCell::new(InternalBatchVerifier::new(0)));

View file

@ -8,6 +8,7 @@ use futures::{
SinkExt, StreamExt,
};
use monero_serai::block::Block;
use rayon::prelude::*;
use tokio::sync::RwLock;
use tower::{Service, ServiceExt};
use tracing::level_filters::LevelFilter;
@ -21,13 +22,13 @@ use cuprate_consensus::{
},
initialize_blockchain_context, initialize_verifier,
rpc::{cache::ScanningCache, init_rpc_load_balancer, RpcConfig},
Database, DatabaseRequest, DatabaseResponse, VerifiedBlockInformation, VerifyBlockRequest,
VerifyBlockResponse,
Database, DatabaseRequest, DatabaseResponse, PrePreparedBlock, VerifiedBlockInformation,
VerifyBlockRequest, VerifyBlockResponse,
};
mod tx_pool;
const MAX_BLOCKS_IN_RANGE: u64 = 1000;
const MAX_BLOCKS_IN_RANGE: u64 = 500;
const MAX_BLOCKS_HEADERS_IN_RANGE: u64 = 1000;
/// Calls for a batch of blocks, returning the response and the time it took.
@ -166,21 +167,33 @@ where
let (block_tx, mut incoming_blocks) = mpsc::channel(3);
let (mut prepped_blocks_tx, mut prepped_blocks_rx) = mpsc::channel(3);
tokio::spawn(async move {
call_blocks(new_tx_chan, block_tx, start_height, chain_height, database).await
});
while let Some(incoming_blocks) = incoming_blocks.next().await {
tokio::spawn(async move {
while let Some(blocks) = incoming_blocks.next().await {
let blocks = rayon_spawn_async(|| {
blocks
.into_par_iter()
.map(|block| PrePreparedBlock::new(block).unwrap())
.collect::<Vec<_>>()
})
.await;
prepped_blocks_tx.send(blocks).await.unwrap();
}
});
while let Some(incoming_blocks) = prepped_blocks_rx.next().await {
let mut height;
for block in incoming_blocks {
let VerifyBlockResponse::MainChain(verified_block_info) = block_verifier
.ready()
.await?
.call(VerifyBlockRequest::MainChain(block))
.await?
else {
panic!("Block verifier sent incorrect response!");
};
.call(VerifyBlockRequest::MainChainPrepared(block))
.await?;
height = verified_block_info.height;
@ -316,3 +329,15 @@ async fn main() {
.await
.unwrap();
}
async fn rayon_spawn_async<F, R>(f: F) -> R
where
F: FnOnce() -> R + Send + 'static,
R: Send + 'static,
{
let (tx, rx) = tokio::sync::oneshot::channel();
rayon::spawn(|| {
let _ = tx.send(f());
});
rx.await.expect("The sender must not be dropped")
}

View file

@ -7,8 +7,11 @@ use std::{
use futures::FutureExt;
use monero_serai::block::Block;
use monero_serai::transaction::Input;
use tower::{Service, ServiceExt};
use monero_consensus::blocks::BlockError;
use monero_consensus::miner_tx::MinerTxError;
use monero_consensus::{
blocks::{calculate_pow_hash, check_block, check_block_pow},
ConsensusError, HardFork,
@ -35,6 +38,31 @@ pub struct PrePreparedBlock {
pub miner_tx_weight: usize,
}
impl PrePreparedBlock {
pub fn new(block: Block) -> Result<PrePreparedBlock, ConsensusError> {
let (hf_version, hf_vote) =
HardFork::from_block_header(&block.header).map_err(BlockError::HardForkError)?;
let Some(Input::Gen(height)) = block.miner_tx.prefix.inputs.first() else {
Err(ConsensusError::Block(BlockError::MinerTxError(
MinerTxError::InputNotOfTypeGen,
)))?
};
Ok(PrePreparedBlock {
block_blob: block.serialize(),
hf_vote,
hf_version,
block_hash: block.hash(),
pow_hash: calculate_pow_hash(&block.serialize_hashable(), *height, &hf_vote)?,
miner_tx_weight: block.miner_tx.weight(),
block,
})
}
}
#[derive(Debug)]
pub struct VerifiedBlockInformation {
pub block: Block,
@ -51,12 +79,11 @@ pub struct VerifiedBlockInformation {
pub enum VerifyBlockRequest {
MainChain(Block),
MainChainPrepared(PrePreparedBlock),
}
pub enum VerifyBlockResponse {
MainChain(VerifiedBlockInformation),
BatchSetup(Vec<PrePreparedBlock>),
}
// TODO: it is probably a bad idea for this to derive clone, if 2 places (RPC, P2P) receive valid but different blocks
@ -138,12 +165,98 @@ where
VerifyBlockRequest::MainChain(block) => {
verify_main_chain_block(block, context_svc, tx_verifier_svc, tx_pool).await
}
VerifyBlockRequest::MainChainPrepared(prepped_block) => {
verify_main_chain_block_prepared(
prepped_block,
context_svc,
tx_verifier_svc,
tx_pool,
)
.await
}
}
}
.boxed()
}
}
async fn verify_main_chain_block_prepared<C, TxV, TxP>(
prepped_block: PrePreparedBlock,
context_svc: C,
tx_verifier_svc: TxV,
tx_pool: TxP,
) -> Result<VerifyBlockResponse, ExtendedConsensusError>
where
C: Service<
BlockChainContextRequest,
Response = BlockChainContextResponse,
Error = tower::BoxError,
> + Send
+ 'static,
C::Future: Send + 'static,
TxV: Service<VerifyTxRequest, Response = VerifyTxResponse, Error = ExtendedConsensusError>,
TxP: Service<TxPoolRequest, Response = TxPoolResponse, Error = TxNotInPool>
+ Clone
+ Send
+ 'static,
{
tracing::debug!("getting blockchain context");
let BlockChainContextResponse::Context(checked_context) = context_svc
.oneshot(BlockChainContextRequest::Get)
.await
.map_err(Into::<ExtendedConsensusError>::into)?
else {
panic!("Context service returned wrong response!");
};
let context = checked_context.unchecked_blockchain_context().clone();
tracing::debug!("got blockchain context: {:?}", context);
let TxPoolResponse::Transactions(txs) = tx_pool
.oneshot(TxPoolRequest::Transactions(prepped_block.block.txs.clone()))
.await?;
tx_verifier_svc
.oneshot(VerifyTxRequest::Block {
txs: txs.clone(),
current_chain_height: context.chain_height,
time_for_time_lock: context.current_adjusted_timestamp_for_time_lock(),
hf: context.current_hf,
re_org_token: context.re_org_token.clone(),
})
.await?;
let block_weight =
prepped_block.miner_tx_weight + txs.iter().map(|tx| tx.tx_weight).sum::<usize>();
let total_fees = txs.iter().map(|tx| tx.fee).sum::<u64>();
let (hf_vote, generated_coins) = check_block(
&prepped_block.block,
total_fees,
block_weight,
prepped_block.block_blob.len(),
&context.context_to_verify_block,
)
.map_err(ConsensusError::Block)?;
check_block_pow(&prepped_block.pow_hash, context.next_difficulty)
.map_err(ConsensusError::Block)?;
Ok(VerifyBlockResponse::MainChain(VerifiedBlockInformation {
block_hash: prepped_block.block_hash,
block: prepped_block.block,
txs,
pow_hash: prepped_block.pow_hash,
generated_coins,
weight: block_weight,
height: context.chain_height,
long_term_weight: context.next_block_long_term_weight(block_weight),
hf_vote,
cumulative_difficulty: context.cumulative_difficulty + context.next_difficulty,
}))
}
async fn verify_main_chain_block<C, TxV, TxP>(
block: Block,
context_svc: C,

View file

@ -7,6 +7,7 @@ use std::{
use monero_consensus::{transactions::OutputOnChain, ConsensusError, HardFork};
//mod batch_verifier;
mod batch_verifier;
pub mod block;
pub mod context;
mod helper;

View file

@ -45,7 +45,8 @@ impl ScanningCache {
pub fn load(file: &Path) -> Result<ScanningCache, tower::BoxError> {
let mut file = std::fs::OpenOptions::new().read(true).open(file)?;
Ok(borsh::from_reader(&mut file)?)
let data: ScanningCache = borsh::from_reader(&mut file)?;
Ok(data)
}
pub fn add_new_block_data(

View file

@ -14,17 +14,16 @@ use tower::{Service, ServiceExt};
use tracing::instrument;
use monero_consensus::{
signatures::verify_contextual_signatures,
transactions::{
check_all_time_locks, check_inputs, check_outputs, check_tx_version, TransactionError,
check_transaction_contextual, check_transaction_semantic, RingCTError, TransactionError,
TxRingMembersInfo,
},
ConsensusError, HardFork, TxVersion,
};
use crate::{
context::ReOrgToken, helper::rayon_spawn_async, Database, DatabaseRequest, DatabaseResponse,
ExtendedConsensusError,
batch_verifier::MultiThreadedBatchVerifier, context::ReOrgToken, helper::rayon_spawn_async,
Database, DatabaseRequest, DatabaseResponse, ExtendedConsensusError,
};
mod contextual_data;
@ -45,12 +44,23 @@ pub struct TransactionVerificationData {
}
impl TransactionVerificationData {
pub fn new(tx: Transaction) -> Result<TransactionVerificationData, ConsensusError> {
pub fn new(
tx: Transaction,
hf: &HardFork,
verifier: Arc<MultiThreadedBatchVerifier>,
) -> Result<TransactionVerificationData, ConsensusError> {
let tx_hash = tx.hash();
let fee = verifier.queue_statement(|verifier| {
check_transaction_semantic(&tx, &tx_hash, hf, verifier)
.map_err(ConsensusError::Transaction)
})?;
Ok(TransactionVerificationData {
tx_hash: tx.hash(),
tx_hash,
tx_blob: tx.serialize(),
tx_weight: tx.weight(),
fee: tx.rct_signatures.base.fee,
fee,
rings_member_info: std::sync::Mutex::new(None),
version: TxVersion::from_raw(tx.prefix.version)
.ok_or(TransactionError::TransactionVersionInvalid)?,
@ -68,7 +78,7 @@ pub enum VerifyTxRequest {
hf: HardFork,
re_org_token: ReOrgToken,
},
/// Batches the setup of [`TransactionVerificationData`], does *minimal* verification, you need to call [`VerifyTxRequest::Block`]
/// Batches the setup of [`TransactionVerificationData`], does *some* verification, you need to call [`VerifyTxRequest::Block`]
/// with the returned data.
BatchSetup {
txs: Vec<Transaction>,
@ -148,14 +158,29 @@ async fn batch_setup_transactions<D>(
where
D: Database + Clone + Sync + Send + 'static,
{
let batch_verifier = Arc::new(MultiThreadedBatchVerifier::new(rayon::current_num_threads()));
let cloned_verifier = batch_verifier.clone();
// Move out of the async runtime and use rayon to parallelize the serialisation and hashing of the txs.
let txs = rayon_spawn_async(|| {
let txs = rayon_spawn_async(move || {
txs.into_par_iter()
.map(|tx| Ok(Arc::new(TransactionVerificationData::new(tx)?)))
.map(|tx| {
Ok(Arc::new(TransactionVerificationData::new(
tx,
&hf,
cloned_verifier.clone(),
)?))
})
.collect::<Result<Vec<_>, ConsensusError>>()
})
.await?;
if !Arc::into_inner(batch_verifier).unwrap().verify() {
Err(ConsensusError::Transaction(TransactionError::RingCTError(
RingCTError::BulletproofsRangeInvalid,
)))?
}
contextual_data::batch_fill_ring_member_info(&txs, &hf, re_org_token, database).await?;
Ok(VerifyTxResponse::BatchSetupOk(txs))
@ -223,44 +248,20 @@ fn verify_transaction_for_block(
hex::encode(tx_verification_data.tx_hash)
);
let tx_version = &tx_verification_data.version;
let rings_member_info_lock = tx_verification_data.rings_member_info.lock().unwrap();
let rings_member_info = match rings_member_info_lock.deref() {
Some(rings_member_info) => rings_member_info,
None => panic!("rings_member_info needs to be set to be able to verify!"),
};
check_tx_version(&rings_member_info.0.decoy_info, tx_version, &hf)?;
check_all_time_locks(
&rings_member_info.0.time_locked_outs,
check_transaction_contextual(
&tx_verification_data.tx,
&rings_member_info.0,
current_chain_height,
time_for_time_lock,
&hf,
)?;
let sum_outputs = check_outputs(&tx_verification_data.tx.prefix.outputs, &hf, tx_version)?;
let sum_inputs = check_inputs(
tx_verification_data.tx.prefix.inputs.as_slice(),
&rings_member_info.0,
current_chain_height,
&hf,
tx_version,
spent_kis,
)?;
if tx_version == &TxVersion::RingSignatures {
if sum_outputs >= sum_inputs {
Err(TransactionError::OutputsTooHigh)?;
}
// check that monero-serai is calculating the correct value here, why can't we just use this
// value? because we don't have this when we create the object.
assert_eq!(tx_verification_data.fee, sum_inputs - sum_outputs);
}
verify_contextual_signatures(&tx_verification_data.tx, &rings_member_info.0.rings)?;
Ok(())
}