mirror of
https://github.com/Cuprate/cuprate.git
synced 2025-01-08 20:09:44 +00:00
add more consensus rules.
This commit is contained in:
parent
ac87f9998e
commit
387278b821
15 changed files with 914 additions and 55 deletions
|
@ -28,10 +28,12 @@ tower = {version = "0.4", features = ["util"]}
|
|||
tracing = "0.1"
|
||||
futures = "0.3"
|
||||
|
||||
sha3 = "0.10"
|
||||
crypto-bigint = "0.5"
|
||||
curve25519-dalek = "4"
|
||||
|
||||
randomx-rs = "1"
|
||||
monero-serai = {git="https://github.com/Cuprate/serai.git", rev = "df4af7b"}
|
||||
monero-serai = {git="https://github.com/Cuprate/serai.git", rev = "39eafae"}
|
||||
|
||||
cuprate-common = {path = "../common"}
|
||||
cryptonight-cuprate = {path = "../cryptonight"}
|
||||
|
@ -45,6 +47,3 @@ tokio = { version = "1", features = ["rt-multi-thread", "macros"], optional = tr
|
|||
tracing-subscriber = {version = "0.3", optional = true}
|
||||
# here to help cargo to pick a version - remove me
|
||||
syn = "2.0.37"
|
||||
|
||||
[profile.dev]
|
||||
opt-level = 3
|
||||
|
|
|
@ -39,7 +39,7 @@ impl Default for ScanningCache {
|
|||
ScanningCache {
|
||||
network: Default::default(),
|
||||
numb_outs: Default::default(),
|
||||
height: 1,
|
||||
height: 1_000_000,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -106,65 +106,101 @@ where
|
|||
tracing::info!("Initialised verifier, begging scan");
|
||||
let batch_size = rpc_config.read().unwrap().block_batch_size();
|
||||
|
||||
let mut next_fut = tokio::spawn(database.clone().ready().await?.call(
|
||||
DatabaseRequest::BlockBatchInRange(
|
||||
cache.height..(cache.height + batch_size).min(chain_height),
|
||||
),
|
||||
));
|
||||
let mut db = database.clone();
|
||||
let mut next_fut = tokio::spawn(async move {
|
||||
let now = std::time::Instant::now();
|
||||
(
|
||||
db.ready()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(DatabaseRequest::BlockBatchInRange(
|
||||
cache.height..(cache.height + batch_size).min(chain_height),
|
||||
))
|
||||
.await
|
||||
.unwrap(),
|
||||
now.elapsed(),
|
||||
)
|
||||
});
|
||||
|
||||
let mut current_height = cache.height;
|
||||
let mut next_batch_start_height = cache.height + batch_size;
|
||||
|
||||
let mut time_to_verify_last_batch: u128 = 0;
|
||||
|
||||
let mut first = true;
|
||||
let mut batches_till_check_batch_size: u64 = 2;
|
||||
|
||||
while next_batch_start_height < chain_height {
|
||||
let next_batch_size = rpc_config.read().unwrap().block_batch_size();
|
||||
let time_to_retrieve_batch = std::time::Instant::now();
|
||||
|
||||
// Call the next batch while we handle this batch.
|
||||
let mut db = database.clone();
|
||||
let current_fut = std::mem::replace(
|
||||
&mut next_fut,
|
||||
tokio::spawn(
|
||||
database
|
||||
.ready()
|
||||
.await?
|
||||
.call(DatabaseRequest::BlockBatchInRange(
|
||||
next_batch_start_height
|
||||
..(next_batch_start_height + next_batch_size).min(chain_height),
|
||||
)),
|
||||
),
|
||||
tokio::spawn(async move {
|
||||
let now = std::time::Instant::now();
|
||||
|
||||
(
|
||||
db.ready()
|
||||
.await
|
||||
.unwrap()
|
||||
.call(DatabaseRequest::BlockBatchInRange(
|
||||
next_batch_start_height
|
||||
..(next_batch_start_height + next_batch_size).min(chain_height),
|
||||
))
|
||||
.await
|
||||
.unwrap(),
|
||||
now.elapsed(),
|
||||
)
|
||||
}),
|
||||
);
|
||||
|
||||
let DatabaseResponse::BlockBatchInRange(blocks) = current_fut.await?? else {
|
||||
let (DatabaseResponse::BlockBatchInRange(blocks), time_to_retrieve_batch) =
|
||||
current_fut.await?
|
||||
else {
|
||||
panic!("Database sent incorrect response!");
|
||||
};
|
||||
|
||||
let time_to_verify_batch = std::time::Instant::now();
|
||||
|
||||
let time_to_retrieve_batch = time_to_retrieve_batch.elapsed().as_millis();
|
||||
let time_to_retrieve_batch = time_to_retrieve_batch.as_millis();
|
||||
|
||||
if time_to_retrieve_batch > time_to_verify_last_batch + 2000
|
||||
&& batches_till_check_batch_size == 0
|
||||
{
|
||||
batches_till_check_batch_size = 3;
|
||||
|
||||
if time_to_retrieve_batch > 2000 && !first {
|
||||
let mut conf = rpc_config.write().unwrap();
|
||||
tracing::info!(
|
||||
"Decreasing batch size time to verify last batch: {}, time_to_retrieve_batch: {}",
|
||||
time_to_verify_last_batch,
|
||||
time_to_retrieve_batch
|
||||
);
|
||||
conf.max_blocks_per_node = (conf.max_blocks_per_node
|
||||
* TryInto::<u64>::try_into(
|
||||
time_to_verify_last_batch
|
||||
/ (time_to_verify_last_batch + time_to_retrieve_batch),
|
||||
)
|
||||
.unwrap())
|
||||
.max(10_u64)
|
||||
.min(MAX_BLOCKS_IN_RANGE);
|
||||
tracing::info!("Decreasing batch size to: {}", conf.max_blocks_per_node);
|
||||
} else if time_to_retrieve_batch < 100 {
|
||||
let mut conf = rpc_config.write().unwrap();
|
||||
conf.max_blocks_per_node = (conf.max_blocks_per_node * 2)
|
||||
* time_to_verify_last_batch as u64
|
||||
/ (time_to_retrieve_batch as u64))
|
||||
.max(10_u64)
|
||||
.min(MAX_BLOCKS_IN_RANGE);
|
||||
tracing::info!("Increasing batch size to: {}", conf.max_blocks_per_node);
|
||||
}
|
||||
tracing::info!("Decreasing batch size to: {}", conf.max_blocks_per_node);
|
||||
} else if time_to_retrieve_batch + 2000 < time_to_verify_last_batch
|
||||
&& batches_till_check_batch_size == 0
|
||||
{
|
||||
batches_till_check_batch_size = 3;
|
||||
|
||||
first = false;
|
||||
let mut conf = rpc_config.write().unwrap();
|
||||
tracing::info!(
|
||||
"Increasing batch size time to verify last batch: {}, time_to_retrieve_batch: {}",
|
||||
time_to_verify_last_batch,
|
||||
time_to_retrieve_batch
|
||||
);
|
||||
conf.max_blocks_per_node = (conf.max_blocks_per_node
|
||||
* (time_to_verify_last_batch as u64)
|
||||
/ time_to_retrieve_batch.max(1) as u64)
|
||||
.max(30_u64)
|
||||
.min(MAX_BLOCKS_IN_RANGE);
|
||||
tracing::info!("Increasing batch size to: {}", conf.max_blocks_per_node);
|
||||
} else {
|
||||
batches_till_check_batch_size = batches_till_check_batch_size.saturating_sub(1);
|
||||
}
|
||||
|
||||
tracing::info!(
|
||||
"Handling batch: {:?}, chain height: {}",
|
||||
|
@ -227,7 +263,10 @@ async fn main() {
|
|||
"http://145.239.97.211:18089".to_string(),
|
||||
];
|
||||
|
||||
let rpc_config = RpcConfig::new(10, INITIAL_MAX_BLOCKS_HEADERS_IN_RANGE);
|
||||
let rpc_config = RpcConfig::new(
|
||||
INITIAL_MAX_BLOCKS_IN_RANGE,
|
||||
INITIAL_MAX_BLOCKS_HEADERS_IN_RANGE,
|
||||
);
|
||||
let rpc_config = Arc::new(RwLock::new(rpc_config));
|
||||
|
||||
let rpc = init_rpc_load_balancer(urls, rpc_config.clone());
|
||||
|
|
|
@ -17,11 +17,36 @@ const BLOCK_FUTURE_TIME_LIMIT: u64 = 60 * 60 * 2;
|
|||
pub struct BlockVerificationData {
|
||||
hf: BlockHFInfo,
|
||||
pow: BlockPOWInfo,
|
||||
current_difficulty: u128,
|
||||
weights: BlockWeightInfo,
|
||||
block_blob: Vec<u8>,
|
||||
block: Block,
|
||||
block_hash: [u8; 32],
|
||||
pow_hash: [u8; 32],
|
||||
// txs: Vec<T>,
|
||||
}
|
||||
|
||||
impl BlockVerificationData {
|
||||
pub fn new(
|
||||
block: Block,
|
||||
difficulty_cache: &DifficultyCache,
|
||||
weight_cache: &BlockWeightsCache,
|
||||
) -> Result<BlockVerificationData, ConsensusError> {
|
||||
let hf = BlockHFInfo::from_block_header(&block.header)?;
|
||||
|
||||
let current_diff = difficulty_cache.next_difficulty(&hf.version);
|
||||
let cum_diff = difficulty_cache.cumulative_difficulty() + current_diff;
|
||||
|
||||
todo!()
|
||||
/*
|
||||
|
||||
Ok(BlockVerificationData {
|
||||
hf: BlockHFInfo::from_block_header(&block.header)?,
|
||||
pow: BlockPOWInfo::new(block.header.timestamp, cum_diff),
|
||||
weights:
|
||||
})
|
||||
*/
|
||||
}
|
||||
}
|
||||
|
||||
/// Sanity check on the block blob size.
|
||||
|
@ -70,7 +95,7 @@ fn check_timestamp(block: &Block, median_timestamp: u64) -> Result<(), Consensus
|
|||
if block.header.timestamp < median_timestamp
|
||||
|| block.header.timestamp > current_time() + BLOCK_FUTURE_TIME_LIMIT
|
||||
{
|
||||
return Err(ConsensusError::BlockTimestampInvalid);
|
||||
Err(ConsensusError::BlockTimestampInvalid)
|
||||
} else {
|
||||
Ok(())
|
||||
}
|
||||
|
|
|
@ -96,6 +96,7 @@ impl DifficultyCache {
|
|||
|
||||
let mut block_start = chain_height.saturating_sub(config.total_block_count());
|
||||
|
||||
// skip the genesis block.
|
||||
if block_start == 0 {
|
||||
block_start = 1;
|
||||
}
|
||||
|
@ -150,6 +151,7 @@ impl DifficultyCache {
|
|||
let mut block_start =
|
||||
(self.last_accounted_height + 1).saturating_sub(self.config.total_block_count());
|
||||
|
||||
// skip the genesis block
|
||||
if block_start == 0 {
|
||||
block_start = 1;
|
||||
}
|
||||
|
@ -216,6 +218,15 @@ impl DifficultyCache {
|
|||
.collect::<Vec<_>>(),
|
||||
)
|
||||
}
|
||||
|
||||
/// Returns the cumulative difficulty of the chain.
|
||||
pub fn cumulative_difficulty(&self) -> u128 {
|
||||
self.cumulative_difficulty
|
||||
}
|
||||
|
||||
pub fn top_block_timestamp(&self) -> Option<u64> {
|
||||
self.timestamps.back().copied()
|
||||
}
|
||||
}
|
||||
|
||||
fn get_window_start_and_end(window_len: usize, accounted_window: usize) -> (usize, usize) {
|
||||
|
|
|
@ -1,5 +1,8 @@
|
|||
use crypto_bigint::{CheckedMul, U256};
|
||||
use cryptonight_cuprate::{cryptonight_hash, Variant};
|
||||
|
||||
use cryptonight_cuprate::{
|
||||
cryptonight_hash_r, cryptonight_hash_v0, cryptonight_hash_v1, cryptonight_hash_v2,
|
||||
};
|
||||
|
||||
use crate::{hardforks::HardFork, ConsensusError};
|
||||
|
||||
|
@ -9,6 +12,15 @@ pub struct BlockPOWInfo {
|
|||
pub cumulative_difficulty: u128,
|
||||
}
|
||||
|
||||
impl BlockPOWInfo {
|
||||
pub fn new(timestamp: u64, cumulative_difficulty: u128) -> BlockPOWInfo {
|
||||
BlockPOWInfo {
|
||||
timestamp,
|
||||
cumulative_difficulty,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns if the blocks POW hash is valid for the current difficulty.
|
||||
///
|
||||
/// See: https://cuprate.github.io/monero-book/consensus_rules/blocks/difficulty.html#checking-a-blocks-proof-of-work
|
||||
|
@ -21,24 +33,29 @@ pub fn check_block_pow(hash: &[u8; 32], difficulty: u128) -> bool {
|
|||
}
|
||||
|
||||
/// Calcualtes the POW hash of this block.
|
||||
pub fn calculate_pow_hash(buf: &[u8], height: u64, hf: &HardFork) -> [u8; 32] {
|
||||
pub fn calculate_pow_hash(
|
||||
buf: &[u8],
|
||||
height: u64,
|
||||
hf: &HardFork,
|
||||
) -> Result<[u8; 32], ConsensusError> {
|
||||
if height == 202612 {
|
||||
return hex::decode("84f64766475d51837ac9efbef1926486e58563c95a19fef4aec3254f03000000")
|
||||
.unwrap()
|
||||
.try_into()
|
||||
.unwrap();
|
||||
return Ok(
|
||||
hex::decode("84f64766475d51837ac9efbef1926486e58563c95a19fef4aec3254f03000000")
|
||||
.unwrap()
|
||||
.try_into()
|
||||
.unwrap(),
|
||||
);
|
||||
}
|
||||
|
||||
if hf.in_range(&HardFork::V1, &HardFork::V7) {
|
||||
cryptonight_hash(buf, &Variant::V0)
|
||||
//cryptonight_hash::cryptonight_hash(buf, &Variant::V0)
|
||||
Ok(if hf.in_range(&HardFork::V1, &HardFork::V7) {
|
||||
cryptonight_hash_v0(buf)
|
||||
} else if hf == &HardFork::V7 {
|
||||
cryptonight_hash(buf, &Variant::V1)
|
||||
cryptonight_hash_v1(buf).map_err(|_| ConsensusError::BlockPOWInvalid)?
|
||||
} else if hf.in_range(&HardFork::V8, &HardFork::V10) {
|
||||
cryptonight_hash(buf, &Variant::V2)
|
||||
cryptonight_hash_v2(buf)
|
||||
} else if hf.in_range(&HardFork::V10, &HardFork::V12) {
|
||||
cryptonight_hash(buf, &Variant::R { height })
|
||||
cryptonight_hash_r(buf, height)
|
||||
} else {
|
||||
todo!("RandomX")
|
||||
}
|
||||
})
|
||||
}
|
||||
|
|
|
@ -19,8 +19,8 @@ const NUMB_OF_HARD_FORKS: usize = 16;
|
|||
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub struct BlockHFInfo {
|
||||
version: HardFork,
|
||||
vote: HardFork,
|
||||
pub version: HardFork,
|
||||
pub vote: HardFork,
|
||||
}
|
||||
|
||||
impl BlockHFInfo {
|
||||
|
|
|
@ -4,6 +4,8 @@ use std::{
|
|||
time::{SystemTime, UNIX_EPOCH},
|
||||
};
|
||||
|
||||
use curve25519_dalek::edwards::CompressedEdwardsY;
|
||||
|
||||
/// Deserializes an object using the give `des` function, checking that all the bytes
|
||||
/// are consumed.
|
||||
pub(crate) fn size_check_decode<T>(
|
||||
|
@ -54,3 +56,16 @@ pub(crate) fn current_time() -> u64 {
|
|||
.unwrap()
|
||||
.as_secs()
|
||||
}
|
||||
|
||||
/// Checks that a point is canonical.
|
||||
///
|
||||
/// https://github.com/dalek-cryptography/curve25519-dalek/issues/380
|
||||
pub(crate) fn check_point(point: &CompressedEdwardsY) -> bool {
|
||||
let bytes = point.as_bytes();
|
||||
|
||||
point
|
||||
.decompress()
|
||||
// Ban points which are either unreduced or -0
|
||||
.filter(|point| point.compress().as_bytes() == bytes)
|
||||
.is_some()
|
||||
}
|
||||
|
|
|
@ -1,3 +1,5 @@
|
|||
use std::collections::{HashMap, HashSet};
|
||||
|
||||
pub mod block;
|
||||
pub mod genesis;
|
||||
pub mod hardforks;
|
||||
|
@ -5,10 +7,27 @@ mod helper;
|
|||
pub mod miner_tx;
|
||||
#[cfg(feature = "binaries")]
|
||||
pub mod rpc;
|
||||
pub mod transactions;
|
||||
pub mod verifier;
|
||||
|
||||
#[derive(Debug, thiserror::Error)]
|
||||
pub enum ConsensusError {
|
||||
#[error("Transaction sig invalid: {0}")]
|
||||
TransactionSignatureInvalid(&'static str),
|
||||
#[error("Transaction inputs overflow")]
|
||||
TransactionInputsOverflow,
|
||||
#[error("Transaction outputs overflow")]
|
||||
TransactionOutputsOverflow,
|
||||
#[error("Transaction has an invalid output: {0}")]
|
||||
TransactionInvalidOutput(&'static str),
|
||||
#[error("Transaction has an invalid version")]
|
||||
TransactionVersionInvalid,
|
||||
#[error("Transaction an invalid input: {0}")]
|
||||
TransactionHasInvalidInput(&'static str),
|
||||
#[error("Transaction has invalid ring: {0}")]
|
||||
TransactionHasInvalidRing(&'static str),
|
||||
#[error("Block has an invalid proof of work")]
|
||||
BlockPOWInvalid,
|
||||
#[error("Block has a timestamp outside of the valid range")]
|
||||
BlockTimestampInvalid,
|
||||
#[error("Block is too large")]
|
||||
|
@ -44,6 +63,9 @@ pub enum DatabaseRequest {
|
|||
|
||||
ChainHeight,
|
||||
|
||||
Outputs(HashMap<u64, HashSet<u64>>),
|
||||
NumberOutputsWithAmount(u64),
|
||||
|
||||
#[cfg(feature = "binaries")]
|
||||
BlockBatchInRange(std::ops::Range<u64>),
|
||||
}
|
||||
|
@ -61,6 +83,9 @@ pub enum DatabaseResponse {
|
|||
|
||||
ChainHeight(u64),
|
||||
|
||||
Outputs(HashMap<u64, HashMap<u64, [curve25519_dalek::EdwardsPoint; 2]>>),
|
||||
NumberOutputsWithAmount(usize),
|
||||
|
||||
#[cfg(feature = "binaries")]
|
||||
BlockBatchInRange(
|
||||
Vec<(
|
||||
|
|
|
@ -293,6 +293,7 @@ impl<R: RpcConnection + Send + Sync + 'static> tower::Service<DatabaseRequest> f
|
|||
let err_slot = self.error_slot.clone();
|
||||
|
||||
match req {
|
||||
_ => todo!(),
|
||||
DatabaseRequest::BlockHash(height) => async move {
|
||||
let res: Result<_, RpcError> = rpc
|
||||
.get_block_hash(height as usize)
|
||||
|
|
67
consensus/src/transactions.rs
Normal file
67
consensus/src/transactions.rs
Normal file
|
@ -0,0 +1,67 @@
|
|||
use monero_serai::transaction::Transaction;
|
||||
use sha3::{Digest, Keccak256};
|
||||
|
||||
use crate::{hardforks::HardFork, ConsensusError, Database};
|
||||
|
||||
mod inputs;
|
||||
mod outputs;
|
||||
mod signatures;
|
||||
mod time_lock;
|
||||
|
||||
#[derive(Debug, Copy, Clone, Eq, PartialEq)]
|
||||
pub enum TxVersion {
|
||||
RingSignatures,
|
||||
RingCT,
|
||||
}
|
||||
|
||||
impl TxVersion {
|
||||
pub fn from_raw(version: u64) -> Result<TxVersion, ConsensusError> {
|
||||
match version {
|
||||
1 => Ok(TxVersion::RingSignatures),
|
||||
2 => Ok(TxVersion::RingCT),
|
||||
_ => Err(ConsensusError::TransactionVersionInvalid),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Data needed to verify a transaction.
|
||||
///
|
||||
pub struct TransactionVerificationData {
|
||||
tx: Transaction,
|
||||
version: TxVersion,
|
||||
tx_blob: Vec<u8>,
|
||||
tx_weight: usize,
|
||||
tx_hash: [u8; 32],
|
||||
rings: signatures::Rings,
|
||||
}
|
||||
|
||||
impl TransactionVerificationData {
|
||||
pub fn new(
|
||||
tx: Transaction,
|
||||
rings: signatures::Rings,
|
||||
) -> Result<TransactionVerificationData, ConsensusError> {
|
||||
let tx_blob = tx.serialize();
|
||||
|
||||
Ok(TransactionVerificationData {
|
||||
tx_hash: Keccak256::digest(&tx_blob).into(),
|
||||
tx_blob,
|
||||
tx_weight: tx.weight(),
|
||||
rings,
|
||||
version: TxVersion::from_raw(tx.prefix.version)?,
|
||||
tx,
|
||||
})
|
||||
}
|
||||
|
||||
pub async fn batch_new<D: Database + Clone>(
|
||||
txs: Vec<Transaction>,
|
||||
hf: &HardFork,
|
||||
database: D,
|
||||
) -> Result<Vec<TransactionVerificationData>, ConsensusError> {
|
||||
let rings = signatures::batch_get_rings(&txs, hf, database.clone()).await?;
|
||||
|
||||
txs.into_iter()
|
||||
.zip(rings.into_iter())
|
||||
.map(|(tx, ring)| TransactionVerificationData::new(tx, ring))
|
||||
.collect()
|
||||
}
|
||||
}
|
244
consensus/src/transactions/inputs.rs
Normal file
244
consensus/src/transactions/inputs.rs
Normal file
|
@ -0,0 +1,244 @@
|
|||
use curve25519_dalek::EdwardsPoint;
|
||||
use std::cmp::{max, min};
|
||||
use std::collections::HashSet;
|
||||
|
||||
use monero_serai::transaction::Input;
|
||||
use tower::{Service, ServiceExt};
|
||||
|
||||
use crate::{hardforks::HardFork, ConsensusError, Database, DatabaseRequest, DatabaseResponse};
|
||||
|
||||
/// A struct holding information about the inputs and their decoys.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/decoys.html
|
||||
pub struct DecoyInfo {
|
||||
/// The number of inputs that have enough outputs on the chain to mix with.
|
||||
mixable: usize,
|
||||
/// The number of inputs that don't have enough outputs on the chain to mix with.
|
||||
not_mixable: usize,
|
||||
/// The minimum amount of decoys used in the transaction.
|
||||
min_decoys: usize,
|
||||
/// The maximum amount of decoys used in the transaction.
|
||||
max_decoys: usize,
|
||||
}
|
||||
|
||||
impl DecoyInfo {
|
||||
/// Creates a new [`DecoyInfo`] struct relating to the passed in inputs.
|
||||
///
|
||||
/// Do not rely on this function to do consensus checks!
|
||||
///
|
||||
pub async fn new<D: Database>(
|
||||
inputs: &[Input],
|
||||
hf: &HardFork,
|
||||
mut database: D,
|
||||
) -> Result<DecoyInfo, ConsensusError> {
|
||||
let mut min_decoys = usize::MAX;
|
||||
let mut max_decoys = usize::MIN;
|
||||
let mut mixable = 0;
|
||||
let mut not_mixable = 0;
|
||||
|
||||
let minimum_decoys = minimum_decoys(hf);
|
||||
|
||||
for inp in inputs {
|
||||
match inp {
|
||||
Input::ToKey {
|
||||
amount,
|
||||
key_offsets,
|
||||
..
|
||||
} => {
|
||||
let DatabaseResponse::NumberOutputsWithAmount(numb_of_outs) = database
|
||||
.ready()
|
||||
.await?
|
||||
.call(DatabaseRequest::NumberOutputsWithAmount(
|
||||
amount.unwrap_or(0),
|
||||
))
|
||||
.await?
|
||||
else {
|
||||
panic!("Database sent incorrect response!");
|
||||
};
|
||||
|
||||
// https://cuprate.github.io/monero-book/consensus_rules/transactions/decoys.html#mixable-and-unmixable-inputs
|
||||
if numb_of_outs <= minimum_decoys {
|
||||
not_mixable += 1;
|
||||
} else {
|
||||
mixable += 1;
|
||||
}
|
||||
|
||||
let numb_decoys = key_offsets
|
||||
.len()
|
||||
.checked_sub(1)
|
||||
.ok_or(ConsensusError::TransactionHasInvalidRing("ring is empty"))?;
|
||||
// https://cuprate.github.io/monero-book/consensus_rules/transactions/decoys.html#minimum-and-maximum-decoys-used
|
||||
min_decoys = min(min_decoys, numb_decoys);
|
||||
max_decoys = max(max_decoys, numb_decoys);
|
||||
}
|
||||
_ => {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"input not ToKey",
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(DecoyInfo {
|
||||
mixable,
|
||||
not_mixable,
|
||||
min_decoys,
|
||||
max_decoys,
|
||||
})
|
||||
}
|
||||
|
||||
/// Checks the decoys are allowed.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#minimum-decoys
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#equal-number-of-decoys
|
||||
pub fn check_decoy_info(&self, hf: &HardFork) -> Result<(), ConsensusError> {
|
||||
if hf == &HardFork::V15 {
|
||||
// Hard-fork 15 allows both v14 and v16 rules
|
||||
return self
|
||||
.check_decoy_info(&HardFork::V14)
|
||||
.or_else(|_| self.check_decoy_info(&HardFork::V16));
|
||||
}
|
||||
|
||||
let current_minimum_decoys = minimum_decoys(hf);
|
||||
|
||||
if self.min_decoys < current_minimum_decoys {
|
||||
if self.not_mixable == 0 {
|
||||
return Err(ConsensusError::TransactionHasInvalidRing(
|
||||
"input does not have enough decoys",
|
||||
));
|
||||
}
|
||||
if self.mixable > 1 {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"more than one mixable input with unmixable inputs",
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
if hf >= &HardFork::V8 && self.min_decoys != current_minimum_decoys {
|
||||
return Err(ConsensusError::TransactionHasInvalidRing(
|
||||
"one ring does not have the minimum number of decoys",
|
||||
));
|
||||
}
|
||||
|
||||
if hf >= &HardFork::V12 && self.min_decoys != self.max_decoys {
|
||||
return Err(ConsensusError::TransactionHasInvalidRing(
|
||||
"rings do not have the same number of members",
|
||||
));
|
||||
}
|
||||
|
||||
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(&self, version: u64, hf: &HardFork) -> Result<(), ConsensusError> {
|
||||
if version == 0 {
|
||||
return Err(ConsensusError::TransactionVersionInvalid);
|
||||
}
|
||||
|
||||
let max = max_tx_version(hf);
|
||||
if version > max {
|
||||
return Err(ConsensusError::TransactionVersionInvalid);
|
||||
}
|
||||
|
||||
// TODO: Doc is wrong here
|
||||
let min = min_tx_version(hf);
|
||||
if version < min && self.not_mixable != 0 {
|
||||
return Err(ConsensusError::TransactionVersionInvalid);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn max_tx_version(hf: &HardFork) -> u64 {
|
||||
if hf <= &HardFork::V3 {
|
||||
1
|
||||
} else {
|
||||
2
|
||||
}
|
||||
}
|
||||
|
||||
fn min_tx_version(hf: &HardFork) -> u64 {
|
||||
if hf >= &HardFork::V6 {
|
||||
2
|
||||
} else {
|
||||
1
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns the minimum amount of decoys for a hard-fork.
|
||||
/// **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
|
||||
fn minimum_decoys(hf: &HardFork) -> usize {
|
||||
use HardFork::*;
|
||||
match hf {
|
||||
V1 => panic!("hard-fork 1 does not use these rules!"),
|
||||
V2 | V3 | V4 | V5 => 2,
|
||||
V6 => 4,
|
||||
V7 => 6,
|
||||
V8 | V9 | V10 | V11 | V12 | V13 | V14 => 10,
|
||||
_ => 15,
|
||||
}
|
||||
}
|
||||
|
||||
/// Sums the inputs checking for overflow.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#inputs-and-outputs-must-not-overflow
|
||||
pub(crate) fn sum_inputs_v1(inputs: &[Input]) -> Result<u64, ConsensusError> {
|
||||
let mut sum: u64 = 0;
|
||||
for inp in inputs {
|
||||
match inp {
|
||||
Input::ToKey { amount, .. } => {
|
||||
sum = sum
|
||||
.checked_add(amount.unwrap_or(0))
|
||||
.ok_or(ConsensusError::TransactionInputsOverflow)?;
|
||||
}
|
||||
_ => {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"input not ToKey",
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(sum)
|
||||
}
|
||||
|
||||
/// Checks the inputs key images for torsion and for duplicates in the transaction.
|
||||
///
|
||||
/// The `spent_kis` parameter is not meant to be a complete list of key images, just a list of related transactions
|
||||
/// key images, for example transactions in a block. The chain will be checked for duplicates later.
|
||||
///
|
||||
/// 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(
|
||||
inputs: &[Input],
|
||||
spent_kis: &mut HashSet<[u8; 32]>,
|
||||
) -> Result<(), ConsensusError> {
|
||||
for inp in inputs {
|
||||
match inp {
|
||||
Input::ToKey { key_image, .. } => {
|
||||
if !key_image.is_torsion_free() {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"key image has torsion",
|
||||
));
|
||||
}
|
||||
if !spent_kis.insert(key_image.compress().to_bytes()) {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"key image already spent",
|
||||
));
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"Input not ToKey",
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
113
consensus/src/transactions/outputs.rs
Normal file
113
consensus/src/transactions/outputs.rs
Normal file
|
@ -0,0 +1,113 @@
|
|||
use std::sync::OnceLock;
|
||||
|
||||
use monero_serai::transaction::Output;
|
||||
|
||||
use crate::{hardforks::HardFork, helper::check_point, transactions::TxVersion, ConsensusError};
|
||||
|
||||
static DECOMPOSED_AMOUNTS: OnceLock<[u64; 172]> = OnceLock::new();
|
||||
|
||||
pub(crate) fn decomposed_amounts() -> &'static [u64; 172] {
|
||||
DECOMPOSED_AMOUNTS.get_or_init(|| {
|
||||
let mut amounts = [1; 172];
|
||||
for zeros in 0..19 {
|
||||
for digit in 1..10 {
|
||||
amounts[zeros * 9 + digit - 1] =
|
||||
(digit * 10_usize.pow(zeros as u32)).try_into().unwrap();
|
||||
}
|
||||
}
|
||||
amounts[171] = 10000000000000000000;
|
||||
amounts
|
||||
})
|
||||
}
|
||||
|
||||
/// Checks the output keys are canonical points.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#output-keys-canonical
|
||||
pub(crate) fn check_output_keys(outputs: &[Output]) -> Result<(), ConsensusError> {
|
||||
for out in outputs {
|
||||
if !check_point(&out.key) {
|
||||
return Err(ConsensusError::TransactionInvalidOutput(
|
||||
"outputs one time key is not a valid point",
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Checks the output types are allowed.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#output-type
|
||||
pub(crate) fn check_output_types(outputs: &[Output], hf: &HardFork) -> Result<(), ConsensusError> {
|
||||
if hf == &HardFork::V15 {
|
||||
for outs in outputs.windows(2) {
|
||||
if outs[0].view_tag.is_some() != outs[0].view_tag.is_some() {
|
||||
return Err(ConsensusError::TransactionInvalidOutput(
|
||||
"v15 output must have one output type",
|
||||
));
|
||||
}
|
||||
}
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
for out in outputs {
|
||||
if hf <= &HardFork::V14 && out.view_tag.is_some() {
|
||||
return Err(ConsensusError::TransactionInvalidOutput(
|
||||
"tagged output used before allowed",
|
||||
));
|
||||
} else if hf >= &HardFork::V16 && out.view_tag.is_none() {
|
||||
return Err(ConsensusError::TransactionInvalidOutput(
|
||||
"output does not have a view tag",
|
||||
));
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Checks that an output amount is decomposed.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#output-amount
|
||||
pub(crate) fn is_decomposed_amount(amount: u64) -> bool {
|
||||
decomposed_amounts().binary_search(&amount).is_ok()
|
||||
}
|
||||
|
||||
/// Checks the outputs amount for version 1 txs.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#output-amount
|
||||
fn check_output_amount_v1(amount: u64, hf: &HardFork) -> Result<(), ConsensusError> {
|
||||
if amount == 0 {
|
||||
return Err(ConsensusError::TransactionInvalidOutput(
|
||||
"zero amount output for v1 tx",
|
||||
));
|
||||
}
|
||||
|
||||
if hf >= &HardFork::V2 && !is_decomposed_amount(amount) {
|
||||
return Err(ConsensusError::TransactionInvalidOutput(
|
||||
"v1 tx does not have decomposed amount",
|
||||
));
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Sums the outputs, checking for overflow and other consensus rules.
|
||||
///
|
||||
/// Should only be used on v1 transactions.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#inputs-and-outputs-must-not-overflow
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#output-amount
|
||||
pub(crate) fn sum_outputs_v1(outputs: &[Output], hf: &HardFork) -> Result<u64, ConsensusError> {
|
||||
let mut sum: u64 = 0;
|
||||
|
||||
for out in outputs {
|
||||
let raw_amount = out.amount.unwrap_or(0);
|
||||
|
||||
check_output_amount_v1(raw_amount, hf)?;
|
||||
|
||||
sum = sum
|
||||
.checked_add(raw_amount)
|
||||
.ok_or(ConsensusError::TransactionOutputsOverflow)?;
|
||||
}
|
||||
|
||||
Ok(sum)
|
||||
}
|
183
consensus/src/transactions/signatures.rs
Normal file
183
consensus/src/transactions/signatures.rs
Normal file
|
@ -0,0 +1,183 @@
|
|||
use std::collections::{HashMap, HashSet};
|
||||
|
||||
use curve25519_dalek::EdwardsPoint;
|
||||
use monero_serai::{
|
||||
ringct::{mlsag::RingMatrix, RctType},
|
||||
transaction::{Input, Transaction},
|
||||
};
|
||||
use tower::ServiceExt;
|
||||
|
||||
use crate::{hardforks::HardFork, ConsensusError, Database, DatabaseRequest, DatabaseResponse};
|
||||
|
||||
mod ring_sigs;
|
||||
|
||||
pub(crate) use ring_sigs::verify_inputs_signatures;
|
||||
|
||||
/// Gets the absolute offsets from the relative offsets.
|
||||
/// This function will return an error if the relative offsets are empty or if the hf version is 6 or higher and
|
||||
/// not all the ring members are unique.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#inputs-must-have-decoys
|
||||
/// TODO: change the URL on this link \/
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#unique-inputs
|
||||
fn get_absolute_offsets(
|
||||
relative_offsets: &[u64],
|
||||
hf: &HardFork,
|
||||
) -> Result<Vec<u64>, ConsensusError> {
|
||||
if relative_offsets.is_empty() {
|
||||
return Err(ConsensusError::TransactionHasInvalidRing(
|
||||
"ring has no members",
|
||||
));
|
||||
}
|
||||
|
||||
let mut offsets = Vec::with_capacity(relative_offsets.len());
|
||||
offsets.push(relative_offsets[0]);
|
||||
|
||||
for i in 1..relative_offsets.len() {
|
||||
if relative_offsets[i] == 0 && hf >= &HardFork::V6 {
|
||||
// all ring members must be unique after v6
|
||||
return Err(ConsensusError::TransactionHasInvalidRing(
|
||||
"ring has duplicate member",
|
||||
));
|
||||
}
|
||||
|
||||
offsets.push(relative_offsets[i - 1] + relative_offsets[i]);
|
||||
}
|
||||
Ok(offsets)
|
||||
}
|
||||
|
||||
/// Returns the outputs that are needed to verify the transaction inputs.
|
||||
///
|
||||
/// The returned value is a hashmap with:
|
||||
/// keys = amount
|
||||
/// values = hashset of amount idxs
|
||||
///
|
||||
pub fn get_ring_member_ids(
|
||||
tx: &Transaction,
|
||||
hf: &HardFork,
|
||||
) -> Result<HashMap<u64, HashSet<u64>>, ConsensusError> {
|
||||
let mut members = HashMap::with_capacity(tx.prefix.inputs.len());
|
||||
|
||||
for input in &tx.prefix.inputs {
|
||||
match input {
|
||||
Input::ToKey {
|
||||
amount,
|
||||
key_offsets,
|
||||
..
|
||||
} => members
|
||||
.entry(amount.unwrap_or(0))
|
||||
.or_insert_with(HashSet::new)
|
||||
.extend(get_absolute_offsets(key_offsets, hf)?),
|
||||
// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#input-type
|
||||
_ => {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"input not ToKey",
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// https://cuprate.github.io/monero-book/consensus_rules/transactions.html#no-empty-inputs
|
||||
if members.is_empty() {
|
||||
return Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"transaction has no inputs",
|
||||
));
|
||||
}
|
||||
|
||||
Ok(members)
|
||||
}
|
||||
|
||||
/// Represents the ring members of the inputs.
|
||||
pub enum Rings {
|
||||
/// Legacy, pre-ringCT, ring.
|
||||
Legacy(Vec<Vec<EdwardsPoint>>),
|
||||
/// TODO:
|
||||
RingCT,
|
||||
}
|
||||
|
||||
impl Rings {
|
||||
/// Builds the rings for the transaction inputs, from the outputs.
|
||||
pub fn new(
|
||||
outputs: &HashMap<u64, HashMap<u64, [EdwardsPoint; 2]>>,
|
||||
inputs: &[Input],
|
||||
rct_type: RctType,
|
||||
hf: &HardFork,
|
||||
) -> Result<Rings, ConsensusError> {
|
||||
match rct_type {
|
||||
RctType::Null => {
|
||||
let legacy_ring = inputs
|
||||
.iter()
|
||||
.map(|inp| match inp {
|
||||
Input::ToKey {
|
||||
amount,
|
||||
key_offsets,
|
||||
..
|
||||
} => {
|
||||
let offsets = get_absolute_offsets(key_offsets, hf)?;
|
||||
Ok(offsets
|
||||
.iter()
|
||||
.map(|offset| {
|
||||
// get the hashmap for this amount.
|
||||
outputs
|
||||
.get(&amount.unwrap_or(0))
|
||||
// get output at the index from the amount hashmap.
|
||||
.and_then(|amount_map| amount_map.get(offset))
|
||||
// this is a legacy ring we only need the one time key.
|
||||
.and_then(|out| Some(out[0]))
|
||||
.ok_or(ConsensusError::TransactionHasInvalidRing(
|
||||
"ring member not in database",
|
||||
))
|
||||
})
|
||||
.collect::<Result<_, ConsensusError>>()?)
|
||||
}
|
||||
_ => Err(ConsensusError::TransactionHasInvalidInput(
|
||||
"input not ToKey",
|
||||
)),
|
||||
})
|
||||
.collect::<Result<_, ConsensusError>>()?;
|
||||
|
||||
Ok(Rings::Legacy(legacy_ring))
|
||||
}
|
||||
_ => todo!("RingCT"),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Get [`Rings`] aka the outputs a transaction references for each transaction.
|
||||
pub async fn batch_get_rings<D: Database>(
|
||||
txs: &[Transaction],
|
||||
hf: &HardFork,
|
||||
database: D,
|
||||
) -> Result<Vec<Rings>, ConsensusError> {
|
||||
let mut output_ids = HashMap::new();
|
||||
|
||||
for tx in txs {
|
||||
let mut tx_out_ids = get_ring_member_ids(tx, hf)?;
|
||||
for (amount, idxs) in tx_out_ids.drain() {
|
||||
output_ids
|
||||
.entry(amount)
|
||||
.or_insert_with(HashSet::new)
|
||||
.extend(idxs);
|
||||
}
|
||||
}
|
||||
|
||||
let DatabaseResponse::Outputs(outputs) = database
|
||||
.oneshot(DatabaseRequest::Outputs(output_ids))
|
||||
.await?
|
||||
else {
|
||||
panic!("Database sent incorrect response!")
|
||||
};
|
||||
|
||||
let mut rings = Vec::with_capacity(txs.len());
|
||||
|
||||
for tx in txs {
|
||||
rings.push(Rings::new(
|
||||
&outputs,
|
||||
&tx.prefix.inputs,
|
||||
tx.rct_signatures.rct_type(),
|
||||
hf,
|
||||
)?);
|
||||
}
|
||||
|
||||
Ok(rings)
|
||||
}
|
48
consensus/src/transactions/signatures/ring_sigs.rs
Normal file
48
consensus/src/transactions/signatures/ring_sigs.rs
Normal file
|
@ -0,0 +1,48 @@
|
|||
//! Version 1 ring signature verification.
|
||||
//!
|
||||
//! Some checks have to be done at deserialization or with data we don't have so we can't do them here, those checks are:
|
||||
//! https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#signatures-must-be-canonical
|
||||
//! this happens at deserialization in monero-serai.
|
||||
//! https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#amount-of-signatures-in-a-ring
|
||||
//! and this happens during ring signature verification in monero-serai.
|
||||
//!
|
||||
use monero_serai::{ring_signatures::RingSignature, transaction::Input};
|
||||
|
||||
use super::Rings;
|
||||
use crate::ConsensusError;
|
||||
|
||||
/// Verifies the ring signature.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#the-ring-signature-must-be-valid
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/pre_rct.html#amount-of-ring-signatures
|
||||
pub fn verify_inputs_signatures(
|
||||
rings: &Rings,
|
||||
inputs: &[Input],
|
||||
signatures: &[RingSignature],
|
||||
tx_sig_hash: &[u8; 32],
|
||||
) -> Result<(), ConsensusError> {
|
||||
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(ConsensusError::TransactionSignatureInvalid(
|
||||
"number of ring sigs != inputs",
|
||||
));
|
||||
}
|
||||
|
||||
for ((input, ring), sig) in inputs.iter().zip(rings).zip(signatures) {
|
||||
let Input::ToKey { key_image, .. } = input else {
|
||||
panic!("How did we build a ring with no decoys?");
|
||||
};
|
||||
|
||||
if !sig.verify_ring_signature(tx_sig_hash, ring, key_image) {
|
||||
return Err(ConsensusError::TransactionSignatureInvalid(
|
||||
"Invalid ring signature",
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
||||
_ => panic!("tried to verify v1 tx with a non v1 ring"),
|
||||
}
|
||||
Ok(())
|
||||
}
|
72
consensus/src/transactions/time_lock.rs
Normal file
72
consensus/src/transactions/time_lock.rs
Normal file
|
@ -0,0 +1,72 @@
|
|||
use std::cmp::min;
|
||||
|
||||
use monero_serai::transaction::Timelock;
|
||||
|
||||
use crate::{block::DifficultyCache, hardforks::HardFork, helper::current_time};
|
||||
|
||||
const BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW: u64 = 60;
|
||||
|
||||
/// Checks if an outputs unlock time has passed.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#unlock-time
|
||||
pub fn output_unlocked(
|
||||
time_lock: &Timelock,
|
||||
difficulty_cache: &DifficultyCache,
|
||||
current_chain_height: u64,
|
||||
hf: &HardFork,
|
||||
) -> bool {
|
||||
match *time_lock {
|
||||
Timelock::None => true,
|
||||
Timelock::Block(unlock_height) => {
|
||||
check_block_time_lock(unlock_height.try_into().unwrap(), current_chain_height)
|
||||
}
|
||||
Timelock::Time(unlock_time) => {
|
||||
check_timestamp_time_lock(unlock_time, difficulty_cache, current_chain_height, hf)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns if a locked output, which uses a block height, can be spend.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#block-height
|
||||
fn check_block_time_lock(unlock_height: u64, current_chain_height: u64) -> bool {
|
||||
// current_chain_height = 1 + top height
|
||||
unlock_height >= current_chain_height
|
||||
}
|
||||
|
||||
/// Returns the timestamp the should be used when checking locked outputs.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#getting-the-current-time
|
||||
fn get_current_timestamp(
|
||||
difficulty_cache: &DifficultyCache,
|
||||
current_chain_height: u64,
|
||||
hf: &HardFork,
|
||||
) -> u64 {
|
||||
if hf < &HardFork::V13 || current_chain_height < BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW {
|
||||
current_time()
|
||||
} else {
|
||||
let median = difficulty_cache
|
||||
.median_timestamp(BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW.try_into().unwrap());
|
||||
let adjusted_median =
|
||||
median + (BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW + 1) * hf.block_time().as_secs() / 2;
|
||||
|
||||
// This is safe as we just check we don't have less than 60 blocks in the chain.
|
||||
let adjusted_top_block =
|
||||
difficulty_cache.top_block_timestamp().unwrap() + hf.block_time().as_secs();
|
||||
|
||||
min(adjusted_median, adjusted_top_block)
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns if a locked output, which uses a block height, can be spend.
|
||||
///
|
||||
/// https://cuprate.github.io/monero-book/consensus_rules/transactions/unlock_time.html#timestamp
|
||||
fn check_timestamp_time_lock(
|
||||
unlock_timestamp: u64,
|
||||
difficulty_cache: &DifficultyCache,
|
||||
current_chain_height: u64,
|
||||
hf: &HardFork,
|
||||
) -> bool {
|
||||
let timestamp = get_current_timestamp(difficulty_cache, current_chain_height, hf);
|
||||
timestamp + hf.block_time().as_secs() >= unlock_timestamp
|
||||
}
|
Loading…
Reference in a new issue