Skip to content

Commit

Permalink
Merge pull request #2102 from EspressoSystems/keyao/remove-commit
Browse files Browse the repository at this point in the history
Remove commitment computation from payload and fix DA proposal
  • Loading branch information
shenkeyao authored Nov 28, 2023
2 parents b9669ce + 13c1e58 commit 7f6ca9a
Show file tree
Hide file tree
Showing 23 changed files with 236 additions and 263 deletions.
3 changes: 1 addition & 2 deletions crates/hotshot/src/demo.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,8 @@ use derivative::Derivative;
use hotshot_signature_key::bn254::BLSPubKey;
use hotshot_types::{
block_impl::{VIDBlockHeader, VIDBlockPayload, VIDTransaction},
data::{fake_commitment, ViewNumber},
data::{fake_commitment, BlockError, ViewNumber},
traits::{
block_contents::BlockError,
election::Membership,
node_implementation::NodeType,
state::{ConsensusTime, TestableState},
Expand Down
18 changes: 14 additions & 4 deletions crates/hotshot/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ use hotshot_task::{
use hotshot_task_impls::{events::HotShotEvent, network::NetworkTaskKind};

use hotshot_types::{
consensus::{BlockPayloadStore, Consensus, ConsensusMetricsValue, View, ViewInner, ViewQueue},
consensus::{Consensus, ConsensusMetricsValue, PayloadStore, View, ViewInner, ViewQueue},
data::Leaf,
error::StorageSnafu,
message::{
Expand All @@ -67,6 +67,7 @@ use hotshot_types::{
signature_key::SignatureKey,
state::ConsensusTime,
storage::StoredView,
BlockPayload,
},
HotShotConfig,
};
Expand Down Expand Up @@ -224,10 +225,19 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>> SystemContext<TYPES, I> {
);

let mut saved_leaves = HashMap::new();
let mut saved_block_payloads = BlockPayloadStore::default();
let mut saved_payloads = PayloadStore::default();
saved_leaves.insert(anchored_leaf.commit(), anchored_leaf.clone());
let payload_commitment = anchored_leaf.get_payload_commitment();
if let Some(payload) = anchored_leaf.get_block_payload() {
saved_block_payloads.insert(payload);
let encoded_txns = match payload.encode() {
// TODO (Keyao) [VALIDATED_STATE] - Avoid collect/copy on the encoded transaction bytes.
// <https://github.com/EspressoSystems/HotShot/issues/2115>
Ok(encoded) => encoded.into_iter().collect(),
Err(e) => {
return Err(HotShotError::BlockError { source: e });
}
};
saved_payloads.insert(payload_commitment, encoded_txns);
}

let start_view = anchored_leaf.get_view_number();
Expand All @@ -237,7 +247,7 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>> SystemContext<TYPES, I> {
cur_view: start_view,
last_decided_view: anchored_leaf.get_view_number(),
saved_leaves,
saved_block_payloads,
saved_payloads,
// TODO this is incorrect
// https://github.com/EspressoSystems/HotShot/issues/560
locked_view: anchored_leaf.get_view_number(),
Expand Down
6 changes: 4 additions & 2 deletions crates/hotshot/src/tasks/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,6 @@

use crate::{async_spawn, types::SystemContextHandle, HotShotConsensusApi};
use async_compatibility_layer::art::async_sleep;
use commit::Committable;
use futures::FutureExt;
use hotshot_task::{
boxed_sync,
Expand All @@ -28,6 +27,7 @@ use hotshot_types::{
event::Event,
message::Messages,
traits::{
block_contents::vid_commitment,
consensus_api::ConsensusSharedApi,
network::{CommunicationChannel, ConsensusIntentEvent, TransmitType},
node_implementation::{NodeImplementation, NodeType},
Expand Down Expand Up @@ -205,13 +205,15 @@ pub async fn add_consensus_task<TYPES: NodeType, I: NodeImplementation<TYPES>>(
};
let registry = task_runner.registry.clone();
let (payload, metadata) = <TYPES::BlockPayload as BlockPayload>::genesis();
// Impossible for `unwrap` to fail on the genesis payload.
let payload_commitment = vid_commitment(payload.encode().unwrap().collect());
// build the consensus task
let consensus_state = ConsensusTaskState {
registry: registry.clone(),
consensus,
timeout: handle.hotshot.inner.config.next_view_timeout,
cur_view: TYPES::Time::new(0),
payload_commitment_and_metadata: Some((payload.commit(), metadata)),
payload_commitment_and_metadata: Some((payload_commitment, metadata)),
api: c_api.clone(),
_pd: PhantomData,
vote_collector: None,
Expand Down
2 changes: 1 addition & 1 deletion crates/hotshot/src/traits/storage/memory_storage.rs
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ mod test {
let payload = VIDBlockPayload::genesis();
let header = VIDBlockHeader {
block_number: 0,
payload_commitment: payload.commit(),
payload_commitment: payload.payload_commitment,
};
let dummy_leaf_commit = fake_commitment::<Leaf<DummyTypes>>();
let data = hotshot_types::simple_vote::QuorumData {
Expand Down
17 changes: 10 additions & 7 deletions crates/task-impls/src/consensus.rs
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@ use async_lock::{RwLock, RwLockUpgradableReadGuard};
#[cfg(async_executor_impl = "async-std")]
use async_std::task::JoinHandle;
use bitvec::prelude::*;
use commit::{Commitment, Committable};
use commit::Committable;
use core::time::Duration;
use either::Either;
use futures::FutureExt;
Expand All @@ -17,7 +17,7 @@ use hotshot_task::{
};
use hotshot_types::{
consensus::{Consensus, View},
data::{Leaf, QuorumProposal},
data::{Leaf, QuorumProposal, VidCommitment},
event::{Event, EventType},
message::{GeneralConsensusMessage, Proposal},
simple_certificate::{DACertificate, QuorumCertificate, TimeoutCertificate, VIDCertificate},
Expand Down Expand Up @@ -52,7 +52,7 @@ use tracing::{debug, error, info, instrument};
pub struct ConsensusTaskError {}

/// Alias for the block payload commitment and the associated metadata.
type CommitmentAndMetadata<PAYLOAD> = (Commitment<PAYLOAD>, <PAYLOAD as BlockPayload>::Metadata);
type CommitmentAndMetadata<PAYLOAD> = (VidCommitment, <PAYLOAD as BlockPayload>::Metadata);

/// The state for the consensus task. Contains all of the information for the implementation
/// of consensus
Expand Down Expand Up @@ -735,11 +735,14 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +

// If the block payload is available for this leaf, include it in
// the leaf chain that we send to the client.
if let Some(payload) = consensus
.saved_block_payloads
.get(leaf.get_payload_commitment())
if let Some(encoded_txns) =
consensus.saved_payloads.get(leaf.get_payload_commitment())
{
if let Err(e) = leaf.fill_block_payload(payload.clone()) {
let payload = BlockPayload::from_bytes(
encoded_txns.clone().into_iter(),
leaf.get_block_header().metadata(),
);
if let Err(e) = leaf.fill_block_payload(payload) {
error!(
"Saved block payload and commitment don't match: {:?}",
e
Expand Down
28 changes: 10 additions & 18 deletions crates/task-impls/src/da.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@ use async_compatibility_layer::art::async_spawn;
use async_lock::RwLock;

use bitvec::prelude::*;
use commit::Committable;
use either::{Either, Left, Right};
use futures::FutureExt;
use hotshot_task::{
Expand All @@ -19,13 +18,13 @@ use hotshot_types::{
message::Proposal,
simple_vote::{DAData, DAVote},
traits::{
block_contents::vid_commitment,
consensus_api::ConsensusApi,
election::Membership,
network::{CommunicationChannel, ConsensusIntentEvent},
node_implementation::{NodeImplementation, NodeType},
signature_key::SignatureKey,
state::ConsensusTime,
BlockPayload,
},
utils::ViewInner,
vote::HasViewNumber,
Expand Down Expand Up @@ -196,11 +195,7 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +
return None;
}

debug!(
"Got a DA block with {} transactions!",
proposal.data.block_payload.transaction_commitments().len()
);
let payload_commitment = proposal.data.block_payload.commit();
let payload_commitment = vid_commitment(proposal.data.encoded_transactions.clone());

// ED Is this the right leader?
let view_leader_key = self.da_membership.get_leader(view);
Expand Down Expand Up @@ -244,15 +239,13 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +
// there is already a view there: the replica task may have inserted a `Leaf` view which
// contains strictly more information.
consensus.state_map.entry(view).or_insert(View {
view_inner: ViewInner::DA {
block: payload_commitment,
},
view_inner: ViewInner::DA { payload_commitment },
});

// Record the block payload we have promised to make available.
// Record the payload we have promised to make available.
consensus
.saved_block_payloads
.insert(proposal.data.block_payload);
.saved_payloads
.insert(payload_commitment, proposal.data.encoded_transactions);
}
HotShotEvent::DAVoteRecv(vote) => {
debug!("DA vote recv, Main Task {:?}", vote.get_view_number(),);
Expand Down Expand Up @@ -371,18 +364,17 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +

return None;
}
HotShotEvent::BlockReady(payload, metadata, view) => {
HotShotEvent::BlockReady(encoded_transactions, metadata, view) => {
self.da_network
.inject_consensus_info(ConsensusIntentEvent::CancelPollForTransactions(*view))
.await;

let payload_commitment = payload.commit();
let payload_commitment = vid_commitment(encoded_transactions.clone());
let signature =
TYPES::SignatureKey::sign(&self.private_key, payload_commitment.as_ref());
// TODO (Keyao) Fix the payload sending and receiving for the DA proposal.
// <https://github.com/EspressoSystems/HotShot/issues/2026>
let data: DAProposal<TYPES> = DAProposal {
block_payload: payload.clone(),
encoded_transactions,
metadata: metadata.clone(),
// Upon entering a new view we want to send a DA Proposal for the next view -> Is it always the case that this is cur_view + 1?
view_number: view,
};
Expand Down
7 changes: 3 additions & 4 deletions crates/task-impls/src/events.rs
Original file line number Diff line number Diff line change
@@ -1,9 +1,8 @@
use crate::view_sync::ViewSyncPhase;

use commit::Commitment;
use either::Either;
use hotshot_types::{
data::{DAProposal, Leaf, QuorumProposal, VidDisperse},
data::{DAProposal, Leaf, QuorumProposal, VidCommitment, VidDisperse},
message::Proposal,
simple_certificate::{
DACertificate, QuorumCertificate, TimeoutCertificate, VIDCertificate,
Expand Down Expand Up @@ -90,12 +89,12 @@ pub enum HotShotEvent<TYPES: NodeType> {
TransactionSend(TYPES::Transaction, TYPES::SignatureKey),
/// Event to send block payload commitment and metadata from DA leader to the quorum; internal event only
SendPayloadCommitmentAndMetadata(
Commitment<TYPES::BlockPayload>,
VidCommitment,
<TYPES::BlockPayload as BlockPayload>::Metadata,
),
/// Event when the transactions task has a block formed
BlockReady(
TYPES::BlockPayload,
Vec<u8>,
<TYPES::BlockPayload as BlockPayload>::Metadata,
TYPES::Time,
),
Expand Down
17 changes: 7 additions & 10 deletions crates/task-impls/src/transactions.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,11 +13,11 @@ use hotshot_task::{
task_impls::HSTWithEvent,
};
use hotshot_types::{
block_impl::{NUM_CHUNKS, NUM_STORAGE_NODES},
consensus::Consensus,
data::{test_srs, Leaf, VidDisperse, VidScheme, VidSchemeTrait},
message::Proposal,
traits::{
block_contents::{NUM_CHUNKS, NUM_STORAGE_NODES},
consensus_api::ConsensusApi,
election::Membership,
node_implementation::{NodeImplementation, NodeType},
Expand Down Expand Up @@ -220,8 +220,8 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +
return None;
}
};
let encoded_txns = match payload.encode() {
Ok(encoded) => encoded,
let encoded_transactions = match payload.encode() {
Ok(encoded) => encoded.into_iter().collect::<Vec<u8>>(),
Err(e) => {
error!("Failed to encode the block payload: {:?}.", e);
return None;
Expand All @@ -233,15 +233,13 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +
// changes.
// TODO <https://github.com/EspressoSystems/HotShot/issues/1693>
let vid = VidScheme::new(NUM_CHUNKS, NUM_STORAGE_NODES, &srs).unwrap();
let vid_disperse = vid
.disperse(encoded_txns.into_iter().collect::<Vec<u8>>())
.unwrap();
let vid_disperse = vid.disperse(encoded_transactions.clone()).unwrap();

// TODO never clone a block
// https://github.com/EspressoSystems/HotShot/issues/1858
self.event_stream
.publish(HotShotEvent::BlockReady(
payload.clone(),
encoded_transactions,
metadata,
view + 1,
))
Expand All @@ -256,14 +254,13 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +
Proposal {
data: VidDisperse {
view_number: view + 1,
payload_commitment: payload.commit(),
payload_commitment: vid_disperse.commit,
shares: vid_disperse.shares,
common: vid_disperse.common,
},
// TODO (Keyao) This is also signed in DA task.
signature: TYPES::SignatureKey::sign(
&self.private_key,
payload.commit().as_ref(),
&vid_disperse.commit,
),
_pd: PhantomData,
},
Expand Down
6 changes: 2 additions & 4 deletions crates/task-impls/src/vid.rs
Original file line number Diff line number Diff line change
Expand Up @@ -322,14 +322,12 @@ impl<TYPES: NodeType, I: NodeImplementation<TYPES>, A: ConsensusApi<TYPES, I> +
// there is already a view there: the replica task may have inserted a `Leaf` view which
// contains strictly more information.
consensus.state_map.entry(view).or_insert(View {
view_inner: ViewInner::DA {
block: payload_commitment,
},
view_inner: ViewInner::DA { payload_commitment },
});

// Record the block we have promised to make available.
// TODO https://github.com/EspressoSystems/HotShot/issues/1692
// consensus.saved_block_payloads.insert(proposal.data.block_payload);
// consensus.saved_payloads.insert(proposal.data.block_payload);
}
HotShotEvent::VidCertRecv(cert) => {
self.network
Expand Down
20 changes: 9 additions & 11 deletions crates/testing/src/overall_safety_task.rs
Original file line number Diff line number Diff line change
@@ -1,13 +1,5 @@
use commit::Commitment;
use either::Either;
use hotshot_task::{event_stream::EventStream, Merge};
use hotshot_task_impls::events::HotShotEvent;
use std::{
collections::{hash_map::Entry, HashMap, HashSet},
sync::Arc,
};

use async_compatibility_layer::channel::UnboundedStream;
use either::Either;
use futures::FutureExt;
use hotshot::{traits::TestableNodeImplementation, HotShotError};
use hotshot_task::{
Expand All @@ -16,14 +8,20 @@ use hotshot_task::{
task_impls::{HSTWithEventAndMessage, TaskBuilder},
MergeN,
};
use hotshot_task::{event_stream::EventStream, Merge};
use hotshot_task_impls::events::HotShotEvent;
use hotshot_types::{
data::Leaf,
data::{Leaf, VidCommitment},
error::RoundTimedoutState,
event::{Event, EventType},
simple_certificate::QuorumCertificate,
traits::node_implementation::NodeType,
};
use snafu::Snafu;
use std::{
collections::{hash_map::Entry, HashMap, HashSet},
sync::Arc,
};

use crate::{test_launcher::TaskGenerator, test_runner::Node};
pub type StateAndBlock<S, B> = (Vec<S>, Vec<B>);
Expand Down Expand Up @@ -106,7 +104,7 @@ pub struct RoundResult<TYPES: NodeType> {
pub leaf_map: HashMap<Leaf<TYPES>, usize>,

/// block -> # entries decided on that block
pub block_map: HashMap<Commitment<TYPES::BlockPayload>, usize>,
pub block_map: HashMap<VidCommitment, usize>,

/// state -> # entries decided on that state
pub state_map: HashMap<(), usize>,
Expand Down
Loading

0 comments on commit 7f6ca9a

Please sign in to comment.