validator/pow: store block difficulty info in sled

This commit is contained in:
aggstam
2023-10-13 19:40:18 +03:00
parent 594946044e
commit 893ea4a52f
9 changed files with 252 additions and 25 deletions

View File

@@ -30,7 +30,7 @@ fn forks() -> Result<()> {
// Create a temporary blockchain and a PoW module // Create a temporary blockchain and a PoW module
let blockchain = Blockchain::new(&sled::Config::new().temporary(true).open()?)?; let blockchain = Blockchain::new(&sled::Config::new().temporary(true).open()?)?;
let module = PoWModule::new(blockchain.clone(), None, Some(90)); let module = PoWModule::new(blockchain.clone(), None, Some(90))?;
// Create a fork // Create a fork
let fork = Fork::new(&blockchain, module)?; let fork = Fork::new(&blockchain, module)?;

View File

@@ -25,6 +25,7 @@ use darkfi_sdk::{
use darkfi_serial::async_trait; use darkfi_serial::async_trait;
use darkfi_serial::{deserialize, serialize, Encodable, SerialDecodable, SerialEncodable}; use darkfi_serial::{deserialize, serialize, Encodable, SerialDecodable, SerialEncodable};
use num_bigint::BigUint;
use crate::{tx::Transaction, Error, Result}; use crate::{tx::Transaction, Error, Result};
@@ -294,7 +295,7 @@ const SLED_BLOCK_ORDER_TREE: &[u8] = b"_block_order";
/// The `BlockOrderStore` is a `sled` tree storing the order of the /// The `BlockOrderStore` is a `sled` tree storing the order of the
/// blockchain's blocks, where the key is the order number, and the value is /// blockchain's blocks, where the key is the order number, and the value is
/// the blocks' hash. [`BlockStore`] can be queried with this hash. /// the blocks' hash. [`BlockOrderStore`] can be queried with this order number.
#[derive(Clone)] #[derive(Clone)]
pub struct BlockOrderStore(pub sled::Tree); pub struct BlockOrderStore(pub sled::Tree);
@@ -488,3 +489,171 @@ impl BlockOrderStoreOverlay {
Ok(self.0.lock().unwrap().is_empty(SLED_BLOCK_ORDER_TREE)?) Ok(self.0.lock().unwrap().is_empty(SLED_BLOCK_ORDER_TREE)?)
} }
} }
/// Auxiliary structure used to keep track of block PoW difficulty information.
/// Note: we only need height cummulative difficulty, but we also keep its actual
/// difficulty, so we can verify the sequence and/or know specific block height
/// difficulty, if ever needed.
#[derive(Debug)]
pub struct BlockDifficulty {
/// Block height number
pub height: u64,
/// Block creation timestamp
pub timestamp: u64,
/// Height difficulty
pub difficulty: BigUint,
/// Height cummulative difficulty (total + height difficulty)
pub cummulative_difficulty: BigUint,
}
impl BlockDifficulty {
pub fn new(
height: u64,
timestamp: u64,
difficulty: BigUint,
cummulative_difficulty: BigUint,
) -> Self {
Self { height, timestamp, difficulty, cummulative_difficulty }
}
}
// Note: Doing all the imports here as this might get obselete if
// we implemented Encodable/Decodable for num_bigint::BigUint.
impl darkfi_serial::Encodable for BlockDifficulty {
fn encode<S: std::io::Write>(&self, mut s: S) -> std::io::Result<usize> {
let mut len = 0;
len += self.height.encode(&mut s)?;
len += self.timestamp.encode(&mut s)?;
len += self.difficulty.to_bytes_be().encode(&mut s)?;
len += self.cummulative_difficulty.to_bytes_be().encode(&mut s)?;
Ok(len)
}
}
impl darkfi_serial::Decodable for BlockDifficulty {
fn decode<D: std::io::Read>(mut d: D) -> std::io::Result<Self> {
let height: u64 = darkfi_serial::Decodable::decode(&mut d)?;
let timestamp: u64 = darkfi_serial::Decodable::decode(&mut d)?;
let bytes: Vec<u8> = darkfi_serial::Decodable::decode(&mut d)?;
let difficulty: BigUint = BigUint::from_bytes_be(&bytes);
let bytes: Vec<u8> = darkfi_serial::Decodable::decode(&mut d)?;
let cummulative_difficulty: BigUint = BigUint::from_bytes_be(&bytes);
let ret = Self { height, timestamp, difficulty, cummulative_difficulty };
Ok(ret)
}
}
/// [`BlockDifficulty`] sled tree
const SLED_BLOCK_DIFFICULTY_TREE: &[u8] = b"_block_difficulty";
/// The `BlockDifficultyStore` is a `sled` tree storing the difficulty information
/// of the blockchain's blocks, where the key is the block height number, and the
/// value is the blocks' hash. [`BlockDifficultyStore`] can be queried with this
/// height number.
#[derive(Clone)]
pub struct BlockDifficultyStore(pub sled::Tree);
impl BlockDifficultyStore {
/// Opens a new or existing `BlockDifficultyStore` on the given sled database.
pub fn new(db: &sled::Db) -> Result<Self> {
let tree = db.open_tree(SLED_BLOCK_DIFFICULTY_TREE)?;
Ok(Self(tree))
}
/// Insert a slice of [`BlockDifficulty`] into the store.
pub fn insert(&self, block_difficulties: &[BlockDifficulty]) -> Result<()> {
let batch = self.insert_batch(block_difficulties)?;
self.0.apply_batch(batch)?;
Ok(())
}
/// Generate the sled batch corresponding to an insert, so caller
/// can handle the write operation.
/// The block's height number is used as the key, while value is
// the serialized [`BlockDifficulty`] itself.
pub fn insert_batch(&self, block_difficulties: &[BlockDifficulty]) -> Result<sled::Batch> {
let mut batch = sled::Batch::default();
for block_difficulty in block_difficulties {
batch.insert(&block_difficulty.height.to_be_bytes(), serialize(block_difficulty));
}
Ok(batch)
}
/// Fetch given block height numbers from the block difficulties store.
/// The resulting vector contains `Option`, which is `Some` if the block
/// height number was found in the block difficulties store, and otherwise
/// it is `None`, if it has not.
/// The second parameter is a boolean which tells the function to fail in
/// case at least one block height number was not found.
pub fn get(&self, heights: &[u64], strict: bool) -> Result<Vec<Option<BlockDifficulty>>> {
let mut ret = Vec::with_capacity(heights.len());
for height in heights {
if let Some(found) = self.0.get(height.to_be_bytes())? {
let block_difficulty = deserialize(&found)?;
ret.push(Some(block_difficulty));
} else {
if strict {
return Err(Error::BlockDifficultyNotFound(*height))
}
ret.push(None);
}
}
Ok(ret)
}
/// Fetch the last N records from the block difficulties store, in order.
pub fn get_last_n(&self, n: usize) -> Result<Vec<BlockDifficulty>> {
// Build an iterator to retrieve last N records
let records = self.0.iter().rev().take(n);
// Since the iterator grabs in right -> left order,
// we deserialize found records, and push them in reverse order
let mut last_n = vec![];
for record in records {
last_n.insert(0, deserialize(&record?.1)?);
}
Ok(last_n)
}
/// Retrieve all blockdifficulties from the block difficulties store in
/// the form of a vector containing (`height`, `difficulty`) tuples.
/// Be careful as this will try to load everything in memory.
pub fn get_all(&self) -> Result<Vec<(u64, BlockDifficulty)>> {
let mut block_difficulties = vec![];
for record in self.0.iter() {
block_difficulties.push(parse_u64_key_record(record.unwrap())?);
}
Ok(block_difficulties)
}
}
/// Overlay structure over a [`BlockDifficultyStore`] instance.
pub struct BlockDifficultyStoreOverlay(SledDbOverlayPtr);
impl BlockDifficultyStoreOverlay {
pub fn new(overlay: &SledDbOverlayPtr) -> Result<Self> {
overlay.lock().unwrap().open_tree(SLED_BLOCK_DIFFICULTY_TREE)?;
Ok(Self(overlay.clone()))
}
/// Insert a slice of [`BlockDifficulty`] into the overlay.
pub fn insert(&self, block_difficulties: &[BlockDifficulty]) -> Result<()> {
let mut lock = self.0.lock().unwrap();
for block_difficulty in block_difficulties {
lock.insert(
SLED_BLOCK_DIFFICULTY_TREE,
&block_difficulty.height.to_be_bytes(),
&serialize(block_difficulty),
)?;
}
Ok(())
}
}

