diff --git a/crates/hotshot/src/demo.rs b/crates/hotshot/src/demo.rs index 000deb351b..a558922ec5 100644 --- a/crates/hotshot/src/demo.rs +++ b/crates/hotshot/src/demo.rs @@ -8,11 +8,11 @@ use crate::traits::election::static_committee::{StaticElectionConfig, StaticVoteToken}; use commit::{Commitment, Committable}; use derivative::Derivative; + use hotshot_signature_key::bn254::BLSPubKey; use hotshot_types::{ block_impl::{BlockPayloadError, VIDBlockHeader, VIDBlockPayload, VIDTransaction}, - certificate::{AssembledSignature, QuorumCertificate}, - data::{fake_commitment, random_commitment, LeafType, ViewNumber}, + data::{fake_commitment, ViewNumber}, traits::{ election::Membership, node_implementation::NodeType, @@ -20,7 +20,7 @@ use hotshot_types::{ BlockPayload, State, }, }; -use rand::Rng; + use serde::{Deserialize, Serialize}; use std::{fmt::Debug, marker::PhantomData}; @@ -176,15 +176,3 @@ where Self::new() } } - -/// Provides a random [`QuorumCertificate`] -pub fn random_quorum_certificate>( - rng: &mut dyn rand::RngCore, -) -> QuorumCertificate> { - QuorumCertificate { - leaf_commitment: random_commitment(rng), - view_number: TYPES::Time::new(rng.gen()), - signatures: AssembledSignature::Genesis(), - is_genesis: rng.gen(), - } -} diff --git a/crates/hotshot/src/lib.rs b/crates/hotshot/src/lib.rs index 3d5c770ea9..48cb49a0db 100644 --- a/crates/hotshot/src/lib.rs +++ b/crates/hotshot/src/lib.rs @@ -56,6 +56,7 @@ use hotshot_task_impls::{events::HotShotEvent, network::NetworkTaskKind}; use hotshot_types::{ certificate::{TimeoutCertificate, VIDCertificate}, data::VidDisperse, + simple_certificate::QuorumCertificate2, traits::node_implementation::TimeoutEx, }; @@ -71,7 +72,7 @@ use hotshot_types::{ }, traits::{ consensus_api::{ConsensusApi, ConsensusSharedApi}, - election::{ConsensusExchange, Membership, SignedCertificate}, + election::{ConsensusExchange, Membership}, network::{CommunicationChannel, NetworkError}, node_implementation::{ ChannelMaps, CommitteeEx, ExchangesType, NodeType, QuorumEx, SendToTasks, VIDEx, @@ -249,7 +250,7 @@ impl> SystemContext { self.inner .internal_event_stream .publish(HotShotEvent::QCFormed(either::Left( - QuorumCertificate::genesis(), + QuorumCertificate2::genesis(), ))) .await; } @@ -1020,7 +1021,7 @@ impl> HotShotInitializer Result> { let state = TYPES::StateType::initialize(); let time = TYPES::Time::genesis(); - let justify_qc = QuorumCertificate::>::genesis(); + let justify_qc = QuorumCertificate2::::genesis(); Ok(Self { inner: LEAF::new(time, justify_qc, genesis_payload, state), diff --git a/crates/hotshot/src/traits/election/static_committee.rs b/crates/hotshot/src/traits/election/static_committee.rs index ccdcfb722f..8252ac2562 100644 --- a/crates/hotshot/src/traits/election/static_committee.rs +++ b/crates/hotshot/src/traits/election/static_committee.rs @@ -17,7 +17,7 @@ use tracing::debug; /// Dummy implementation of [`Membership`] -#[derive(Clone, Debug, Eq, PartialEq)] +#[derive(Clone, Debug, Eq, PartialEq, Hash)] pub struct GeneralStaticCommittee, PUBKEY: SignatureKey> { /// All the nodes participating and their stake nodes_with_stake: Vec, diff --git a/crates/hotshot/src/traits/storage/memory_storage.rs b/crates/hotshot/src/traits/storage/memory_storage.rs index 7b7b1ccd01..58d56a6c55 100644 --- a/crates/hotshot/src/traits/storage/memory_storage.rs +++ b/crates/hotshot/src/traits/storage/memory_storage.rs @@ -121,11 +121,11 @@ mod test { use hotshot_signature_key::bn254::BLSPubKey; use hotshot_types::{ block_impl::{VIDBlockHeader, VIDBlockPayload, VIDTransaction}, - certificate::{AssembledSignature, QuorumCertificate}, data::{fake_commitment, genesis_proposer_id, ValidatingLeaf, ViewNumber}, + simple_certificate::QuorumCertificate2, traits::{node_implementation::NodeType, state::dummy::DummyState, state::ConsensusTime}, }; - use std::{fmt::Debug, hash::Hash}; + use std::{fmt::Debug, hash::Hash, marker::PhantomData}; use tracing::instrument; #[derive( @@ -165,12 +165,18 @@ mod test { payload_commitment: payload.commit(), }; let dummy_leaf_commit = fake_commitment::>(); + let data = hotshot_types::simple_vote::QuorumData { + leaf_commit: dummy_leaf_commit, + }; + let commit = data.commit(); StoredView::from_qc_block_and_state( - QuorumCertificate { + QuorumCertificate2 { is_genesis: view_number == ::Time::genesis(), - leaf_commitment: dummy_leaf_commit, - signatures: AssembledSignature::Genesis(), + data, + vote_commitment: commit, + signatures: None, view_number, + _pd: PhantomData, }, header, Some(payload), diff --git a/crates/hotshot/src/types/handle.rs b/crates/hotshot/src/types/handle.rs index 16ca56d549..9500698bbc 100644 --- a/crates/hotshot/src/types/handle.rs +++ b/crates/hotshot/src/types/handle.rs @@ -1,6 +1,7 @@ //! Provides an event-streaming handle for a [`HotShot`] running in the background -use crate::{traits::NodeImplementation, types::Event, Message, QuorumCertificate, SystemContext}; +use crate::QuorumCertificate2; +use crate::{traits::NodeImplementation, types::Event, SystemContext}; use async_compatibility_layer::channel::UnboundedStream; use async_lock::RwLock; use commit::Committable; @@ -13,20 +14,20 @@ use hotshot_task::{ BoxSyncFuture, }; use hotshot_task_impls::events::HotShotEvent; +use hotshot_types::simple_vote::QuorumData; use hotshot_types::{ consensus::Consensus, data::LeafType, error::HotShotError, event::EventType, - message::{GeneralConsensusMessage, MessageKind}, + message::MessageKind, traits::{ - election::{ConsensusExchange, QuorumExchangeType, SignedCertificate}, - node_implementation::{ExchangesType, NodeType, QuorumEx}, + election::{ConsensusExchange, QuorumExchangeType}, + node_implementation::{ExchangesType, NodeType}, state::ConsensusTime, storage::Storage, }, }; - use std::sync::Arc; use tracing::error; @@ -190,8 +191,10 @@ impl + 'static> SystemContextHandl if let Ok(anchor_leaf) = self.storage().get_anchored_view().await { if anchor_leaf.view_number == TYPES::Time::genesis() { let leaf: I::Leaf = I::Leaf::from_stored_view(anchor_leaf); - let mut qc = QuorumCertificate::>::genesis(); - qc.leaf_commitment = leaf.commit(); + let mut qc = QuorumCertificate2::::genesis(); + qc.data = QuorumData { + leaf_commit: leaf.commit(), + }; let event = Event { view_number: TYPES::Time::genesis(), event: EventType::Decide { @@ -329,31 +332,6 @@ impl + 'static> SystemContextHandl .sign_validating_or_commitment_proposal::(leaf_commitment) } - /// create a yes message - #[cfg(feature = "hotshot-testing")] - pub fn create_yes_message( - &self, - justify_qc_commitment: Commitment>>, - leaf_commitment: Commitment, - current_view: TYPES::Time, - vote_token: TYPES::VoteTokenType, - ) -> GeneralConsensusMessage - where - QuorumEx: ConsensusExchange< - TYPES, - Message, - Certificate = QuorumCertificate>, - >, - { - let inner = self.hotshot.inner.clone(); - inner.exchanges.quorum_exchange().create_yes_message( - justify_qc_commitment, - leaf_commitment, - current_view, - vote_token, - ) - } - /// Wrapper around `HotShotConsensusApi`'s `send_broadcast_consensus_message` function #[cfg(feature = "hotshot-testing")] pub async fn send_broadcast_consensus_message(&self, msg: I::ConsensusMessage) { diff --git a/crates/task-impls/src/consensus.rs b/crates/task-impls/src/consensus.rs index bf09bd01f6..f5b755cd8a 100644 --- a/crates/task-impls/src/consensus.rs +++ b/crates/task-impls/src/consensus.rs @@ -22,18 +22,23 @@ use hotshot_types::{ data::{Leaf, LeafType, ProposalType, QuorumProposal}, event::{Event, EventType}, message::{GeneralConsensusMessage, Message, Proposal, SequencingMessage}, + simple_certificate::QuorumCertificate2, + simple_vote::{QuorumData, QuorumVote}, traits::{ block_contents::BlockHeader, consensus_api::ConsensusApi, election::{ConsensusExchange, QuorumExchangeType, SignedCertificate, TimeoutExchangeType}, network::{CommunicationChannel, ConsensusIntentEvent}, - node_implementation::{CommitteeEx, NodeImplementation, NodeType, QuorumEx, TimeoutEx}, + node_implementation::{ + CommitteeEx, NodeImplementation, NodeType, QuorumEx, QuorumMembership, TimeoutEx, + }, signature_key::SignatureKey, state::ConsensusTime, BlockPayload, }, utils::{Terminator, ViewInner}, - vote::{QuorumVote, QuorumVoteAccumulator, TimeoutVoteAccumulator, VoteType}, + vote::{TimeoutVoteAccumulator, VoteType}, + vote2::{Certificate2, HasViewNumber, VoteAccumulator2}, }; use tracing::warn; @@ -169,13 +174,12 @@ pub struct VoteCollectionTaskState< #[allow(clippy::type_complexity)] /// Accumulator for votes pub accumulator: Either< - >> as SignedCertificate< + VoteAccumulator2< TYPES, - TYPES::Time, - TYPES::VoteTokenType, - Commitment>, - >>::VoteAccumulator, - QuorumCertificate>>, + QuorumVote>, + QuorumCertificate2, + >, + QuorumCertificate2, >, /// Accumulator for votes @@ -243,57 +247,49 @@ where >, { match event { - HotShotEvent::QuorumVoteRecv(vote) => match vote.clone() { - QuorumVote::Yes(vote_internal) => { - // For the case where we receive votes after we've made a certificate - if state.accumulator.is_right() { - return (None, state); - } + HotShotEvent::QuorumVoteRecv(vote) => { + // For the case where we receive votes after we've made a certificate + if state.accumulator.is_right() { + return (None, state); + } - if vote_internal.current_view != state.cur_view { - error!( - "Vote view does not match! vote view is {} current view is {}", - *vote_internal.current_view, *state.cur_view - ); + if vote.get_view_number() != state.cur_view { + error!( + "Vote view does not match! vote view is {} current view is {}", + *vote.get_view_number(), + *state.cur_view + ); + return (None, state); + } + + let accumulator = state.accumulator.left().unwrap(); + + match accumulator.accumulate(&vote, state.quorum_exchange.membership()) { + Either::Left(acc) => { + state.accumulator = Either::Left(acc); return (None, state); } + Either::Right(qc) => { + debug!("QCFormed! {:?}", qc.view_number); + state + .event_stream + .publish(HotShotEvent::QCFormed(either::Left(qc.clone()))) + .await; + state.accumulator = Either::Right(qc.clone()); - let accumulator = state.accumulator.left().unwrap(); - - match state.quorum_exchange.accumulate_vote( - accumulator, - &vote, - &vote_internal.leaf_commitment, - ) { - Either::Left(acc) => { - state.accumulator = Either::Left(acc); - return (None, state); - } - Either::Right(qc) => { - debug!("QCFormed! {:?}", qc.view_number); - state - .event_stream - .publish(HotShotEvent::QCFormed(either::Left(qc.clone()))) - .await; - state.accumulator = Either::Right(qc.clone()); - - // No longer need to poll for votes - state - .quorum_exchange - .network() - .inject_consensus_info(ConsensusIntentEvent::CancelPollForVotes( - *qc.view_number, - )) - .await; + // No longer need to poll for votes + state + .quorum_exchange + .network() + .inject_consensus_info(ConsensusIntentEvent::CancelPollForVotes( + *qc.view_number, + )) + .await; - return (Some(HotShotTaskCompleted::ShutDown), state); - } + return (Some(HotShotTaskCompleted::ShutDown), state); } } - QuorumVote::No(_) => { - error!("The next leader has received an unexpected vote!"); - } - }, + } // TODO: Code below is redundant of code above; can be fixed // during exchange refactor // https://github.com/EspressoSystems/HotShot/issues/1799 @@ -414,7 +410,7 @@ where async fn vote_if_able(&self) -> bool { if let Some(proposal) = &self.current_proposal { // ED Need to account for the genesis DA cert - if proposal.justify_qc.is_genesis() && proposal.view_number == TYPES::Time::new(1) { + if proposal.justify_qc.is_genesis && proposal.view_number == TYPES::Time::new(1) { // warn!("Proposal is genesis!"); let view = TYPES::Time::new(*proposal.view_number); @@ -427,16 +423,16 @@ where Ok(None) => { debug!("We were not chosen for consensus committee on {:?}", view); } - Ok(Some(vote_token)) => { + Ok(Some(_vote_token)) => { let justify_qc = proposal.justify_qc.clone(); - let parent = if justify_qc.is_genesis() { + let parent = if justify_qc.is_genesis { self.genesis_leaf().await } else { self.consensus .read() .await .saved_leaves - .get(&justify_qc.leaf_commitment()) + .get(&justify_qc.get_data().leaf_commit) .cloned() }; @@ -444,7 +440,7 @@ where let Some(parent) = parent else { error!( "Proposal's parent missing from storage with commitment: {:?}, proposal view {:?}", - justify_qc.leaf_commitment(), + justify_qc.get_data().leaf_commit, proposal.view_number, ); return false; @@ -461,19 +457,19 @@ where timestamp: time::OffsetDateTime::now_utc().unix_timestamp_nanos(), proposer_id: self.quorum_exchange.get_leader(view).to_bytes(), }; - - let message: GeneralConsensusMessage = - self.quorum_exchange.create_yes_message( - proposal.justify_qc.commit(), - leaf.commit(), + let vote = + QuorumVote::>::create_signed_vote( + QuorumData { leaf_commit: leaf.commit() }, view, - vote_token, + self.quorum_exchange.public_key(), + self.quorum_exchange.private_key(), ); + let message = GeneralConsensusMessage::::Vote(vote); if let GeneralConsensusMessage::Vote(vote) = message { debug!( "Sending vote to next quorum leader {:?}", - vote.get_view() + 1 + vote.get_view_number() + 1 ); self.event_stream .publish(HotShotEvent::QuorumVoteSend(vote)) @@ -497,16 +493,16 @@ where Ok(None) => { debug!("We were not chosen for consensus committee on {:?}", view); } - Ok(Some(vote_token)) => { + Ok(Some(_vote_token)) => { let justify_qc = proposal.justify_qc.clone(); - let parent = if justify_qc.is_genesis() { + let parent = if justify_qc.is_genesis { self.genesis_leaf().await } else { self.consensus .read() .await .saved_leaves - .get(&justify_qc.leaf_commitment()) + .get(&justify_qc.get_data().leaf_commit) .cloned() }; @@ -514,7 +510,7 @@ where let Some(parent) = parent else { error!( "Proposal's parent missing from storage with commitment: {:?}, proposal view {:?}", - justify_qc.leaf_commitment(), + justify_qc.get_data().leaf_commit, proposal.view_number, ); return false; @@ -531,30 +527,35 @@ where timestamp: time::OffsetDateTime::now_utc().unix_timestamp_nanos(), proposer_id: self.quorum_exchange.get_leader(view).to_bytes(), }; - let message: GeneralConsensusMessage= + // Validate the DAC. - if self - .committee_exchange - .is_valid_cert(cert) - { + let message = if self.committee_exchange.is_valid_cert(cert) { // Validate the block payload commitment for non-genesis DAC. - if !cert.is_genesis() && cert.leaf_commitment() != proposal.block_header.payload_commitment() { + if !cert.is_genesis() + && cert.leaf_commitment() + != proposal.block_header.payload_commitment() + { error!("Block payload commitment does not equal parent commitment"); return false; } - self.quorum_exchange.create_yes_message( - proposal.justify_qc.commit(), - leaf.commit(), - cert.view_number, - vote_token) + let vote = + QuorumVote::>::create_signed_vote( + QuorumData { leaf_commit: leaf.commit() }, + view, + self.quorum_exchange.public_key(), + self.quorum_exchange.private_key(), + ); + GeneralConsensusMessage::::Vote(vote) } else { error!("Invalid DAC in proposal! Skipping proposal. {:?} cur view is: {:?}", cert, self.cur_view ); return false; - }; if let GeneralConsensusMessage::Vote(vote) = message { - debug!("Sending vote to next quorum leader {:?}", vote.get_view()); + debug!( + "Sending vote to next quorum leader {:?}", + vote.get_view_number() + ); self.event_stream .publish(HotShotEvent::QuorumVoteSend(vote)) .await; @@ -680,7 +681,7 @@ where } // Verify a timeout certificate exists and is valid - if proposal.data.justify_qc.view_number() != view - 1 { + if proposal.data.justify_qc.get_view_number() != view - 1 { let Some(timeout_cert) = proposal.data.timeout_certificate.clone() else { warn!( "Quorum proposal for view {} needed a timeout certificate but did not have one", @@ -704,7 +705,7 @@ where let justify_qc = proposal.data.justify_qc.clone(); - if !self.quorum_exchange.is_valid_cert(&justify_qc) { + if !justify_qc.is_valid_cert(self.quorum_exchange.membership()) { error!("Invalid justify_qc in proposal for view {}", *view); let consensus = self.consensus.write().await; consensus.metrics.invalid_qc.update(1); @@ -719,12 +720,12 @@ where let consensus = self.consensus.upgradable_read().await; // Construct the leaf. - let parent = if justify_qc.is_genesis() { + let parent = if justify_qc.is_genesis { self.genesis_leaf().await } else { consensus .saved_leaves - .get(&justify_qc.leaf_commitment()) + .get(&justify_qc.get_data().leaf_commit) .cloned() }; @@ -734,12 +735,12 @@ where // If no parent then just update our state map and return. We will not vote. error!( "Proposal's parent missing from storage with commitment: {:?}", - justify_qc.leaf_commitment() + justify_qc.get_data().leaf_commit ); let leaf = Leaf { view_number: view, justify_qc: justify_qc.clone(), - parent_commitment: justify_qc.leaf_commitment(), + parent_commitment: justify_qc.get_data().leaf_commit, block_header: proposal.data.block_header, block_payload: None, rejected: Vec::new(), @@ -782,12 +783,12 @@ where // passes. // Liveness check. - let liveness_check = justify_qc.view_number > consensus.locked_view; + let liveness_check = justify_qc.get_view_number() > consensus.locked_view; // Safety check. // Check if proposal extends from the locked leaf. let outcome = consensus.visit_leaf_ancestors( - justify_qc.view_number, + justify_qc.get_view_number(), Terminator::Inclusive(consensus.locked_view), false, |leaf| { @@ -818,7 +819,7 @@ where let mut leaf_views = Vec::new(); let mut included_txns = HashSet::new(); let old_anchor_view = consensus.last_decided_view; - let parent_view = leaf.justify_qc.view_number; + let parent_view = leaf.justify_qc.get_view_number(); let mut current_chain_length = 0usize; if parent_view + 1 == view { current_chain_length += 1; @@ -981,115 +982,91 @@ where } } HotShotEvent::QuorumVoteRecv(vote) => { - debug!("Received quroum vote: {:?}", vote.get_view()); + debug!("Received quroum vote: {:?}", vote.get_view_number()); - if !self.quorum_exchange.is_leader(vote.get_view() + 1) { + if !self.quorum_exchange.is_leader(vote.get_view_number() + 1) { error!( "We are not the leader for view {} are we the leader for view + 1? {}", - *vote.get_view() + 1, - self.quorum_exchange.is_leader(vote.get_view() + 2) + *vote.get_view_number() + 1, + self.quorum_exchange.is_leader(vote.get_view_number() + 2) ); return; } - match vote.clone() { - QuorumVote::Yes(vote_internal) => { - let handle_event = HandleEvent(Arc::new(move |event, state| { - async move { vote_handle(state, event).await }.boxed() - })); - let collection_view = if let Some((collection_view, collection_task, _)) = - &self.vote_collector - { - if vote_internal.current_view > *collection_view { - // ED I think we'd want to let that task timeout to avoid a griefing vector - self.registry.shutdown_task(*collection_task).await; - } - *collection_view - } else { - TYPES::Time::new(0) - }; + let handle_event = HandleEvent(Arc::new(move |event, state| { + async move { vote_handle(state, event).await }.boxed() + })); + let collection_view = + if let Some((collection_view, collection_task, _)) = &self.vote_collector { + if vote.get_view_number() > *collection_view { + // ED I think we'd want to let that task timeout to avoid a griefing vector + self.registry.shutdown_task(*collection_task).await; + } + *collection_view + } else { + TYPES::Time::new(0) + }; - // Todo check if we are the leader - let new_accumulator = QuorumVoteAccumulator { - total_vote_outcomes: HashMap::new(), - yes_vote_outcomes: HashMap::new(), - no_vote_outcomes: HashMap::new(), + // Todo check if we are the leader + let new_accumulator = VoteAccumulator2 { + vote_outcomes: HashMap::new(), + sig_lists: Vec::new(), + signers: bitvec![0; self.quorum_exchange.total_nodes()], + phantom: PhantomData, + }; - success_threshold: self.quorum_exchange.success_threshold(), - failure_threshold: self.quorum_exchange.failure_threshold(), + let accumulator = + new_accumulator.accumulate(&vote, self.quorum_exchange.membership()); - sig_lists: Vec::new(), - signers: bitvec![0; self.quorum_exchange.total_nodes()], - phantom: PhantomData, - }; + // TODO Create default functions for accumulators + // https://github.com/EspressoSystems/HotShot/issues/1797 + let timeout_accumulator = TimeoutVoteAccumulator { + da_vote_outcomes: HashMap::new(), + success_threshold: self.timeout_exchange.success_threshold(), + sig_lists: Vec::new(), + signers: bitvec![0; self.timeout_exchange.total_nodes()], + phantom: PhantomData, + }; - let accumulator = self.quorum_exchange.accumulate_vote( - new_accumulator, - &vote, - &vote_internal.clone().leaf_commitment, - ); - - // TODO Create default functions for accumulators - // https://github.com/EspressoSystems/HotShot/issues/1797 - let timeout_accumulator = TimeoutVoteAccumulator { - da_vote_outcomes: HashMap::new(), - success_threshold: self.timeout_exchange.success_threshold(), - sig_lists: Vec::new(), - signers: bitvec![0; self.timeout_exchange.total_nodes()], - phantom: PhantomData, - }; + if vote.get_view_number() > collection_view { + let state = VoteCollectionTaskState { + quorum_exchange: self.quorum_exchange.clone(), + timeout_exchange: self.timeout_exchange.clone(), + accumulator, + timeout_accumulator: either::Left(timeout_accumulator), + cur_view: vote.get_view_number(), + event_stream: self.event_stream.clone(), + id: self.id, + }; + let name = "Quorum Vote Collection"; + let filter = FilterEvent(Arc::new(|event| { + matches!( + event, + HotShotEvent::QuorumVoteRecv(_) | HotShotEvent::TimeoutVoteRecv(_) + ) + })); - if vote_internal.current_view > collection_view { - let state = VoteCollectionTaskState { - quorum_exchange: self.quorum_exchange.clone(), - timeout_exchange: self.timeout_exchange.clone(), - accumulator, - timeout_accumulator: either::Left(timeout_accumulator), - cur_view: vote_internal.current_view, - event_stream: self.event_stream.clone(), - id: self.id, - }; - let name = "Quorum Vote Collection"; - let filter = FilterEvent(Arc::new(|event| { - matches!( - event, - HotShotEvent::QuorumVoteRecv(_) - | HotShotEvent::TimeoutVoteRecv(_) - ) - })); - - let builder = - TaskBuilder::>::new(name.to_string()) - .register_event_stream(self.event_stream.clone(), filter) - .await - .register_registry(&mut self.registry.clone()) - .await - .register_state(state) - .register_event_handler(handle_event); - let id = builder.get_task_id().unwrap(); - let stream_id = builder.get_stream_id().unwrap(); - - self.vote_collector = Some((vote_internal.current_view, id, stream_id)); - - let _task = async_spawn(async move { - VoteCollectionTypes::build(builder).launch().await; - }); - debug!( - "Starting vote handle for view {:?}", - vote_internal.current_view - ); - } else if let Some((_, _, stream_id)) = self.vote_collector { - self.event_stream - .direct_message( - stream_id, - HotShotEvent::QuorumVoteRecv(QuorumVote::Yes(vote_internal)), - ) - .await; - } - } - QuorumVote::No(_) => { - error!("The next leader has received an unexpected vote!"); - } + let builder = + TaskBuilder::>::new(name.to_string()) + .register_event_stream(self.event_stream.clone(), filter) + .await + .register_registry(&mut self.registry.clone()) + .await + .register_state(state) + .register_event_handler(handle_event); + let id = builder.get_task_id().unwrap(); + let stream_id = builder.get_stream_id().unwrap(); + + self.vote_collector = Some((vote.get_view_number(), id, stream_id)); + + let _task = async_spawn(async move { + VoteCollectionTypes::build(builder).launch().await; + }); + debug!("Starting vote handle for view {:?}", vote.get_view_number()); + } else if let Some((_, _, stream_id)) = self.vote_collector { + self.event_stream + .direct_message(stream_id, HotShotEvent::QuorumVoteRecv(vote)) + .await; } } HotShotEvent::TimeoutVoteRecv(vote) => { @@ -1133,14 +1110,8 @@ where &vote.get_view().commit(), ); - let quorum_accumulator = QuorumVoteAccumulator { - total_vote_outcomes: HashMap::new(), - yes_vote_outcomes: HashMap::new(), - no_vote_outcomes: HashMap::new(), - - success_threshold: self.quorum_exchange.success_threshold(), - failure_threshold: self.quorum_exchange.failure_threshold(), - + let quorum_accumulator = VoteAccumulator2 { + vote_outcomes: HashMap::new(), sig_lists: Vec::new(), signers: bitvec![0; self.quorum_exchange.total_nodes()], phantom: PhantomData, @@ -1313,7 +1284,7 @@ where #[allow(clippy::too_many_lines)] pub async fn publish_proposal_if_able( &mut self, - _qc: QuorumCertificate>, + _qc: QuorumCertificate2, view: TYPES::Time, timeout_certificate: Option>, ) -> bool { @@ -1326,7 +1297,7 @@ where } let consensus = self.consensus.read().await; - let parent_view_number = &consensus.high_qc.view_number(); + let parent_view_number = &consensus.high_qc.get_view_number(); let mut reached_decided = false; let Some(parent_view) = consensus.state_map.get(parent_view_number) else { @@ -1343,12 +1314,12 @@ where ); return false; }; - if leaf_commitment != consensus.high_qc.leaf_commitment() { + if leaf_commitment != consensus.high_qc.get_data().leaf_commit { // NOTE: This happens on the genesis block debug!( "They don't equal: {:?} {:?}", leaf_commitment, - consensus.high_qc.leaf_commitment() + consensus.high_qc.get_data().leaf_commit ); } let Some(leaf) = consensus.saved_leaves.get(&leaf_commitment) else { diff --git a/crates/task-impls/src/events.rs b/crates/task-impls/src/events.rs index 37b050d9b2..8bf8db5de2 100644 --- a/crates/task-impls/src/events.rs +++ b/crates/task-impls/src/events.rs @@ -1,14 +1,17 @@ use crate::view_sync::ViewSyncPhase; + use commit::Commitment; use either::Either; use hotshot_types::{ - certificate::{DACertificate, QuorumCertificate, TimeoutCertificate, VIDCertificate}, + certificate::{DACertificate, TimeoutCertificate, VIDCertificate}, data::{DAProposal, VidDisperse}, message::Proposal, + simple_certificate::QuorumCertificate2, + simple_vote::QuorumVote, traits::node_implementation::{ - NodeImplementation, NodeType, QuorumProposalType, ViewSyncProposalType, + NodeImplementation, NodeType, QuorumMembership, QuorumProposalType, ViewSyncProposalType, }, - vote::{DAVote, QuorumVote, TimeoutVote, VIDVote, ViewSyncVote}, + vote::{DAVote, TimeoutVote, VIDVote, ViewSyncVote}, }; /// All of the possible events that can be passed between Sequecning `HotShot` tasks @@ -19,7 +22,7 @@ pub enum HotShotEvent> { /// A quorum proposal has been received from the network; handled by the consensus task QuorumProposalRecv(Proposal>, TYPES::SignatureKey), /// A quorum vote has been received from the network; handled by the consensus task - QuorumVoteRecv(QuorumVote>), + QuorumVoteRecv(QuorumVote>), /// A timeout vote recevied from the network; handled by consensus task TimeoutVoteRecv(TimeoutVote), /// Send a timeout vote to the network; emitted by consensus task replicas @@ -33,13 +36,13 @@ pub enum HotShotEvent> { /// Send a quorum proposal to the network; emitted by the leader in the consensus task QuorumProposalSend(Proposal>, TYPES::SignatureKey), /// Send a quorum vote to the next leader; emitted by a replica in the consensus task after seeing a valid quorum proposal - QuorumVoteSend(QuorumVote>), + QuorumVoteSend(QuorumVote>), /// Send a DA proposal to the DA committee; emitted by the DA leader (which is the same node as the leader of view v + 1) in the DA task DAProposalSend(Proposal>, TYPES::SignatureKey), /// Send a DA vote to the DA leader; emitted by DA committee members in the DA task after seeing a valid DA proposal DAVoteSend(DAVote), /// The next leader has collected enough votes to form a QC; emitted by the next leader in the consensus task; an internal event only - QCFormed(Either>, TimeoutCertificate>), + QCFormed(Either, TimeoutCertificate>), /// The DA leader has collected enough votes to form a DAC; emitted by the DA leader in the DA task; sent to the entire network via the networking task DACSend(DACertificate, TYPES::SignatureKey), /// The current view has changed; emitted by the replica in the consensus task or replica in the view sync task; received by almost all other tasks diff --git a/crates/task-impls/src/network.rs b/crates/task-impls/src/network.rs index 523dbd1f70..217030cfd6 100644 --- a/crates/task-impls/src/network.rs +++ b/crates/task-impls/src/network.rs @@ -18,6 +18,7 @@ use hotshot_types::{ node_implementation::{NodeImplementation, NodeType}, }, vote::VoteType, + vote2::{HasViewNumber, Vote2}, }; use snafu::Snafu; use std::{marker::PhantomData, sync::Arc}; @@ -206,12 +207,12 @@ impl< // ED Each network task is subscribed to all these message types. Need filters per network task HotShotEvent::QuorumVoteSend(vote) => ( - vote.signature_key(), + vote.get_signing_key(), MessageKind::::from_consensus_message(SequencingMessage(Left( GeneralConsensusMessage::Vote(vote.clone()), ))), TransmitType::Direct, - Some(membership.get_leader(vote.get_view() + 1)), + Some(membership.get_leader(vote.get_view_number() + 1)), ), HotShotEvent::VidDisperseSend(proposal, sender) => ( sender, diff --git a/crates/testing/src/overall_safety_task.rs b/crates/testing/src/overall_safety_task.rs index 7bc4e83bdc..058d4768c1 100644 --- a/crates/testing/src/overall_safety_task.rs +++ b/crates/testing/src/overall_safety_task.rs @@ -20,10 +20,10 @@ use hotshot_task::{ MergeN, }; use hotshot_types::{ - certificate::QuorumCertificate, data::{LeafBlockPayload, LeafType}, error::RoundTimedoutState, event::{Event, EventType}, + simple_certificate::QuorumCertificate2, traits::node_implementation::NodeType, }; use snafu::Snafu; @@ -95,7 +95,7 @@ pub struct RoundResult> { /// id -> (leaf, qc) // TODO GG: isn't it infeasible to store a Vec? #[allow(clippy::type_complexity)] - success_nodes: HashMap, QuorumCertificate>)>, + success_nodes: HashMap, QuorumCertificate2)>, /// Nodes that failed to commit this round pub failed_nodes: HashMap>>>, @@ -189,7 +189,7 @@ impl> RoundResult pub fn insert_into_result( &mut self, idx: usize, - result: (Vec, QuorumCertificate>), + result: (Vec, QuorumCertificate2), maybe_block_size: Option, ) -> Option { self.success_nodes.insert(idx as u64, result.clone()); diff --git a/crates/testing/src/task_helpers.rs b/crates/testing/src/task_helpers.rs index 3aa8b87ffb..719e2203d4 100644 --- a/crates/testing/src/task_helpers.rs +++ b/crates/testing/src/task_helpers.rs @@ -4,7 +4,6 @@ use crate::{ }; use commit::Committable; use hotshot::{ - certificate::QuorumCertificate, traits::{NodeImplementation, TestableNodeImplementation}, types::{bn254::BLSPubKey, SignatureKey, SystemContextHandle}, HotShotConsensusApi, HotShotInitializer, SystemContext, @@ -16,14 +15,16 @@ use hotshot_types::{ consensus::ConsensusMetricsValue, data::{Leaf, QuorumProposal, VidScheme, ViewNumber}, message::{Message, Proposal}, + simple_certificate::QuorumCertificate2, traits::{ block_contents::BlockHeader, consensus_api::ConsensusSharedApi, - election::{ConsensusExchange, Membership, SignedCertificate}, + election::{ConsensusExchange, Membership}, node_implementation::{CommitteeEx, ExchangesType, NodeType, QuorumEx}, signature_key::EncodedSignature, state::{ConsensusTime, TestableBlock}, }, + vote2::HasViewNumber, }; pub async fn build_system_handle( @@ -99,7 +100,7 @@ async fn build_quorum_proposal_and_signature( }; let _quorum_exchange = api.inner.exchanges.quorum_exchange().clone(); - let parent_view_number = &consensus.high_qc.view_number(); + let parent_view_number = &consensus.high_qc.get_view_number(); let Some(parent_view) = consensus.state_map.get(parent_view_number) else { panic!("Couldn't find high QC parent in state map."); }; @@ -130,7 +131,7 @@ async fn build_quorum_proposal_and_signature( let proposal = QuorumProposal::> { block_header, view_number: ViewNumber::new(view), - justify_qc: QuorumCertificate::genesis(), + justify_qc: QuorumCertificate2::genesis(), timeout_certificate: None, proposer_id: leaf.proposer_id, dac: None, diff --git a/crates/testing/tests/consensus_task.rs b/crates/testing/tests/consensus_task.rs index 12feb914a1..f68b224929 100644 --- a/crates/testing/tests/consensus_task.rs +++ b/crates/testing/tests/consensus_task.rs @@ -1,4 +1,3 @@ -use commit::Commitment; use commit::Committable; use hotshot::{ tasks::add_consensus_task, @@ -11,13 +10,15 @@ use hotshot_testing::{ node_types::{MemoryImpl, TestTypes}, task_helpers::{build_quorum_proposal, key_pair_for_id}, }; +use hotshot_types::simple_vote::QuorumData; +use hotshot_types::simple_vote::QuorumVote; +use hotshot_types::traits::node_implementation::QuorumMembership; +use hotshot_types::vote2::Certificate2; use hotshot_types::{ data::{Leaf, QuorumProposal, ViewNumber}, message::GeneralConsensusMessage, traits::{ - election::{ConsensusExchange, QuorumExchangeType, SignedCertificate}, - node_implementation::ExchangesType, - state::ConsensusTime, + election::ConsensusExchange, node_implementation::ExchangesType, state::ConsensusTime, }, }; @@ -26,7 +27,6 @@ use std::collections::HashMap; async fn build_vote( handle: &SystemContextHandle, proposal: QuorumProposal>, - view: ViewNumber, ) -> GeneralConsensusMessage { let consensus_lock = handle.get_consensus(); let consensus = consensus_lock.read().await; @@ -34,11 +34,10 @@ async fn build_vote( inner: handle.hotshot.inner.clone(), }; let quorum_exchange = api.inner.exchanges.quorum_exchange().clone(); - let vote_token = quorum_exchange.make_vote_token(view).unwrap().unwrap(); let justify_qc = proposal.justify_qc.clone(); let view = ViewNumber::new(*proposal.view_number); - let parent = if justify_qc.is_genesis() { + let parent = if justify_qc.is_genesis { let Some(genesis_view) = consensus.state_map.get(&ViewNumber::new(0)) else { panic!("Couldn't find genesis view in state map."); }; @@ -52,7 +51,7 @@ async fn build_vote( } else { consensus .saved_leaves - .get(&justify_qc.leaf_commitment()) + .get(&justify_qc.get_data().leaf_commit) .cloned() .unwrap() }; @@ -69,13 +68,14 @@ async fn build_vote( timestamp: 0, proposer_id: quorum_exchange.get_leader(view).to_bytes(), }; - - quorum_exchange.create_yes_message( - proposal.justify_qc.commit(), - leaf.commit(), + let vote = + QuorumVote::, QuorumMembership>::create_signed_vote( + QuorumData { leaf_commit: leaf.commit() }, view, - vote_token, - ) + quorum_exchange.public_key(), + quorum_exchange.private_key(), + ); + GeneralConsensusMessage::::Vote(vote) } #[cfg(test)] @@ -88,7 +88,7 @@ async fn build_vote( async fn test_consensus_task() { use hotshot_task_impls::harness::run_harness; use hotshot_testing::task_helpers::build_system_handle; - use hotshot_types::certificate::QuorumCertificate; + use hotshot_types::simple_certificate::QuorumCertificate2; async_compatibility_layer::logging::setup_logging(); async_compatibility_layer::logging::setup_backtrace(); @@ -100,7 +100,7 @@ async fn test_consensus_task() { let mut output = HashMap::new(); // Trigger a proposal to send by creating a new QC. Then recieve that proposal and update view based on the valid QC in the proposal - let qc = QuorumCertificate::>>::genesis(); + let qc = QuorumCertificate2::>::genesis(); let proposal = build_quorum_proposal(&handle, &private_key, 1).await; input.push(HotShotEvent::QCFormed(either::Left(qc.clone()))); @@ -160,9 +160,7 @@ async fn test_consensus_vote() { 1, ); let proposal = proposal.data; - if let GeneralConsensusMessage::Vote(vote) = - build_vote(&handle, proposal, ViewNumber::new(1)).await - { + if let GeneralConsensusMessage::Vote(vote) = build_vote(&handle, proposal).await { output.insert(HotShotEvent::QuorumVoteSend(vote.clone()), 1); input.push(HotShotEvent::QuorumVoteRecv(vote.clone())); output.insert(HotShotEvent::QuorumVoteRecv(vote), 1); diff --git a/crates/testing/tests/timeout.rs b/crates/testing/tests/timeout.rs index 3b7e2e2bfb..98c275868c 100644 --- a/crates/testing/tests/timeout.rs +++ b/crates/testing/tests/timeout.rs @@ -22,7 +22,7 @@ async fn test_timeout_web() { async_compatibility_layer::logging::setup_logging(); async_compatibility_layer::logging::setup_backtrace(); let timing_data = TimingData { - next_view_timeout: 1000, + next_view_timeout: 2000, ..Default::default() }; @@ -50,7 +50,7 @@ async fn test_timeout_web() { metadata.completion_task_description = CompletionTaskDescription::TimeBasedCompletionTaskBuilder( TimeBasedCompletionTaskDescription { - duration: Duration::from_secs(30), + duration: Duration::from_secs(60), }, ); @@ -85,7 +85,7 @@ async fn test_timeout_libp2p() { async_compatibility_layer::logging::setup_logging(); async_compatibility_layer::logging::setup_backtrace(); let timing_data = TimingData { - next_view_timeout: 1000, + next_view_timeout: 2000, ..Default::default() }; @@ -113,7 +113,7 @@ async fn test_timeout_libp2p() { metadata.completion_task_description = CompletionTaskDescription::TimeBasedCompletionTaskBuilder( TimeBasedCompletionTaskDescription { - duration: Duration::from_secs(30), + duration: Duration::from_secs(60), }, ); diff --git a/crates/types/src/consensus.rs b/crates/types/src/consensus.rs index 99a8b00c51..4091e0555b 100644 --- a/crates/types/src/consensus.rs +++ b/crates/types/src/consensus.rs @@ -7,9 +7,9 @@ pub use crate::{ use displaydoc::Display; use crate::{ - certificate::QuorumCertificate, data::LeafType, error::HotShotError, + simple_certificate::QuorumCertificate2, traits::{ metrics::{Counter, Gauge, Histogram, Label, Metrics}, node_implementation::NodeType, @@ -57,7 +57,7 @@ pub struct Consensus> { pub locked_view: TYPES::Time, /// the highqc per spec - pub high_qc: QuorumCertificate>, + pub high_qc: QuorumCertificate2, /// A reference to the metrics trait #[debug(skip)] diff --git a/crates/types/src/data.rs b/crates/types/src/data.rs index 7a00900131..824d965422 100644 --- a/crates/types/src/data.rs +++ b/crates/types/src/data.rs @@ -4,10 +4,8 @@ //! `HotShot`'s version of a block, and proposals, messages upon which to reach the consensus. use crate::{ - certificate::{ - AssembledSignature, DACertificate, QuorumCertificate, TimeoutCertificate, - ViewSyncCertificate, - }, + certificate::{AssembledSignature, DACertificate, TimeoutCertificate, ViewSyncCertificate}, + simple_certificate::QuorumCertificate2, traits::{ block_contents::BlockHeader, node_implementation::NodeType, @@ -16,6 +14,7 @@ use crate::{ storage::StoredView, BlockPayload, State, }, + vote2::Certificate2, }; use ark_bls12_381::Bls12_381; use ark_serialize::{CanonicalDeserialize, CanonicalSerialize, Read, SerializationError, Write}; @@ -116,45 +115,6 @@ pub type Transaction = <::BlockPayload as BlockPayload>:: /// `Commitment` to the `Transaction` type associated with a `State`, as a syntactic shortcut pub type TxnCommitment = Commitment>; -/// subset of state that we stick into a leaf. -/// original hotstuff proposal -#[derive(custom_debug::Debug, Serialize, Deserialize, Clone, Derivative, Eq)] -#[serde(bound(deserialize = ""))] -#[derivative(PartialEq, Hash)] -pub struct ValidatingProposal> -where - LEAF: Committable, -{ - /// Current view's block payload commitment - pub payload_commitment: Commitment, - - /// CurView from leader when proposing leaf - pub view_number: TYPES::Time, - - /// Height from leader when proposing leaf - pub height: u64, - - /// Per spec, justification - pub justify_qc: QuorumCertificate>, - - /// The hash of the parent `Leaf` - /// So we can ask if it extends - #[debug(skip)] - pub parent_commitment: Commitment, - - /// BlockPayload leaf wants to apply - pub deltas: TYPES::BlockPayload, - - /// What the state should be after applying `self.deltas` - pub state_commitment: Commitment, - - /// Transactions that were marked for rejection while collecting deltas - pub rejected: Vec<::Transaction>, - - /// the propser id - pub proposer_id: EncodedPublicKey, -} - /// A proposal to start providing data availability for a block. #[derive(custom_debug::Debug, Serialize, Deserialize, Clone, Eq, PartialEq, Hash)] pub struct DAProposal { @@ -213,7 +173,7 @@ pub struct QuorumProposal> { pub view_number: TYPES::Time, /// Per spec, justification - pub justify_qc: QuorumCertificate>, + pub justify_qc: QuorumCertificate2, /// Possible timeout certificate. Only present if the justify_qc is not for the preceding view pub timeout_certificate: Option>, @@ -226,15 +186,6 @@ pub struct QuorumProposal> { pub dac: Option>, } -impl> ProposalType - for ValidatingProposal -{ - type NodeType = TYPES; - fn get_view_number(&self) -> ::Time { - self.view_number - } -} - impl ProposalType for DAProposal { type NodeType = TYPES; fn get_view_number(&self) -> ::Time { @@ -358,7 +309,7 @@ pub trait LeafType: /// Create a new leaf from its components. fn new( view_number: LeafTime, - justify_qc: QuorumCertificate>, + justify_qc: QuorumCertificate2, deltas: LeafBlockPayload, state: LeafState, ) -> Self; @@ -369,7 +320,7 @@ pub trait LeafType: /// Equivalently, this is the number of leaves before this one in the chain. fn get_height(&self) -> u64; /// The QC linking this leaf to its parent in the chain. - fn get_justify_qc(&self) -> QuorumCertificate>; + fn get_justify_qc(&self) -> QuorumCertificate2; /// Commitment to this leaf's parent. fn get_parent_commitment(&self) -> Commitment; /// The block header contained in this leaf. @@ -441,7 +392,7 @@ pub struct ValidatingLeaf { pub height: u64, /// Per spec, justification - pub justify_qc: QuorumCertificate>, + pub justify_qc: QuorumCertificate2, /// The hash of the parent `Leaf` /// So we can ask if it extends @@ -477,7 +428,7 @@ pub struct Leaf { pub view_number: TYPES::Time, /// Per spec, justification - pub justify_qc: QuorumCertificate>, + pub justify_qc: QuorumCertificate2, /// The hash of the parent `Leaf` /// So we can ask if it extends @@ -537,7 +488,7 @@ impl LeafType for ValidatingLeaf { fn new( view_number: ::Time, - justify_qc: QuorumCertificate>, + justify_qc: QuorumCertificate2, deltas: ::BlockPayload, state: ::StateType, ) -> Self { @@ -562,7 +513,7 @@ impl LeafType for ValidatingLeaf { self.height } - fn get_justify_qc(&self) -> QuorumCertificate> { + fn get_justify_qc(&self) -> QuorumCertificate2 { self.justify_qc.clone() } @@ -646,7 +597,7 @@ impl LeafType for Leaf { fn new( view_number: ::Time, - justify_qc: QuorumCertificate>, + justify_qc: QuorumCertificate2, payload: ::BlockPayload, _state: ::StateType, ) -> Self { @@ -670,7 +621,7 @@ impl LeafType for Leaf { self.block_header.block_number() } - fn get_justify_qc(&self) -> QuorumCertificate> { + fn get_justify_qc(&self) -> QuorumCertificate2 { self.justify_qc.clone() } @@ -764,6 +715,7 @@ pub fn random_commitment(rng: &mut dyn rand::RngCore) -> Commitm /// Serialization for the QC assembled signature /// # Panics /// if serialization fails +// TODO: Remove after new QC is integrated pub fn serialize_signature(signature: &AssembledSignature) -> Vec { let mut signatures_bytes = vec![]; let signatures: Option<::QCType> = match &signature { @@ -820,9 +772,37 @@ pub fn serialize_signature(signature: &AssembledSignature( + signatures: &::QCType, +) -> Vec { + let mut signatures_bytes = vec![]; + signatures_bytes.extend("Yes".as_bytes()); + + let (sig, proof) = TYPES::SignatureKey::get_sig_proof(signatures); + let proof_bytes = bincode_opts() + .serialize(&proof.as_bitslice()) + .expect("This serialization shouldn't be able to fail"); + signatures_bytes.extend("bitvec proof".as_bytes()); + signatures_bytes.extend(proof_bytes.as_slice()); + let sig_bytes = bincode_opts() + .serialize(&sig) + .expect("This serialization shouldn't be able to fail"); + signatures_bytes.extend("aggregated signature".as_bytes()); + signatures_bytes.extend(sig_bytes.as_slice()); + signatures_bytes +} impl Committable for ValidatingLeaf { fn commit(&self) -> commit::Commitment { - let signatures_bytes = serialize_signature(&self.justify_qc.signatures); + let signatures_bytes = if self.justify_qc.is_genesis { + let mut bytes = vec![]; + bytes.extend("genesis".as_bytes()); + bytes + } else { + serialize_signature2::(self.justify_qc.signatures.as_ref().unwrap()) + }; commit::RawCommitmentBuilder::new("leaf commitment") .u64_field("view number", *self.view_number) @@ -834,7 +814,7 @@ impl Committable for ValidatingLeaf { .u64(*self.justify_qc.view_number) .field( "justify_qc leaf commitment", - self.justify_qc.leaf_commitment, + self.justify_qc.get_data().leaf_commit, ) .constant_str("justify_qc signatures") .var_size_bytes(&signatures_bytes) @@ -848,7 +828,13 @@ impl Committable for ValidatingLeaf { impl Committable for Leaf { fn commit(&self) -> commit::Commitment { - let signatures_bytes = serialize_signature(&self.justify_qc.signatures); + let signatures_bytes = if self.justify_qc.is_genesis { + let mut bytes = vec![]; + bytes.extend("genesis".as_bytes()); + bytes + } else { + serialize_signature2::(self.justify_qc.signatures.as_ref().unwrap()) + }; // Skip the transaction commitments, so that the repliacs can reconstruct the leaf. commit::RawCommitmentBuilder::new("leaf commitment") @@ -860,7 +846,7 @@ impl Committable for Leaf { .u64(*self.justify_qc.view_number) .field( "justify_qc leaf commitment", - self.justify_qc.leaf_commitment, + self.justify_qc.get_data().leaf_commit, ) .constant_str("justify_qc signatures") .var_size_bytes(&signatures_bytes) @@ -868,24 +854,6 @@ impl Committable for Leaf { } } -impl From> - for ValidatingProposal> -{ - fn from(leaf: ValidatingLeaf) -> Self { - Self { - view_number: leaf.view_number, - height: leaf.height, - justify_qc: leaf.justify_qc, - parent_commitment: leaf.parent_commitment, - deltas: leaf.deltas.clone(), - state_commitment: leaf.state.commit(), - rejected: leaf.rejected, - proposer_id: leaf.proposer_id, - payload_commitment: leaf.deltas.commit(), - } - } -} - impl From for StoredView where TYPES: NodeType, diff --git a/crates/types/src/event.rs b/crates/types/src/event.rs index 5d091cd400..c70bb2a8e7 100644 --- a/crates/types/src/event.rs +++ b/crates/types/src/event.rs @@ -1,10 +1,10 @@ //! Events that a `HotShot` instance can emit use crate::{ - certificate::QuorumCertificate, data::LeafType, error::HotShotError, + data::LeafType, error::HotShotError, simple_certificate::QuorumCertificate2, traits::node_implementation::NodeType, }; -use commit::Commitment; + use std::sync::Arc; /// A status event emitted by a `HotShot` instance /// @@ -43,7 +43,7 @@ pub enum EventType> { /// /// Note that the QC for each additional leaf in the chain can be obtained from the leaf /// before it using - qc: Arc>>, + qc: Arc>, /// Optional information of the number of transactions in the block, for logging purposes. block_size: Option, }, diff --git a/crates/types/src/lib.rs b/crates/types/src/lib.rs index ff296cf28d..8021122547 100644 --- a/crates/types/src/lib.rs +++ b/crates/types/src/lib.rs @@ -18,6 +18,8 @@ pub mod data; pub mod error; pub mod event; pub mod message; +pub mod simple_certificate; +pub mod simple_vote; pub mod traits; pub mod utils; pub mod vote; diff --git a/crates/types/src/message.rs b/crates/types/src/message.rs index e85c9c51df..cc1acc80f7 100644 --- a/crates/types/src/message.rs +++ b/crates/types/src/message.rs @@ -3,19 +3,22 @@ //! This module contains types used to represent the various types of messages that //! `HotShot` nodes can send among themselves. +use crate::vote2::HasViewNumber; use crate::{ certificate::{DACertificate, VIDCertificate}, data::{DAProposal, ProposalType, VidDisperse}, + simple_vote::QuorumVote, traits::{ network::{NetworkMsg, ViewMessage}, node_implementation::{ - ExchangesType, NodeImplementation, NodeType, QuorumProposalType, ViewSyncProposalType, + ExchangesType, NodeImplementation, NodeType, QuorumMembership, QuorumProposalType, + ViewSyncProposalType, }, signature_key::EncodedSignature, }, - vote::{DAVote, QuorumVote, TimeoutVote, VIDVote, ViewSyncVote, VoteType}, + vote::{DAVote, TimeoutVote, VIDVote, ViewSyncVote, VoteType}, }; -use commit::Commitment; + use derivative::Derivative; use either::Either::{self, Left, Right}; use serde::{Deserialize, Serialize}; @@ -148,7 +151,10 @@ where /// Message with a quorum proposal. Proposal(Proposal>, TYPES::SignatureKey), /// Message with a quorum vote. - Vote(QuorumVote>, TYPES::SignatureKey), + Vote( + QuorumVote>, + TYPES::SignatureKey, + ), /// Message with a view sync vote. ViewSyncVote(ViewSyncVote), /// Message with a view sync certificate. @@ -303,7 +309,7 @@ impl< } } -#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Eq, Hash)] #[serde(bound(deserialize = "", serialize = ""))] /// Messages related to both validating and sequencing consensus. pub enum GeneralConsensusMessage> @@ -314,7 +320,7 @@ where Proposal(Proposal>), /// Message with a quorum vote. - Vote(QuorumVote>), + Vote(QuorumVote>), /// Message with a view sync vote. ViewSyncVote(ViewSyncVote), @@ -383,7 +389,7 @@ pub trait SequencingMessageType>: } /// Messages for sequencing consensus. -#[derive(Clone, Debug, Deserialize, Serialize, Hash, PartialEq, Eq)] +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq, Eq, Hash)] #[serde(bound(deserialize = "", serialize = ""))] pub struct SequencingMessage< TYPES: NodeType, @@ -409,7 +415,7 @@ impl< // this should match replica upon receipt p.data.get_view_number() } - GeneralConsensusMessage::Vote(vote_message) => vote_message.get_view(), + GeneralConsensusMessage::Vote(vote_message) => vote_message.get_view_number(), GeneralConsensusMessage::InternalTrigger(trigger) => match trigger { InternalTrigger::Timeout(time) => *time, }, diff --git a/crates/types/src/simple_certificate.rs b/crates/types/src/simple_certificate.rs new file mode 100644 index 0000000000..766c4eaa69 --- /dev/null +++ b/crates/types/src/simple_certificate.rs @@ -0,0 +1,128 @@ +//! Implementations of the simple certificate type. Used for Quorum, DA, and Timeout Certificates + +use std::{ + fmt::{self, Debug, Display, Formatter}, + hash::Hash, + marker::PhantomData, +}; + +use commit::{Commitment, CommitmentBoundsArkless, Committable}; +use ethereum_types::U256; + +use crate::{ + simple_vote::{QuorumData, Voteable}, + traits::{ + election::Membership, node_implementation::NodeType, signature_key::SignatureKey, + state::ConsensusTime, + }, + vote2::{Certificate2, HasViewNumber}, +}; + +use serde::{Deserialize, Serialize}; + +/// A certificate which can be created by aggregating many simple votes on the commitment. +#[derive(Serialize, Deserialize, Eq, Hash, PartialEq, Debug, Clone)] +pub struct SimpleCertificate { + /// The data this certificate is for. I.e the thing that was voted on to create this Certificate + pub data: VOTEABLE, + /// commitment of all the votes this cert should be signed over + pub vote_commitment: Commitment, + /// Which view this QC relates to + pub view_number: TYPES::Time, + /// assembled signature for certificate aggregation + pub signatures: Option<::QCType>, + /// If this QC is for the genesis block + pub is_genesis: bool, + /// phantom data for `MEMBERSHIP` and `TYPES` + pub _pd: PhantomData, +} + +impl Certificate2 + for SimpleCertificate +{ + type Voteable = VOTEABLE; + + fn create_signed_certificate( + vote_commitment: Commitment, + data: Self::Voteable, + sig: ::QCType, + view: TYPES::Time, + ) -> Self { + SimpleCertificate { + data, + vote_commitment, + view_number: view, + signatures: Some(sig), + is_genesis: false, + _pd: PhantomData, + } + } + fn is_valid_cert>(&self, membership: &MEMBERSHIP) -> bool { + if self.is_genesis && self.view_number == TYPES::Time::genesis() { + return true; + } + let real_qc_pp = ::get_public_parameter( + membership.get_committee_qc_stake_table(), + U256::from(membership.success_threshold().get()), + ); + ::check( + &real_qc_pp, + self.vote_commitment.as_ref(), + self.signatures.as_ref().unwrap(), + ) + } + fn threshold>(membership: &MEMBERSHIP) -> u64 { + membership.success_threshold().into() + } + fn get_data(&self) -> &Self::Voteable { + &self.data + } + fn get_data_commitment(&self) -> Commitment { + self.vote_commitment + } +} + +impl HasViewNumber + for SimpleCertificate +{ + fn get_view_number(&self) -> TYPES::Time { + self.view_number + } +} +impl Display + for QuorumCertificate2 +{ + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!( + f, + "view: {:?}, is_genesis: {:?}", + self.view_number, self.is_genesis + ) + } +} + +impl< + TYPES: NodeType, + LEAF: Committable + Committable + Clone + Serialize + Debug + PartialEq + Hash + Eq + 'static, + > QuorumCertificate2 +{ + #[must_use] + /// Creat the Genisis certificate + pub fn genesis() -> Self { + let data = QuorumData { + leaf_commit: Commitment::::default_commitment_no_preimage(), + }; + let commit = data.commit(); + Self { + data, + vote_commitment: commit, + view_number: ::genesis(), + signatures: None, + is_genesis: true, + _pd: PhantomData, + } + } +} + +/// Type alias for a `QuorumCertificate`, which is a `SimpleCertificate` of `QuorumVotes` +pub type QuorumCertificate2 = SimpleCertificate>; diff --git a/crates/types/src/simple_vote.rs b/crates/types/src/simple_vote.rs new file mode 100644 index 0000000000..62438ea4e9 --- /dev/null +++ b/crates/types/src/simple_vote.rs @@ -0,0 +1,225 @@ +//! Implementations of the simple vote types. + +use std::{fmt::Debug, hash::Hash, marker::PhantomData}; + +use commit::{Commitment, Committable}; +use serde::{Deserialize, Serialize}; + +use crate::{ + traits::{ + election::Membership, + node_implementation::NodeType, + signature_key::{EncodedPublicKey, EncodedSignature, SignatureKey}, + }, + vote2::{HasViewNumber, Vote2}, +}; + +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a yes vote. +pub struct QuorumData { + /// Commitment to the leaf + pub leaf_commit: Commitment, +} +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a DA vote. +pub struct DAData { + /// Commitment to a block payload + pub payload_commit: Commitment, +} +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a timeout vote. +pub struct TimeoutData { + /// View the timeout is for + pub view: TYPES::Time, +} +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a VID vote. +pub struct VIDData { + /// Commitment to the block payload the VID vote is on. + pub payload_commit: Commitment, +} +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a Pre Commit vote. +pub struct ViewSyncPreCommitData { + /// The relay this vote is intended for + pub relay: EncodedPublicKey, + /// The view number we are trying to sync on + pub round: TYPES::Time, +} +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a Commit vote. +pub struct ViewSyncCommitData { + /// The relay this vote is intended for + pub relay: EncodedPublicKey, + /// The view number we are trying to sync on + pub round: TYPES::Time, +} +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +/// Data used for a Finalize vote. +pub struct ViewSyncFinalizeData { + /// The relay this vote is intended for + pub relay: EncodedPublicKey, + /// The view number we are trying to sync on + pub round: TYPES::Time, +} + +/// Marker trait for data or commitments that can be voted on. +/// Only structs in this file can implement voteable. This is enforced with the `Sealed` trait +/// Sealing this trait prevents creating new vote types outside this file. +pub trait Voteable: + sealed::Sealed + Committable + Clone + Serialize + Debug + PartialEq + Hash + Eq +{ +} + +/// Sealed is used to make sure no other files can implement the Voteable trait. +/// All simple voteable types should be implemented here. This prevents us from +/// creating/using improper types when using the vote types. +mod sealed { + use commit::Committable; + + /// Only structs in this file can impl `Sealed` + pub trait Sealed {} + + // TODO: Does the implement for things outside this file that are commitable? + impl Sealed for C {} +} + +/// A simple yes vote over some votable type. +#[derive(Serialize, Deserialize, Clone, Debug, PartialEq, Hash, Eq)] +pub struct SimpleVote> { + /// The signature share associated with this vote + pub signature: (EncodedPublicKey, EncodedSignature), + /// The leaf commitment being voted on. + pub data: DATA, + /// The view this vote was cast for + pub view_number: TYPES::Time, + /// phantom data for `MEMBERSHIP` + _pd: PhantomData, +} + +impl> HasViewNumber + for SimpleVote +{ + fn get_view_number(&self) -> ::Time { + self.view_number + } +} + +impl> Vote2 + for SimpleVote +{ + type Commitment = DATA; + type Membership = MEMBERSHIP; + + fn get_signing_key(&self) -> ::SignatureKey { + ::from_bytes(&self.signature.0).unwrap() + } + + fn get_signature(&self) -> EncodedSignature { + self.signature.1.clone() + } + + fn get_data(&self) -> &DATA { + &self.data + } + + fn get_data_commitment(&self) -> Commitment { + self.data.commit() + } +} + +impl> + SimpleVote +{ + /// Creates and signs a simple vote + pub fn create_signed_vote( + data: DATA, + view: TYPES::Time, + pub_key: &TYPES::SignatureKey, + private_key: &::PrivateKey, + ) -> Self { + let signature = TYPES::SignatureKey::sign(private_key, data.commit().as_ref()); + Self { + signature: (pub_key.to_bytes(), signature), + data, + view_number: view, + _pd: PhantomData, + } + } +} + +impl Committable for QuorumData { + fn commit(&self) -> Commitment { + commit::RawCommitmentBuilder::new("Yes Vote") + .var_size_bytes(self.leaf_commit.as_ref()) + .finalize() + } +} + +impl Committable for DAData { + fn commit(&self) -> Commitment { + commit::RawCommitmentBuilder::new("DA Vote") + .var_size_bytes(self.payload_commit.as_ref()) + .finalize() + } +} +impl Committable for VIDData { + fn commit(&self) -> Commitment { + commit::RawCommitmentBuilder::new("VID Vote") + .var_size_bytes(self.payload_commit.as_ref()) + .finalize() + } +} + +/// This implements commit for all the types which contain a view and relay public key. +fn view_and_relay_commit( + view: TYPES::Time, + relay: &EncodedPublicKey, + tag: &str, +) -> Commitment { + let builder = commit::RawCommitmentBuilder::new(tag); + builder + .var_size_field("Relay public key", &relay.0) + .u64(*view) + .finalize() +} + +impl Committable for ViewSyncPreCommitData { + fn commit(&self) -> Commitment { + view_and_relay_commit::(self.round, &self.relay, "View Sync Precommit") + } +} + +impl Committable for ViewSyncFinalizeData { + fn commit(&self) -> Commitment { + view_and_relay_commit::(self.round, &self.relay, "View Sync Finalize") + } +} +impl Committable for ViewSyncCommitData { + fn commit(&self) -> Commitment { + view_and_relay_commit::(self.round, &self.relay, "View Sync Commit") + } +} + +// impl votable for all the data types in this file sealed marker should ensure nothing is accidently +// implemented for structs that aren't "voteable" +impl Voteable + for V +{ +} + +// Type aliases for simple use of all the main votes. We should never see `SimpleVote` outside this file +/// Yes vote Alias +pub type QuorumVote = SimpleVote, M>; +/// DA vote type alias +pub type DAVote = SimpleVote, M>; +/// VID vote type alias +pub type VIDVote = SimpleVote, M>; +/// Timeout Vote type alias +pub type TimeoutVote = SimpleVote, M>; +/// View Sync Commit Vote type alias +pub type ViewSyncCommitVote = SimpleVote, M>; +/// View Sync Pre Commit Vote type alias +pub type ViewSyncPreCommitVote = SimpleVote, M>; +/// View Sync Finalize Vote type alias +pub type ViewSyncFinalizeVote = SimpleVote, M>; diff --git a/crates/types/src/traits/consensus_api.rs b/crates/types/src/traits/consensus_api.rs index a2ee27c3b1..ac48fc593f 100644 --- a/crates/types/src/traits/consensus_api.rs +++ b/crates/types/src/traits/consensus_api.rs @@ -1,11 +1,11 @@ //! Contains the [`ConsensusApi`] trait. use crate::{ - certificate::QuorumCertificate, data::LeafType, error::HotShotError, event::{Event, EventType}, message::{DataMessage, SequencingMessage}, + simple_certificate::QuorumCertificate2, traits::{ network::NetworkError, node_implementation::{NodeImplementation, NodeType}, @@ -14,7 +14,7 @@ use crate::{ }, }; use async_trait::async_trait; -use commit::Commitment; + use std::{num::NonZeroUsize, sync::Arc, time::Duration}; /// The API that [`HotStuff`] needs to talk to the system, implemented for both validating and @@ -97,7 +97,7 @@ pub trait ConsensusSharedApi< &self, view_number: TYPES::Time, leaf_views: Vec, - decide_qc: QuorumCertificate>, + decide_qc: QuorumCertificate2, ) { self.send_event(Event { view_number, diff --git a/crates/types/src/traits/election.rs b/crates/types/src/traits/election.rs index 98966d9dc4..d2872cc2f5 100644 --- a/crates/types/src/traits/election.rs +++ b/crates/types/src/traits/election.rs @@ -29,7 +29,7 @@ use crate::{ signature_key::SignatureKey, state::ConsensusTime, }, - vote::{Accumulator, DAVote, QuorumVote, ViewSyncData, ViewSyncVote, VoteType, YesOrNoVote}, + vote::{Accumulator, DAVote, QuorumVote, ViewSyncData, ViewSyncVote, VoteType}, }; use bincode::Options; use commit::{Commitment, CommitmentBounds, Committable}; @@ -206,7 +206,7 @@ where /// A protocol for determining membership in and participating in a committee. pub trait Membership: - Clone + Debug + Eq + PartialEq + Send + Sync + 'static + Clone + Debug + Eq + PartialEq + Send + Sync + Hash + 'static { /// generate a default election configuration fn default_election_config(num_nodes: u64) -> TYPES::ElectionConfigType; @@ -787,19 +787,6 @@ impl< pub trait QuorumExchangeType, M: NetworkMsg>: ConsensusExchange { - /// Create a message with a positive vote on validating or commitment proposal. - // TODO ED This returns just a general message type, it's not even bound to a proposal, and this is just a function on the QC. Make proprosal doesn't really apply to all cert types. - fn create_yes_message>( - &self, - justify_qc_commitment: Commitment, - leaf_commitment: Commitment, - current_view: TYPES::Time, - vote_token: TYPES::VoteTokenType, - ) -> GeneralConsensusMessage - where - >::Certificate: commit::Committable, - I::Exchanges: ExchangesType>; - /// Sign a validating or commitment proposal. fn sign_validating_or_commitment_proposal>( &self, @@ -822,27 +809,6 @@ pub trait QuorumExchangeType, &self, leaf_commitment: Commitment, ) -> (EncodedPublicKey, EncodedSignature); - - /// Sign a neagtive vote on validating or commitment proposal. - /// - /// The leaf commitment and the type of the vote (no) are signed, which is the minimum amount - /// of information necessary for any user of the subsequently constructed QC to check that this - /// node voted `No` on that leaf. - fn sign_no_vote( - &self, - leaf_commitment: Commitment, - ) -> (EncodedPublicKey, EncodedSignature); - - /// Create a message with a negative vote on validating or commitment proposal. - fn create_no_message>( - &self, - justify_qc_commitment: Commitment>>, - leaf_commitment: Commitment, - current_view: TYPES::Time, - vote_token: TYPES::VoteTokenType, - ) -> GeneralConsensusMessage - where - I::Exchanges: ExchangesType>; } /// Standard implementation of [`QuroumExchangeType`] based on Hot Stuff consensus. @@ -881,27 +847,6 @@ impl< > QuorumExchangeType for QuorumExchange { - /// Create a message with a positive vote on validating or commitment proposal. - fn create_yes_message>( - &self, - justify_qc_commitment: Commitment>>, - leaf_commitment: Commitment, - current_view: TYPES::Time, - vote_token: TYPES::VoteTokenType, - ) -> GeneralConsensusMessage - where - I::Exchanges: ExchangesType>, - { - let signature = self.sign_yes_vote(leaf_commitment); - GeneralConsensusMessage::::Vote(QuorumVote::Yes(YesOrNoVote { - justify_qc_commitment, - signature, - leaf_commitment, - current_view, - vote_token, - vote_data: VoteData::Yes(leaf_commitment), - })) - } /// Sign a validating or commitment proposal. fn sign_validating_or_commitment_proposal>( &self, @@ -936,45 +881,6 @@ impl< ); (self.public_key.to_bytes(), signature) } - - /// Sign a neagtive vote on validating or commitment proposal. - /// - /// The leaf commitment and the type of the vote (no) are signed, which is the minimum amount - /// of information necessary for any user of the subsequently constructed QC to check that this - /// node voted `No` on that leaf. - /// TODO GG: why return the pubkey? Some other `sign_xxx` methods do not return the pubkey. - fn sign_no_vote( - &self, - leaf_commitment: Commitment, - ) -> (EncodedPublicKey, EncodedSignature) { - let signature = TYPES::SignatureKey::sign( - &self.private_key, - VoteData::No(leaf_commitment).commit().as_ref(), - ); - (self.public_key.to_bytes(), signature) - } - - /// Create a message with a negative vote on validating or commitment proposal. - fn create_no_message>( - &self, - justify_qc_commitment: Commitment>>, - leaf_commitment: Commitment, - current_view: TYPES::Time, - vote_token: TYPES::VoteTokenType, - ) -> GeneralConsensusMessage - where - I::Exchanges: ExchangesType>, - { - let signature = self.sign_no_vote(leaf_commitment); - GeneralConsensusMessage::::Vote(QuorumVote::No(YesOrNoVote { - justify_qc_commitment, - signature, - leaf_commitment, - current_view, - vote_token, - vote_data: VoteData::No(leaf_commitment), - })) - } } impl< diff --git a/crates/types/src/traits/storage.rs b/crates/types/src/traits/storage.rs index 24c7e0459f..b7078b151c 100644 --- a/crates/types/src/traits/storage.rs +++ b/crates/types/src/traits/storage.rs @@ -2,9 +2,8 @@ use super::{node_implementation::NodeType, signature_key::EncodedPublicKey}; use crate::{ - certificate::QuorumCertificate, - data::LeafType, - traits::{election::SignedCertificate, BlockPayload}, + data::LeafType, simple_certificate::QuorumCertificate2, traits::BlockPayload, + vote2::HasViewNumber, }; use async_trait::async_trait; use commit::Commitment; @@ -130,7 +129,7 @@ pub struct StoredView> { /// The parent of this view pub parent: Commitment, /// The justify QC of this view. See the hotstuff paper for more information on this. - pub justify_qc: QuorumCertificate>, + pub justify_qc: QuorumCertificate2, /// The state of this view pub state: LEAF::MaybeState, /// Block header. @@ -159,7 +158,7 @@ where /// Note that this will set the `parent` to `LeafHash::default()`, so this will not have a /// parent. pub fn from_qc_block_and_state( - qc: QuorumCertificate>, + qc: QuorumCertificate2, block_header: TYPES::BlockHeader, block_payload: Option, state: LEAF::MaybeState, @@ -168,7 +167,7 @@ where proposer_id: EncodedPublicKey, ) -> Self { Self { - view_number: qc.view_number(), + view_number: qc.get_view_number(), parent: parent_commitment, justify_qc: qc, state, diff --git a/crates/types/src/vote2.rs b/crates/types/src/vote2.rs index 3e452c102d..e510bef4c6 100644 --- a/crates/types/src/vote2.rs +++ b/crates/types/src/vote2.rs @@ -7,29 +7,34 @@ use std::{ use bincode::Options; use bitvec::vec::BitVec; -use commit::CommitmentBounds; +use commit::Commitment; use either::Either; use ethereum_types::U256; use hotshot_utils::bincode::bincode_opts; use tracing::error; -use crate::traits::{ - election::Membership, - node_implementation::NodeType, - signature_key::{EncodedPublicKey, EncodedSignature, SignatureKey}, +use crate::{ + simple_vote::Voteable, + traits::{ + election::Membership, + node_implementation::NodeType, + signature_key::{EncodedPublicKey, EncodedSignature, SignatureKey}, + }, }; /// A simple vote that has a signer and commitment to the data voted on. -pub trait Vote2 { +pub trait Vote2: HasViewNumber { /// The membership of those that send this vote type type Membership: Membership; /// Type of data commitment this vote uses. - type Commitment: CommitmentBounds; + type Commitment: Voteable; /// Get the signature of the vote sender fn get_signature(&self) -> EncodedSignature; - /// Gets the Data commitment the vote references - fn get_data_commitment(&self) -> Self::Commitment; + /// Gets the data which was voted on by this vote + fn get_data(&self) -> &Self::Commitment; + /// Gets the Data commitment of the vote + fn get_data_commitment(&self) -> Commitment; /// Gets the public signature key of the votes creator/sender fn get_signing_key(&self) -> TYPES::SignatureKey; @@ -37,43 +42,47 @@ pub trait Vote2 { } /// Any type that is associated with a view -pub trait ViewNumber { +pub trait HasViewNumber { /// Returns the view number the type refers to. fn get_view_number(&self) -> TYPES::Time; } -/// The certificate formed from the collection of signatures a committee. -/// The committee is defined by the `Membership` associated type. -/// The votes all must be over the `Commitment` associated type. -pub trait Certificate2 { - /// Type that defines membership for voters on the certificate - type Membership: Membership; +/** +The certificate formed from the collection of signatures a committee. +The committee is defined by the `Membership` associated type. +The votes all must be over the `Commitment` associated type. +*/ +pub trait Certificate2: HasViewNumber { /// The data commitment this certificate certifies. - type Commitment: CommitmentBounds; + type Voteable: Voteable; /// Build a certificate from the data commitment and the quorum of signers fn create_signed_certificate( - data_commitment: Self::Commitment, + vote_commitment: Commitment, + data: Self::Voteable, sig: ::QCType, + view: TYPES::Time, ) -> Self; /// Checks if the cert is valid - fn is_valid_cert(&self) -> bool; + fn is_valid_cert>(&self, membership: &MEMBERSHIP) -> bool; /// Returns the amount of stake needed to create this certificate // TODO: Make this a static ratio of the total stake of `Membership` - fn threshold() -> u64; - /// Get the data commitment the certificate is referencing - fn get_data_commitment(&self) -> Self::Commitment; + fn threshold>(membership: &MEMBERSHIP) -> u64; + /// Get the commitment which was voted on + fn get_data(&self) -> &Self::Voteable; + /// Get the vote commitment which the votes commit to + fn get_data_commitment(&self) -> Commitment; } /// Accumulates votes until a certificate is formed. This implementation works for all simple vote and certificate pairs pub struct VoteAccumulator2< TYPES: NodeType, VOTE: Vote2, - CERT: Certificate2, + CERT: Certificate2, > { /// Map of all signatures accumlated so far - pub vote_outcomes: VoteMap2, + pub vote_outcomes: VoteMap2>, /// A list of valid signatures for certificate aggregation pub sig_lists: Vec<::PureAssembledSignatureType>, /// A bitvec to indicate which node is active and send out a valid signature for certificate aggregation, this automatically do uniqueness check @@ -85,22 +94,20 @@ pub struct VoteAccumulator2< impl< TYPES: NodeType, VOTE: Vote2, - CERT: Certificate2, + CERT: Certificate2, > VoteAccumulator2 { /// Add a vote to the total accumulated votes. Returns the accumulator or the certificate if we /// have accumulated enough votes to exceed the threshold for creating a certificate. - #[allow(dead_code)] - fn accumulate(mut self, vote: &VOTE, membership: &VOTE::Membership) -> Either { + /// + /// # Panics + /// Panics if the vote comes from a node not in the stake table + pub fn accumulate(mut self, vote: &VOTE, membership: &VOTE::Membership) -> Either { let key = vote.get_signing_key(); let vote_commitment = vote.get_data_commitment(); - if !key.validate( - &vote.get_signature(), - &bincode_opts().serialize(&vote_commitment).unwrap(), - ) { - error!("Vote data is {:?}", vote.get_data_commitment()); - error!("Invalid vote! Data"); + if !key.validate(&vote.get_signature(), vote_commitment.as_ref()) { + error!("Invalid vote! Vote Data {:?}", vote.get_data()); return Either::Left(self); } @@ -147,12 +154,13 @@ impl< (vote.get_signature(), vote.get_data_commitment()), ); - if *total_stake_casted >= CERT::threshold() { + if *total_stake_casted >= CERT::threshold(membership) { // Assemble QC - let real_qc_pp = ::get_public_parameter( - stake_table.clone(), - U256::from(CERT::threshold()), - ); + let real_qc_pp: <::SignatureKey as SignatureKey>::QCParams = + ::get_public_parameter( + stake_table.clone(), + U256::from(CERT::threshold(membership)), + ); let real_qc_sig = ::assemble( &real_qc_pp, @@ -160,7 +168,12 @@ impl< &self.sig_lists[..], ); - let cert = CERT::create_signed_certificate(vote.get_data_commitment(), real_qc_sig); + let cert = CERT::create_signed_certificate( + vote.get_data_commitment(), + vote.get_data().clone(), + real_qc_sig, + vote.get_view_number(), + ); return Either::Right(cert); } Either::Left(self)