Skip to content

Commit

Permalink
BFT-498: Add genesis to BatchVotes operations
Browse files Browse the repository at this point in the history
  • Loading branch information
aakoshh committed Jul 26, 2024
1 parent 0d27fcf commit b835dff
Show file tree
Hide file tree
Showing 18 changed files with 105 additions and 31 deletions.
4 changes: 3 additions & 1 deletion node/actors/executor/src/attestation.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,8 @@ impl AttesterRunner {
return Ok(());
}

let genesis = self.block_store.genesis().hash();

// Find the initial range of batches that we want to (re)sign after a (re)start.
let last_batch_number = self
.batch_store
Expand Down Expand Up @@ -79,7 +81,7 @@ impl AttesterRunner {

// We only have to publish a vote once; future peers can pull it from the register.
self.publisher
.publish(attesters, &self.attester.key, batch)
.publish(attesters, &genesis, &self.attester.key, batch)
.await
.context("publish")?;

Expand Down
20 changes: 18 additions & 2 deletions node/actors/network/src/gossip/batch_votes.rs
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@ impl BatchVotes {
pub(super) fn update(
&mut self,
attesters: &attester::Committee,
genesis: &attester::GenesisHash,
data: &[Arc<attester::Signed<attester::Batch>>],
) -> anyhow::Result<BatchUpdateStats> {
let mut stats = BatchUpdateStats::default();
Expand All @@ -93,6 +94,14 @@ impl BatchVotes {
}
done.insert(d.key.clone());

// Disallow votes from different genesis. It might indicate a reorg,
// in which case either this node or the remote peer has to be restarted.
anyhow::ensure!(
d.msg.genesis == *genesis,
"vote for batch with different genesis hash: {:?}",
d.msg.genesis
);

if d.msg.number < self.min_batch_number {
continue;
}
Expand Down Expand Up @@ -130,6 +139,7 @@ impl BatchVotes {
pub(super) fn find_quorums(
&self,
attesters: &attester::Committee,
genesis: &attester::GenesisHash,
skip: impl Fn(attester::BatchNumber) -> bool,
) -> Vec<attester::BatchQC> {
let threshold = attesters.threshold();
Expand All @@ -154,6 +164,8 @@ impl BatchVotes {
message: attester::Batch {
number: *number,
hash: *hash,
// This was checked during insertion; we could look up the first in `votes`
genesis: *genesis,
},
signatures: sigs,
}
Expand Down Expand Up @@ -230,11 +242,12 @@ impl BatchVotesWatch {
pub(crate) async fn update(
&self,
attesters: &attester::Committee,
genesis: &attester::GenesisHash,
data: &[Arc<attester::Signed<attester::Batch>>],
) -> anyhow::Result<()> {
let this = self.0.lock().await;
let mut votes = this.borrow().clone();
let stats = votes.update(attesters, data)?;
let stats = votes.update(attesters, genesis, data)?;

if let Some(last_added) = stats.last_added {
this.send_replace(votes);
Expand Down Expand Up @@ -288,6 +301,7 @@ impl BatchVotesPublisher {
pub async fn publish(
&self,
attesters: &attester::Committee,
genesis: &attester::GenesisHash,
attester: &attester::SecretKey,
batch: attester::Batch,
) -> anyhow::Result<()> {
Expand All @@ -300,6 +314,8 @@ impl BatchVotesPublisher {
.last_signed_batch_number
.set(attestation.msg.number.0);

self.0.update(attesters, &[Arc::new(attestation)]).await
self.0
.update(attesters, genesis, &[Arc::new(attestation)])
.await
}
}
3 changes: 2 additions & 1 deletion node/actors/network/src/gossip/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -154,14 +154,15 @@ impl Network {
let Some(attesters) = self.genesis().attesters.as_ref() else {
return Ok(());
};
let genesis = self.genesis().hash();
let mut sub = self.batch_votes.subscribe();
loop {
// In the future when we might be gossiping about multiple batches at the same time,
// we can collect the ones we submitted into a skip list until we see them confirmed
// on L1 and we can finally increase the minimum as well.
let quorums = {
let votes = sync::changed(ctx, &mut sub).await?;
votes.find_quorums(attesters, |_| false)
votes.find_quorums(attesters, &genesis, |_| false)
};

for qc in quorums {
Expand Down
1 change: 1 addition & 0 deletions node/actors/network/src/gossip/runner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@ impl rpc::Handler<rpc::push_batch_votes::Rpc> for &PushServer<'_> {
.batch_votes
.update(
self.net.genesis().attesters.as_ref().context("attesters")?,
&self.net.genesis().hash(),
&req.0,
)
.await?;
Expand Down
59 changes: 47 additions & 12 deletions node/actors/network/src/gossip/tests/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -114,10 +114,12 @@ fn mk_batch<R: Rng>(
rng: &mut R,
key: &attester::SecretKey,
number: attester::BatchNumber,
genesis: attester::GenesisHash,
) -> attester::Signed<attester::Batch> {
key.sign_msg(attester::Batch {
number,
hash: rng.gen(),
genesis,
})
}

Expand All @@ -136,10 +138,12 @@ fn random_netaddr<R: Rng>(
fn random_batch_vote<R: Rng>(
rng: &mut R,
key: &attester::SecretKey,
genesis: attester::GenesisHash,
) -> Arc<attester::Signed<attester::Batch>> {
let batch = attester::Batch {
number: attester::BatchNumber(rng.gen_range(0..1000)),
hash: rng.gen(),
genesis,
};
Arc::new(key.sign_msg(batch.to_owned()))
}
Expand Down Expand Up @@ -168,6 +172,7 @@ fn update_signature<R: Rng>(
let batch = attester::Batch {
number: attester::BatchNumber((batch.number.0 as i64 + batch_number_diff) as u64),
hash: rng.gen(),
genesis: batch.genesis,
};
Arc::new(key.sign_msg(batch.to_owned()))
}
Expand Down Expand Up @@ -552,12 +557,17 @@ async fn test_batch_votes() {
let votes = BatchVotesWatch::default();
let mut sub = votes.subscribe();

let genesis = rng.gen::<attester::GenesisHash>();

// Initial values.
let mut want = Signatures::default();
for k in &keys[0..6] {
want.insert(random_batch_vote(rng, k));
want.insert(random_batch_vote(rng, k, genesis));
}
votes.update(&attesters, &want.as_vec()).await.unwrap();
votes
.update(&attesters, &genesis, &want.as_vec())
.await
.unwrap();
assert_eq!(want.0, sub.borrow_and_update().votes);

// newer batch number, should be updated
Expand All @@ -567,10 +577,10 @@ async fn test_batch_votes() {
// older batch number, should be ignored
let k4v2 = update_signature(rng, &want.get(&keys[4]).msg, &keys[4], -1);
// first entry for a key in the config, should be inserted
let k6v1 = random_batch_vote(rng, &keys[6]);
let k6v1 = random_batch_vote(rng, &keys[6], genesis);
// entry for a key outside of the config, should be ignored
let k8 = rng.gen();
let k8v1 = random_batch_vote(rng, &k8);
let k8v1 = random_batch_vote(rng, &k8, genesis);

// Update the ones we expect to succeed
want.insert(k0v2.clone());
Expand All @@ -586,22 +596,40 @@ async fn test_batch_votes() {
// no entry at all for keys[7]
k8v1.clone(),
];
votes.update(&attesters, &update).await.unwrap();
votes.update(&attesters, &genesis, &update).await.unwrap();
assert_eq!(want.0, sub.borrow_and_update().votes);

// Invalid signature, should be rejected.
let mut k0v3 = mk_batch(
rng,
&keys[1],
attester::BatchNumber(want.get(&keys[0]).msg.number.0 + 1),
genesis,
);
k0v3.key = keys[0].public();
assert!(votes.update(&attesters, &[Arc::new(k0v3)]).await.is_err());
assert!(votes
.update(&attesters, &genesis, &[Arc::new(k0v3)])
.await
.is_err());

// Invalid genesis, should be rejected.
let other_genesis = rng.gen();
let k1v3 = mk_batch(
rng,
&keys[1],
attester::BatchNumber(want.get(&keys[1]).msg.number.0 + 1),
other_genesis,
);
assert!(votes
.update(&attesters, &genesis, &[Arc::new(k1v3)])
.await
.is_err());

assert_eq!(want.0, sub.borrow_and_update().votes);

// Duplicate entry in the update, should be rejected.
assert!(votes
.update(&attesters, &[k8v1.clone(), k8v1])
.update(&attesters, &genesis, &[k8v1.clone(), k8v1])
.await
.is_err());
assert_eq!(want.0, sub.borrow_and_update().votes);
Expand All @@ -625,7 +653,9 @@ fn test_batch_votes_quorum() {
let batch1 = attester::Batch {
number: batch0.number.next(),
hash: rng.gen(),
genesis: batch0.genesis,
};
let genesis = batch0.genesis;
let mut batches = [(batch0, 0u64), (batch1, 0u64)];

let mut votes = BatchVotes::default();
Expand All @@ -634,12 +664,14 @@ fn test_batch_votes_quorum() {
let b = usize::from(rng.gen_range(0..100) < 80);
let batch = &batches[b].0;
let vote = sk.sign_msg(batch.clone());
votes.update(&attesters, &[Arc::new(vote)]).unwrap();
votes
.update(&attesters, &genesis, &[Arc::new(vote)])
.unwrap();
batches[b].1 += attesters.weight(&sk.public()).unwrap();

// Check that as soon as we have quorum it's found.
if batches[b].1 >= attesters.threshold() {
let qs = votes.find_quorums(&attesters, |_| false);
let qs = votes.find_quorums(&attesters, &genesis, |_| false);
assert!(!qs.is_empty(), "should find quorum");
assert!(qs[0].message == *batch);
assert!(qs[0].signatures.keys().count() > 0);
Expand All @@ -653,11 +685,13 @@ fn test_batch_votes_quorum() {
{
// Check that a quorum can be skipped
assert!(votes
.find_quorums(&attesters, |b| b == quorum.number)
.find_quorums(&attesters, &genesis, |b| b == quorum.number)
.is_empty());
} else {
// Check that if there was no quoroum then we don't find any.
assert!(votes.find_quorums(&attesters, |_| false).is_empty());
assert!(votes
.find_quorums(&attesters, &genesis, |_| false)
.is_empty());
}

// Check that the minimum batch number prunes data.
Expand Down Expand Up @@ -705,6 +739,7 @@ async fn test_batch_votes_propagation() {
let batch = attester::Batch {
number: batch.number,
hash: rng.gen(),
genesis: setup.genesis.hash(),
};

// Start all nodes.
Expand All @@ -728,7 +763,7 @@ async fn test_batch_votes_propagation() {
for (node, key) in nodes.iter().zip(setup.attester_keys.iter()) {
node.net
.batch_vote_publisher()
.publish(attesters, key, batch.clone())
.publish(attesters, &setup.genesis.hash(), key, batch.clone())
.await
.unwrap();
}
Expand Down
3 changes: 2 additions & 1 deletion node/actors/network/src/proto/consensus.proto
Original file line number Diff line number Diff line change
Expand Up @@ -2,13 +2,14 @@ syntax = "proto3";

package zksync.network.consensus;

import "zksync/roles/common.proto";
import "zksync/roles/validator.proto";
import "zksync/std.proto";

// First message exchanged in the encrypted session.
message Handshake {
optional roles.validator.Signed session_id = 1; // required
optional roles.validator.GenesisHash genesis = 2; // required
optional roles.common.GenesisHash genesis = 2; // required
}

message ConsensusReq {
Expand Down
3 changes: 2 additions & 1 deletion node/actors/network/src/proto/gossip.proto
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,15 @@ syntax = "proto3";

package zksync.network.gossip;

import "zksync/roles/common.proto";
import "zksync/roles/node.proto";
import "zksync/roles/validator.proto";
import "zksync/roles/attester.proto";

// First message exchanged in the encrypted session.
message Handshake {
optional roles.node.Signed session_id = 1; // required
optional roles.validator.GenesisHash genesis = 3; // required
optional roles.common.GenesisHash genesis = 3; // required
optional bool is_static = 2; // required
}

Expand Down
4 changes: 3 additions & 1 deletion node/libs/roles/src/attester/conv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,14 +27,16 @@ impl ProtoFmt for Batch {
type Proto = proto::Batch;
fn read(r: &Self::Proto) -> anyhow::Result<Self> {
Ok(Self {
hash: read_required(&r.hash).context("hash")?,
number: BatchNumber(*required(&r.number).context("number")?),
hash: read_required(&r.hash).context("hash")?,
genesis: read_required(&r.genesis).context("genesis")?,
})
}
fn build(&self) -> Self::Proto {
Self::Proto {
number: Some(self.number.0),
hash: Some(self.hash.build()),
genesis: Some(self.genesis.build()),
}
}
}
Expand Down
8 changes: 7 additions & 1 deletion node/libs/roles/src/attester/messages/batch.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
use super::Signed;
use super::{GenesisHash, Signed};
use crate::{
attester,
validator::{Genesis, Payload},
Expand Down Expand Up @@ -92,6 +92,12 @@ pub struct Batch {
pub number: BatchNumber,
/// Hash of the batch.
pub hash: BatchHash,
/// Hash of the genesis.
///
/// This includes the chain ID and the current fork number, which prevents
/// replay attacks from other chains where the same attesters might operate,
/// or from earlier forks, which are created after a revert of L1 batches.
pub genesis: GenesisHash,
}

/// A certificate for a batch of L2 blocks to be sent to L1.
Expand Down
1 change: 1 addition & 0 deletions node/libs/roles/src/attester/messages/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,5 +2,6 @@
mod batch;
mod msg;

pub use crate::validator::GenesisHash;
pub use batch::*;
pub use msg::*;
1 change: 1 addition & 0 deletions node/libs/roles/src/attester/testonly.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ impl Distribution<Batch> for Standard {
Batch {
number: rng.gen(),
hash: rng.gen(),
genesis: rng.gen(),
}
}
}
Expand Down
3 changes: 2 additions & 1 deletion node/libs/roles/src/proto/attester.proto
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ syntax = "proto3";
package zksync.roles.attester;

import "zksync/std.proto";
import "zksync/roles/common.proto";

message SyncBatch {
optional uint64 number = 1; // required
Expand All @@ -14,10 +15,10 @@ message BatchHash {
optional bytes keccak256 = 1; // required
}


message Batch {
optional uint64 number = 1; // required
optional BatchHash hash = 2; // required
optional common.GenesisHash genesis = 3; // required
}

message BatchQC {
Expand Down
7 changes: 7 additions & 0 deletions node/libs/roles/src/proto/common.proto
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
syntax = "proto3";

package zksync.roles.common;

message GenesisHash {
optional bytes keccak256 = 1; // required
}
Loading

0 comments on commit b835dff

Please sign in to comment.