View File

@@ -29,7 +29,8 @@ use crate::{tx::Transaction, Error, Result};
/// Block related definitions and storage implementations /// Block related definitions and storage implementations
pub mod block_store; pub mod block_store;
pub use block_store::{ pub use block_store::{
Block, BlockInfo, BlockOrderStore, BlockOrderStoreOverlay, BlockStore, BlockStoreOverlay, Block, BlockDifficultyStore, BlockDifficultyStoreOverlay, BlockInfo, BlockOrderStore,
BlockOrderStoreOverlay, BlockStore, BlockStoreOverlay,
}; };
/// Header definition and storage implementation /// Header definition and storage implementation
@@ -65,6 +66,8 @@ pub struct Blockchain {
pub slots: SlotStore, pub slots: SlotStore,
/// Blocks Slots sled tree /// Blocks Slots sled tree
pub blocks_slots: BlocksSlotsStore, pub blocks_slots: BlocksSlotsStore,
/// Block height difficulties sled tree,
pub difficulties: BlockDifficultyStore,
/// Transactions sled tree /// Transactions sled tree
pub transactions: TxStore, pub transactions: TxStore,
/// Pending transactions sled tree /// Pending transactions sled tree
@@ -85,6 +88,7 @@ impl Blockchain {
let order = BlockOrderStore::new(db)?; let order = BlockOrderStore::new(db)?;
let slots = SlotStore::new(db)?; let slots = SlotStore::new(db)?;
let blocks_slots = BlocksSlotsStore::new(db)?; let blocks_slots = BlocksSlotsStore::new(db)?;
let difficulties = BlockDifficultyStore::new(db)?;
let transactions = TxStore::new(db)?; let transactions = TxStore::new(db)?;
let pending_txs = PendingTxStore::new(db)?; let pending_txs = PendingTxStore::new(db)?;
let pending_txs_order = PendingTxOrderStore::new(db)?; let pending_txs_order = PendingTxOrderStore::new(db)?;
@@ -98,6 +102,7 @@ impl Blockchain {
order, order,
slots, slots,
blocks_slots, blocks_slots,
difficulties,
transactions, transactions,
pending_txs, pending_txs,
pending_txs_order, pending_txs_order,
@@ -409,6 +414,8 @@ pub struct BlockchainOverlay {
pub slots: SlotStoreOverlay, pub slots: SlotStoreOverlay,
/// Blocks slots overlay /// Blocks slots overlay
pub blocks_slots: BlocksSlotsStoreOverlay, pub blocks_slots: BlocksSlotsStoreOverlay,
/// Block height difficulties overlay,
pub difficulties: BlockDifficultyStoreOverlay,
/// Transactions overlay /// Transactions overlay
pub transactions: TxStoreOverlay, pub transactions: TxStoreOverlay,
/// Contract states overlay /// Contract states overlay
@@ -426,6 +433,7 @@ impl BlockchainOverlay {
let order = BlockOrderStoreOverlay::new(&overlay)?; let order = BlockOrderStoreOverlay::new(&overlay)?;
let slots = SlotStoreOverlay::new(&overlay)?; let slots = SlotStoreOverlay::new(&overlay)?;
let blocks_slots = BlocksSlotsStoreOverlay::new(&overlay)?; let blocks_slots = BlocksSlotsStoreOverlay::new(&overlay)?;
let difficulties = BlockDifficultyStoreOverlay::new(&overlay)?;
let transactions = TxStoreOverlay::new(&overlay)?; let transactions = TxStoreOverlay::new(&overlay)?;
let contracts = ContractStateStoreOverlay::new(&overlay)?; let contracts = ContractStateStoreOverlay::new(&overlay)?;
let wasm_bincode = WasmStoreOverlay::new(&overlay)?; let wasm_bincode = WasmStoreOverlay::new(&overlay)?;
@@ -437,6 +445,7 @@ impl BlockchainOverlay {
order, order,
slots, slots,
blocks_slots, blocks_slots,
difficulties,
transactions, transactions,
contracts, contracts,
wasm_bincode, wasm_bincode,
@@ -578,6 +587,7 @@ impl BlockchainOverlay {
let order = BlockOrderStoreOverlay::new(&overlay)?; let order = BlockOrderStoreOverlay::new(&overlay)?;
let slots = SlotStoreOverlay::new(&overlay)?; let slots = SlotStoreOverlay::new(&overlay)?;
let blocks_slots = BlocksSlotsStoreOverlay::new(&overlay)?; let blocks_slots = BlocksSlotsStoreOverlay::new(&overlay)?;
let difficulties = BlockDifficultyStoreOverlay::new(&overlay)?;
let transactions = TxStoreOverlay::new(&overlay)?; let transactions = TxStoreOverlay::new(&overlay)?;
let contracts = ContractStateStoreOverlay::new(&overlay)?; let contracts = ContractStateStoreOverlay::new(&overlay)?;
let wasm_bincode = WasmStoreOverlay::new(&overlay)?; let wasm_bincode = WasmStoreOverlay::new(&overlay)?;
@@ -589,6 +599,7 @@ impl BlockchainOverlay {
order, order,
slots, slots,
blocks_slots, blocks_slots,
difficulties,
transactions, transactions,
contracts, contracts,
wasm_bincode, wasm_bincode,

View File

@@ -381,6 +381,9 @@ pub enum Error {
#[error("Block with order number {0} not found in database")] #[error("Block with order number {0} not found in database")]
BlockNumberNotFound(u64), BlockNumberNotFound(u64),
#[error("Block difficulty for height number {0} not found in database")]
BlockDifficultyNotFound(u64),
#[error("Block {0} contains 0 transactions")] #[error("Block {0} contains 0 transactions")]
BlockContainsNoTransactions(String), BlockContainsNoTransactions(String),

View File

@@ -67,9 +67,9 @@ impl Consensus {
time_keeper: TimeKeeper, time_keeper: TimeKeeper,
pow_target: Option<usize>, pow_target: Option<usize>,
testing_mode: bool, testing_mode: bool,
) -> Self { ) -> Result<Self> {
let module = PoWModule::new(blockchain.clone(), None, pow_target); let module = PoWModule::new(blockchain.clone(), None, pow_target)?;
Self { Ok(Self {
blockchain, blockchain,
time_keeper, time_keeper,
participating: false, participating: false,
@@ -77,7 +77,7 @@ impl Consensus {
forks: vec![], forks: vec![],
module, module,
testing_mode, testing_mode,
} })
} }
/// Generate next hot/live PoW slot for all current forks. /// Generate next hot/live PoW slot for all current forks.

View File

@@ -27,7 +27,10 @@ use log::{debug, error, info, warn};
use smol::lock::RwLock; use smol::lock::RwLock;
use crate::{ use crate::{
blockchain::{BlockInfo, Blockchain, BlockchainOverlay}, blockchain::{
block_store::{BlockDifficulty, BlockInfo},
Blockchain, BlockchainOverlay,
},
error::TxVerifyFailed, error::TxVerifyFailed,
tx::Transaction, tx::Transaction,
util::time::TimeKeeper, util::time::TimeKeeper,
@@ -144,8 +147,12 @@ impl Validator {
overlay.lock().unwrap().overlay.lock().unwrap().apply()?; overlay.lock().unwrap().overlay.lock().unwrap().apply()?;
info!(target: "validator::new", "Initializing Consensus"); info!(target: "validator::new", "Initializing Consensus");
let consensus = let consensus = Consensus::new(
Consensus::new(blockchain.clone(), config.time_keeper, config.pow_target, testing_mode); blockchain.clone(),
config.time_keeper,
config.pow_target,
testing_mode,
)?;
// Create the actual state // Create the actual state
let state = let state =
@@ -376,7 +383,17 @@ impl Validator {
// Update PoW module // Update PoW module
if block.header.version == 1 { if block.header.version == 1 {
module.append(block.header.timestamp.0, &module.next_difficulty()?); // Generate block difficulty
let difficulty = module.next_difficulty()?;
let cummulative_difficulty =
module.cummulative_difficulty.clone() + difficulty.clone();
let block_difficulty = BlockDifficulty::new(
block.header.height,
block.header.timestamp.0,
difficulty,
cummulative_difficulty,
);
module.append_difficulty(&overlay, block_difficulty)?;
} }
// Store block transactions // Store block transactions
@@ -529,7 +546,7 @@ impl Validator {
// Create a time keeper and a PoW module to validate each block // Create a time keeper and a PoW module to validate each block
let mut time_keeper = self.consensus.time_keeper.clone(); let mut time_keeper = self.consensus.time_keeper.clone();
let mut module = PoWModule::new(blockchain.clone(), None, pow_target); let mut module = PoWModule::new(blockchain.clone(), None, pow_target)?;
// Deploy native wasm contracts // Deploy native wasm contracts
deploy_native_contracts(&overlay, &time_keeper, &faucet_pubkeys)?; deploy_native_contracts(&overlay, &time_keeper, &faucet_pubkeys)?;

View File

@@ -35,7 +35,10 @@ use randomx::{RandomXCache, RandomXDataset, RandomXFlags, RandomXVM};
use smol::channel::Receiver; use smol::channel::Receiver;
use crate::{ use crate::{
blockchain::{BlockInfo, Blockchain}, blockchain::{
block_store::{BlockDifficulty, BlockInfo},
Blockchain, BlockchainOverlayPtr,
},
util::{ringbuffer::RingBuffer, time::Timestamp}, util::{ringbuffer::RingBuffer, time::Timestamp},
validator::utils::median, validator::utils::median,
Error, Result, Error, Result,
@@ -79,8 +82,6 @@ const BLOCK_FUTURE_TIME_LIMIT: u64 = 60 * 60 * 2;
/// This struct represents the information required by the PoW algorithm /// This struct represents the information required by the PoW algorithm
#[derive(Clone)] #[derive(Clone)]
pub struct PoWModule { pub struct PoWModule {
/// Canonical (finalized) blockchain
pub blockchain: Blockchain,
/// Number of threads to use for hashing, /// Number of threads to use for hashing,
/// if None provided will use N_THREADS /// if None provided will use N_THREADS
pub threads: usize, pub threads: usize,
@@ -92,18 +93,33 @@ pub struct PoWModule {
/// Latest block cummulative difficulties ringbuffer /// Latest block cummulative difficulties ringbuffer
pub difficulties: RingBuffer<BigUint, BUF_SIZE>, pub difficulties: RingBuffer<BigUint, BUF_SIZE>,
/// Total blocks cummulative difficulty /// Total blocks cummulative difficulty
/// Note: we keep this as a struct field for faster
/// access(optimization), since its always same as
/// difficulties buffer last.
pub cummulative_difficulty: BigUint, pub cummulative_difficulty: BigUint,
} }
impl PoWModule { impl PoWModule {
pub fn new(blockchain: Blockchain, threads: Option<usize>, target: Option<usize>) -> Self { pub fn new(
blockchain: Blockchain,
threads: Option<usize>,
target: Option<usize>,
) -> Result<Self> {
let threads = if let Some(t) = threads { t } else { N_THREADS }; let threads = if let Some(t) = threads { t } else { N_THREADS };
let target = if let Some(t) = target { t } else { DIFFICULTY_TARGET }; let target = if let Some(t) = target { t } else { DIFFICULTY_TARGET };
// TODO: store/retrieve info in/from sled
let timestamps = RingBuffer::<u64, BUF_SIZE>::new(); // Retrieving last BUF_ZISE difficulties from blockchain to build the buffers
let difficulties = RingBuffer::<BigUint, BUF_SIZE>::new(); let mut timestamps = RingBuffer::<u64, BUF_SIZE>::new();
let cummulative_difficulty = BigUint::zero(); let mut difficulties = RingBuffer::<BigUint, BUF_SIZE>::new();
Self { blockchain, threads, target, timestamps, difficulties, cummulative_difficulty } let mut cummulative_difficulty = BigUint::zero();
let last_n = blockchain.difficulties.get_last_n(BUF_SIZE)?;
for difficulty in last_n {
timestamps.push(difficulty.timestamp);
difficulties.push(difficulty.cummulative_difficulty.clone());
cummulative_difficulty = difficulty.cummulative_difficulty;
}
Ok(Self { threads, target, timestamps, difficulties, cummulative_difficulty })
} }
/// Compute the next mining difficulty, based on current ring buffers. /// Compute the next mining difficulty, based on current ring buffers.
@@ -247,6 +263,17 @@ impl PoWModule {
self.difficulties.push(self.cummulative_difficulty.clone()); self.difficulties.push(self.cummulative_difficulty.clone());
} }
/// Append provided block difficulty to the ring buffers and insert
/// it to provided overlay
pub fn append_difficulty(
&mut self,
overlay: &BlockchainOverlayPtr,
difficulty: BlockDifficulty,
) -> Result<()> {
self.append(difficulty.timestamp, &difficulty.difficulty);
overlay.lock().unwrap().difficulties.insert(&[difficulty])
}
/// Mine provided block, based on provided PoW module next mine target and difficulty /// Mine provided block, based on provided PoW module next mine target and difficulty
pub fn mine_block( pub fn mine_block(
&self, &self,
@@ -369,7 +396,7 @@ mod tests {
fn test_wide_difficulty() -> Result<()> { fn test_wide_difficulty() -> Result<()> {
let sled_db = sled::Config::new().temporary(true).open()?; let sled_db = sled::Config::new().temporary(true).open()?;
let blockchain = Blockchain::new(&sled_db)?; let blockchain = Blockchain::new(&sled_db)?;
let mut module = PoWModule::new(blockchain, None, Some(DEFAULT_TEST_DIFFICULTY_TARGET)); let mut module = PoWModule::new(blockchain, None, Some(DEFAULT_TEST_DIFFICULTY_TARGET))?;
let output = Command::new("./script/research/pow/gen_wide_data.py").output().unwrap(); let output = Command::new("./script/research/pow/gen_wide_data.py").output().unwrap();
let reader = Cursor::new(output.stdout); let reader = Cursor::new(output.stdout);
@@ -402,7 +429,7 @@ mod tests {
// Default setup // Default setup
let sled_db = sled::Config::new().temporary(true).open()?; let sled_db = sled::Config::new().temporary(true).open()?;
let blockchain = Blockchain::new(&sled_db)?; let blockchain = Blockchain::new(&sled_db)?;
let module = PoWModule::new(blockchain, None, Some(DEFAULT_TEST_DIFFICULTY_TARGET)); let module = PoWModule::new(blockchain, None, Some(DEFAULT_TEST_DIFFICULTY_TARGET))?;
let (_, recvr) = smol::channel::bounded(1); let (_, recvr) = smol::channel::bounded(1);
let genesis_block = BlockInfo::default(); let genesis_block = BlockInfo::default();

View File

@@ -335,7 +335,7 @@ pub fn validate_pos_slot(
/// Be careful as this will try to load everything in memory. /// Be careful as this will try to load everything in memory.
pub fn validate_blockchain(blockchain: &Blockchain, pow_target: Option<usize>) -> Result<()> { pub fn validate_blockchain(blockchain: &Blockchain, pow_target: Option<usize>) -> Result<()> {
// Generate a PoW module // Generate a PoW module
let mut module = PoWModule::new(blockchain.clone(), None, pow_target); let mut module = PoWModule::new(blockchain.clone(), None, pow_target)?;
// We use block order store here so we have all blocks in order // We use block order store here so we have all blocks in order
let blocks = blockchain.order.get_all()?; let blocks = blockchain.order.get_all()?;
for (index, block) in blocks[1..].iter().enumerate() { for (index, block) in blocks[1..].iter().enumerate() {

View File

@@ -40,7 +40,7 @@ struct Node {
impl Node { impl Node {
fn new() -> Result<Self> { fn new() -> Result<Self> {
let blockchain = Blockchain::new(&sled::Config::new().temporary(true).open()?)?; let blockchain = Blockchain::new(&sled::Config::new().temporary(true).open()?)?;
let module = PoWModule::new(blockchain.clone(), None, POW_TARGET); let module = PoWModule::new(blockchain.clone(), None, POW_TARGET)?;
Ok(Self { blockchain, module }) Ok(Self { blockchain, module })
} }
} }