src/consensus: split state, fully sled integration, modified nodes-tool to extract all sled db data

This commit is contained in:
aggstam
2022-04-14 13:19:00 +03:00
parent 3658317512
commit 7a34ec3d45
16 changed files with 691 additions and 270 deletions

View File

@@ -1,3 +1,3 @@
/target
Cargo.lock
validatord_state_*
validatord_*

View File

@@ -9,6 +9,7 @@ features = ["blockchain"]
[dependencies]
blake3 = "1.3.1"
sled = "0.34.7"
[workspace]

View File

@@ -1,19 +1,328 @@
use std::{fs::File, io::Write};
use darkfi::{consensus::state::State, util::expand_path, Result};
use darkfi::{
consensus::{
block::{Block, BlockProposal, BlockStore},
blockchain::{Blockchain, ProposalsChain},
metadata::{Metadata, MetadataStore, OuroborosMetadata, StreamletMetadata},
participant::Participant,
state::{ConsensusState, ValidatorState},
tx::{Tx, TxStore},
util::Timestamp,
vote::Vote,
},
util::expand_path,
Result,
};
#[derive(Debug)]
struct ParticipantInfo {
_id: u64,
_joined: u64,
_voted: Option<u64>,
}
impl ParticipantInfo {
pub fn new(participant: &Participant) -> ParticipantInfo {
let _id = participant.id;
let _joined = participant.joined;
let _voted = participant.voted;
ParticipantInfo { _id, _joined, _voted }
}
}
#[derive(Debug)]
struct VoteInfo {
_proposal: blake3::Hash,
_sl: u64,
_id: u64,
}
impl VoteInfo {
pub fn new(vote: &Vote) -> VoteInfo {
let _proposal = vote.proposal;
let _sl = vote.sl;
let _id = vote.id;
VoteInfo { _proposal, _sl, _id }
}
}
#[derive(Debug)]
struct StreamletMetadataInfo {
_votes: Vec<VoteInfo>,
_notarized: bool,
_finalized: bool,
_participants: Vec<ParticipantInfo>,
}
impl StreamletMetadataInfo {
pub fn new(metadata: &StreamletMetadata) -> StreamletMetadataInfo {
let mut _votes = Vec::new();
for vote in &metadata.votes {
_votes.push(VoteInfo::new(&vote));
}
let _notarized = metadata.notarized;
let _finalized = metadata.finalized;
let mut _participants = Vec::new();
for participant in &metadata.participants {
_participants.push(ParticipantInfo::new(&participant));
}
StreamletMetadataInfo { _votes, _notarized, _finalized, _participants }
}
}
#[derive(Debug)]
struct OuroborosMetadataInfo {
_proof: String,
_r: String,
_s: String,
}
impl OuroborosMetadataInfo {
pub fn new(metadata: &OuroborosMetadata) -> OuroborosMetadataInfo {
let _proof = metadata.proof.clone();
let _r = metadata.r.clone();
let _s = metadata.s.clone();
OuroborosMetadataInfo { _proof, _r, _s }
}
}
#[derive(Debug)]
struct MetadataInfo {
_timestamp: Timestamp,
_om: OuroborosMetadataInfo,
_sm: StreamletMetadataInfo,
}
impl MetadataInfo {
pub fn new(metadata: &Metadata) -> MetadataInfo {
let _timestamp = metadata.timestamp.clone();
let _om = OuroborosMetadataInfo::new(&metadata.om);
let _sm = StreamletMetadataInfo::new(&metadata.sm);
MetadataInfo { _timestamp, _om, _sm }
}
}
#[derive(Debug)]
struct ProposalInfo {
_id: u64,
_st: blake3::Hash,
_sl: u64,
_txs: Vec<Tx>,
_metadata: MetadataInfo,
}
impl ProposalInfo {
pub fn new(proposal: &BlockProposal) -> ProposalInfo {
let _id = proposal.id;
let _st = proposal.st;
let _sl = proposal.sl;
let _txs = proposal.txs.clone();
let _metadata = MetadataInfo::new(&proposal.metadata);
ProposalInfo { _id, _st, _sl, _txs, _metadata }
}
}
#[derive(Debug)]
struct ProposalsInfoChain {
_proposals: Vec<ProposalInfo>,
}
impl ProposalsInfoChain {
pub fn new(proposals: &ProposalsChain) -> ProposalsInfoChain {
let mut _proposals = Vec::new();
for proposal in &proposals.proposals {
_proposals.push(ProposalInfo::new(&proposal));
}
ProposalsInfoChain { _proposals }
}
}
#[derive(Debug)]
struct ConsensusInfo {
_genesis: Timestamp,
_last_block: blake3::Hash,
_last_sl: u64,
_proposals: Vec<ProposalsInfoChain>,
}
impl ConsensusInfo {
pub fn new(consensus: &ConsensusState) -> ConsensusInfo {
let _genesis = consensus.genesis.clone();
let _last_block = consensus.last_block.clone();
let _last_sl = consensus.last_sl.clone();
let mut _proposals = Vec::new();
for proposal in &consensus.proposals {
_proposals.push(ProposalsInfoChain::new(&proposal));
}
ConsensusInfo { _genesis, _last_block, _last_sl, _proposals }
}
}
#[derive(Debug)]
struct BlockInfo {
_hash: blake3::Hash,
_st: blake3::Hash,
_sl: u64,
_txs: Vec<blake3::Hash>,
}
impl BlockInfo {
pub fn new(_hash: blake3::Hash, block: &Block) -> BlockInfo {
let _st = block.st;
let _sl = block.sl;
let _txs = block.txs.clone();
BlockInfo { _hash, _st, _sl, _txs }
}
}
#[derive(Debug)]
struct BlockInfoChain {
_blocks: Vec<BlockInfo>,
}
impl BlockInfoChain {
pub fn new(blockstore: &BlockStore) -> BlockInfoChain {
let mut _blocks = Vec::new();
let result = blockstore.get_all();
match result {
Ok(iter) => {
for item in iter.iter() {
match item {
Some((hash, block)) => _blocks.push(BlockInfo::new(hash.clone(), &block)),
None => (),
};
}
}
Err(e) => println!("Error: {:?}", e),
}
BlockInfoChain { _blocks }
}
}
#[derive(Debug)]
struct TxInfo {
_hash: blake3::Hash,
_payload: String,
}
impl TxInfo {
pub fn new(_hash: blake3::Hash, tx: &Tx) -> TxInfo {
let _payload = tx.payload.clone();
TxInfo { _hash, _payload }
}
}
#[derive(Debug)]
struct TxStoreInfo {
_transactions: Vec<TxInfo>,
}
impl TxStoreInfo {
pub fn new(txstore: &TxStore) -> TxStoreInfo {
let mut _transactions = Vec::new();
let result = txstore.get_all();
match result {
Ok(iter) => {
for item in iter.iter() {
match item {
Some((hash, tx)) => _transactions.push(TxInfo::new(hash.clone(), &tx)),
None => (),
};
}
}
Err(e) => println!("Error: {:?}", e),
}
TxStoreInfo { _transactions }
}
}
#[derive(Debug)]
struct HashedMetadataInfo {
_block: blake3::Hash,
_metadata: MetadataInfo,
}
impl HashedMetadataInfo {
pub fn new(_block: blake3::Hash, metadata: &Metadata) -> HashedMetadataInfo {
let _metadata = MetadataInfo::new(&metadata);
HashedMetadataInfo { _block, _metadata }
}
}
#[derive(Debug)]
struct MetadataStoreInfo {
_metadata: Vec<HashedMetadataInfo>,
}
impl MetadataStoreInfo {
pub fn new(metadatastore: &MetadataStore) -> MetadataStoreInfo {
let mut _metadata = Vec::new();
let result = metadatastore.get_all();
match result {
Ok(iter) => {
for item in iter.iter() {
match item {
Some((hash, m)) => {
_metadata.push(HashedMetadataInfo::new(hash.clone(), &m))
}
None => (),
};
}
}
Err(e) => println!("Error: {:?}", e),
}
MetadataStoreInfo { _metadata }
}
}
#[derive(Debug)]
struct BlockchainInfo {
_blocks: BlockInfoChain,
_transactions: TxStoreInfo,
_metadata: MetadataStoreInfo,
}
impl BlockchainInfo {
pub fn new(blockchain: &Blockchain) -> BlockchainInfo {
let _blocks = BlockInfoChain::new(&blockchain.blocks);
let _transactions = TxStoreInfo::new(&blockchain.transactions);
let _metadata = MetadataStoreInfo::new(&blockchain.metadata);
BlockchainInfo { _blocks, _transactions, _metadata }
}
}
#[derive(Debug)]
struct StateInfo {
_id: u64,
_consensus: ConsensusInfo,
_blockchain: BlockchainInfo,
_unconfirmed_txs: Vec<Tx>,
}
impl StateInfo {
pub fn new(state: &ValidatorState) -> StateInfo {
let _id = state.id;
let _consensus = ConsensusInfo::new(&state.consensus);
let _blockchain = BlockchainInfo::new(&state.blockchain);
let _unconfirmed_txs = state.unconfirmed_txs.clone();
StateInfo { _id, _consensus, _blockchain, _unconfirmed_txs }
}
}
fn main() -> Result<()> {
let nodes = 4;
let genesis = 1648383795;
for i in 0..4 {
for i in 0..nodes {
let path = format!("~/.config/darkfi/validatord_db_{:?}", i);
let database_path = expand_path(&path).unwrap();
println!("Export state from sled database: {:?}", database_path);
let database = sled::open(database_path).unwrap();
let state = State::load_current_state(genesis, i, &database).unwrap();
let state_string = format!("{:#?}", state.read().unwrap());
println!("Export data from sled database: {:?}", database_path);
let state = ValidatorState::new(database_path, i, genesis).unwrap();
let info = StateInfo::new(&*state.read().unwrap());
let info_string = format!("{:#?}", info);
let path = format!("validatord_state_{:?}", i);
let mut file = File::create(path)?;
file.write(state_string.as_bytes())?;
file.write(info_string.as_bytes())?;
}
Ok(())

View File

@@ -13,7 +13,7 @@ use structopt_toml::StructOptToml;
use darkfi::{
consensus::{
participant::Participant,
state::{State, StatePtr},
state::{ValidatorState, ValidatorStatePtr},
tx::Tx,
},
net,
@@ -95,7 +95,7 @@ struct Opt {
verbose: u8,
}
async fn proposal_task(p2p: net::P2pPtr, state: StatePtr, database: &sled::Db) {
async fn proposal_task(p2p: net::P2pPtr, state: ValidatorStatePtr) {
// Node signals the network that it starts participating
let participant =
Participant::new(state.read().unwrap().id, state.read().unwrap().current_epoch());
@@ -136,7 +136,11 @@ async fn proposal_task(p2p: net::P2pPtr, state: StatePtr, database: &sled::Db) {
debug!("Node did not vote for the proposed block.");
} else {
let vote = x.unwrap();
state.write().unwrap().receive_vote(&vote);
let result = state.write().unwrap().receive_vote(&vote);
match result {
Ok(_) => info!("Vote saved successfuly."),
Err(e) => error!("Vote save failed. Error: {:?}", e),
}
// Broadcasting block
let result = p2p.broadcast(unwrapped).await;
match result {
@@ -161,7 +165,7 @@ async fn proposal_task(p2p: net::P2pPtr, state: StatePtr, database: &sled::Db) {
}
// Current node state is flushed to sled database
let result = state.read().unwrap().save(database);
let result = state.read().unwrap().save_consensus_state();
match result {
Ok(()) => (),
Err(e) => {
@@ -196,9 +200,8 @@ async fn start(executor: Arc<Executor<'_>>, opts: &Opt) -> Result<()> {
// State setup
let genesis = opts.genesis;
let database_path = expand_path(&opts.database).unwrap();
let database = sled::open(database_path).unwrap();
let id = opts.id.clone();
let state = State::load_current_state(genesis, id, &database).unwrap();
let state = ValidatorState::new(database_path, id, genesis).unwrap();
// P2P registry setup
let p2p = net::P2p::new(network_settings).await;
@@ -265,13 +268,13 @@ async fn start(executor: Arc<Executor<'_>>, opts: &Opt) -> Result<()> {
.spawn(async move { listen_and_serve(rpc_server_config, rpc_interface, ex3).await })
.detach();
proposal_task(p2p, state, &database).await;
proposal_task(p2p, state).await;
Ok(())
}
struct JsonRpcInterface {
state: StatePtr,
state: ValidatorStatePtr,
p2p: net::P2pPtr,
_rpc_listen_addr: SocketAddr,
}

View File

@@ -2,7 +2,7 @@ use async_executor::Executor;
use async_trait::async_trait;
use darkfi::{
consensus::{participant::Participant, state::StatePtr},
consensus::{participant::Participant, state::ValidatorStatePtr},
net::{
ChannelPtr, MessageSubscription, P2pPtr, ProtocolBase, ProtocolBasePtr,
ProtocolJobsManager, ProtocolJobsManagerPtr,
@@ -15,12 +15,16 @@ use std::sync::Arc;
pub struct ProtocolParticipant {
participant_sub: MessageSubscription<Participant>,
jobsman: ProtocolJobsManagerPtr,
state: StatePtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
}
impl ProtocolParticipant {
pub async fn init(channel: ChannelPtr, state: StatePtr, p2p: P2pPtr) -> ProtocolBasePtr {
pub async fn init(
channel: ChannelPtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
) -> ProtocolBasePtr {
let message_subsytem = channel.get_message_subsystem();
message_subsytem.add_dispatch::<Participant>().await;
@@ -46,7 +50,8 @@ impl ProtocolParticipant {
participant
);
if self.state.write().unwrap().append_participant((*participant).clone()) {
let pending_participants = self.state.read().unwrap().pending_participants.clone();
let pending_participants =
self.state.read().unwrap().consensus.pending_participants.clone();
for pending_participant in pending_participants {
self.p2p.broadcast(pending_participant.clone()).await?;
}

View File

@@ -2,7 +2,7 @@ use async_executor::Executor;
use async_trait::async_trait;
use darkfi::{
consensus::{block::BlockProposal, state::StatePtr},
consensus::{block::BlockProposal, state::ValidatorStatePtr},
net::{
ChannelPtr, MessageSubscription, P2pPtr, ProtocolBase, ProtocolBasePtr,
ProtocolJobsManager, ProtocolJobsManagerPtr,
@@ -15,12 +15,16 @@ use std::sync::Arc;
pub struct ProtocolProposal {
proposal_sub: MessageSubscription<BlockProposal>,
jobsman: ProtocolJobsManagerPtr,
state: StatePtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
}
impl ProtocolProposal {
pub async fn init(channel: ChannelPtr, state: StatePtr, p2p: P2pPtr) -> ProtocolBasePtr {
pub async fn init(
channel: ChannelPtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
) -> ProtocolBasePtr {
let message_subsytem = channel.get_message_subsystem();
message_subsytem.add_dispatch::<BlockProposal>().await;
@@ -53,7 +57,7 @@ impl ProtocolProposal {
debug!("Node did not vote for the proposed block.");
} else {
let vote = x.unwrap();
self.state.write().unwrap().receive_vote(&vote);
self.state.write().unwrap().receive_vote(&vote)?;
// Broadcasting block to rest nodes
self.p2p.broadcast(proposal_copy).await?;
// Broadcasting vote

View File

@@ -2,7 +2,7 @@ use async_executor::Executor;
use async_trait::async_trait;
use darkfi::{
consensus::{state::StatePtr, tx::Tx},
consensus::{state::ValidatorStatePtr, tx::Tx},
net::{
ChannelPtr, MessageSubscription, P2pPtr, ProtocolBase, ProtocolBasePtr,
ProtocolJobsManager, ProtocolJobsManagerPtr,
@@ -15,12 +15,16 @@ use std::sync::Arc;
pub struct ProtocolTx {
tx_sub: MessageSubscription<Tx>,
jobsman: ProtocolJobsManagerPtr,
state: StatePtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
}
impl ProtocolTx {
pub async fn init(channel: ChannelPtr, state: StatePtr, p2p: P2pPtr) -> ProtocolBasePtr {
pub async fn init(
channel: ChannelPtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
) -> ProtocolBasePtr {
let message_subsytem = channel.get_message_subsystem();
message_subsytem.add_dispatch::<Tx>().await;

View File

@@ -2,7 +2,7 @@ use async_executor::Executor;
use async_trait::async_trait;
use darkfi::{
consensus::{state::StatePtr, vote::Vote},
consensus::{state::ValidatorStatePtr, vote::Vote},
net::{
ChannelPtr, MessageSubscription, P2pPtr, ProtocolBase, ProtocolBasePtr,
ProtocolJobsManager, ProtocolJobsManagerPtr,
@@ -15,12 +15,16 @@ use std::sync::Arc;
pub struct ProtocolVote {
vote_sub: MessageSubscription<Vote>,
jobsman: ProtocolJobsManagerPtr,
state: StatePtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
}
impl ProtocolVote {
pub async fn init(channel: ChannelPtr, state: StatePtr, p2p: P2pPtr) -> ProtocolBasePtr {
pub async fn init(
channel: ChannelPtr,
state: ValidatorStatePtr,
p2p: P2pPtr,
) -> ProtocolBasePtr {
let message_subsytem = channel.get_message_subsystem();
message_subsytem.add_dispatch::<Vote>().await;
@@ -45,7 +49,7 @@ impl ProtocolVote {
vote
);
let vote_copy = (*vote).clone();
if self.state.write().unwrap().receive_vote(&vote_copy) {
if self.state.write().unwrap().receive_vote(&vote_copy)? {
self.p2p.broadcast(vote_copy).await?;
};
}

View File

@@ -69,7 +69,7 @@ impl BlockStore {
Ok(self.0.contains_key(blockhash.as_bytes())?)
}
/// Fetch the first (oldest) block in the tree.
/// Fetch the first block in the tree, based on the Ord implementation for Vec<u8>.
pub fn get_first(&self) -> Result<Option<(blake3::Hash, Block)>> {
if let Some(found) = self.0.first()? {
let hash_bytes: [u8; 32] = found.0.as_ref().try_into().unwrap();
@@ -80,7 +80,7 @@ impl BlockStore {
Ok(None)
}
/// Fetch the last (newest) block in the tree.
/// Fetch the last block in the tree, based on the Ord implementation for Vec<u8>.
pub fn get_last(&self) -> Result<Option<(blake3::Hash, Block)>> {
if let Some(found) = self.0.last()? {
let hash_bytes: [u8; 32] = found.0.as_ref().try_into().unwrap();

View File

@@ -3,53 +3,83 @@ use std::io;
use crate::{
crypto::{keypair::PublicKey, schnorr::Signature},
impl_vec, net,
util::serial::{Decodable, Encodable, SerialDecodable, SerialEncodable, VarInt},
util::serial::{
deserialize, serialize, Decodable, Encodable, SerialDecodable, SerialEncodable, VarInt,
},
Result,
};
use super::{metadata::Metadata, participant::Participant, tx::Tx, util::Timestamp};
use super::{
metadata::Metadata,
participant::Participant,
tx::Tx,
util::{Timestamp, GENESIS_HASH_BYTES},
};
/// This struct represents a tuple of the form (st, sl, txs, metadata).
/// Each blocks parent hash st may be computed simply as a hash of the parent block.
const SLED_BLOCK_TREE: &[u8] = b"_blocks";
/// This struct represents a tuple of the form (st, sl, txs).
#[derive(Debug, Clone, SerialEncodable, SerialDecodable)]
pub struct Block {
/// Previous block hash
pub st: blake3::Hash,
/// Slot uid, generated by the beacon
pub sl: u64,
/// Transactions payload
pub txs: Vec<Tx>,
/// Additional block information
pub metadata: Metadata,
/// Transaction hashes
/// The actual transactions are in [`TxStore`]
pub txs: Vec<blake3::Hash>,
}
impl Block {
pub fn new(
st: blake3::Hash,
sl: u64,
txs: Vec<Tx>,
timestamp: Timestamp,
proof: String,
r: String,
s: String,
participants: Vec<Participant>,
) -> Block {
Block { st, sl, txs, metadata: Metadata::new(timestamp, proof, r, s, participants) }
}
pub fn from_proposal(proposal: BlockProposal) -> Block {
Block { st: proposal.st, sl: proposal.sl, txs: proposal.txs, metadata: proposal.metadata }
pub fn new(st: blake3::Hash, sl: u64, txs: Vec<blake3::Hash>) -> Block {
Block { st, sl, txs }
}
}
impl PartialEq for Block {
fn eq(&self, other: &Self) -> bool {
self.st == other.st && self.sl == other.sl && self.txs == other.txs
#[derive(Debug)]
pub struct BlockStore(sled::Tree);
impl BlockStore {
/// Opens a new or existing blockstore tree given a sled database.
pub fn new(db: &sled::Db) -> Result<Self> {
let tree = db.open_tree(SLED_BLOCK_TREE)?;
let store = Self(tree);
if store.0.is_empty() {
// Genesis block is generated.
let hash = blake3::Hash::from(GENESIS_HASH_BYTES);
let genesis_block = Block::new(hash, 0, vec![]);
store.insert(&genesis_block)?;
}
Ok(store)
}
/// Insert a block into the blockstore.
/// The block is hashed with blake3 and this blockhash is used as
/// the key, where value is the serialized block itself.
pub fn insert(&self, block: &Block) -> Result<blake3::Hash> {
let serialized = serialize(block);
let blockhash = blake3::hash(&serialized);
self.0.insert(blockhash.as_bytes(), serialized)?;
Ok(blockhash)
}
/// Retrieve all blocks.
/// Be carefull as this will try to load everything in memory.
pub fn get_all(&self) -> Result<Vec<Option<(blake3::Hash, Block)>>> {
let mut blocks = Vec::new();
let mut iterator = self.0.into_iter().enumerate();
while let Some((_, r)) = iterator.next() {
let (k, v) = r.unwrap();
let hash_bytes: [u8; 32] = k.as_ref().try_into().unwrap();
let block = deserialize(&v)?;
blocks.push(Some((hash_bytes.into(), block)));
}
Ok(blocks)
}
}
impl_vec!(Block);
/// This struct represents a Block proposal, used for consensus.
#[derive(Debug, Clone, SerialEncodable, SerialDecodable)]
pub struct BlockProposal {

View File

@@ -3,74 +3,54 @@ use std::io;
use log::debug;
use crate::{
encode_payload, impl_vec,
util::serial::{serialize, Decodable, Encodable, SerialDecodable, SerialEncodable, VarInt},
impl_vec,
util::serial::{Decodable, Encodable, SerialDecodable, SerialEncodable, VarInt},
Result,
};
use super::{
block::{Block, BlockProposal},
util::GENESIS_HASH_BYTES,
block::{Block, BlockProposal, BlockStore},
metadata::MetadataStore,
tx::TxStore,
util::{to_block_serial, GENESIS_HASH_BYTES},
};
/// This struct represents a sequence of blocks starting with the genesis block.
#[derive(Debug, Clone, PartialEq, SerialEncodable, SerialDecodable)]
#[derive(Debug)]
pub struct Blockchain {
pub blocks: Vec<Block>,
pub blocks: BlockStore,
pub transactions: TxStore,
pub metadata: MetadataStore,
}
impl Blockchain {
pub fn new(initial_block: Block) -> Blockchain {
Blockchain { blocks: vec![initial_block] }
pub fn new(db: &sled::Db) -> Result<Blockchain> {
let blocks = BlockStore::new(db)?;
let transactions = TxStore::new(db)?;
let metadata = MetadataStore::new(db)?;
Ok(Blockchain { blocks, transactions, metadata })
}
/// A block is considered valid when its parent hash is equal to the hash of the
/// previous block and their epochs are incremental, exluding genesis.
/// Additional validity rules can be applied.
pub fn check_block(&self, block: &Block, previous: &Block) -> Result<bool> {
if block.st.as_bytes() == &GENESIS_HASH_BYTES {
debug!("Genesis block provided.");
return Ok(false)
/// Insertion of a block proposal.
pub fn add(&mut self, proposal: BlockProposal) -> Result<blake3::Hash> {
// Storing transactions
let mut txs = Vec::new();
for tx in proposal.txs {
let hash = self.transactions.insert(&tx)?;
txs.push(hash);
}
let mut buf = vec![];
encode_payload!(&mut buf, previous.st, previous.sl, previous.txs);
let previous_hash = blake3::hash(&serialize(&buf));
if block.st != previous_hash || block.sl <= previous.sl {
debug!("Provided block is invalid.");
return Ok(false)
}
Ok(true)
}
/// A blockchain is considered valid, when every block is valid, based on check_block function.
pub fn check_chain(&self) -> bool {
for (index, block) in self.blocks[1..].iter().enumerate() {
if !self.check_block(block, &self.blocks[index]).unwrap() {
return false
}
}
true
}
// Storing block
let block = Block { st: proposal.st, sl: proposal.sl, txs };
let hash = self.blocks.insert(&block)?;
/// Insertion of a valid block.
pub fn add(&mut self, block: &Block) {
self.check_block(block, self.blocks.last().unwrap()).unwrap();
self.blocks.push(block.clone());
}
// Storing metadata
self.metadata.insert(&proposal.metadata, hash)?;
/// Blockchain notarization check.
pub fn notarized(&self) -> bool {
for block in &self.blocks {
if !block.metadata.sm.notarized {
return false
}
}
true
Ok(hash)
}
}
impl_vec!(Blockchain);
/// This struct represents a sequence of block proposals.
#[derive(Debug, Clone, PartialEq, SerialEncodable, SerialDecodable)]
pub struct ProposalsChain {
@@ -94,9 +74,7 @@ impl ProposalsChain {
debug!("Genesis block proposal provided.");
return Ok(false)
}
let mut buf = vec![];
encode_payload!(&mut buf, previous.st, previous.sl, previous.txs);
let previous_hash = blake3::hash(&serialize(&buf));
let previous_hash = blake3::hash(&to_block_serial(previous.st, previous.sl, &previous.txs));
if proposal.st != previous_hash || proposal.sl <= previous.sl {
debug!("Provided proposal is invalid.");
return Ok(false)

View File

@@ -1,7 +1,12 @@
use crate::util::serial::{SerialDecodable, SerialEncodable};
use crate::{
util::serial::{deserialize, serialize, SerialDecodable, SerialEncodable},
Result,
};
use super::{participant::Participant, util::Timestamp, vote::Vote};
const SLED_METADATA_TREE: &[u8] = b"_metadata";
/// This struct represents additional Block information used by the consensus protocol.
#[derive(Debug, Clone, SerialEncodable, SerialDecodable)]
pub struct Metadata {
@@ -64,3 +69,34 @@ impl StreamletMetadata {
StreamletMetadata { votes: Vec::new(), notarized: false, finalized: false, participants }
}
}
#[derive(Debug)]
pub struct MetadataStore(sled::Tree);
impl MetadataStore {
pub fn new(db: &sled::Db) -> Result<Self> {
let tree = db.open_tree(SLED_METADATA_TREE)?;
Ok(Self(tree))
}
/// Insert metadata into the metadatastore.
/// The block hash for the madatad is used as the key, where value is the serialized metadata.
pub fn insert(&self, metadata: &Metadata, block: blake3::Hash) -> Result<()> {
self.0.insert(block.as_bytes(), serialize(metadata))?;
Ok(())
}
/// Retrieve all metadata.
/// Be carefull as this will try to load everything in memory.
pub fn get_all(&self) -> Result<Vec<Option<(blake3::Hash, Metadata)>>> {
let mut metadata = Vec::new();
let mut iterator = self.0.into_iter().enumerate();
while let Some((_, r)) = iterator.next() {
let (k, v) = r.unwrap();
let hash_bytes: [u8; 32] = k.as_ref().try_into().unwrap();
let m = deserialize(&v)?;
metadata.push(Some((hash_bytes.into(), m)));
}
Ok(metadata)
}
}

View File

@@ -11,6 +11,6 @@ pub use block::{Block, BlockProposal};
pub use blockchain::Blockchain;
pub use metadata::Metadata;
pub use participant::Participant;
pub use state::State;
pub use state::ValidatorState;
pub use tx::Tx;
pub use vote::Vote;

View File

@@ -4,6 +4,7 @@ use rand::rngs::OsRng;
use std::{
collections::{hash_map::DefaultHasher, BTreeMap},
hash::{Hash, Hasher},
path::PathBuf,
sync::{Arc, RwLock},
time::Duration,
};
@@ -13,61 +14,106 @@ use crate::{
keypair::{PublicKey, SecretKey},
schnorr::{SchnorrPublic, SchnorrSecret},
},
encode_payload,
util::serial::{deserialize, serialize, Encodable, SerialDecodable, SerialEncodable},
Error, Result,
};
use super::{
block::{Block, BlockProposal},
block::BlockProposal,
blockchain::{Blockchain, ProposalsChain},
participant::Participant,
tx::Tx,
util::{get_current_time, Timestamp, GENESIS_HASH_BYTES},
util::{get_current_time, to_block_serial, Timestamp, GENESIS_HASH_BYTES},
vote::Vote,
};
const DELTA: u64 = 60;
const SLED_STATE_TREE: &[u8] = b"_state";
const SLED_CONSESUS_STATE_TREE: &[u8] = b"_consensus_state";
/// Atomic pointer to state.
pub type StatePtr = Arc<RwLock<State>>;
/// This struct represents the state of a consensus node.
/// Each node is numbered and has a secret-public keys pair, to sign messages.
/// Nodes hold the canonical(finalized) blockchain, a set of fork chains containing proposals
/// and a set of unconfirmed pending transactions.
/// Additionally, each node keeps tracks of all participating nodes.
/// This struct represents the information required by the consensus algorithm.
/// Last finalized block hash and slot are used because SLED order follows the Ord implementation for Vec<u8>.
#[derive(Debug, SerialEncodable, SerialDecodable)]
pub struct State {
pub id: u64,
pub struct ConsensusState {
/// Genesis block creation timestamp
pub genesis: Timestamp,
pub secret: SecretKey,
pub public: PublicKey,
pub blockchain: Blockchain,
/// Last finalized block hash,
pub last_block: blake3::Hash,
/// Last finalized block slot,
pub last_sl: u64,
/// Fork chains containing block proposals
pub proposals: Vec<ProposalsChain>,
pub unconfirmed_txs: Vec<Tx>,
/// Orphan votes pool, in case a vote reaches a node before the corresponding block
pub orphan_votes: Vec<Vote>,
/// Validators currently participating in the concensus
pub participants: BTreeMap<u64, Participant>,
/// Validators to be added on next epoch as participants
pub pending_participants: Vec<Participant>,
}
impl State {
pub fn new(id: u64, genesis: Timestamp, init_block: Block) -> State {
impl ConsensusState {
pub fn new(db: &sled::Db, id: u64, genesis: i64) -> Result<ConsensusState> {
let tree = db.open_tree(SLED_CONSESUS_STATE_TREE)?;
let consensus = if let Some(found) = tree.get(id.to_ne_bytes())? {
deserialize(&found).unwrap()
} else {
let hash = blake3::Hash::from(GENESIS_HASH_BYTES);
let genesis_hash = blake3::hash(&to_block_serial(hash, 0, &vec![]));
let consensus = ConsensusState {
genesis: Timestamp(genesis),
last_block: genesis_hash,
last_sl: 0,
proposals: Vec::new(),
orphan_votes: Vec::new(),
participants: BTreeMap::new(),
pending_participants: Vec::new(),
};
let serialized = serialize(&consensus);
tree.insert(id.to_ne_bytes(), serialized)?;
consensus
};
Ok(consensus)
}
}
/// Atomic pointer to validator state.
pub type ValidatorStatePtr = Arc<RwLock<ValidatorState>>;
/// This struct represents the state of a validator node.
pub struct ValidatorState {
/// Validator id
pub id: u64,
/// Secret key, to sign messages
pub secret: SecretKey,
/// Validator public key
pub public: PublicKey,
/// Sled database for storage
pub db: sled::Db,
/// Hot/live data used by the consensus algorithm
pub consensus: ConsensusState,
/// Canonical (finalized) blockchain
pub blockchain: Blockchain,
/// Pending transactions
pub unconfirmed_txs: Vec<Tx>,
}
impl ValidatorState {
pub fn new(db_path: PathBuf, id: u64, genesis: i64) -> Result<ValidatorStatePtr> {
// TODO: clock sync
let secret = SecretKey::random(&mut OsRng);
State {
let db = sled::open(db_path)?;
let public = PublicKey::from_secret(secret);
let consensus = ConsensusState::new(&db, id, genesis)?;
let blockchain = Blockchain::new(&db)?;
let unconfirmed_txs = Vec::new();
Ok(Arc::new(RwLock::new(ValidatorState {
id,
genesis,
secret,
public: PublicKey::from_secret(secret),
blockchain: Blockchain::new(init_block),
proposals: Vec::new(),
unconfirmed_txs: Vec::new(),
orphan_votes: Vec::new(),
participants: BTreeMap::new(),
pending_participants: Vec::new(),
}
public,
db,
consensus,
blockchain,
unconfirmed_txs,
})))
}
/// Node retreives a transaction and append it to the unconfirmed transactions list.
@@ -83,7 +129,7 @@ impl State {
/// Node calculates seconds until next epoch starting time.
/// Epochs duration is configured using the delta value.
pub fn next_epoch_start(&self) -> Duration {
let start_time = NaiveDateTime::from_timestamp(self.genesis.0, 0);
let start_time = NaiveDateTime::from_timestamp(self.consensus.genesis.0, 0);
let current_epoch = self.current_epoch() + 1;
let next_epoch_start_timestamp =
(current_epoch * (2 * DELTA)) + (start_time.timestamp() as u64);
@@ -97,7 +143,7 @@ impl State {
/// Node calculates current epoch, based on elapsed time from the genesis block.
/// Epochs duration is configured using the delta value.
pub fn current_epoch(&self) -> u64 {
self.genesis.clone().elapsed() / (2 * DELTA)
self.consensus.genesis.clone().elapsed() / (2 * DELTA)
}
/// Node finds epochs leader, using a simple hash method.
@@ -107,8 +153,8 @@ impl State {
let mut hasher = DefaultHasher::new();
epoch.hash(&mut hasher);
self.zero_participants_check();
let pos = hasher.finish() % (self.participants.len() as u64);
self.participants.iter().nth(pos as usize).unwrap().1.id
let pos = hasher.finish() % (self.consensus.participants.len() as u64);
self.consensus.participants.iter().nth(pos as usize).unwrap().1.id
}
/// Node checks if they are the current epoch leader.
@@ -124,9 +170,8 @@ impl State {
let epoch = self.current_epoch();
let previous_hash = self.longest_notarized_chain_last_hash().unwrap();
let unproposed_txs = self.unproposed_txs();
let mut encoded_block = vec![];
encode_payload!(&mut encoded_block, previous_hash, epoch, unproposed_txs);
let signed_block = self.secret.sign(&encoded_block[..]);
let signed_block =
self.secret.sign(&to_block_serial(previous_hash, epoch, &unproposed_txs)[..]);
Ok(Some(BlockProposal::new(
self.public,
signed_block,
@@ -138,14 +183,14 @@ impl State {
String::from("proof"),
String::from("r"),
String::from("s"),
self.participants.values().cloned().collect(),
self.consensus.participants.values().cloned().collect(),
)))
}
/// Node retrieves all unconfiremd transactions not proposed in previous blocks.
pub fn unproposed_txs(&self) -> Vec<Tx> {
let mut unproposed_txs = self.unconfirmed_txs.clone();
for chain in &self.proposals {
for chain in &self.consensus.proposals {
for proposal in &chain.proposals {
for tx in &proposal.txs {
if let Some(pos) = unproposed_txs.iter().position(|txs| *txs == *tx) {
@@ -159,12 +204,11 @@ impl State {
/// Finds the longest fully notarized blockchain the node holds and returns the last block hash.
pub fn longest_notarized_chain_last_hash(&self) -> Result<blake3::Hash> {
let mut buf = vec![];
if !self.proposals.is_empty() {
let mut longest_notarized_chain = &self.proposals[0];
let hash = if !self.consensus.proposals.is_empty() {
let mut longest_notarized_chain = &self.consensus.proposals[0];
let mut length = longest_notarized_chain.proposals.len();
if self.proposals.len() > 1 {
for chain in &self.proposals[1..] {
if self.consensus.proposals.len() > 1 {
for chain in &self.consensus.proposals[1..] {
if chain.notarized() && chain.proposals.len() > length {
length = chain.proposals.len();
longest_notarized_chain = chain;
@@ -172,12 +216,11 @@ impl State {
}
}
let last = longest_notarized_chain.proposals.last().unwrap();
encode_payload!(&mut buf, last.st, last.sl, last.txs);
blake3::hash(&to_block_serial(last.st, last.sl, &last.txs))
} else {
let last = self.blockchain.blocks.last().unwrap();
encode_payload!(&mut buf, last.st, last.sl, last.txs);
self.consensus.last_block
};
Ok(blake3::hash(&serialize(&buf)))
Ok(hash)
}
/// Node receives the proposed block, verifies its sender(epoch leader),
@@ -191,9 +234,10 @@ impl State {
);
return Ok(None)
}
let mut encoded_block = vec![];
encode_payload!(&mut encoded_block, proposal.st, proposal.sl, proposal.txs);
if !proposal.public_key.verify(&encoded_block[..], &proposal.signature) {
if !proposal.public_key.verify(
&to_block_serial(proposal.st, proposal.sl, &proposal.txs)[..],
&proposal.signature,
) {
debug!("Proposer signature couldn't be verified. Proposer: {:?}", proposal.id);
return Ok(None)
}
@@ -208,20 +252,18 @@ impl State {
let mut proposal = proposal.clone();
// Generate proposal hash
let mut buf = vec![];
encode_payload!(&mut buf, proposal.st, proposal.sl, proposal.txs);
let proposal_hash = blake3::hash(&serialize(&buf));
let proposal_hash = blake3::hash(&to_block_serial(proposal.st, proposal.sl, &proposal.txs));
// Add orphan votes
let mut orphans = Vec::new();
for vote in self.orphan_votes.iter() {
for vote in self.consensus.orphan_votes.iter() {
if vote.proposal == proposal_hash {
proposal.metadata.sm.votes.push(vote.clone());
orphans.push(vote.clone());
}
}
for vote in orphans {
self.orphan_votes.retain(|v| *v != vote);
self.consensus.orphan_votes.retain(|v| *v != vote);
}
let index = self.find_extended_chain_index(&proposal).unwrap();
@@ -232,19 +274,17 @@ impl State {
let chain = match index {
-1 => {
let proposalschain = ProposalsChain::new(proposal.clone());
self.proposals.push(proposalschain);
self.proposals.last().unwrap()
self.consensus.proposals.push(proposalschain);
self.consensus.proposals.last().unwrap()
}
_ => {
self.proposals[index as usize].add(&proposal);
&self.proposals[index as usize]
self.consensus.proposals[index as usize].add(&proposal);
&self.consensus.proposals[index as usize]
}
};
if self.extends_notarized_chain(chain) {
let mut encoded_hash = vec![];
encode_payload!(&mut encoded_hash, proposal_hash);
let signed_hash = self.secret.sign(&encoded_hash[..]);
let signed_hash = self.secret.sign(&serialize(&proposal_hash)[..]);
return Ok(Some(Vote::new(
self.public,
signed_hash,
@@ -268,11 +308,9 @@ impl State {
/// Given a proposal, node finds the index of the chain it extends.
pub fn find_extended_chain_index(&self, proposal: &BlockProposal) -> Result<i64> {
for (index, chain) in self.proposals.iter().enumerate() {
for (index, chain) in self.consensus.proposals.iter().enumerate() {
let last = chain.proposals.last().unwrap();
let mut buf = vec![];
encode_payload!(&mut buf, last.st, last.sl, last.txs);
let hash = blake3::hash(&serialize(&buf));
let hash = blake3::hash(&to_block_serial(last.st, last.sl, &last.txs));
if proposal.st == hash && proposal.sl > last.sl {
return Ok(index as i64)
}
@@ -282,14 +320,11 @@ impl State {
}
}
let last = self.blockchain.blocks.last().unwrap();
let mut buf = vec![];
encode_payload!(&mut buf, last.st, last.sl, last.txs);
let hash = blake3::hash(&serialize(&buf));
if proposal.st != hash || proposal.sl <= last.sl {
if proposal.st != self.consensus.last_block || proposal.sl <= self.consensus.last_sl {
debug!("Proposal doesn't extend any known chains.");
return Ok(-2)
}
Ok(-1)
}
@@ -302,32 +337,32 @@ impl State {
/// nodes unconfirmed transactions list.
/// Finally, we check if the notarization of the proposal can finalize parent proposals
/// in its chain.
pub fn receive_vote(&mut self, vote: &Vote) -> bool {
pub fn receive_vote(&mut self, vote: &Vote) -> Result<bool> {
let mut encoded_proposal = vec![];
let result = vote.proposal.encode(&mut encoded_proposal);
match result {
Ok(_) => (),
Err(e) => {
error!("Proposal encoding failed. Error: {:?}", e);
return false
return Ok(false)
}
};
if !vote.public_key.verify(&encoded_proposal[..], &vote.vote) {
debug!("Voter signature couldn't be verified. Voter: {:?}", vote.id);
return false
return Ok(false)
}
let nodes_count = self.participants.len();
let nodes_count = self.consensus.participants.len();
self.zero_participants_check();
let proposal = self.find_proposal(&vote.proposal).unwrap();
if proposal == None {
debug!("Received vote for unknown proposal.");
if !self.orphan_votes.contains(vote) {
self.orphan_votes.push(vote.clone());
if !self.consensus.orphan_votes.contains(vote) {
self.consensus.orphan_votes.push(vote.clone());
}
return false
return Ok(false)
}
let (unwrapped, chain_index) = proposal.unwrap();
@@ -338,21 +373,21 @@ impl State {
unwrapped.metadata.sm.votes.len() > (2 * nodes_count / 3)
{
unwrapped.metadata.sm.notarized = true;
self.chain_finalization(chain_index);
self.chain_finalization(chain_index)?;
}
// updating participant vote
let exists = self.participants.get(&vote.id);
let exists = self.consensus.participants.get(&vote.id);
let mut participant = match exists {
Some(p) => p.clone(),
None => Participant::new(vote.id, vote.sl),
};
participant.voted = Some(vote.sl);
self.participants.insert(participant.id, participant);
self.consensus.participants.insert(participant.id, participant);
return true
return Ok(true)
}
false
Ok(false)
}
/// Node searches it the chains it holds for provided proposal.
@@ -360,11 +395,10 @@ impl State {
&mut self,
vote_proposal: &blake3::Hash,
) -> Result<Option<(&mut BlockProposal, i64)>> {
for (index, chain) in &mut self.proposals.iter_mut().enumerate() {
for (index, chain) in &mut self.consensus.proposals.iter_mut().enumerate() {
for proposal in chain.proposals.iter_mut().rev() {
let mut buf = vec![];
encode_payload!(&mut buf, proposal.st, proposal.sl, proposal.txs);
let proposal_hash = blake3::hash(&serialize(&buf));
let proposal_hash =
blake3::hash(&to_block_serial(proposal.st, proposal.sl, &proposal.txs));
if vote_proposal == &proposal_hash {
return Ok(Some((proposal, index as i64)))
}
@@ -377,8 +411,8 @@ impl State {
/// Consensus finalization logic: If node has observed the notarization of 3 consecutive
/// proposals in a fork chain, it finalizes (appends to canonical blockchain) all proposals up to the middle block.
/// When fork chain proposals are finalized, rest fork chains not starting by those proposals are removed.
pub fn chain_finalization(&mut self, chain_index: i64) {
let chain = &mut self.proposals[chain_index as usize];
pub fn chain_finalization(&mut self, chain_index: i64) -> Result<()> {
let chain = &mut self.consensus.proposals[chain_index as usize];
let len = chain.proposals.len();
if len > 2 {
let mut consecutive = 0;
@@ -403,53 +437,55 @@ impl State {
}
chain.proposals.drain(0..(consecutive - 1));
for proposal in &finalized {
self.blockchain.blocks.push(Block::from_proposal(proposal.clone()));
let hash = self.blockchain.add(proposal.clone())?;
self.consensus.last_block = hash;
self.consensus.last_sl = proposal.sl;
}
let last = self.blockchain.blocks.last().unwrap();
let hash = blake3::hash(&serialize(last));
let mut dropped = Vec::new();
for chain in self.proposals.iter() {
for chain in self.consensus.proposals.iter() {
let first = chain.proposals.first().unwrap();
if first.st != hash || first.sl <= last.sl {
if first.st != self.consensus.last_block || first.sl <= self.consensus.last_sl {
dropped.push(chain.clone());
}
}
for chain in dropped {
self.proposals.retain(|c| *c != chain);
self.consensus.proposals.retain(|c| *c != chain);
}
// Remove orphan votes
let mut orphans = Vec::new();
for vote in self.orphan_votes.iter() {
if vote.sl <= last.sl {
for vote in self.consensus.orphan_votes.iter() {
if vote.sl <= self.consensus.last_sl {
orphans.push(vote.clone());
}
}
for vote in orphans {
self.orphan_votes.retain(|v| *v != vote);
self.consensus.orphan_votes.retain(|v| *v != vote);
}
}
}
Ok(())
}
/// Node retreives a new participant and appends it to the pending participants list.
pub fn append_participant(&mut self, participant: Participant) -> bool {
if self.pending_participants.contains(&participant) {
if self.consensus.pending_participants.contains(&participant) {
return false
}
self.pending_participants.push(participant);
self.consensus.pending_participants.push(participant);
true
}
/// This prevent the extreme case scenario where network is initialized, but some nodes
/// have not pushed the initial participants in the map.
pub fn zero_participants_check(&mut self) {
if self.participants.len() == 0 {
for participant in &self.pending_participants {
self.participants.insert(participant.id, participant.clone());
if self.consensus.participants.len() == 0 {
for participant in &self.consensus.pending_participants {
self.consensus.participants.insert(participant.id, participant.clone());
}
self.pending_participants = Vec::new();
self.consensus.pending_participants = Vec::new();
}
}
@@ -457,14 +493,14 @@ impl State {
/// Active nodes are considered those who joined or voted on previous epoch.
pub fn refresh_participants(&mut self) {
// adding pending participants
for participant in &self.pending_participants {
self.participants.insert(participant.id, participant.clone());
for participant in &self.consensus.pending_participants {
self.consensus.participants.insert(participant.id, participant.clone());
}
self.pending_participants = Vec::new();
self.consensus.pending_participants = Vec::new();
let mut inactive = Vec::new();
let previous_epoch = self.current_epoch() - 1;
for (index, participant) in self.participants.clone().iter() {
for (index, participant) in self.consensus.participants.clone().iter() {
match participant.voted {
Some(epoch) => {
if epoch < previous_epoch {
@@ -479,57 +515,17 @@ impl State {
}
}
for index in inactive {
self.participants.remove(&index);
self.consensus.participants.remove(&index);
}
}
/// Util function to save the current node state to provided file path.
pub fn save(&self, db: &sled::Db) -> Result<()> {
let tree = db.open_tree(SLED_STATE_TREE).unwrap();
let serialized = serialize(self);
/// Util function to save the current consensus state to provided file path.
pub fn save_consensus_state(&self) -> Result<()> {
let tree = self.db.open_tree(SLED_CONSESUS_STATE_TREE).unwrap();
let serialized = serialize(&self.consensus);
match tree.insert(self.id.to_ne_bytes(), serialized) {
Err(_) => Err(Error::OperationFailed),
_ => Ok(()),
}
}
/// Util function to load current node state by the provided file path.
// If file is not found, node state is reset.
pub fn load_or_create(genesis: i64, id: u64, db: &sled::Db) -> Result<State> {
let tree = db.open_tree(SLED_STATE_TREE).unwrap();
if let Some(found) = tree.get(id.to_ne_bytes()).unwrap() {
Ok(deserialize(&found).unwrap())
} else {
Self::reset(genesis, id, db)
}
}
/// Util function to load the current node state by the provided folder path.
pub fn load_current_state(genesis: i64, id: u64, db: &sled::Db) -> Result<StatePtr> {
let state = Self::load_or_create(genesis, id, db)?;
Ok(Arc::new(RwLock::new(state)))
}
/// Util function to reset node state.
pub fn reset(genesis: i64, id: u64, db: &sled::Db) -> Result<State> {
// Genesis block is generated.
let mut genesis_block = Block::new(
blake3::Hash::from(GENESIS_HASH_BYTES),
0,
vec![],
get_current_time(),
String::from("proof"),
String::from("r"),
String::from("s"),
vec![],
);
genesis_block.metadata.sm.notarized = true;
genesis_block.metadata.sm.finalized = true;
let genesis_time = Timestamp(genesis);
let state = Self::new(id, genesis_time, genesis_block);
state.save(db)?;
Ok(state)
}
}

View File

@@ -2,10 +2,14 @@ use std::io;
use crate::{
impl_vec, net,
util::serial::{Decodable, Encodable, SerialDecodable, SerialEncodable, VarInt},
util::serial::{
deserialize, serialize, Decodable, Encodable, SerialDecodable, SerialEncodable, VarInt,
},
Result,
};
const SLED_TX_TREE: &[u8] = b"_transactions";
/// Temporary structure used to represent transactions.
#[derive(Debug, Clone, PartialEq, SerialEncodable, SerialDecodable)]
pub struct Tx {
@@ -19,3 +23,38 @@ impl net::Message for Tx {
}
impl_vec!(Tx);
#[derive(Debug)]
pub struct TxStore(sled::Tree);
impl TxStore {
pub fn new(db: &sled::Db) -> Result<Self> {
let tree = db.open_tree(SLED_TX_TREE)?;
Ok(Self(tree))
}
/// Insert a tx into the txstore.
/// The tx is hashed with blake3 and this txhash is used as
/// the key, where value is the serialized tx itself.
pub fn insert(&self, tx: &Tx) -> Result<blake3::Hash> {
let serialized = serialize(tx);
let txhash = blake3::hash(&serialized);
self.0.insert(txhash.as_bytes(), serialized)?;
Ok(txhash)
}
/// Retrieve all transactions.
/// Be carefull as this will try to load everything in memory.
pub fn get_all(&self) -> Result<Vec<Option<(blake3::Hash, Tx)>>> {
let mut txs = Vec::new();
let mut iterator = self.0.into_iter().enumerate();
while let Some((_, r)) = iterator.next() {
let (k, v) = r.unwrap();
let hash_bytes: [u8; 32] = k.as_ref().try_into().unwrap();
let tx = deserialize(&v)?;
txs.push(Some((hash_bytes.into(), tx)));
}
Ok(txs)
}
}

View File

@@ -1,6 +1,8 @@
use chrono::{NaiveDateTime, Utc};
use crate::util::serial::{SerialDecodable, SerialEncodable};
use crate::util::serial::{serialize, SerialDecodable, SerialEncodable};
use super::{block::Block, tx::Tx};
/// Serialized blake3 hash bytes for character "⊥"
pub const GENESIS_HASH_BYTES: [u8; 32] = [
@@ -26,3 +28,13 @@ impl Timestamp {
pub fn get_current_time() -> Timestamp {
Timestamp(Utc::now().timestamp())
}
/// Util function to create a dummy block and encode it, to produce the correct hash
pub fn to_block_serial(st: blake3::Hash, sl: u64, transactions: &Vec<Tx>) -> Vec<u8> {
let mut txs = Vec::new();
for tx in transactions {
let hash = blake3::hash(&serialize(tx));
txs.push(hash);
}
serialize(&Block::new(st, sl, txs))
}