From fec4b6ed39637db75d742a7bf002eb9b56008e9f Mon Sep 17 00:00:00 2001 From: Romain Ruetschi Date: Mon, 10 Feb 2025 12:15:28 +0100 Subject: [PATCH] feat(code/app/starknet): Adapt Starknet app to latest P2P protos (#819) * Update Starknet protos to their latest version * Adapt Starknet app to new protos * Use height and round as stream id in example app * Use `ConsensusStreamId` proto message for stream id in Starknet app * Fix formatting of stream ids * Update protos for interop * Add `ProposalCommitment` type * Add `BlockInfo` type * Update proposal building and assembly from parts to account for BlockInfo and ProposalCommitment * Update protos * Adapt to latest proto changes * Post-merge fixes * Disable integration tests with proposal mode, keep parts only * Set parts-only in spwan.bash * Add detail to panic when explicit proposal * Fix fmt --------- Co-authored-by: Anca Zamfir --- code/crates/engine/src/util/streaming.rs | 33 ++- code/crates/starknet/host/src/actor.rs | 56 +++-- code/crates/starknet/host/src/block_store.rs | 4 +- code/crates/starknet/host/src/codec.rs | 141 +++++-------- .../crates/starknet/host/src/host/proposal.rs | 159 ++++++-------- .../crates/starknet/host/src/host/starknet.rs | 48 ++--- code/crates/starknet/host/src/host/state.rs | 133 +++++++----- code/crates/starknet/host/src/mempool.rs | 6 +- code/crates/starknet/host/src/spawn.rs | 2 +- code/crates/starknet/host/src/streaming.rs | 11 +- code/crates/starknet/p2p-proto/build.rs | 4 +- .../crates/starknet/p2p-proto/export-proto.sh | 2 +- .../p2p-proto/proto/certificate.proto | 2 +- .../proto/p2p/proto/capabilities.proto | 2 +- .../p2p-proto/proto/p2p/proto/class.proto | 17 +- .../p2p-proto/proto/p2p/proto/common.proto | 23 +- .../p2p-proto/proto/p2p/proto/consensus.proto | 96 --------- .../proto/p2p/proto/consensus/consensus.proto | 125 +++++++++++ .../p2p-proto/proto/p2p/proto/discovery.proto | 2 + .../p2p-proto/proto/p2p/proto/mempool.proto | 30 --- .../proto/p2p/proto/mempool/transaction.proto | 16 ++ .../p2p-proto/proto/p2p/proto/snapshot.proto | 9 +- .../p2p-proto/proto/p2p/proto/streaming.proto | 15 -- .../proto/p2p/proto/sync/class.proto | 18 ++ .../proto/p2p/proto/sync/common.proto | 24 +++ .../proto/p2p/proto/{ => sync}/event.proto | 3 + .../proto/p2p/proto/{ => sync}/header.proto | 21 +- .../proto/p2p/proto/{ => sync}/receipt.proto | 5 +- .../proto/p2p/proto/{ => sync}/state.proto | 3 + .../proto/p2p/proto/sync/transaction.proto | 106 ++++++++++ .../proto/p2p/proto/transaction.proto | 199 +++++------------- .../starknet/p2p-proto/proto/sync.proto | 11 +- code/crates/starknet/p2p-proto/src/lib.rs | 15 ++ code/crates/starknet/p2p-types/src/block.rs | 6 +- .../starknet/p2p-types/src/block_info.rs | 64 ++++++ code/crates/starknet/p2p-types/src/context.rs | 14 +- .../starknet/p2p-types/src/context/impls.rs | 29 ++- code/crates/starknet/p2p-types/src/lib.rs | 14 +- .../crates/starknet/p2p-types/src/proposal.rs | 53 +---- .../p2p-types/src/proposal_commitment.rs | 140 ++++++++++++ .../starknet/p2p-types/src/proposal_part.rs | 89 ++++---- code/crates/starknet/p2p-types/src/signing.rs | 7 + .../p2p-types/src/signing/provider.rs | 34 +-- .../starknet/p2p-types/src/streaming.rs | 30 +-- .../starknet/p2p-types/src/transaction.rs | 28 ++- code/crates/starknet/p2p-types/src/vote.rs | 49 +---- .../test/src/tests/n3f0_consensus_mode.rs | 2 + .../starknet/test/src/tests/value_sync.rs | 2 + .../starknet/test/src/tests/vote_sync.rs | 2 + code/crates/starknet/test/src/tests/wal.rs | 4 + code/crates/test/proto/consensus.proto | 2 +- code/crates/test/src/codec/proto/mod.rs | 10 +- code/examples/channel/src/app.rs | 2 +- code/examples/channel/src/state.rs | 23 +- code/examples/channel/src/streaming.rs | 7 +- code/scripts/spawn.bash | 2 +- code/scripts/spawn.fish | 2 +- 57 files changed, 1089 insertions(+), 867 deletions(-) delete mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/consensus.proto create mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/consensus/consensus.proto delete mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/mempool.proto create mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/mempool/transaction.proto delete mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/streaming.proto create mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/sync/class.proto create mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/sync/common.proto rename code/crates/starknet/p2p-proto/proto/p2p/proto/{ => sync}/event.proto (81%) rename code/crates/starknet/p2p-proto/proto/p2p/proto/{ => sync}/header.proto (84%) rename code/crates/starknet/p2p-proto/proto/p2p/proto/{ => sync}/receipt.proto (91%) rename code/crates/starknet/p2p-proto/proto/p2p/proto/{ => sync}/state.proto (91%) create mode 100644 code/crates/starknet/p2p-proto/proto/p2p/proto/sync/transaction.proto create mode 100644 code/crates/starknet/p2p-types/src/block_info.rs create mode 100644 code/crates/starknet/p2p-types/src/proposal_commitment.rs diff --git a/code/crates/engine/src/util/streaming.rs b/code/crates/engine/src/util/streaming.rs index be79ce643..237b866ea 100644 --- a/code/crates/engine/src/util/streaming.rs +++ b/code/crates/engine/src/util/streaming.rs @@ -1,6 +1,31 @@ -pub type StreamId = u64; +use core::fmt; + +use bytes::Bytes; + pub type Sequence = u64; +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] +pub struct StreamId(Bytes); + +impl StreamId { + pub fn new(bytes: Bytes) -> Self { + Self(bytes) + } + + pub fn to_bytes(&self) -> Bytes { + self.0.clone() + } +} + +impl fmt::Display for StreamId { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + for byte in &self.0 { + write!(f, "{byte:02x}")?; + } + Ok(()) + } +} + #[derive(Clone, Debug, PartialEq, Eq)] pub struct StreamMessage { /// Receivers identify streams by (sender, stream_id). @@ -38,8 +63,8 @@ pub enum StreamContent { /// Serialized content. Data(T), - /// Fin must be set to true. - Fin(bool), + /// Indicates the end of the stream. + Fin, } impl StreamContent { @@ -58,6 +83,6 @@ impl StreamContent { } pub fn is_fin(&self) -> bool { - matches!(self, Self::Fin(true)) + matches!(self, Self::Fin) } } diff --git a/code/crates/starknet/host/src/actor.rs b/code/crates/starknet/host/src/actor.rs index 68a97d531..170a603ad 100644 --- a/code/crates/starknet/host/src/actor.rs +++ b/code/crates/starknet/host/src/actor.rs @@ -21,7 +21,6 @@ use malachitebft_engine::util::streaming::{StreamContent, StreamMessage}; use malachitebft_metrics::Metrics; use malachitebft_sync::RawDecidedValue; -use crate::host::proposal::compute_proposal_signature; use crate::host::state::HostState; use crate::host::{Host as _, StarknetHost}; use crate::mempool::{MempoolMsg, MempoolRef}; @@ -51,10 +50,12 @@ impl Host { std::fs::create_dir_all(&db_dir).map_err(|e| SpawnErr::StartupFailed(e.into()))?; let db_path = db_dir.join("blocks.db"); + let ctx = MockContext::new(); + let (actor_ref, _) = Actor::spawn( None, Self::new(mempool, network, metrics, span), - HostState::new(host, db_path, &mut StdRng::from_entropy()), + HostState::new(ctx, host, db_path, &mut StdRng::from_entropy()), ) .await?; @@ -330,7 +331,7 @@ async fn on_get_value( let (mut rx_part, rx_hash) = state.host.build_new_proposal(height, round, deadline).await; - let stream_id = state.next_stream_id(); + let stream_id = state.stream_id(); let mut sequence = 0; @@ -340,7 +341,11 @@ async fn on_get_value( if state.host.params.value_payload.include_parts() { debug!(%stream_id, %sequence, "Broadcasting proposal part"); - let msg = StreamMessage::new(stream_id, sequence, StreamContent::Data(part.clone())); + let msg = StreamMessage::new( + stream_id.clone(), + sequence, + StreamContent::Data(part.clone()), + ); network.cast(NetworkMsg::PublishProposalPart(msg))?; } @@ -348,7 +353,7 @@ async fn on_get_value( } if state.host.params.value_payload.include_parts() { - let msg = StreamMessage::new(stream_id, sequence, StreamContent::Fin(true)); + let msg = StreamMessage::new(stream_id, sequence, StreamContent::Fin); network.cast(NetworkMsg::PublishProposalPart(msg))?; } @@ -382,14 +387,14 @@ async fn on_get_value( } async fn on_extend_vote( - state: &mut HostState, - height: Height, - round: Round, + _state: &mut HostState, + _height: Height, + _round: Round, _value_id: ValueId, reply_to: RpcReplyPort>, ) -> Result<(), ActorProcessingErr> { - let extension = state.host.generate_vote_extension(height, round); - reply_to.send(extension)?; + // let extension = state.host.generate_vote_extension(height, round); + reply_to.send(None)?; Ok(()) } @@ -430,27 +435,30 @@ async fn on_restream_value( network: &NetworkRef, height: Height, round: Round, - value_id: Hash, + proposal_commitment_hash: Hash, valid_round: Round, - address: Address, + proposer: Address, ) -> Result<(), ActorProcessingErr> { debug!(%height, %round, "Restreaming existing proposal..."); - let mut rx_part = state.host.send_known_proposal(value_id).await; + let mut rx_part = state + .host + .send_known_proposal(proposal_commitment_hash) + .await; - let stream_id = state.next_stream_id(); + let stream_id = state.stream_id(); let init = ProposalInit { height, - proposal_round: round, + round, valid_round, - proposer: address, + proposer, }; - let signature = compute_proposal_signature(&init, &value_id, &state.host.private_key); - let init_part = ProposalPart::Init(init); - let fin_part = ProposalPart::Fin(ProposalFin { signature }); + let fin_part = ProposalPart::Fin(ProposalFin { + proposal_commitment_hash, + }); debug!(%height, %round, "Created new Init part: {init_part:?}"); @@ -459,8 +467,10 @@ async fn on_restream_value( while let Some(part) = rx_part.recv().await { let new_part = match part.part_type() { PartType::Init => init_part.clone(), + PartType::BlockInfo => part, + PartType::Transactions => part, + PartType::ProposalCommitment => part, PartType::Fin => fin_part.clone(), - PartType::Transactions | PartType::BlockProof => part, }; state.host.part_store.store(height, round, new_part.clone()); @@ -468,7 +478,8 @@ async fn on_restream_value( if state.host.params.value_payload.include_parts() { debug!(%stream_id, %sequence, "Broadcasting proposal part"); - let msg = StreamMessage::new(stream_id, sequence, StreamContent::Data(new_part)); + let msg = + StreamMessage::new(stream_id.clone(), sequence, StreamContent::Data(new_part)); network.cast(NetworkMsg::PublishProposalPart(msg))?; @@ -544,7 +555,8 @@ async fn on_received_proposal_part( from: PeerId, reply_to: RpcReplyPort>, ) -> Result<(), ActorProcessingErr> { - // TODO - use state.host.receive_proposal() and move some of the logic below there + // TODO: Use state.host.receive_proposal() and move some of the logic below there + let sequence = part.sequence; let Some(parts) = state.part_streams_map.insert(from, part) else { diff --git a/code/crates/starknet/host/src/block_store.rs b/code/crates/starknet/host/src/block_store.rs index 4f04aecce..91c2ec9fd 100644 --- a/code/crates/starknet/host/src/block_store.rs +++ b/code/crates/starknet/host/src/block_store.rs @@ -16,7 +16,7 @@ use malachitebft_proto::Protobuf; use crate::codec::{self, ProtobufCodec}; use crate::proto::{self as proto, Error as ProtoError}; use crate::types::MockContext; -use crate::types::{Block, BlockHash, Height, Transaction, Transactions}; +use crate::types::{Block, BlockHash, Height, Transaction, TransactionBatch}; mod keys; use keys::{HeightKey, UndecidedValueKey}; @@ -276,7 +276,7 @@ impl BlockStore { block: Block { height: certificate.height, block_hash: certificate.value_id, - transactions: Transactions::new(txes.to_vec()), + transactions: TransactionBatch::new(txes.to_vec()), }, certificate: certificate.clone(), }; diff --git a/code/crates/starknet/host/src/codec.rs b/code/crates/starknet/host/src/codec.rs index 9c1bb19aa..a813f88e8 100644 --- a/code/crates/starknet/host/src/codec.rs +++ b/code/crates/starknet/host/src/codec.rs @@ -1,10 +1,11 @@ use bytes::Bytes; +use malachitebft_app::streaming::StreamId; +use malachitebft_starknet_p2p_types::Signature; use prost::Message; use malachitebft_codec::Codec; use malachitebft_core_types::{ - AggregatedSignature, CommitCertificate, CommitSignature, Round, SignedExtension, - SignedProposal, SignedVote, Validity, + AggregatedSignature, CommitCertificate, CommitSignature, Round, SignedVote, Validity, }; use malachitebft_engine::util::streaming::{StreamContent, StreamMessage}; use malachitebft_sync::{ @@ -12,9 +13,7 @@ use malachitebft_sync::{ }; use malachitebft_core_consensus::{PeerId, ProposedValue, SignedConsensusMsg}; -use malachitebft_starknet_p2p_proto::ConsensusMessage; -use crate::proto::consensus_message::Messages; use crate::proto::{self as proto, Error as ProtoError, Protobuf}; use crate::types::{self as p2p, Address, BlockHash, Height, MockContext, ProposalPart, Vote}; @@ -69,42 +68,42 @@ impl Codec for ProtobufCodec { } } -pub fn decode_extension(ext: proto::Extension) -> Result, ProtoError> { - let signature = ext - .signature - .ok_or_else(|| ProtoError::missing_field::("signature")) - .and_then(p2p::Signature::from_proto)?; - - Ok(SignedExtension::new(ext.data, signature)) -} - -pub fn encode_extension( - ext: &SignedExtension, -) -> Result { - Ok(proto::Extension { - data: ext.message.clone(), - signature: Some(ext.signature.to_proto()?), - }) -} - -impl Codec> for ProtobufCodec { - type Error = ProtoError; - - fn decode(&self, bytes: Bytes) -> Result, Self::Error> { - decode_extension(proto::Extension::decode(bytes)?) - } - - fn encode(&self, msg: &SignedExtension) -> Result { - encode_extension(msg).map(|proto| proto.encode_to_bytes()) - } -} +// impl Codec> for ProtobufCodec { +// type Error = ProtoError; +// +// fn decode(&self, bytes: Bytes) -> Result, Self::Error> { +// decode_extension(proto::Extension::decode(bytes)?) +// } +// +// fn encode(&self, msg: &SignedExtension) -> Result { +// encode_extension(msg).map(|proto| proto.encode_to_bytes()) +// } +// } +// +// pub fn decode_extension(ext: proto::Extension) -> Result, ProtoError> { +// let signature = ext +// .signature +// .ok_or_else(|| ProtoError::missing_field::("signature")) +// .and_then(p2p::Signature::from_proto)?; +// +// Ok(SignedExtension::new(ext.data, signature)) +// } +// +// pub fn encode_extension( +// ext: &SignedExtension, +// ) -> Result { +// Ok(proto::Extension { +// data: ext.message.clone(), +// signature: Some(ext.signature.to_proto()?), +// }) +// } pub fn decode_proposed_value( proto: proto::sync::ProposedValue, ) -> Result, ProtoError> { let proposer = proto .proposer - .ok_or_else(|| ProtoError::missing_field::("proposer"))?; + .ok_or_else(|| ProtoError::missing_field::("proposer"))?; Ok(ProposedValue { height: Height::new(proto.block_number, proto.fork_id), @@ -126,10 +125,7 @@ pub fn encode_proposed_value( valid_round: msg.valid_round.as_u32(), value: msg.value.to_bytes()?, proposer: Some(msg.proposer.to_proto()?), - validity: match msg.validity { - Validity::Valid => true, - Validity::Invalid => false, - }, + validity: msg.validity.to_bool(), }; Ok(proto) @@ -351,39 +347,25 @@ impl Codec> for ProtobufCodec { } pub fn decode_consensus_message( - proto: proto::ConsensusMessage, + proto: proto::Vote, ) -> Result, ProtoError> { - let proto_signature = proto - .signature - .ok_or_else(|| ProtoError::missing_field::("signature"))?; - - let message = proto - .messages - .ok_or_else(|| ProtoError::missing_field::("messages"))?; + let vote = Vote::from_proto(proto)?; + let signature = p2p::Signature::dummy(); - let signature = p2p::Signature::from_proto(proto_signature)?; - - match message { - Messages::Vote(v) => { - Vote::from_proto(v).map(|v| SignedConsensusMsg::Vote(SignedVote::new(v, signature))) - } - Messages::Proposal(p) => p2p::Proposal::from_proto(p) - .map(|p| SignedConsensusMsg::Proposal(SignedProposal::new(p, signature))), - } + Ok(SignedConsensusMsg::Vote(SignedVote::new(vote, signature))) } pub fn encode_consensus_message( msg: &SignedConsensusMsg, -) -> Result { +) -> Result { let message = match msg { - SignedConsensusMsg::Vote(v) => proto::ConsensusMessage { - messages: Some(Messages::Vote(v.to_proto()?)), - signature: Some(v.signature.to_proto()?), - }, - SignedConsensusMsg::Proposal(p) => proto::ConsensusMessage { - messages: Some(Messages::Proposal(p.to_proto()?)), - signature: Some(p.signature.to_proto()?), - }, + SignedConsensusMsg::Vote(v) => v.to_proto()?, + SignedConsensusMsg::Proposal(_) => { + panic!("explicit proposal not supported by starknet test application") + } // SignedConsensusMsg::Proposal(p) => proto::ConsensusMessage { + // messages: Some(Messages::Proposal(p.to_proto()?)), + // signature: Some(p.signature.to_proto()?), + // }, }; Ok(message) @@ -393,7 +375,7 @@ impl Codec> for ProtobufCodec { type Error = ProtoError; fn decode(&self, bytes: Bytes) -> Result, Self::Error> { - decode_consensus_message(proto::ConsensusMessage::decode(bytes)?) + decode_consensus_message(proto::Vote::decode(bytes)?) } fn encode(&self, msg: &SignedConsensusMsg) -> Result { @@ -411,24 +393,24 @@ where let p2p_msg = p2p::StreamMessage::from_bytes(&bytes)?; Ok(StreamMessage { - stream_id: p2p_msg.id, + stream_id: StreamId::new(p2p_msg.id), sequence: p2p_msg.sequence, content: match p2p_msg.content { p2p::StreamContent::Data(data) => { StreamContent::Data(T::from_bytes(data.as_ref())?) } - p2p::StreamContent::Fin(fin) => StreamContent::Fin(fin), + p2p::StreamContent::Fin => StreamContent::Fin, }, }) } fn encode(&self, msg: &StreamMessage) -> Result { let p2p_msg = p2p::StreamMessage { - id: msg.stream_id, + id: msg.stream_id.to_bytes(), sequence: msg.sequence, content: match &msg.content { StreamContent::Data(data) => p2p::StreamContent::Data(data.to_bytes()?), - StreamContent::Fin(fin) => p2p::StreamContent::Fin(*fin), + StreamContent::Fin => p2p::StreamContent::Fin, }, }; @@ -602,13 +584,6 @@ pub(crate) fn encode_vote_set( }) } -pub(crate) fn encode_vote(vote: &SignedVote) -> Result { - Ok(ConsensusMessage { - messages: Some(Messages::Vote(vote.message.to_proto()?)), - signature: Some(vote.signature.to_proto()?), - }) -} - pub(crate) fn decode_vote_set( vote_set: proto::sync::VoteSet, ) -> Result, ProtoError> { @@ -621,14 +596,12 @@ pub(crate) fn decode_vote_set( }) } -pub(crate) fn decode_vote(msg: ConsensusMessage) -> Option> { - let signature = msg.signature?; - let vote = match msg.messages { - Some(Messages::Vote(v)) => Some(v), - _ => None, - }?; +pub(crate) fn encode_vote(vote: &SignedVote) -> Result { + vote.message.to_proto() +} - let signature = p2p::Signature::from_proto(signature).ok()?; - let vote = Vote::from_proto(vote).ok()?; +pub(crate) fn decode_vote(msg: proto::Vote) -> Option> { + let signature = Signature::dummy(); + let vote = Vote::from_proto(msg).ok()?; Some(SignedVote::new(vote, signature)) } diff --git a/code/crates/starknet/host/src/host/proposal.rs b/code/crates/starknet/host/src/host/proposal.rs index ab8efb5fa..23b3fa9b6 100644 --- a/code/crates/starknet/host/src/host/proposal.rs +++ b/code/crates/starknet/host/src/host/proposal.rs @@ -1,29 +1,28 @@ #![allow(clippy::too_many_arguments)] use std::sync::Arc; +use std::time::SystemTime; -use bytes::Bytes; use bytesize::ByteSize; use eyre::eyre; -use rand::rngs::StdRng; -use rand::{RngCore, SeedableRng}; -use sha3::Digest; use tokio::sync::{mpsc, oneshot}; use tokio::time::Instant; use tracing::{error, trace}; -use malachitebft_core_types::{Round, VoteExtensions}; +use malachitebft_core_types::Round; use crate::host::starknet::StarknetParams; use crate::mempool::{MempoolMsg, MempoolRef}; use crate::types::*; +const PROTOCOL_VERSION: &str = "0.13.0"; + pub async fn build_proposal_task( height: Height, round: Round, proposer: Address, private_key: PrivateKey, - vote_extensions: VoteExtensions, + // vote_extensions: VoteExtensions, params: StarknetParams, deadline: Instant, mempool: MempoolRef, @@ -35,7 +34,7 @@ pub async fn build_proposal_task( round, proposer, private_key, - vote_extensions, + // vote_extensions, params, deadline, mempool, @@ -52,44 +51,58 @@ async fn run_build_proposal_task( height: Height, round: Round, proposer: Address, - private_key: PrivateKey, - vote_extensions: VoteExtensions, + _private_key: PrivateKey, + // vote_extensions: VoteExtensions, params: StarknetParams, deadline: Instant, mempool: MempoolRef, tx_part: mpsc::Sender, - tx_block_hash: oneshot::Sender, + tx_value_id: oneshot::Sender, ) -> Result<(), Box> { let start = Instant::now(); let build_duration = (deadline - start).mul_f32(params.time_allowance_factor); let mut sequence = 0; let mut block_tx_count = 0; - let mut block_hasher = sha3::Keccak256::new(); - let vote_extensions_size = - (params.vote_extensions.size.as_u64() * vote_extensions.extensions.len() as u64) as usize; - let mut block_size = vote_extensions_size; + let mut block_size = 0; + + trace!(%height, %round, "Building local value"); // Init - let init = { - let init = ProposalInit { + { + let part = ProposalPart::Init(ProposalInit { height, - proposal_round: round, + round, proposer, valid_round: Round::Nil, - }; + }); - tx_part.send(ProposalPart::Init(init.clone())).await?; + tx_part.send(part).await?; sequence += 1; + } - init - }; + let now = SystemTime::UNIX_EPOCH.elapsed().unwrap().as_secs(); + + // Block Info + { + let part = ProposalPart::BlockInfo(BlockInfo { + height, + builder: proposer, + timestamp: now, + l1_gas_price_wei: 0, + l1_data_gas_price_wei: 0, + l2_gas_price_fri: 0, + eth_to_strk_rate: 0, + l1_da_mode: L1DataAvailabilityMode::Blob, + }); + + tx_part.send(part).await?; + sequence += 1; + } let max_block_size = params.max_block_size.as_u64() as usize; 'reap: loop { - trace!(%height, %round, %sequence, "Building local value"); - let reaped_txes = mempool .call( |reply| MempoolMsg::Reap { @@ -136,9 +149,7 @@ async fn run_build_proposal_task( // Transactions { - let part = ProposalPart::Transactions(Transactions::new(txes)); - - block_hasher.update(part.to_sign_bytes()); + let part = ProposalPart::Transactions(TransactionBatch::new(txes)); tx_part.send(part).await?; sequence += 1; } @@ -152,51 +163,39 @@ async fn run_build_proposal_task( } } - // Vote extensions - if !vote_extensions.extensions.is_empty() { - let transactions = vote_extensions - .extensions - .into_iter() - .map(|(_, e)| e.message) - .take_while(|e| { - let keep_going = block_size + e.len() <= max_block_size; - if keep_going { - block_size += e.len(); - block_tx_count += 1; - } - keep_going - }) - .map(Transaction::new) - .collect(); - - let part = ProposalPart::Transactions(Transactions::new(transactions)); - - block_hasher.update(part.to_sign_bytes()); - tx_part.send(part).await?; - sequence += 1; - } - - // BlockProof + // Proposal Commitment { - // TODO: Compute actual "proof" - let mut rng = StdRng::from_entropy(); - let mut proof = vec![0; 32]; - rng.fill_bytes(&mut proof); - - let part = ProposalPart::BlockProof(BlockProof::new(vec![Bytes::from(proof)])); + let part = ProposalPart::Commitment(Box::new(ProposalCommitment { + height, + parent_commitment: Hash::new([0; 32]), + builder: proposer, + timestamp: now, + protocol_version: PROTOCOL_VERSION.to_string(), + old_state_root: Hash::new([0; 32]), + state_diff_commitment: Hash::new([0; 32]), + transaction_commitment: Hash::new([0; 32]), + event_commitment: Hash::new([0; 32]), + receipt_commitment: Hash::new([0; 32]), + concatenated_counts: Felt::ONE, + l1_gas_price_fri: 0, + l1_data_gas_price_fri: 0, + l2_gas_price_fri: 0, + l2_gas_used: 0, + l1_da_mode: L1DataAvailabilityMode::Blob, + })); - block_hasher.update(part.to_sign_bytes()); tx_part.send(part).await?; sequence += 1; } - let block_hash = BlockHash::new(block_hasher.finalize().into()); + // TODO: Compute the actual propoosal commitment hash + let proposal_commitment_hash = Hash::new([42; 32]); // Fin { - let signature = compute_proposal_signature(&init, &block_hash, &private_key); - - let part = ProposalPart::Fin(ProposalFin { signature }); + let part = ProposalPart::Fin(ProposalFin { + proposal_commitment_hash, + }); tx_part.send(part).await?; sequence += 1; } @@ -207,13 +206,13 @@ async fn run_build_proposal_task( let block_size = ByteSize::b(block_size as u64); trace!( - tx_count = %block_tx_count, size = %block_size, hash = %block_hash, parts = %sequence, + tx_count = %block_tx_count, size = %block_size, %proposal_commitment_hash, parts = %sequence, "Built block in {:?}", start.elapsed() ); - tx_block_hash - .send(block_hash) - .map_err(|_| "Failed to send block hash")?; + tx_value_id + .send(proposal_commitment_hash) + .map_err(|_| "Failed to send proposal commitment hash")?; Ok(()) } @@ -239,31 +238,3 @@ async fn run_repropose_task( } Ok(()) } - -pub fn compute_proposal_hash(init: &ProposalInit, block_hash: &BlockHash) -> Hash { - use sha3::Digest; - - let mut hasher = sha3::Keccak256::new(); - - // 1. Block number - hasher.update(init.height.block_number.to_be_bytes()); - // 2. Fork id - hasher.update(init.height.fork_id.to_be_bytes()); - // 3. Proposal round - hasher.update(init.proposal_round.as_i64().to_be_bytes()); - // 4. Valid round - hasher.update(init.valid_round.as_i64().to_be_bytes()); - // 5. Block hash - hasher.update(block_hash.as_bytes()); - - Hash::new(hasher.finalize().into()) -} - -pub fn compute_proposal_signature( - init: &ProposalInit, - block_hash: &BlockHash, - private_key: &PrivateKey, -) -> Signature { - let hash = compute_proposal_hash(init, block_hash); - private_key.sign(&hash.as_felt()) -} diff --git a/code/crates/starknet/host/src/host/starknet.rs b/code/crates/starknet/host/src/host/starknet.rs index d00509c58..535ec5831 100644 --- a/code/crates/starknet/host/src/host/starknet.rs +++ b/code/crates/starknet/host/src/host/starknet.rs @@ -1,16 +1,15 @@ -use std::collections::{BTreeSet, HashMap}; +use std::collections::BTreeSet; use std::time::Duration; use async_trait::async_trait; -use bytes::Bytes; use bytesize::ByteSize; use tokio::sync::{mpsc, oneshot}; use tokio::time::Instant; -use tracing::{debug, Instrument}; +use tracing::Instrument; -use malachitebft_config::VoteExtensionsConfig; +// use malachitebft_config::VoteExtensionsConfig; use malachitebft_core_consensus::ValuePayload; -use malachitebft_core_types::{CommitCertificate, Round, SignedVote, VoteExtensions}; +use malachitebft_core_types::{CommitCertificate, Round, SignedVote}; use crate::host::Host; use crate::mempool::MempoolRef; @@ -28,7 +27,7 @@ pub struct StarknetParams { pub time_allowance_factor: f32, pub exec_time_per_tx: Duration, pub max_retain_blocks: usize, - pub vote_extensions: VoteExtensionsConfig, + // pub vote_extensions: VoteExtensionsConfig, } pub struct StarknetHost { @@ -38,7 +37,7 @@ pub struct StarknetHost { pub private_key: PrivateKey, pub validator_set: ValidatorSet, pub part_store: PartStore, - pub vote_extensions: HashMap>, + // pub vote_extensions: HashMap>, } impl StarknetHost { @@ -56,25 +55,25 @@ impl StarknetHost { private_key, validator_set, part_store: Default::default(), - vote_extensions: Default::default(), + // vote_extensions: Default::default(), } } - pub fn generate_vote_extension(&self, _height: Height, _round: Round) -> Option { - use rand::RngCore; - - if !self.params.vote_extensions.enabled { - return None; - } - - let size = self.params.vote_extensions.size.as_u64() as usize; - debug!(%size, "Vote extensions are enabled"); - - let mut bytes = vec![0u8; size]; - rand::thread_rng().fill_bytes(&mut bytes); - - Some(Bytes::from(bytes)) - } + // pub fn generate_vote_extension(&self, _height: Height, _round: Round) -> Option { + // use rand::RngCore; + // + // if !self.params.vote_extensions.enabled { + // return None; + // } + // + // let size = self.params.vote_extensions.size.as_u64() as usize; + // debug!(%size, "Vote extensions are enabled"); + // + // let mut bytes = vec![0u8; size]; + // rand::thread_rng().fill_bytes(&mut bytes); + // + // Some(Bytes::from(bytes)) + // } } #[async_trait] @@ -101,7 +100,7 @@ impl Host for StarknetHost { let (tx_part, rx_content) = mpsc::channel(self.params.txs_per_part); let (tx_block_hash, rx_block_hash) = oneshot::channel(); - let vote_extensions = self.vote_extensions.remove(&height).unwrap_or_default(); + // let vote_extensions = self.vote_extensions.remove(&height).unwrap_or_default(); tokio::spawn( build_proposal_task( @@ -109,7 +108,6 @@ impl Host for StarknetHost { round, self.address, self.private_key, - vote_extensions, self.params, deadline, self.mempool.clone(), diff --git a/code/crates/starknet/host/src/host/state.rs b/code/crates/starknet/host/src/host/state.rs index 4fd300d9b..81d2e617b 100644 --- a/code/crates/starknet/host/src/host/state.rs +++ b/code/crates/starknet/host/src/host/state.rs @@ -2,21 +2,21 @@ use std::path::Path; use std::sync::Arc; use rand::RngCore; -use sha3::Digest; use tracing::{debug, error, trace}; -use malachitebft_core_types::{Round, Validity}; +use malachitebft_core_types::{Context, Round, Validity}; use malachitebft_engine::consensus::ConsensusRef; use malachitebft_engine::host::ProposedValue; use malachitebft_engine::util::streaming::StreamId; +use malachitebft_starknet_p2p_proto as p2p_proto; use crate::block_store::BlockStore; -use crate::host::proposal::compute_proposal_hash; use crate::host::{Host, StarknetHost}; use crate::streaming::PartStreamsMap; use crate::types::*; pub struct HostState { + pub ctx: MockContext, pub height: Height, pub round: Round, pub proposer: Option
, @@ -24,15 +24,21 @@ pub struct HostState { pub consensus: Option>, pub block_store: BlockStore, pub part_streams_map: PartStreamsMap, - pub next_stream_id: StreamId, + pub nonce: u64, } impl HostState { - pub fn new(host: StarknetHost, db_path: impl AsRef, rng: &mut R) -> Self + pub fn new( + ctx: MockContext, + host: StarknetHost, + db_path: impl AsRef, + rng: &mut R, + ) -> Self where R: RngCore, { Self { + ctx, height: Height::new(0, 0), round: Round::Nil, proposer: None, @@ -40,16 +46,21 @@ impl HostState { consensus: None, block_store: BlockStore::new(db_path).unwrap(), part_streams_map: PartStreamsMap::default(), - next_stream_id: rng.next_u64(), + nonce: rng.next_u64(), } } - pub fn next_stream_id(&mut self) -> StreamId { - let stream_id = self.next_stream_id; - // Wrap around if we get to u64::MAX, which may happen if the initial - // stream id was close to it already. - self.next_stream_id = self.next_stream_id.wrapping_add(1); - stream_id + pub fn stream_id(&mut self) -> StreamId { + let stream_id = p2p_proto::ConsensusStreamId { + height: self.height.as_u64(), + round: self.round.as_u32().expect("round is non-nil"), + nonce: self.nonce, + }; + + self.nonce += 1; + + let bytes = prost::Message::encode_to_vec(&stream_id); + StreamId::new(bytes.into()) } #[tracing::instrument(skip_all, fields(%height, %round))] @@ -86,69 +97,75 @@ impl HostState { } let Some(init) = parts.iter().find_map(|part| part.as_init()) else { - error!("No Init part found in the proposal parts"); + error!("Part not found: Init"); return None; }; - let valid_round = init.valid_round; - if valid_round.is_defined() { - debug!("Reassembling a Proposal we might have seen before: {init:?}"); - } - let Some(fin) = parts.iter().find_map(|part| part.as_fin()) else { - error!("No Fin part found in the proposal parts"); + error!("Part not found: Fin"); return None; }; - trace!(parts.len = %parts.len(), "Building proposal content from parts"); - - let block_hash = { - let mut block_hasher = sha3::Keccak256::new(); - for part in parts { - if part.as_init().is_some() || part.as_fin().is_some() { - // NOTE: We do not hash over Init, so restreaming returns the same hash - // NOTE: We do not hash over Fin, because Fin includes a signature over the block hash - // TODO: we should probably still include height - continue; - } - - block_hasher.update(part.to_sign_bytes()); - } + let Some(_block_info) = parts.iter().find_map(|part| part.as_block_info()) else { + error!("Part not found: BlockInfo"); + return None; + }; - BlockHash::new(block_hasher.finalize().into()) + let Some(commitment) = parts.iter().find_map(|part| part.as_commitment()) else { + error!("Part not found: ProposalCommitment"); + return None; }; - trace!(%block_hash, "Computed block hash"); + let validity = self.verify_proposal_validity(init, fin, commitment).await?; - let proposal_hash = compute_proposal_hash(init, &block_hash); + let valid_round = init.valid_round; + if valid_round.is_defined() { + debug!("Reassembling a proposal we might have seen before: {init:?}"); + } - let validity = self - .verify_proposal_validity(init, &proposal_hash, &fin.signature) - .await?; + trace!(parts.len = %parts.len(), "Building proposal content from parts"); - Some((valid_round, block_hash, init.proposer, validity)) + Some(( + valid_round, + fin.proposal_commitment_hash, + init.proposer, + validity, + )) } async fn verify_proposal_validity( &self, init: &ProposalInit, - proposal_hash: &Hash, - signature: &Signature, + _fin: &ProposalFin, + _commitment: &ProposalCommitment, ) -> Option { let validators = self.host.validators(init.height).await?; - let public_key = validators - .iter() - .find(|v| v.address == init.proposer) - .map(|v| v.public_key); - - let Some(public_key) = public_key else { + if !validators.iter().any(|v| v.address == init.proposer) { error!(proposer = %init.proposer, "No validator found for the proposer"); return None; }; - let valid = public_key.verify(&proposal_hash.as_felt(), signature); - Some(Validity::from_bool(valid)) + let validator_set = ValidatorSet::new(validators); + let proposer = self + .ctx + .select_proposer(&validator_set, init.height, init.round); + + if proposer.address != init.proposer { + error!( + height = %init.height, + round = %init.round, + proposer = %init.proposer, + expected = %proposer.address, + "Proposer is not the selected proposer for this height and round" + ); + + return None; + } + + // TODO: Check that the hash of `commitment` matches `fin.proposal_commitment_hash` + + Some(Validity::Valid) } #[tracing::instrument(skip_all, fields( @@ -164,12 +181,12 @@ impl HostState { ) -> Option> { self.host.part_store.store(height, round, part.clone()); - if let ProposalPart::Transactions(_txes) = &part { + if let ProposalPart::Transactions(txes) = &part { debug!("Simulating tx execution and proof verification"); // Simulate Tx execution and proof verification (assumes success) // TODO: Add config knob for invalid blocks - let num_txes = part.tx_count() as u32; + let num_txes = txes.len() as u32; let exec_time = self.host.params.exec_time_per_tx * num_txes; tokio::time::sleep(exec_time).await; @@ -186,7 +203,17 @@ impl HostState { // TODO: Do more validations, e.g. there is no higher tx proposal part, // check that we have received the proof, etc. let Some(_fin) = parts.iter().find_map(|part| part.as_fin()) else { - debug!("Final proposal part has not been received yet"); + debug!("Proposal part has not been received yet: Fin"); + return None; + }; + + let Some(_block_info) = parts.iter().find_map(|part| part.as_block_info()) else { + debug!("Proposal part has not been received yet: BlockInfo"); + return None; + }; + + let Some(_proposal_commitment) = parts.iter().find_map(|part| part.as_commitment()) else { + debug!("Proposal part has not been received yet: ProposalCommitment"); return None; }; diff --git a/code/crates/starknet/host/src/mempool.rs b/code/crates/starknet/host/src/mempool.rs index 6f10b057a..798ab6747 100644 --- a/code/crates/starknet/host/src/mempool.rs +++ b/code/crates/starknet/host/src/mempool.rs @@ -11,7 +11,7 @@ use malachitebft_test_mempool::types::MempoolTransactionBatch; use malachitebft_test_mempool::{Event as NetworkEvent, NetworkMsg, PeerId}; use crate::proto::Protobuf; -use crate::types::{Hash, Transaction, Transactions}; +use crate::types::{Hash, Transaction, TransactionBatch}; pub mod network; use network::{MempoolNetworkMsg, MempoolNetworkRef}; @@ -146,7 +146,7 @@ impl Mempool { ) -> Result<(), ractor::ActorProcessingErr> { match msg { NetworkMsg::TransactionBatch(batch) => { - let Ok(batch) = Transactions::from_any(&batch.transaction_batch) else { + let Ok(batch) = TransactionBatch::from_any(&batch.transaction_batch) else { // TODO: Log error return Ok(()); }; @@ -261,7 +261,7 @@ fn generate_and_broadcast_txes( let initial_count = transactions.len(); - let mut tx_batch = Transactions::default(); + let mut tx_batch = TransactionBatch::default(); let mut rng = rand::thread_rng(); for _ in initial_count..count { diff --git a/code/crates/starknet/host/src/spawn.rs b/code/crates/starknet/host/src/spawn.rs index 0796c2f0a..2e9a777f8 100644 --- a/code/crates/starknet/host/src/spawn.rs +++ b/code/crates/starknet/host/src/spawn.rs @@ -345,7 +345,7 @@ async fn spawn_host_actor( time_allowance_factor: cfg.test.time_allowance_factor, exec_time_per_tx: cfg.test.exec_time_per_tx, max_retain_blocks: cfg.test.max_retain_blocks, - vote_extensions: cfg.test.vote_extensions, + // vote_extensions: cfg.test.vote_extensions, }; let mock_host = StarknetHost::new( diff --git a/code/crates/starknet/host/src/streaming.rs b/code/crates/starknet/host/src/streaming.rs index dbe05f0da..6295ffd0e 100644 --- a/code/crates/starknet/host/src/streaming.rs +++ b/code/crates/starknet/host/src/streaming.rs @@ -113,8 +113,11 @@ impl PartStreamsMap { peer_id: PeerId, msg: StreamMessage, ) -> Option { - let stream_id = msg.stream_id; - let state = self.streams.entry((peer_id, stream_id)).or_default(); + let stream_id = msg.stream_id.clone(); + let state = self + .streams + .entry((peer_id, stream_id.clone())) + .or_default(); if !state.seen_sequences.insert(msg.sequence) { // We have already seen a message with this sequence number. @@ -148,7 +151,7 @@ impl PartStreamsMap { Some(ProposalParts { height: init_info.height, - round: init_info.proposal_round, + round: init_info.round, proposer: init_info.proposer, parts: to_emit, }) @@ -176,7 +179,7 @@ impl PartStreamsMap { Some(ProposalParts { height: init_info.height, - round: init_info.proposal_round, + round: init_info.round, proposer: init_info.proposer, parts: to_emit, }) diff --git a/code/crates/starknet/p2p-proto/build.rs b/code/crates/starknet/p2p-proto/build.rs index 6eb174db3..98d316e8d 100644 --- a/code/crates/starknet/p2p-proto/build.rs +++ b/code/crates/starknet/p2p-proto/build.rs @@ -3,10 +3,8 @@ fn main() -> Result<(), Box> { "./proto/sync.proto", "./proto/certificate.proto", "./proto/p2p/proto/common.proto", - "./proto/p2p/proto/header.proto", "./proto/p2p/proto/transaction.proto", - "./proto/p2p/proto/consensus.proto", - "./proto/p2p/proto/streaming.proto", + "./proto/p2p/proto/consensus/consensus.proto", ]; for proto in protos { diff --git a/code/crates/starknet/p2p-proto/export-proto.sh b/code/crates/starknet/p2p-proto/export-proto.sh index 5fbe4470c..b39eae681 100755 --- a/code/crates/starknet/p2p-proto/export-proto.sh +++ b/code/crates/starknet/p2p-proto/export-proto.sh @@ -2,7 +2,7 @@ SCRIPT_PATH="$(dirname "$(realpath "$0")")" -ref="5d5fdc5983224cc594828442cd61cfe2" +ref="72dda96ebb88492581b1bb2591fd2314" output="$SCRIPT_PATH/proto" echo "Exporting proto files from 'buf.build/romac/starknet-p2p:$ref' to '$output'..." diff --git a/code/crates/starknet/p2p-proto/proto/certificate.proto b/code/crates/starknet/p2p-proto/proto/certificate.proto index c2ef1e5a0..5e069c366 100644 --- a/code/crates/starknet/p2p-proto/proto/certificate.proto +++ b/code/crates/starknet/p2p-proto/proto/certificate.proto @@ -3,7 +3,7 @@ syntax = "proto3"; package certificate; import "p2p/proto/common.proto"; -import "p2p/proto/consensus.proto"; +import "p2p/proto/consensus/consensus.proto"; import "p2p/proto/transaction.proto"; message CommitSignature { diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/capabilities.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/capabilities.proto index 9566262f9..caa8e41d4 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/capabilities.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/capabilities.proto @@ -1,6 +1,6 @@ syntax = "proto3"; -import "p2p/proto/common.proto"; +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/capabilities"; // A capability for one of the following protocols: // 1. /starknet/headers/ diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/class.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/class.proto index 244551685..4fc1be532 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/class.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/class.proto @@ -1,6 +1,7 @@ syntax = "proto3"; import "p2p/proto/common.proto"; +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/class"; message EntryPoint { Felt252 selector = 1; @@ -42,19 +43,3 @@ message Class { uint32 domain = 3; Hash class_hash = 4; } - -message Classes { - repeated Class classes = 1; -} - -message ClassesRequest { - Iteration iteration = 1; -} - -// Responses are sent ordered by the order given in the request. -message ClassesResponse { - oneof class_message { - Class class = 1; - Fin fin = 2; // Fin is sent after the peer sent all the data or when it encountered a block that it doesn't have its classes. - } -} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/common.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/common.proto index 1ab04a1d8..b8fdf7c46 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/common.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/common.proto @@ -1,5 +1,7 @@ syntax = "proto3"; +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/common"; + message Felt252 { bytes elements = 1; } @@ -43,11 +45,6 @@ message Patricia { Hash root = 2; } -message StateDiffCommitment { - uint64 state_diff_length = 1; - Hash root = 2; -} - message BlockID { uint64 number = 1; Hash header = 2; @@ -63,21 +60,9 @@ enum VolitionDomain { L2 = 1; } -message Iteration { - enum Direction { - Forward = 0; - Backward = 1; - } - oneof start { - uint64 block_number = 1; - Hash header = 2; - } - Direction direction = 3; - uint64 limit = 4; - uint64 step = 5; // to allow interleaving from several nodes - // bool interleave = 6; // return results in any order of blocks, per block the messages should still be in the order specified +message BlockProof { + repeated bytes proof = 1; } // mark the end of a stream of messages -// TBD: may not be required if we open a stream per request. message Fin {} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/consensus.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/consensus.proto deleted file mode 100644 index 9715307ca..000000000 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/consensus.proto +++ /dev/null @@ -1,96 +0,0 @@ -syntax = "proto3"; - -import "p2p/proto/common.proto"; -import "p2p/proto/header.proto"; -import "p2p/proto/transaction.proto"; - -// WIP - will change - -message Vote { - enum VoteType { - Prevote = 0; - Precommit = 1; - }; - - // We use a type field to distinguish between prevotes and precommits instead of different - // messages, to make sure the data, and therefore the signatures, are unambiguous between - // Prevote and Precommit. - VoteType vote_type = 1; - uint64 block_number = 3; - uint64 fork_id = 4; - uint32 round = 5; - // This is optional since a vote can be NIL. - optional Hash block_hash = 6; - // Identifies the voter. - Address voter = 7; - - /// ADDED - optional Extension extension = 8; -} - -message ProposalInit { - uint64 block_number = 1; - uint64 fork_id = 2; - uint32 proposal_round = 3; - Address proposer = 4; - optional uint32 valid_round = 5; -} - -// Finalize the Tendermint Proposal. When a validator receives this message it will presume that no -// more content for the proposal should be sent. The signature supplied with ProposalFin should be -// for the full Tendermint proposal: -// 1. height -// 2. fork_id -// 3. proposal_round -// 4. valid_round -// 5. block_hash - the validator calculates the block_hash on its own from the content stream and -// confirms the signature with that value. -message ProposalFin { - ConsensusSignature signature = 1; -} - -// The timestamp of a proposal can impact consensus, specifically the lower bound applied. If nodes -// apply a lower bound validation based on their local time, then we risk a scenario where in round -// `R` proposal P is locked. Then in a later round the timestamp in P has gone stale. Therefore the -// lower bound should be "greater than the previous timestamp". Upper bounds don't suffer from this -// problem. -message ProposalPart { - oneof messages { - ProposalInit init = 1; - ProposalFin fin = 2; - // Once block `H` is decided there remains a question; which set of validators receive a - // reward? More specifically, what is the canonical set of precommits for block `H`? Our - // current plan is for the proposer to set the first transaction in `H+1` to be writing the - // list of precommits for `H` to the staking contract in startknet. - Transactions transactions = 3; - BlockProof proof = 4; - } -} - -// ADDED -// TODO: Remove this message -message Proposal { - uint64 fork_id = 1; - uint64 block_number = 2; - uint32 round = 3; - Hash block_hash = 4; - optional uint32 pol_round = 5; - Address proposer = 6; -} - -message ConsensusMessage { - oneof messages { - Vote vote = 1; - Proposal proposal = 2; - } - - // Signature by the initial sender (e.g. proposer, voter) of the message. - ConsensusSignature signature = 3; -} - -// ADDED -message Extension { - bytes data = 1; - ConsensusSignature signature = 2; -} - diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/consensus/consensus.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/consensus/consensus.proto new file mode 100644 index 000000000..f8a83a82a --- /dev/null +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/consensus/consensus.proto @@ -0,0 +1,125 @@ +syntax = "proto3"; + +import "p2p/proto/common.proto"; +import "p2p/proto/transaction.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/consensus/consensus"; + +// WIP - will change + +// Contains all variants of mempool and an L1Handler variant to cover all transactions that can be +// in a new block. +message ConsensusTransaction { + oneof txn { + DeclareV3WithClass declare_v3 = 1; + DeployAccountV3 deploy_account_v3 = 2; + InvokeV3 invoke_v3 = 3; + L1HandlerV0 l1_handler = 4; + bytes dummy = 6; + } + Hash transaction_hash = 5; +} + +message Vote { + enum VoteType { + Prevote = 0; + Precommit = 1; + }; + + // We use a type field to distinguish between prevotes and precommits instead of different + // messages, to make sure the data, and therefore the signatures, are unambiguous between + // Prevote and Precommit. + VoteType vote_type = 1; + uint64 block_number = 3; + uint64 fork_id = 4; + uint32 round = 5; + // This is optional since a vote can be NIL. + optional Hash block_hash = 6; + // Identifies the voter. + Address voter = 7; +} + +// Streaming of proposals is done on the "consensus_proposal" topic. +message ConsensusStreamId { + uint64 height = 1; + uint32 round = 2; + uint64 nonce = 3; +} + +// The timestamp of a proposal can impact consensus, specifically the lower bound applied. If nodes +// apply a lower bound validation based on their local time, then we risk a scenario where in round +// `R` proposal P is locked. Then in a later round the timestamp in P has gone stale. Therefore the +// lower bound should be "greater than the previous timestamp". Upper bounds don't suffer from this +// problem. +message ProposalPart { + oneof messages { + ProposalInit init = 1; + ProposalFin fin = 2; + BlockInfo block_info = 3; + TransactionBatch transactions = 4; + ProposalCommitment commitment = 5; + } +} + +message ProposalInit { + uint64 height = 1; + uint32 round = 2; + optional uint32 valid_round = 3; + Address proposer = 4; +} + +// Identifies the content proposed (and executed). Consensus is reached on the value contained here. +message ProposalFin { + Hash proposal_commitment = 1; +} + +message TransactionBatch { + repeated ConsensusTransaction transactions = 1; +} + +// The content and stream_id are generic fields. The user of the stream can choose to pass whatever +// message that they want. The messages are then encoded in bytes. +message StreamMessage { + oneof message { + bytes content = 1; + Fin fin = 2; + } + bytes stream_id = 3; + uint64 message_id = 4; +} + +message ProposalCommitment { + uint64 block_number = 1; + uint64 fork_id = 2; + Hash parent_commitment = 3; + Address builder = 4; + uint64 timestamp = 5; + string protocol_version = 6; // Starknet version + // State root of block `H-K`, where `K` is defined by the protocol's version. + Hash old_state_root = 7; + // The state diff commitment returned by the Starknet Feeder Gateway + Hash state_diff_commitment = 8; + Hash transaction_commitment = 9; + Hash event_commitment = 10; + Hash receipt_commitment = 11; + // TODO: Just put the sizes explicitly? This is relatively free compared actually hashing the + // lists we commit to. If not, should we also pack other fields? + Felt252 concatenated_counts = 12; + Uint128 l1_gas_price_fri = 13; + Uint128 l1_data_gas_price_fri = 14; + Uint128 l2_gas_price_fri = 15; + Uint128 l2_gas_used = 16; + L1DataAvailabilityMode l1_da_mode = 17; +} + +message BlockInfo { + uint64 block_number = 1; + uint64 fork_id = 2; + Address builder = 3; + uint64 timestamp = 4; + Uint128 l2_gas_price_fri = 5; + Uint128 l1_gas_price_wei = 6; + Uint128 l1_data_gas_price_wei = 7; + Uint128 eth_to_strk_rate = 8; + L1DataAvailabilityMode l1_da_mode = 9; +} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/discovery.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/discovery.proto index 22107658c..d7b3a9846 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/discovery.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/discovery.proto @@ -3,6 +3,8 @@ syntax = "proto3"; import "p2p/proto/common.proto"; import "google/protobuf/descriptor.proto"; +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/discovery"; + extend google.protobuf.MessageOptions { optional bytes powDifficulty = 1001; } diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/mempool.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/mempool.proto deleted file mode 100644 index 3392c7836..000000000 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/mempool.proto +++ /dev/null @@ -1,30 +0,0 @@ -syntax = "proto3"; - -import "p2p/proto/common.proto"; -import "p2p/proto/transaction.proto"; - -// Support also non-validating node that wants to know of the mempool (e.g. to estimate fee in case of first price) -// Result is PooledTransactions+ -message PooledTransactionsRequest -{ - message Known { - oneof known { - Hashes txs = 1; // for mempool of 2000 txs, this will be 64K. Can use Hash32 instead (8K)... - uint64 marker = 2; // since last returned marker. - } - } - optional Known known = 1; -} - -// Can be also a push, similar to NewBlock. So a full node that accepts a new transaction from a wallet -// can propagate it without being pulled -// nodes should track state diffs to know when txs have been included (the contract nonce increases) -message PolledTransactionsResponse { - optional uint64 marker = 1; // optional, if the peer supports that. - bool baseline = 2; // means treat all data as baseline, not diff (may be if 'known' was sent but the mempool was reset/reorged - - oneof responses { - Transactions pending = 3; // if 'known' is given, they will be only txs added after the known - Fin fin = 4; - } -} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/mempool/transaction.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/mempool/transaction.proto new file mode 100644 index 000000000..f1dc0b245 --- /dev/null +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/mempool/transaction.proto @@ -0,0 +1,16 @@ +syntax = "proto3"; + +import "p2p/proto/common.proto"; +import "p2p/proto/transaction.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/mempool/transaction"; + +// Doesn't contain L1Handler, as those don't need to be propagated and can be downloaded from L1. +message MempoolTransaction { + oneof txn { + DeclareV3WithClass declare_v3 = 1; + DeployAccountV3 deploy_account_v3 = 2; + InvokeV3 invoke_v3 = 3; + } + Hash transaction_hash = 4; +} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/snapshot.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/snapshot.proto index 3d3cbbfbc..d812ac496 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/snapshot.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/snapshot.proto @@ -1,9 +1,11 @@ syntax = "proto3"; import "p2p/proto/common.proto"; -import "p2p/proto/state.proto"; +import "p2p/proto/sync/state.proto"; import "p2p/proto/class.proto"; +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/snapshot"; + message PatriciaNode { message Edge { uint32 length = 1; @@ -60,6 +62,11 @@ message ContractRangeResponse { } } +message Classes { + uint32 domain = 1; + repeated Class classes = 2; +} + // duplicate of GetContractRange. Can introduce a 'type' instead. // result is (Classes+, PatriciaRangeProof)* message ClassRangeRequest { diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/streaming.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/streaming.proto deleted file mode 100644 index 79974f7dc..000000000 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/streaming.proto +++ /dev/null @@ -1,15 +0,0 @@ -syntax = "proto3"; - -message Stream { - // Receivers identify streams by (sender, stream_id). This means each node can allocate - // `stream_id`s independently and that many streams can be sent on a single network topic. - uint64 stream_id = 1; - // Identifies the sequence of each message in the stream starting from 0. - uint64 sequence_number = 2; - oneof content { - // Serialized content. - bytes data = 3; - // Fin must be set to true. - bool fin = 4; - } -} \ No newline at end of file diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/class.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/class.proto new file mode 100644 index 000000000..dbbdc4305 --- /dev/null +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/class.proto @@ -0,0 +1,18 @@ +syntax = "proto3"; +import "p2p/proto/class.proto"; +import "p2p/proto/common.proto"; +import "p2p/proto/sync/common.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/class"; + +message ClassesRequest { + Iteration iteration = 1; +} + +// Responses are sent ordered by the order given in the request. +message ClassesResponse { + oneof class_message { + Class class = 1; + Fin fin = 2; // Fin is sent after the peer sent all the data or when it encountered a block that it doesn't have its classes. + } +} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/common.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/common.proto new file mode 100644 index 000000000..2155e302e --- /dev/null +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/common.proto @@ -0,0 +1,24 @@ +syntax = "proto3"; +import "p2p/proto/common.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/common"; + +message StateDiffCommitment { + uint64 state_diff_length = 1; + Hash root = 2; +} + +message Iteration { + enum Direction { + Forward = 0; + Backward = 1; + } + oneof start { + uint64 block_number = 1; + Hash header = 2; + } + Direction direction = 3; + uint64 limit = 4; + uint64 step = 5; // to allow interleaving from several nodes + // bool interleave = 6; // return results in any order of blocks, per block the messages should still be in the order specified +} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/event.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/event.proto similarity index 81% rename from code/crates/starknet/p2p-proto/proto/p2p/proto/event.proto rename to code/crates/starknet/p2p-proto/proto/p2p/proto/sync/event.proto index 89e023917..fb938118d 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/event.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/event.proto @@ -1,5 +1,8 @@ syntax = "proto3"; import "p2p/proto/common.proto"; +import "p2p/proto/sync/common.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/event"; message Event { Hash transaction_hash = 1; diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/header.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/header.proto similarity index 84% rename from code/crates/starknet/p2p-proto/proto/p2p/proto/header.proto rename to code/crates/starknet/p2p-proto/proto/p2p/proto/sync/header.proto index 44736d076..6c461c758 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/header.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/header.proto @@ -1,5 +1,8 @@ syntax = "proto3"; import "p2p/proto/common.proto"; +import "p2p/proto/sync/common.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/header"; // Note: commitments may change to be for the previous blocks like comet/tendermint // hash of block header sent to L1 @@ -17,13 +20,15 @@ message SignedBlockHeader { Patricia events = 9; // By order of issuance. TBD: in receipts? Hash receipts = 10; // By order of issuance. This is a patricia root. No need for length because it's the same length as transactions. string protocol_version = 11; // Starknet version - Uint128 gas_price_fri = 12; - Uint128 gas_price_wei = 13; - Uint128 data_gas_price_fri = 14; - Uint128 data_gas_price_wei = 15; - L1DataAvailabilityMode l1_data_availability_mode = 16; + Uint128 l1_gas_price_fri = 12; + Uint128 l1_gas_price_wei = 13; + Uint128 l1_data_gas_price_fri = 14; + Uint128 l1_data_gas_price_wei = 15; + Uint128 l2_gas_price_fri = 16; + Uint128 l2_gas_price_wei = 17; + L1DataAvailabilityMode l1_data_availability_mode = 18; // for now, we assume a small consensus, so this fits in 1M. Else, these will be repeated and extracted from this message. - repeated ConsensusSignature signatures = 17; + repeated ConsensusSignature signatures = 19; // can be more explicit here about the signature structure as this is not part of account abstraction } @@ -48,7 +53,3 @@ message BlockHeadersResponse { Fin fin = 2; // Fin is sent after the peer sent all the data or when it encountered a block that it doesn't have its header. } } - -message BlockProof { - repeated bytes proof = 1; -} \ No newline at end of file diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/receipt.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/receipt.proto similarity index 91% rename from code/crates/starknet/p2p-proto/proto/p2p/proto/receipt.proto rename to code/crates/starknet/p2p-proto/proto/p2p/proto/sync/receipt.proto index 025888d7c..ebd603c6a 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/receipt.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/receipt.proto @@ -1,6 +1,8 @@ syntax = "proto3"; import "p2p/proto/common.proto"; +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/receipt"; + message MessageToL1 { Felt252 from_address = 2; repeated Felt252 payload = 3; @@ -37,7 +39,8 @@ message Receipt { uint32 memory_holes = 3; Felt252 l1_gas = 4; Felt252 l1_data_gas = 5; - Felt252 total_l1_gas = 6; + Felt252 l2_gas = 6; + Felt252 total_l1_gas = 7; } message Common { diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/state.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/state.proto similarity index 91% rename from code/crates/starknet/p2p-proto/proto/p2p/proto/state.proto rename to code/crates/starknet/p2p-proto/proto/p2p/proto/sync/state.proto index ff157d81d..9eb14e2b9 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/state.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/state.proto @@ -1,5 +1,8 @@ syntax = "proto3"; import "p2p/proto/common.proto"; +import "p2p/proto/sync/common.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/state"; // optimized for flat storage, not through a trie (not sharing key prefixes) diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/transaction.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/transaction.proto new file mode 100644 index 000000000..49694085a --- /dev/null +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/sync/transaction.proto @@ -0,0 +1,106 @@ +syntax = "proto3"; +import "p2p/proto/common.proto"; +import "p2p/proto/sync/common.proto"; +import "p2p/proto/sync/receipt.proto"; +import "p2p/proto/transaction.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/sync/transaction"; + +// TBD: can support a flag to return tx hashes only, good for standalone mempool to remove them, +// or any node that keeps track of transaction streaming in the consensus. +message TransactionsRequest { + Iteration iteration = 1; +} + +// Responses are sent ordered by the order given in the request. The order inside each block is +// according to the execution order. +message TransactionsResponse { + oneof transaction_message { + TransactionWithReceipt transaction_with_receipt = 1; + Fin fin = 2; // Fin is sent after the peer sent all the data or when it encountered a block that it doesn't have its transactions. + } +} + +message TransactionWithReceipt { + TransactionInBlock transaction = 1; + Receipt receipt = 2; +} + +message TransactionInBlock { + message DeclareV0WithoutClass { + Address sender = 1; + Felt252 max_fee = 2; + AccountSignature signature = 3; + Hash class_hash = 4; + } + + message DeclareV1WithoutClass { + Address sender = 1; + Felt252 max_fee = 2; + AccountSignature signature = 3; + Hash class_hash = 4; + Felt252 nonce = 5; + } + + message DeclareV2WithoutClass { + Address sender = 1; + Felt252 max_fee = 2; + AccountSignature signature = 3; + Hash class_hash = 4; + Felt252 nonce = 5; + Hash compiled_class_hash = 6; + } + + // see https://external.integration.starknet.io/feeder_gateway/get_transaction?transactionHash=0x41d1f5206ef58a443e7d3d1ca073171ec25fa75313394318fc83a074a6631c3 + message DeclareV3WithoutClass { + DeclareV3Common common = 1; + Hash class_hash = 2; + } + + message Deploy { + Hash class_hash = 1; + Felt252 address_salt = 2; + repeated Felt252 calldata = 3; + uint32 version = 4; + } + + message DeployAccountV1 { + Felt252 max_fee = 1; + AccountSignature signature = 2; + Hash class_hash = 3; + Felt252 nonce = 4; + Felt252 address_salt = 5; + repeated Felt252 calldata = 6; + } + + message InvokeV0 { + Felt252 max_fee = 1; + AccountSignature signature = 2; + Address address = 3; + Felt252 entry_point_selector = 4; + repeated Felt252 calldata = 5; + } + + message InvokeV1 { + Address sender = 1; + Felt252 max_fee = 2; + AccountSignature signature = 3; + repeated Felt252 calldata = 4; + Felt252 nonce = 5; + } + + oneof txn { + DeclareV0WithoutClass declare_v0 = 1; + DeclareV1WithoutClass declare_v1 = 2; + DeclareV2WithoutClass declare_v2 = 3; + DeclareV3WithoutClass declare_v3 = 4; + Deploy deploy = 5; + DeployAccountV1 deploy_account_v1 = 6; + DeployAccountV3 deploy_account_v3 = 7; + InvokeV0 invoke_v0 = 8; + InvokeV1 invoke_v1 = 9; + InvokeV3 invoke_v3 = 10; + L1HandlerV0 l1_handler = 11; + } + Hash transaction_hash = 12; +} diff --git a/code/crates/starknet/p2p-proto/proto/p2p/proto/transaction.proto b/code/crates/starknet/p2p-proto/proto/p2p/proto/transaction.proto index e5c9d2e5d..1ef63c730 100644 --- a/code/crates/starknet/p2p-proto/proto/p2p/proto/transaction.proto +++ b/code/crates/starknet/p2p-proto/proto/p2p/proto/transaction.proto @@ -1,6 +1,8 @@ syntax = "proto3"; import "p2p/proto/common.proto"; -import "p2p/proto/receipt.proto"; +import "p2p/proto/class.proto"; + +option go_package = "github.com/starknet-io/starknet-p2pspecs/p2p/proto/transaction"; message ResourceLimits { Felt252 max_amount = 1; @@ -9,165 +11,66 @@ message ResourceLimits { message ResourceBounds { ResourceLimits l1_gas = 1; - ResourceLimits l2_gas = 2; + // This can be None only in transactions that don't support l2 gas. + // Starting from 0.14.0, MempoolTransaction and ConsensusTransaction shouldn't have None here. + optional ResourceLimits l1_data_gas = 2; + ResourceLimits l2_gas = 3; } message AccountSignature { repeated Felt252 parts = 1; } -// This is a transaction that is already accepted in a block. Once we have a mempool, we will define -// a separate message for BroadcastedTransaction. -message Transaction -{ - message Dummy { - bytes bytes = 1; - } - - message DeclareV0 { - Address sender = 1; - Felt252 max_fee = 2; - AccountSignature signature = 3; - Hash class_hash = 4; - } - - message DeclareV1 { - Address sender = 1; - Felt252 max_fee = 2; - AccountSignature signature = 3; - Hash class_hash = 4; - Felt252 nonce = 5; - } - - message DeclareV2 { - Address sender = 1; - Felt252 max_fee = 2; - AccountSignature signature = 3; - Hash class_hash = 4; - Felt252 nonce = 5; - Hash compiled_class_hash = 6; - } - - // see https://external.integration.starknet.io/feeder_gateway/get_transaction?transactionHash=0x41d1f5206ef58a443e7d3d1ca073171ec25fa75313394318fc83a074a6631c3 - message DeclareV3 { - Address sender = 1; - AccountSignature signature = 2; - Hash class_hash = 3; - Felt252 nonce = 4; - Hash compiled_class_hash = 5; - ResourceBounds resource_bounds = 6; - uint64 tip = 7; - repeated Felt252 paymaster_data = 8; - repeated Felt252 account_deployment_data = 9; - VolitionDomain nonce_data_availability_mode = 10; - VolitionDomain fee_data_availability_mode = 11; - } - - message Deploy { - Hash class_hash = 1; - Felt252 address_salt = 2; - repeated Felt252 calldata = 3; - uint32 version = 4; - } - - message DeployAccountV1 { - Felt252 max_fee = 1; - AccountSignature signature = 2; - Hash class_hash = 3; - Felt252 nonce = 4; - Felt252 address_salt = 5; - repeated Felt252 calldata = 6; - } - - // see https://external.integration.starknet.io/feeder_gateway/get_transaction?transactionHash=0x29fd7881f14380842414cdfdd8d6c0b1f2174f8916edcfeb1ede1eb26ac3ef0 - message DeployAccountV3 { - AccountSignature signature = 1; - Hash class_hash = 2; - Felt252 nonce = 3; - Felt252 address_salt = 4; - repeated Felt252 calldata = 5; - ResourceBounds resource_bounds = 6; - uint64 tip = 7; - repeated Felt252 paymaster_data = 8; - VolitionDomain nonce_data_availability_mode = 9; - VolitionDomain fee_data_availability_mode = 10; - } - - message InvokeV0 { - Felt252 max_fee = 1; - AccountSignature signature = 2; - Address address = 3; - Felt252 entry_point_selector = 4; - repeated Felt252 calldata = 5; - } - - message InvokeV1 { - Address sender = 1; - Felt252 max_fee = 2; - AccountSignature signature = 3; - repeated Felt252 calldata = 4; - Felt252 nonce = 5; - } - - // see https://external.integration.starknet.io/feeder_gateway/get_transaction?transactionHash=0x41906f1c314cca5f43170ea75d3b1904196a10101190d2b12a41cc61cfd17c - message InvokeV3 { - Address sender = 1; - AccountSignature signature = 2; - repeated Felt252 calldata = 3; - ResourceBounds resource_bounds = 4; - uint64 tip = 5; - repeated Felt252 paymaster_data = 6; - repeated Felt252 account_deployment_data = 7; - VolitionDomain nonce_data_availability_mode = 8; - VolitionDomain fee_data_availability_mode = 9; - Felt252 nonce = 10; - } - - message L1HandlerV0 { - Felt252 nonce = 1; - Address address = 2; - Felt252 entry_point_selector = 3; - repeated Felt252 calldata = 4; - } - - oneof txn { - Dummy dummy = 42; - - DeclareV0 declare_v0 = 1; - DeclareV1 declare_v1 = 2; - DeclareV2 declare_v2 = 3; - DeclareV3 declare_v3 = 4; - Deploy deploy = 5; - DeployAccountV1 deploy_account_v1 = 6; - DeployAccountV3 deploy_account_v3 = 7; - InvokeV0 invoke_v0 = 8; - InvokeV1 invoke_v1 = 9; - InvokeV3 invoke_v3 = 10; - L1HandlerV0 l1_handler = 11; - } - Hash transaction_hash = 12; +message L1HandlerV0 { + Felt252 nonce = 1; + Address address = 2; + Felt252 entry_point_selector = 3; + repeated Felt252 calldata = 4; } -message TransactionWithReceipt { - Transaction transaction = 1; - Receipt receipt = 2; +message DeclareV3Common { + Address sender = 1; + AccountSignature signature = 2; + Felt252 nonce = 4; + Hash compiled_class_hash = 5; + ResourceBounds resource_bounds = 6; + uint64 tip = 7; + repeated Felt252 paymaster_data = 8; + repeated Felt252 account_deployment_data = 9; + VolitionDomain nonce_data_availability_mode = 10; + VolitionDomain fee_data_availability_mode = 11; } -// TBD: can support a flag to return tx hashes only, good for standalone mempool to remove them, -// or any node that keeps track of transaction streaming in the consensus. -message TransactionsRequest { - Iteration iteration = 1; +message DeclareV3WithClass { + DeclareV3Common common = 1; + Cairo1Class class = 2; } -// Responses are sent ordered by the order given in the request. The order inside each block is -// according to the execution order. -message TransactionsResponse { - oneof transaction_message { - TransactionWithReceipt transaction_with_receipt = 1; - Fin fin = 2; // Fin is sent after the peer sent all the data or when it encountered a block that it doesn't have its transactions. - } + +// see https://external.integration.starknet.io/feeder_gateway/get_transaction?transactionHash=0x41906f1c314cca5f43170ea75d3b1904196a10101190d2b12a41cc61cfd17c +message InvokeV3 { + Address sender = 1; + AccountSignature signature = 2; + repeated Felt252 calldata = 3; + ResourceBounds resource_bounds = 4; + uint64 tip = 5; + repeated Felt252 paymaster_data = 6; + repeated Felt252 account_deployment_data = 7; + VolitionDomain nonce_data_availability_mode = 8; + VolitionDomain fee_data_availability_mode = 9; + Felt252 nonce = 10; } -message Transactions { - repeated Transaction transactions = 1; +// see https://external.integration.starknet.io/feeder_gateway/get_transaction?transactionHash=0x29fd7881f14380842414cdfdd8d6c0b1f2174f8916edcfeb1ede1eb26ac3ef0 +message DeployAccountV3 { + AccountSignature signature = 1; + Hash class_hash = 2; + Felt252 nonce = 3; + Felt252 address_salt = 4; + repeated Felt252 calldata = 5; + ResourceBounds resource_bounds = 6; + uint64 tip = 7; + repeated Felt252 paymaster_data = 8; + VolitionDomain nonce_data_availability_mode = 9; + VolitionDomain fee_data_availability_mode = 10; } diff --git a/code/crates/starknet/p2p-proto/proto/sync.proto b/code/crates/starknet/p2p-proto/proto/sync.proto index 9d28fa103..389099fe6 100644 --- a/code/crates/starknet/p2p-proto/proto/sync.proto +++ b/code/crates/starknet/p2p-proto/proto/sync.proto @@ -3,8 +3,7 @@ syntax = "proto3"; package sync; import "p2p/proto/common.proto"; -import "p2p/proto/consensus.proto"; -import "p2p/proto/transaction.proto"; +import "p2p/proto/consensus/consensus.proto"; message Status { PeerID peer_id = 1; @@ -33,10 +32,14 @@ message SyncedValue { message Block { uint64 fork_id = 1; uint64 block_number = 2; - Transactions transactions = 3; + TransactionBatch transactions = 3; Hash block_hash = 4; } +message Extension { + bytes data = 1; +} + message CommitSignature { // TODO - add flag (no vote, nil, value?) Address validator_address = 1; @@ -79,7 +82,7 @@ message VoteSetResponse { } message VoteSet { - repeated ConsensusMessage signed_votes = 1; + repeated Vote signed_votes = 1; } message SyncRequest { diff --git a/code/crates/starknet/p2p-proto/src/lib.rs b/code/crates/starknet/p2p-proto/src/lib.rs index 05d20e0fe..8e10780b3 100644 --- a/code/crates/starknet/p2p-proto/src/lib.rs +++ b/code/crates/starknet/p2p-proto/src/lib.rs @@ -9,3 +9,18 @@ pub mod sync { pub mod certificate { include!(concat!(env!("OUT_DIR"), "/certificate.rs")); } + +impl From for u128 { + fn from(value: Uint128) -> Self { + value.low as u128 | (value.high as u128) << 64 + } +} + +impl From for Uint128 { + fn from(value: u128) -> Self { + Self { + low: value as u64, + high: (value >> 64) as u64, + } + } +} diff --git a/code/crates/starknet/p2p-types/src/block.rs b/code/crates/starknet/p2p-types/src/block.rs index 2be2dfd0d..3ee8e7a12 100644 --- a/code/crates/starknet/p2p-types/src/block.rs +++ b/code/crates/starknet/p2p-types/src/block.rs @@ -1,4 +1,4 @@ -use crate::{BlockHash, Height, Transactions}; +use crate::{BlockHash, Height, TransactionBatch}; use malachitebft_proto::{Error as ProtoError, Protobuf}; use malachitebft_starknet_p2p_proto as proto; @@ -6,7 +6,7 @@ use malachitebft_starknet_p2p_proto as proto; #[derive(Clone, Debug)] pub struct Block { pub height: Height, - pub transactions: Transactions, + pub transactions: TransactionBatch, pub block_hash: BlockHash, } @@ -24,7 +24,7 @@ impl Protobuf for Block { Ok(Self { height: Height::new(proto.block_number, proto.fork_id), - transactions: Transactions::from_proto(transactions)?, + transactions: TransactionBatch::from_proto(transactions)?, block_hash: BlockHash::from_proto(block_hash)?, }) } diff --git a/code/crates/starknet/p2p-types/src/block_info.rs b/code/crates/starknet/p2p-types/src/block_info.rs new file mode 100644 index 000000000..ff22adbd3 --- /dev/null +++ b/code/crates/starknet/p2p-types/src/block_info.rs @@ -0,0 +1,64 @@ +use malachitebft_proto::{Error as ProtoError, Protobuf}; +use malachitebft_starknet_p2p_proto::{self as p2p_proto}; + +use crate::proposal_commitment::L1DataAvailabilityMode; +use crate::{Address, Height}; + +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct BlockInfo { + pub height: Height, + pub builder: Address, + pub timestamp: u64, + pub l1_gas_price_wei: u128, + pub l1_data_gas_price_wei: u128, + pub l2_gas_price_fri: u128, + pub eth_to_strk_rate: u128, + pub l1_da_mode: L1DataAvailabilityMode, +} + +impl Protobuf for BlockInfo { + type Proto = p2p_proto::BlockInfo; + + fn from_proto(proto: Self::Proto) -> Result { + Ok(Self { + height: Height::new(proto.block_number, proto.fork_id), + builder: Address::from_proto( + proto + .builder + .ok_or_else(|| ProtoError::missing_field::("builder"))?, + )?, + timestamp: proto.timestamp, + l1_gas_price_wei: proto + .l1_gas_price_wei + .ok_or_else(|| ProtoError::missing_field::("l1_gas_price_wei"))? + .into(), + l1_data_gas_price_wei: proto + .l1_data_gas_price_wei + .ok_or_else(|| ProtoError::missing_field::("l1_data_gas_price_wei"))? + .into(), + l2_gas_price_fri: proto + .l2_gas_price_fri + .ok_or_else(|| ProtoError::missing_field::("l2_gas_price_fri"))? + .into(), + eth_to_strk_rate: proto + .eth_to_strk_rate + .ok_or_else(|| ProtoError::missing_field::("eth_to_strk_rate"))? + .into(), + l1_da_mode: L1DataAvailabilityMode::from_proto(proto.l1_da_mode)?, + }) + } + + fn to_proto(&self) -> Result { + Ok(Self::Proto { + block_number: self.height.block_number, + fork_id: self.height.fork_id, + builder: Some(self.builder.to_proto()?), + timestamp: self.timestamp, + l2_gas_price_fri: Some(self.l2_gas_price_fri.into()), + l1_gas_price_wei: Some(self.l1_gas_price_wei.into()), + l1_data_gas_price_wei: Some(self.l1_data_gas_price_wei.into()), + eth_to_strk_rate: Some(self.eth_to_strk_rate.into()), + l1_da_mode: self.l1_da_mode.to_proto()?, + }) + } +} diff --git a/code/crates/starknet/p2p-types/src/context.rs b/code/crates/starknet/p2p-types/src/context.rs index a8157b4e1..d655eb114 100644 --- a/code/crates/starknet/p2p-types/src/context.rs +++ b/code/crates/starknet/p2p-types/src/context.rs @@ -2,9 +2,7 @@ use bytes::Bytes; use malachitebft_core_types::{Context, NilOrVal, Round, ValidatorSet as _}; -use crate::{ - Address, BlockHash, Ecdsa, Height, Proposal, ProposalPart, Validator, ValidatorSet, Vote, -}; +use crate::{Address, Ecdsa, Hash, Height, Proposal, ProposalPart, Validator, ValidatorSet, Vote}; mod impls; @@ -24,7 +22,7 @@ impl Context for MockContext { type Proposal = Proposal; type ValidatorSet = ValidatorSet; type Validator = Validator; - type Value = BlockHash; + type Value = Hash; type Vote = Vote; type Extension = Bytes; type SigningScheme = Ecdsa; @@ -53,17 +51,17 @@ impl Context for MockContext { fn new_proposal( height: Height, round: Round, - block_hash: BlockHash, + value_id: Hash, pol_round: Round, address: Address, ) -> Proposal { - Proposal::new(height, round, block_hash, pol_round, address) + Proposal::new(height, round, value_id, pol_round, address) } fn new_prevote( height: Height, round: Round, - value_id: NilOrVal, + value_id: NilOrVal, address: Address, ) -> Vote { Vote::new_prevote(height, round, value_id, address) @@ -72,7 +70,7 @@ impl Context for MockContext { fn new_precommit( height: Height, round: Round, - value_id: NilOrVal, + value_id: NilOrVal, address: Address, ) -> Vote { Vote::new_precommit(height, round, value_id, address) diff --git a/code/crates/starknet/p2p-types/src/context/impls.rs b/code/crates/starknet/p2p-types/src/context/impls.rs index 7a5c39598..f7b0bdb1e 100644 --- a/code/crates/starknet/p2p-types/src/context/impls.rs +++ b/code/crates/starknet/p2p-types/src/context/impls.rs @@ -3,8 +3,8 @@ use malachitebft_core_types::{ }; use crate::{ - Address, BlockHash, Height, MockContext, PartType, Proposal, ProposalPart, PublicKey, - Validator, ValidatorSet, Vote, + Address, Hash, Height, MockContext, PartType, Proposal, ProposalPart, PublicKey, Validator, + ValidatorSet, Vote, }; impl common::ProposalPart for ProposalPart { @@ -26,12 +26,12 @@ impl common::Proposal for Proposal { self.round } - fn value(&self) -> &BlockHash { - &self.block_hash + fn value(&self) -> &Hash { + &self.value_id } - fn take_value(self) -> BlockHash { - self.block_hash + fn take_value(self) -> Hash { + self.value_id } fn pol_round(&self) -> Round { @@ -52,11 +52,11 @@ impl common::Vote for Vote { self.round } - fn value(&self) -> &NilOrVal { + fn value(&self) -> &NilOrVal { &self.block_hash } - fn take_value(self) -> NilOrVal { + fn take_value(self) -> NilOrVal { self.block_hash } @@ -69,18 +69,15 @@ impl common::Vote for Vote { } fn extension(&self) -> Option<&SignedExtension> { - self.extension.as_ref() + None } - fn take_extension(&mut self) -> Option> { - self.extension.take() + fn extend(self, _extension: SignedExtension) -> Self { + self } - fn extend(self, extension: SignedExtension) -> Self { - Self { - extension: Some(extension), - ..self - } + fn take_extension(&mut self) -> Option> { + None } } diff --git a/code/crates/starknet/p2p-types/src/lib.rs b/code/crates/starknet/p2p-types/src/lib.rs index f23090602..067cc53e0 100644 --- a/code/crates/starknet/p2p-types/src/lib.rs +++ b/code/crates/starknet/p2p-types/src/lib.rs @@ -15,11 +15,8 @@ pub use height::Height; mod vote; pub use vote::Vote; -mod proposal; -pub use proposal::Proposal; - mod transaction; -pub use transaction::{Transaction, Transactions}; +pub use transaction::{Transaction, TransactionBatch}; mod validator; pub use validator::Validator; @@ -27,12 +24,21 @@ pub use validator::Validator; mod validator_set; pub use validator_set::ValidatorSet; +mod proposal; +pub use proposal::Proposal; + +mod proposal_commitment; +pub use proposal_commitment::{L1DataAvailabilityMode, ProposalCommitment}; + mod proposal_part; pub use proposal_part::{PartType, ProposalFin, ProposalInit, ProposalPart}; mod block; pub use block::Block; +mod block_info; +pub use block_info::BlockInfo; + mod block_proof; pub use block_proof::BlockProof; diff --git a/code/crates/starknet/p2p-types/src/proposal.rs b/code/crates/starknet/p2p-types/src/proposal.rs index b0b4a148f..0539e76e2 100644 --- a/code/crates/starknet/p2p-types/src/proposal.rs +++ b/code/crates/starknet/p2p-types/src/proposal.rs @@ -1,16 +1,16 @@ -use bytes::Bytes; +// use bytes::Bytes; use malachitebft_core_types::Round; -use malachitebft_proto as proto; -use malachitebft_starknet_p2p_proto as p2p_proto; +// use malachitebft_proto as proto; +// use malachitebft_starknet_p2p_proto as p2p_proto; -use crate::{Address, BlockHash, Height}; +use crate::{Address, Hash, Height}; /// A proposal for a value in a round #[derive(Clone, Debug, PartialEq, Eq)] pub struct Proposal { pub height: Height, pub round: Round, - pub block_hash: BlockHash, + pub value_id: Hash, pub pol_round: Round, pub proposer: Address, } @@ -19,55 +19,16 @@ impl Proposal { pub fn new( height: Height, round: Round, - block_hash: BlockHash, + value_id: Hash, pol_round: Round, proposer: Address, ) -> Self { Self { height, round, - block_hash, + value_id, pol_round, proposer, } } - - pub fn to_sign_bytes(&self) -> Bytes { - proto::Protobuf::to_bytes(self).unwrap() - } -} - -impl proto::Protobuf for Proposal { - type Proto = p2p_proto::Proposal; - - #[cfg_attr(coverage_nightly, coverage(off))] - fn to_proto(&self) -> Result { - Ok(Self::Proto { - block_number: self.height.block_number, - fork_id: self.height.fork_id, - round: self.round.as_u32().expect("round should not be nil"), - block_hash: Some(self.block_hash.to_proto()?), - pol_round: self.pol_round.as_u32(), - proposer: Some(self.proposer.to_proto()?), - }) - } - - #[cfg_attr(coverage_nightly, coverage(off))] - fn from_proto(proto: Self::Proto) -> Result { - Ok(Self { - height: Height::new(proto.block_number, proto.fork_id), - round: Round::new(proto.round), - block_hash: BlockHash::from_proto( - proto - .block_hash - .ok_or_else(|| proto::Error::missing_field::("block_hash"))?, - )?, - pol_round: Round::from(proto.pol_round), - proposer: Address::from_proto( - proto - .proposer - .ok_or_else(|| proto::Error::missing_field::("proposer"))?, - )?, - }) - } } diff --git a/code/crates/starknet/p2p-types/src/proposal_commitment.rs b/code/crates/starknet/p2p-types/src/proposal_commitment.rs new file mode 100644 index 000000000..1a8107dfe --- /dev/null +++ b/code/crates/starknet/p2p-types/src/proposal_commitment.rs @@ -0,0 +1,140 @@ +use malachitebft_proto::{Error as ProtoError, Protobuf}; +use malachitebft_starknet_p2p_proto::{self as p2p_proto}; + +use crate::felt::FeltExt; +use crate::{Address, Felt, Hash, Height}; + +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct ProposalCommitment { + pub height: Height, + pub parent_commitment: Hash, + pub builder: Address, + pub timestamp: u64, + pub protocol_version: String, + pub old_state_root: Hash, + pub state_diff_commitment: Hash, + pub transaction_commitment: Hash, + pub event_commitment: Hash, + pub receipt_commitment: Hash, + pub concatenated_counts: Felt, + pub l1_gas_price_fri: u128, + pub l1_data_gas_price_fri: u128, + pub l2_gas_price_fri: u128, + pub l2_gas_used: u128, + pub l1_da_mode: L1DataAvailabilityMode, +} + +#[derive(Copy, Clone, Debug, PartialEq, Eq)] +pub enum L1DataAvailabilityMode { + Calldata = 0, + Blob = 1, +} + +impl Protobuf for L1DataAvailabilityMode { + type Proto = i32; + + fn from_proto(proto: Self::Proto) -> Result { + let proto = p2p_proto::L1DataAvailabilityMode::try_from(proto).map_err(|_| { + ProtoError::invalid_data::("invalid value for L1DataAvailabilityMode") + })?; + + match proto { + p2p_proto::L1DataAvailabilityMode::Calldata => Ok(Self::Calldata), + p2p_proto::L1DataAvailabilityMode::Blob => Ok(Self::Blob), + } + } + + fn to_proto(&self) -> Result { + match self { + Self::Calldata => Ok(p2p_proto::L1DataAvailabilityMode::Calldata as i32), + Self::Blob => Ok(p2p_proto::L1DataAvailabilityMode::Blob as i32), + } + } +} + +impl Protobuf for ProposalCommitment { + type Proto = p2p_proto::ProposalCommitment; + + fn from_proto(proto: Self::Proto) -> Result { + Ok(Self { + height: Height::new(proto.block_number, proto.fork_id), + parent_commitment: Hash::from_proto( + proto + .parent_commitment + .ok_or_else(|| ProtoError::missing_field::("parent_commitment"))?, + )?, + builder: Address::from_proto( + proto + .builder + .ok_or_else(|| ProtoError::missing_field::("builder"))?, + )?, + timestamp: proto.timestamp, + protocol_version: proto.protocol_version, + old_state_root: Hash::from_proto( + proto + .old_state_root + .ok_or_else(|| ProtoError::missing_field::("old_state_root"))?, + )?, + state_diff_commitment: Hash::from_proto(proto.state_diff_commitment.ok_or_else( + || ProtoError::missing_field::("state_diff_commitment"), + )?)?, + transaction_commitment: Hash::from_proto(proto.transaction_commitment.ok_or_else( + || ProtoError::missing_field::("transaction_commitment"), + )?)?, + event_commitment: Hash::from_proto( + proto + .event_commitment + .ok_or_else(|| ProtoError::missing_field::("event_commitment"))?, + )?, + receipt_commitment: Hash::from_proto( + proto.receipt_commitment.ok_or_else(|| { + ProtoError::missing_field::("receipt_commitment") + })?, + )?, + concatenated_counts: Felt::from_proto( + proto.concatenated_counts.ok_or_else(|| { + ProtoError::missing_field::("concatenated_counts") + })?, + )?, + l1_gas_price_fri: proto + .l1_gas_price_fri + .ok_or_else(|| ProtoError::missing_field::("l1_gas_price_fri"))? + .into(), + l1_data_gas_price_fri: proto + .l1_data_gas_price_fri + .ok_or_else(|| ProtoError::missing_field::("l1_data_gas_price_fri"))? + .into(), + l2_gas_price_fri: proto + .l2_gas_price_fri + .ok_or_else(|| ProtoError::missing_field::("l2_gas_price_fri"))? + .into(), + l2_gas_used: proto + .l2_gas_used + .ok_or_else(|| ProtoError::missing_field::("l2_gas_used"))? + .into(), + l1_da_mode: L1DataAvailabilityMode::from_proto(proto.l1_da_mode)?, + }) + } + + fn to_proto(&self) -> Result { + Ok(Self::Proto { + block_number: self.height.block_number, + fork_id: self.height.fork_id, + parent_commitment: Some(self.parent_commitment.to_proto()?), + builder: Some(self.builder.to_proto()?), + timestamp: self.timestamp, + protocol_version: self.protocol_version.clone(), + old_state_root: Some(self.old_state_root.to_proto()?), + state_diff_commitment: Some(self.state_diff_commitment.to_proto()?), + transaction_commitment: Some(self.transaction_commitment.to_proto()?), + event_commitment: Some(self.event_commitment.to_proto()?), + receipt_commitment: Some(self.receipt_commitment.to_proto()?), + concatenated_counts: Some(self.concatenated_counts.to_proto()?), + l1_gas_price_fri: Some(self.l1_gas_price_fri.into()), + l1_data_gas_price_fri: Some(self.l1_data_gas_price_fri.into()), + l2_gas_price_fri: Some(self.l2_gas_price_fri.into()), + l2_gas_used: Some(self.l2_gas_used.into()), + l1_da_mode: self.l1_da_mode.to_proto()?, + }) + } +} diff --git a/code/crates/starknet/p2p-types/src/proposal_part.rs b/code/crates/starknet/p2p-types/src/proposal_part.rs index cff88f9f7..be4140e02 100644 --- a/code/crates/starknet/p2p-types/src/proposal_part.rs +++ b/code/crates/starknet/p2p-types/src/proposal_part.rs @@ -1,36 +1,38 @@ use bytes::Bytes; use malachitebft_core_types::Round; use malachitebft_proto as proto; -use malachitebft_starknet_p2p_proto as p2p_proto; +use malachitebft_starknet_p2p_proto::{self as p2p_proto}; -use crate::{Address, BlockProof, Height, Signature, Transactions}; +use crate::{Address, BlockInfo, Hash, Height, ProposalCommitment, TransactionBatch}; #[derive(Clone, Debug, PartialEq, Eq)] pub struct ProposalInit { pub height: Height, - pub proposal_round: Round, + pub round: Round, pub valid_round: Round, pub proposer: Address, } #[derive(Clone, Debug, PartialEq, Eq)] pub struct ProposalFin { - pub signature: Signature, + pub proposal_commitment_hash: Hash, } #[derive(Clone, Debug, PartialEq, Eq)] pub enum ProposalPart { Init(ProposalInit), - Transactions(Transactions), - BlockProof(BlockProof), + BlockInfo(BlockInfo), + Transactions(TransactionBatch), + Commitment(Box), Fin(ProposalFin), } #[derive(Copy, Clone, Debug, PartialEq, Eq)] pub enum PartType { Init, + BlockInfo, Transactions, - BlockProof, + ProposalCommitment, Fin, } @@ -38,8 +40,9 @@ impl ProposalPart { pub fn part_type(&self) -> PartType { match self { Self::Init(_) => PartType::Init, + Self::BlockInfo(_) => PartType::BlockInfo, Self::Transactions(_) => PartType::Transactions, - Self::BlockProof(_) => PartType::BlockProof, + Self::Commitment(_) => PartType::ProposalCommitment, Self::Fin(_) => PartType::Fin, } } @@ -67,7 +70,15 @@ impl ProposalPart { } } - pub fn as_transactions(&self) -> Option<&Transactions> { + pub fn as_block_info(&self) -> Option<&BlockInfo> { + if let Self::BlockInfo(v) = self { + Some(v) + } else { + None + } + } + + pub fn as_transactions(&self) -> Option<&TransactionBatch> { if let Self::Transactions(v) = self { Some(v) } else { @@ -75,8 +86,8 @@ impl ProposalPart { } } - pub fn as_block_proof(&self) -> Option<&BlockProof> { - if let Self::BlockProof(v) = self { + pub fn as_commitment(&self) -> Option<&ProposalCommitment> { + if let Self::Commitment(v) = self { Some(v) } else { None @@ -105,8 +116,8 @@ impl proto::Protobuf for ProposalPart { Ok(match message { Messages::Init(init) => ProposalPart::Init(ProposalInit { - height: Height::new(init.block_number, init.fork_id), - proposal_round: Round::new(init.proposal_round), + height: Height::new(init.height, 0), + round: Round::new(init.round), valid_round: init.valid_round.into(), proposer: Address::from_proto( init.proposer @@ -114,21 +125,24 @@ impl proto::Protobuf for ProposalPart { )?, }), - Messages::Fin(fin) => ProposalPart::Fin(ProposalFin { - signature: Signature::from_proto( - fin.signature - .ok_or_else(|| proto::Error::missing_field::("signature"))?, - )?, - }), + Messages::BlockInfo(block_info) => { + ProposalPart::BlockInfo(BlockInfo::from_proto(block_info)?) + } Messages::Transactions(txes) => { - let transactions = Transactions::from_proto(txes)?; + let transactions = TransactionBatch::from_proto(txes)?; ProposalPart::Transactions(transactions) } - Messages::Proof(proof) => { - let block_proof = BlockProof::from_proto(proof)?; - ProposalPart::BlockProof(block_proof) + + Messages::Commitment(commitment) => { + ProposalPart::Commitment(Box::new(ProposalCommitment::from_proto(commitment)?)) } + + Messages::Fin(fin) => ProposalPart::Fin(ProposalFin { + proposal_commitment_hash: Hash::from_proto(fin.proposal_commitment.ok_or_else( + || proto::Error::missing_field::("proposal_commitment"), + )?)?, + }), }) } @@ -138,26 +152,25 @@ impl proto::Protobuf for ProposalPart { let message = match self { ProposalPart::Init(init) => Messages::Init(p2p_proto::ProposalInit { - block_number: init.height.block_number, - fork_id: init.height.fork_id, - proposal_round: init - .proposal_round - .as_u32() - .expect("round should not be nil"), + height: init.height.block_number, + round: init.round.as_u32().expect("round should not be nil"), valid_round: init.valid_round.as_u32(), proposer: Some(init.proposer.to_proto()?), }), + ProposalPart::BlockInfo(block_info) => Messages::BlockInfo(block_info.to_proto()?), + ProposalPart::Transactions(txes) => { + Messages::Transactions(p2p_proto::TransactionBatch { + transactions: txes + .as_slice() + .iter() + .map(|tx| tx.to_proto()) + .collect::, _>>()?, + }) + } + ProposalPart::Commitment(commitment) => Messages::Commitment(commitment.to_proto()?), ProposalPart::Fin(fin) => Messages::Fin(p2p_proto::ProposalFin { - signature: Some(fin.signature.to_proto()?), - }), - ProposalPart::Transactions(txes) => Messages::Transactions(p2p_proto::Transactions { - transactions: txes - .as_slice() - .iter() - .map(|tx| tx.to_proto()) - .collect::, _>>()?, + proposal_commitment: Some(fin.proposal_commitment_hash.to_proto()?), }), - ProposalPart::BlockProof(block_proof) => Messages::Proof(block_proof.to_proto()?), }; Ok(p2p_proto::ProposalPart { diff --git a/code/crates/starknet/p2p-types/src/signing.rs b/code/crates/starknet/p2p-types/src/signing.rs index 95d702d3e..ce65bcac4 100644 --- a/code/crates/starknet/p2p-types/src/signing.rs +++ b/code/crates/starknet/p2p-types/src/signing.rs @@ -76,6 +76,13 @@ impl Signature { pub fn inner(&self) -> &starknet_crypto::Signature { &self.0 } + + pub fn dummy() -> Self { + Self(starknet_crypto::Signature { + r: Felt::ZERO, + s: Felt::ZERO, + }) + } } impl Clone for Signature { diff --git a/code/crates/starknet/p2p-types/src/signing/provider.rs b/code/crates/starknet/p2p-types/src/signing/provider.rs index dfa04981e..6b605a276 100644 --- a/code/crates/starknet/p2p-types/src/signing/provider.rs +++ b/code/crates/starknet/p2p-types/src/signing/provider.rs @@ -23,35 +23,37 @@ impl EcdsaProvider { impl SigningProvider for EcdsaProvider { fn sign_vote(&self, vote: Vote) -> SignedVote { - let hash = starknet_keccak(&vote.to_sign_bytes()); - let signature = self.private_key.sign(&hash); - SignedVote::new(vote, signature) + // Votes are not signed for now + // let hash = starknet_keccak(&vote.to_sign_bytes()); + // let signature = self.private_key.sign(&hash); + SignedVote::new(vote, Signature::dummy()) } fn verify_signed_vote( &self, - vote: &Vote, - signature: &Signature, - public_key: &PublicKey, + _vote: &Vote, + _signature: &Signature, + _public_key: &PublicKey, ) -> bool { - let hash = starknet_keccak(&vote.to_sign_bytes()); - public_key.verify(&hash, signature) + // Votes are not signed for now + true + // let hash = starknet_keccak(&vote.to_sign_bytes()); + // public_key.verify(&hash, signature) } fn sign_proposal(&self, proposal: Proposal) -> SignedProposal { - let hash = starknet_keccak(&proposal.to_sign_bytes()); - let signature = self.private_key.sign(&hash); - SignedProposal::new(proposal, signature) + // Proposals are never sent over the network + SignedProposal::new(proposal, Signature::dummy()) } fn verify_signed_proposal( &self, - proposal: &Proposal, - signature: &Signature, - public_key: &PublicKey, + _proposal: &Proposal, + _signature: &Signature, + _public_key: &PublicKey, ) -> bool { - let hash = starknet_keccak(&proposal.to_sign_bytes()); - public_key.verify(&hash, signature) + // Proposals are never sent over the network + true } fn sign_proposal_part(&self, proposal_part: ProposalPart) -> SignedProposalPart { diff --git a/code/crates/starknet/p2p-types/src/streaming.rs b/code/crates/starknet/p2p-types/src/streaming.rs index 90de695f0..0c4d114af 100644 --- a/code/crates/starknet/p2p-types/src/streaming.rs +++ b/code/crates/starknet/p2p-types/src/streaming.rs @@ -6,7 +6,7 @@ pub struct StreamMessage { /// Receivers identify streams by (sender, stream_id). /// This means each node can allocate stream_ids independently /// and that many streams can be sent on a single network topic. - pub id: u64, + pub id: Bytes, /// Identifies the sequence of each message in the stream starting from 0. pub sequence: u64, @@ -18,26 +18,26 @@ pub struct StreamMessage { pub enum StreamContent { /// Serialized content. Data(Bytes), - /// Fin must be set to true. - Fin(bool), + /// Final message. + Fin, } impl Protobuf for StreamMessage { - type Proto = p2p_proto::Stream; + type Proto = p2p_proto::StreamMessage; #[cfg_attr(coverage_nightly, coverage(off))] fn from_proto(proto: Self::Proto) -> Result { let content = match proto - .content + .message .ok_or_else(|| malachitebft_proto::Error::missing_field::("content"))? { - p2p_proto::stream::Content::Data(data) => StreamContent::Data(data), - p2p_proto::stream::Content::Fin(fin) => StreamContent::Fin(fin), + p2p_proto::stream_message::Message::Content(data) => StreamContent::Data(data), + p2p_proto::stream_message::Message::Fin(_) => StreamContent::Fin, }; Ok(Self { id: proto.stream_id, - sequence: proto.sequence_number, + sequence: proto.message_id, content, }) } @@ -45,11 +45,15 @@ impl Protobuf for StreamMessage { #[cfg_attr(coverage_nightly, coverage(off))] fn to_proto(&self) -> Result { Ok(Self::Proto { - stream_id: self.id, - sequence_number: self.sequence, - content: match &self.content { - StreamContent::Data(data) => Some(p2p_proto::stream::Content::Data(data.clone())), - StreamContent::Fin(fin) => Some(p2p_proto::stream::Content::Fin(*fin)), + stream_id: self.id.clone(), + message_id: self.sequence, + message: match &self.content { + StreamContent::Data(data) => { + Some(p2p_proto::stream_message::Message::Content(data.clone())) + } + StreamContent::Fin => { + Some(p2p_proto::stream_message::Message::Fin(p2p_proto::Fin {})) + } }, }) } diff --git a/code/crates/starknet/p2p-types/src/transaction.rs b/code/crates/starknet/p2p-types/src/transaction.rs index c0ee34bef..ac4864748 100644 --- a/code/crates/starknet/p2p-types/src/transaction.rs +++ b/code/crates/starknet/p2p-types/src/transaction.rs @@ -59,10 +59,10 @@ impl fmt::Debug for Transaction { } impl proto::Protobuf for Transaction { - type Proto = p2p_proto::Transaction; + type Proto = p2p_proto::ConsensusTransaction; fn from_proto(proto: Self::Proto) -> Result { - use malachitebft_starknet_p2p_proto::transaction::Txn; + use malachitebft_starknet_p2p_proto::consensus_transaction::Txn; let txn = proto .txn @@ -73,8 +73,8 @@ impl proto::Protobuf for Transaction { .ok_or_else(|| proto::Error::missing_field::("transaction_hash"))?; match txn { - Txn::Dummy(dummy) => Ok(Self { - data: dummy.bytes, + Txn::Dummy(bytes) => Ok(Self { + data: bytes, hash: Hash::from_proto(hash)?, }), _ => Err(proto::Error::invalid_data::( @@ -84,25 +84,23 @@ impl proto::Protobuf for Transaction { } fn to_proto(&self) -> Result { - use malachitebft_starknet_p2p_proto::transaction::{Dummy, Txn}; + use malachitebft_starknet_p2p_proto::consensus_transaction::Txn; Ok(Self::Proto { transaction_hash: Some(self.hash.to_proto()?), - txn: Some(Txn::Dummy(Dummy { - bytes: self.to_bytes(), - })), + txn: Some(Txn::Dummy(self.to_bytes())), }) } } /// Transaction batch (used by mempool and proposal part) #[derive(Clone, Debug, Default, PartialEq, Eq)] -pub struct Transactions(Vec); +pub struct TransactionBatch(Vec); -impl Transactions { +impl TransactionBatch { /// Create a new transaction batch pub fn new(txes: Vec) -> Self { - Transactions(txes) + TransactionBatch(txes) } /// Add a transaction to the batch @@ -111,7 +109,7 @@ impl Transactions { } /// Add a set of transaction to the batch - pub fn append(&mut self, txes: Transactions) { + pub fn append(&mut self, txes: TransactionBatch) { let mut txes1 = txes.clone(); self.0.append(&mut txes1.0); } @@ -150,8 +148,8 @@ impl Transactions { } } -impl proto::Protobuf for Transactions { - type Proto = p2p_proto::Transactions; +impl proto::Protobuf for TransactionBatch { + type Proto = p2p_proto::TransactionBatch; fn from_proto(proto: Self::Proto) -> Result { Ok(Self::new( @@ -164,7 +162,7 @@ impl proto::Protobuf for Transactions { } fn to_proto(&self) -> Result { - Ok(p2p_proto::Transactions { + Ok(p2p_proto::TransactionBatch { transactions: self .as_slice() .iter() diff --git a/code/crates/starknet/p2p-types/src/vote.rs b/code/crates/starknet/p2p-types/src/vote.rs index ebcfc49c0..e1d54a923 100644 --- a/code/crates/starknet/p2p-types/src/vote.rs +++ b/code/crates/starknet/p2p-types/src/vote.rs @@ -1,10 +1,10 @@ use bytes::Bytes; -use malachitebft_core_types::{NilOrVal, Round, SignedExtension, VoteType}; +use malachitebft_core_types::{NilOrVal, Round, VoteType}; use malachitebft_proto as proto; use malachitebft_starknet_p2p_proto as p2p_proto; -use crate::{Address, BlockHash, Height, MockContext, Signature}; +use crate::{Address, BlockHash, Height}; #[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Ord)] pub struct Vote { @@ -13,7 +13,6 @@ pub struct Vote { pub round: Round, pub block_hash: NilOrVal, pub voter: Address, - pub extension: Option>, } impl Vote { @@ -29,7 +28,6 @@ impl Vote { round, block_hash, voter, - extension: None, } } @@ -45,24 +43,6 @@ impl Vote { round, block_hash: value, voter: address, - extension: None, - } - } - - pub fn new_precommit_with_extension( - height: Height, - round: Round, - value: NilOrVal, - address: Address, - extension: SignedExtension, - ) -> Self { - Self { - vote_type: VoteType::Precommit, - height, - round, - block_hash: value, - voter: address, - extension: Some(extension), } } @@ -78,20 +58,6 @@ impl proto::Protobuf for Vote { fn from_proto(proto: Self::Proto) -> Result { let vote_type = proto_to_common_vote_type(proto.vote_type()); - let extension = proto - .extension - .map(|data| -> Result<_, proto::Error> { - let signature = data.signature.ok_or_else(|| { - proto::Error::missing_field::("extension.signature") - })?; - - Ok(SignedExtension::new( - data.data, - Signature::from_proto(signature)?, - )) - }) - .transpose()?; - Ok(Self { vote_type, height: Height::new(proto.block_number, proto.fork_id), @@ -105,7 +71,6 @@ impl proto::Protobuf for Vote { .voter .ok_or_else(|| proto::Error::missing_field::("voter"))?, )?, - extension, }) } @@ -121,16 +86,6 @@ impl proto::Protobuf for Vote { NilOrVal::Val(v) => Some(v.to_proto()?), }, voter: Some(self.voter.to_proto()?), - extension: self - .extension - .as_ref() - .map(|ext| -> Result<_, proto::Error> { - Ok(p2p_proto::Extension { - data: ext.message.clone(), - signature: Some(ext.signature.to_proto()?), - }) - }) - .transpose()?, }) } } diff --git a/code/crates/starknet/test/src/tests/n3f0_consensus_mode.rs b/code/crates/starknet/test/src/tests/n3f0_consensus_mode.rs index 468a18c6f..caf7138b7 100644 --- a/code/crates/starknet/test/src/tests/n3f0_consensus_mode.rs +++ b/code/crates/starknet/test/src/tests/n3f0_consensus_mode.rs @@ -31,6 +31,7 @@ pub async fn parts_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode pub async fn proposal_only() { let params = TestParams { value_payload: ValuePayload::ProposalOnly, @@ -41,6 +42,7 @@ pub async fn proposal_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode pub async fn proposal_and_parts() { let params = TestParams { value_payload: ValuePayload::ProposalAndParts, diff --git a/code/crates/starknet/test/src/tests/value_sync.rs b/code/crates/starknet/test/src/tests/value_sync.rs index 865f72153..4f4ab8be8 100644 --- a/code/crates/starknet/test/src/tests/value_sync.rs +++ b/code/crates/starknet/test/src/tests/value_sync.rs @@ -66,6 +66,7 @@ pub async fn crash_restart_from_start_parts_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode pub async fn crash_restart_from_start_proposal_only() { let params = TestParams { value_payload: ValuePayload::ProposalOnly, @@ -76,6 +77,7 @@ pub async fn crash_restart_from_start_proposal_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode pub async fn crash_restart_from_start_proposal_and_parts() { let params = TestParams { value_payload: ValuePayload::ProposalAndParts, diff --git a/code/crates/starknet/test/src/tests/vote_sync.rs b/code/crates/starknet/test/src/tests/vote_sync.rs index 4346a1db1..6c02b90d2 100644 --- a/code/crates/starknet/test/src/tests/vote_sync.rs +++ b/code/crates/starknet/test/src/tests/vote_sync.rs @@ -57,6 +57,7 @@ pub async fn crash_restart_from_start_parts_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode pub async fn crash_restart_from_start_proposal_only() { let params = TestParams { value_payload: ValuePayload::ProposalOnly, @@ -67,6 +68,7 @@ pub async fn crash_restart_from_start_proposal_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode pub async fn crash_restart_from_start_proposal_and_parts() { let params = TestParams { value_payload: ValuePayload::ProposalAndParts, diff --git a/code/crates/starknet/test/src/tests/wal.rs b/code/crates/starknet/test/src/tests/wal.rs index 88b14a0db..ed03f4f6b 100644 --- a/code/crates/starknet/test/src/tests/wal.rs +++ b/code/crates/starknet/test/src/tests/wal.rs @@ -21,6 +21,7 @@ async fn proposer_crashes_after_proposing_parts_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode async fn proposer_crashes_after_proposing_proposal_and_parts() { proposer_crashes_after_proposing(TestParams { value_payload: ValuePayload::ProposalAndParts, @@ -30,6 +31,7 @@ async fn proposer_crashes_after_proposing_proposal_and_parts() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode async fn proposer_crashes_after_proposing_proposal_only() { proposer_crashes_after_proposing(TestParams { value_payload: ValuePayload::ProposalOnly, @@ -113,6 +115,7 @@ async fn non_proposer_crashes_after_voting_parts_only() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode async fn non_proposer_crashes_after_voting_proposal_and_parts() { non_proposer_crashes_after_voting(TestParams { value_payload: ValuePayload::ProposalAndParts, @@ -122,6 +125,7 @@ async fn non_proposer_crashes_after_voting_proposal_and_parts() { } #[tokio::test] +#[ignore] // Starknet app only supports parts only mode async fn non_proposer_crashes_after_voting_proposal_only() { non_proposer_crashes_after_voting(TestParams { value_payload: ValuePayload::ProposalOnly, diff --git a/code/crates/test/proto/consensus.proto b/code/crates/test/proto/consensus.proto index 8ff28c523..5c30642de 100644 --- a/code/crates/test/proto/consensus.proto +++ b/code/crates/test/proto/consensus.proto @@ -75,7 +75,7 @@ message Extension { } message StreamMessage { - uint64 stream_id = 1; + bytes stream_id = 1; uint64 sequence = 2; oneof content { // Serialized content. diff --git a/code/crates/test/src/codec/proto/mod.rs b/code/crates/test/src/codec/proto/mod.rs index e9a499ae4..cf7089d96 100644 --- a/code/crates/test/src/codec/proto/mod.rs +++ b/code/crates/test/src/codec/proto/mod.rs @@ -1,7 +1,7 @@ use bytes::Bytes; use prost::Message; -use malachitebft_app::streaming::{StreamContent, StreamMessage}; +use malachitebft_app::streaming::{StreamContent, StreamId, StreamMessage}; use malachitebft_codec::Codec; use malachitebft_core_consensus::{ProposedValue, SignedConsensusMsg}; use malachitebft_core_types::{ @@ -127,11 +127,11 @@ impl Codec> for ProtobufCodec { proto::stream_message::Content::Data(data) => { StreamContent::Data(ProposalPart::from_bytes(&data)?) } - proto::stream_message::Content::Fin(end) => StreamContent::Fin(end), + proto::stream_message::Content::Fin(_) => StreamContent::Fin, }; Ok(StreamMessage { - stream_id: proto.stream_id, + stream_id: StreamId::new(proto.stream_id), sequence: proto.sequence, content, }) @@ -139,13 +139,13 @@ impl Codec> for ProtobufCodec { fn encode(&self, msg: &StreamMessage) -> Result { let proto = proto::StreamMessage { - stream_id: msg.stream_id, + stream_id: msg.stream_id.to_bytes(), sequence: msg.sequence, content: match &msg.content { StreamContent::Data(data) => { Some(proto::stream_message::Content::Data(data.to_bytes()?)) } - StreamContent::Fin(end) => Some(proto::stream_message::Content::Fin(*end)), + StreamContent::Fin => Some(proto::stream_message::Content::Fin(true)), }, }; diff --git a/code/examples/channel/src/app.rs b/code/examples/channel/src/app.rs index 596642c26..5bb5bbec9 100644 --- a/code/examples/channel/src/app.rs +++ b/code/examples/channel/src/app.rs @@ -135,7 +135,7 @@ pub async fn run(state: &mut State, channels: &mut Channels) -> eyr AppMsg::ReceivedProposalPart { from, part, reply } => { let part_type = match &part.content { StreamContent::Data(part) => part.get_type(), - StreamContent::Fin(_) => "end of stream", + StreamContent::Fin => "end of stream", }; info!(%from, %part.sequence, part.type = %part_type, "Received proposal part"); diff --git a/code/examples/channel/src/state.rs b/code/examples/channel/src/state.rs index 1d1146254..952ad0513 100644 --- a/code/examples/channel/src/state.rs +++ b/code/examples/channel/src/state.rs @@ -11,7 +11,7 @@ use sha3::Digest; use tracing::{debug, error}; use malachitebft_app_channel::app::consensus::ProposedValue; -use malachitebft_app_channel::app::streaming::{StreamContent, StreamMessage}; +use malachitebft_app_channel::app::streaming::{StreamContent, StreamId, StreamMessage}; use malachitebft_app_channel::app::types::codec::Codec; use malachitebft_app_channel::app::types::core::{ CommitCertificate, Round, Validity, VoteExtensions, @@ -36,7 +36,6 @@ pub struct State { address: Address, store: Store, vote_extensions: HashMap>, - stream_id: u64, streams_map: PartStreamsMap, rng: StdRng, @@ -91,7 +90,6 @@ impl State { address, store, vote_extensions: HashMap::new(), - stream_id: 0, streams_map: PartStreamsMap::new(), rng: StdRng::seed_from_u64(seed_from_address(&address)), peers: HashSet::new(), @@ -304,28 +302,29 @@ impl State { value: LocallyProposedValue, ) -> impl Iterator> { let parts = self.value_to_parts(value); - - let stream_id = self.stream_id; - self.stream_id += 1; + let stream_id = self.stream_id(); let mut msgs = Vec::with_capacity(parts.len() + 1); let mut sequence = 0; for part in parts { - let msg = StreamMessage::new(stream_id, sequence, StreamContent::Data(part)); + let msg = StreamMessage::new(stream_id.clone(), sequence, StreamContent::Data(part)); sequence += 1; msgs.push(msg); } - msgs.push(StreamMessage::new( - stream_id, - sequence, - StreamContent::Fin(true), - )); + msgs.push(StreamMessage::new(stream_id, sequence, StreamContent::Fin)); msgs.into_iter() } + fn stream_id(&self) -> StreamId { + let mut bytes = Vec::with_capacity(size_of::() + size_of::()); + bytes.extend_from_slice(&self.current_height.as_u64().to_be_bytes()); + bytes.extend_from_slice(&self.current_round.as_u32().unwrap().to_be_bytes()); + StreamId::new(bytes.into()) + } + fn value_to_parts(&self, value: LocallyProposedValue) -> Vec { let mut hasher = sha3::Keccak256::new(); let mut parts = Vec::new(); diff --git a/code/examples/channel/src/streaming.rs b/code/examples/channel/src/streaming.rs index 6792b0a5a..036262391 100644 --- a/code/examples/channel/src/streaming.rs +++ b/code/examples/channel/src/streaming.rs @@ -117,8 +117,11 @@ impl PartStreamsMap { peer_id: PeerId, msg: StreamMessage, ) -> Option { - let stream_id = msg.stream_id; - let state = self.streams.entry((peer_id, stream_id)).or_default(); + let stream_id = msg.stream_id.clone(); + let state = self + .streams + .entry((peer_id, stream_id.clone())) + .or_default(); if !state.seen_sequences.insert(msg.sequence) { // We have already seen a message with this sequence number. diff --git a/code/scripts/spawn.bash b/code/scripts/spawn.bash index 680a80bc9..cb991c2cd 100755 --- a/code/scripts/spawn.bash +++ b/code/scripts/spawn.bash @@ -38,7 +38,7 @@ fi # Environment variables export MALACHITE__CONSENSUS__P2P__PROTOCOL__TYPE="gossipsub" -export MALACHITE__CONSENSUS__VALUE_PAYLOAD="proposal-and-parts" +export MALACHITE__CONSENSUS__VALUE_PAYLOAD="parts-only" export MALACHITE__CONSENSUS__MAX_BLOCK_SIZE="50KiB" export MALACHITE__CONSENSUS__TIMEOUT_PROPOSE="2s" export MALACHITE__CONSENSUS__TIMEOUT_PROPOSE_DELTA="1s" diff --git a/code/scripts/spawn.fish b/code/scripts/spawn.fish index dc87e2382..69582c43b 100755 --- a/code/scripts/spawn.fish +++ b/code/scripts/spawn.fish @@ -46,7 +46,7 @@ if ! set -q _flag_home return 1 end -set app_name "informalsystems-malachitebft-example-channel" +set app_name "informalsystems-malachitebft-starknet-app" set profile false set debug false set lldb false