Further expand clippy workspace lints

Achieves a notable amount of reduced async and clones.
This commit is contained in:
Luke Parker
2023-12-17 00:01:41 -05:00
parent ea3af28139
commit 065d314e2a
113 changed files with 596 additions and 724 deletions

View File

@@ -85,13 +85,13 @@ impl FirstPreprocessDb {
network: NetworkId,
id_type: RecognizedIdType,
id: &[u8],
preprocess: Vec<Vec<u8>>,
preprocess: &Vec<Vec<u8>>,
) {
if let Some(existing) = FirstPreprocessDb::get(txn, network, id_type, id) {
assert_eq!(existing, preprocess, "saved a distinct first preprocess");
assert_eq!(&existing, preprocess, "saved a distinct first preprocess");
return;
}
FirstPreprocessDb::set(txn, network, id_type, id, &preprocess);
FirstPreprocessDb::set(txn, network, id_type, id, preprocess);
}
}
@@ -114,7 +114,7 @@ impl HandoverBatchDb {
}
}
impl QueuedBatchesDb {
pub fn queue(txn: &mut impl DbTxn, set: ValidatorSet, batch: Transaction) {
pub fn queue(txn: &mut impl DbTxn, set: ValidatorSet, batch: &Transaction) {
let mut batches = Self::get(txn, set).unwrap_or_default();
batch.write(&mut batches).unwrap();
Self::set(txn, set, &batches);

View File

@@ -159,17 +159,17 @@ async fn handle_processor_message<D: Db, P: P2p>(
// We'll only receive these if we fired GenerateKey, which we'll only do if if we're
// in-set, making the Tributary relevant
ProcessorMessage::KeyGen(inner_msg) => match inner_msg {
key_gen::ProcessorMessage::Commitments { id, .. } => Some(id.session),
key_gen::ProcessorMessage::InvalidCommitments { id, .. } => Some(id.session),
key_gen::ProcessorMessage::Shares { id, .. } => Some(id.session),
key_gen::ProcessorMessage::InvalidShare { id, .. } => Some(id.session),
key_gen::ProcessorMessage::GeneratedKeyPair { id, .. } => Some(id.session),
key_gen::ProcessorMessage::Commitments { id, .. } |
key_gen::ProcessorMessage::InvalidCommitments { id, .. } |
key_gen::ProcessorMessage::Shares { id, .. } |
key_gen::ProcessorMessage::InvalidShare { id, .. } |
key_gen::ProcessorMessage::GeneratedKeyPair { id, .. } |
key_gen::ProcessorMessage::Blame { id, .. } => Some(id.session),
},
ProcessorMessage::Sign(inner_msg) => match inner_msg {
// We'll only receive InvalidParticipant/Preprocess/Share if we're actively signing
sign::ProcessorMessage::InvalidParticipant { id, .. } => Some(id.session),
sign::ProcessorMessage::Preprocess { id, .. } => Some(id.session),
sign::ProcessorMessage::InvalidParticipant { id, .. } |
sign::ProcessorMessage::Preprocess { id, .. } |
sign::ProcessorMessage::Share { id, .. } => Some(id.session),
// While the Processor's Scanner will always emit Completed, that's routed through the
// Signer and only becomes a ProcessorMessage::Completed if the Signer is present and
@@ -233,9 +233,9 @@ async fn handle_processor_message<D: Db, P: P2p>(
None
}
// We'll only fire these if we are the Substrate signer, making the Tributary relevant
coordinator::ProcessorMessage::InvalidParticipant { id, .. } => Some(id.session),
coordinator::ProcessorMessage::CosignPreprocess { id, .. } => Some(id.session),
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => Some(id.session),
coordinator::ProcessorMessage::InvalidParticipant { id, .. } |
coordinator::ProcessorMessage::CosignPreprocess { id, .. } |
coordinator::ProcessorMessage::BatchPreprocess { id, .. } |
coordinator::ProcessorMessage::SubstrateShare { id, .. } => Some(id.session),
coordinator::ProcessorMessage::CosignedBlock { block_number, block, signature } => {
let cosigned_block = CosignedBlock {
@@ -486,7 +486,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
network,
RecognizedIdType::Plan,
&id.id,
preprocesses,
&preprocesses,
);
vec![]
@@ -566,7 +566,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
};
id.to_le_bytes()
},
preprocesses.into_iter().map(Into::into).collect(),
&preprocesses.into_iter().map(Into::into).collect::<Vec<_>>(),
);
let intended = Transaction::Batch {
@@ -611,8 +611,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
// the prior Batch hasn't been verified yet...
if (last_received != 0) &&
LastVerifiedBatchDb::get(&txn, msg.network)
.map(|last_verified| last_verified < (last_received - 1))
.unwrap_or(true)
.map_or(true, |last_verified| last_verified < (last_received - 1))
{
// Withhold this TX until we verify all prior `Batch`s
queue = true;
@@ -620,7 +619,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
}
if queue {
QueuedBatchesDb::queue(&mut txn, spec.set(), intended);
QueuedBatchesDb::queue(&mut txn, spec.set(), &intended);
vec![]
} else {
// Because this is post-verification of the handover batch, take all queued `Batch`s
@@ -650,10 +649,11 @@ async fn handle_processor_message<D: Db, P: P2p>(
signed: Transaction::empty_signed(),
})]
}
#[allow(clippy::match_same_arms)] // Allowed to preserve layout
coordinator::ProcessorMessage::CosignedBlock { .. } => unreachable!(),
},
ProcessorMessage::Substrate(inner_msg) => match inner_msg {
processor_messages::substrate::ProcessorMessage::Batch { .. } => unreachable!(),
processor_messages::substrate::ProcessorMessage::Batch { .. } |
processor_messages::substrate::ProcessorMessage::SignedBatch { .. } => unreachable!(),
},
};
@@ -823,9 +823,8 @@ async fn handle_cosigns_and_batch_publication<D: Db, P: P2p>(
let _hvq_lock = HANDOVER_VERIFY_QUEUE_LOCK.get_or_init(|| Mutex::new(())).lock().await;
let mut txn = db.txn();
let mut to_publish = vec![];
let start_id = LastVerifiedBatchDb::get(&txn, network)
.map(|already_verified| already_verified + 1)
.unwrap_or(0);
let start_id =
LastVerifiedBatchDb::get(&txn, network).map_or(0, |already_verified| already_verified + 1);
if let Some(last_id) =
substrate::verify_published_batches::<D>(&mut txn, network, u32::MAX).await
{
@@ -847,7 +846,7 @@ async fn handle_cosigns_and_batch_publication<D: Db, P: P2p>(
to_publish.push((set.session, queued.remove(0)));
// Re-queue the remaining batches
for remaining in queued {
QueuedBatchesDb::queue(&mut txn, set, remaining);
QueuedBatchesDb::queue(&mut txn, set, &remaining);
}
}

View File

@@ -59,11 +59,10 @@ pub enum P2pMessageKind {
impl P2pMessageKind {
fn genesis(&self) -> Option<[u8; 32]> {
match self {
P2pMessageKind::KeepAlive => None,
P2pMessageKind::Tributary(genesis) => Some(*genesis),
P2pMessageKind::Heartbeat(genesis) => Some(*genesis),
P2pMessageKind::KeepAlive | P2pMessageKind::CosignedBlock => None,
P2pMessageKind::Tributary(genesis) |
P2pMessageKind::Heartbeat(genesis) |
P2pMessageKind::Block(genesis) => Some(*genesis),
P2pMessageKind::CosignedBlock => None,
}
}
@@ -303,7 +302,7 @@ impl LibP2p {
let mut time_of_last_p2p_message = Instant::now();
#[allow(clippy::needless_pass_by_ref_mut)] // False positive
async fn broadcast_raw(
fn broadcast_raw(
p2p: &mut Swarm<Behavior>,
time_of_last_p2p_message: &mut Instant,
genesis: Option<[u8; 32]>,
@@ -364,7 +363,7 @@ impl LibP2p {
&mut time_of_last_p2p_message,
genesis,
msg,
).await;
);
}
// Handle new incoming messages
@@ -416,7 +415,7 @@ impl LibP2p {
&mut time_of_last_p2p_message,
None,
P2pMessageKind::KeepAlive.serialize()
).await;
);
}
}
}
@@ -689,16 +688,8 @@ pub async fn handle_p2p_task<D: Db, P: P2p>(
let msg = p2p.receive().await;
match msg.kind {
P2pMessageKind::KeepAlive => {}
P2pMessageKind::Tributary(genesis) => {
if let Some(channel) = channels.read().await.get(&genesis) {
channel.send(msg).unwrap();
}
}
P2pMessageKind::Heartbeat(genesis) => {
if let Some(channel) = channels.read().await.get(&genesis) {
channel.send(msg).unwrap();
}
}
P2pMessageKind::Tributary(genesis) |
P2pMessageKind::Heartbeat(genesis) |
P2pMessageKind::Block(genesis) => {
if let Some(channel) = channels.read().await.get(&genesis) {
channel.send(msg).unwrap();

View File

@@ -18,7 +18,7 @@ pub use inner_db::{NextBlock, BatchInstructionsHashDb};
pub struct HandledEvent;
impl HandledEvent {
fn next_to_handle_event(getter: &impl Get, block: [u8; 32]) -> u32 {
inner_db::HandledEvent::get(getter, block).map(|last| last + 1).unwrap_or(0)
inner_db::HandledEvent::get(getter, block).map_or(0, |last| last + 1)
}
pub fn is_unhandled(getter: &impl Get, block: [u8; 32], event_id: u32) -> bool {
let next = Self::next_to_handle_event(getter, block);

View File

@@ -396,9 +396,8 @@ pub async fn scan_task<D: Db, Pro: Processors>(
Ok(latest) => {
if latest.header.number >= next_substrate_block {
return latest;
} else {
sleep(Duration::from_secs(3)).await;
}
sleep(Duration::from_secs(3)).await;
}
Err(e) => {
log::error!("couldn't communicate with serai node: {e}");
@@ -493,7 +492,7 @@ pub(crate) async fn verify_published_batches<D: Db>(
) -> Option<u32> {
// TODO: Localize from MainDb to SubstrateDb
let last = crate::LastVerifiedBatchDb::get(txn, network);
for id in last.map(|last| last + 1).unwrap_or(0) ..= optimistic_up_to {
for id in last.map_or(0, |last| last + 1) ..= optimistic_up_to {
let Some(on_chain) = BatchInstructionsHashDb::get(txn, network, id) else {
break;
};

View File

@@ -60,7 +60,7 @@ fn random_sign_data<R: RngCore, Id: Clone + PartialEq + Eq + Debug + Encode + De
data: {
let mut res = vec![];
for _ in 0 .. ((rng.next_u64() % 255) + 1) {
for _ in 0 ..= (rng.next_u64() % 256) {
res.push(random_vec(&mut OsRng, 512));
}
res
@@ -70,8 +70,8 @@ fn random_sign_data<R: RngCore, Id: Clone + PartialEq + Eq + Debug + Encode + De
}
}
fn test_read_write<RW: Eq + Debug + ReadWrite>(value: RW) {
assert_eq!(value, RW::read::<&[u8]>(&mut value.serialize().as_ref()).unwrap());
fn test_read_write<RW: Eq + Debug + ReadWrite>(value: &RW) {
assert_eq!(value, &RW::read::<&[u8]>(&mut value.serialize().as_ref()).unwrap());
}
#[test]
@@ -102,36 +102,36 @@ fn tx_size_limit() {
#[test]
fn serialize_sign_data() {
fn test_read_write<Id: Clone + PartialEq + Eq + Debug + Encode + Decode>(value: SignData<Id>) {
fn test_read_write<Id: Clone + PartialEq + Eq + Debug + Encode + Decode>(value: &SignData<Id>) {
let mut buf = vec![];
value.write(&mut buf).unwrap();
assert_eq!(value, SignData::read(&mut buf.as_slice()).unwrap())
assert_eq!(value, &SignData::read(&mut buf.as_slice()).unwrap())
}
let mut plan = [0; 3];
OsRng.fill_bytes(&mut plan);
test_read_write(random_sign_data::<_, _>(
test_read_write(&random_sign_data::<_, _>(
&mut OsRng,
plan,
if (OsRng.next_u64() % 2) == 0 { Label::Preprocess } else { Label::Share },
));
let mut plan = [0; 5];
OsRng.fill_bytes(&mut plan);
test_read_write(random_sign_data::<_, _>(
test_read_write(&random_sign_data::<_, _>(
&mut OsRng,
plan,
if (OsRng.next_u64() % 2) == 0 { Label::Preprocess } else { Label::Share },
));
let mut plan = [0; 8];
OsRng.fill_bytes(&mut plan);
test_read_write(random_sign_data::<_, _>(
test_read_write(&random_sign_data::<_, _>(
&mut OsRng,
plan,
if (OsRng.next_u64() % 2) == 0 { Label::Preprocess } else { Label::Share },
));
let mut plan = [0; 24];
OsRng.fill_bytes(&mut plan);
test_read_write(random_sign_data::<_, _>(
test_read_write(&random_sign_data::<_, _>(
&mut OsRng,
plan,
if (OsRng.next_u64() % 2) == 0 { Label::Preprocess } else { Label::Share },
@@ -140,7 +140,7 @@ fn serialize_sign_data() {
#[test]
fn serialize_transaction() {
test_read_write(Transaction::RemoveParticipantDueToDkg {
test_read_write(&Transaction::RemoveParticipantDueToDkg {
attempt: u32::try_from(OsRng.next_u64() >> 32).unwrap(),
participant: frost::Participant::new(
u16::try_from(OsRng.next_u64() >> 48).unwrap().saturating_add(1),
@@ -155,7 +155,7 @@ fn serialize_transaction() {
OsRng.fill_bytes(&mut temp);
commitments.push(temp);
}
test_read_write(Transaction::DkgCommitments {
test_read_write(&Transaction::DkgCommitments {
attempt: random_u32(&mut OsRng),
commitments,
signed: random_signed_with_nonce(&mut OsRng, 0),
@@ -170,7 +170,7 @@ fn serialize_transaction() {
// Create a valid vec of shares
let mut shares = vec![];
// Create up to 150 participants
for _ in 0 .. ((OsRng.next_u64() % 150) + 1) {
for _ in 0 ..= (OsRng.next_u64() % 150) {
// Give each sender multiple shares
let mut sender_shares = vec![];
for _ in 0 .. amount_of_shares {
@@ -181,7 +181,7 @@ fn serialize_transaction() {
shares.push(sender_shares);
}
test_read_write(Transaction::DkgShares {
test_read_write(&Transaction::DkgShares {
attempt: random_u32(&mut OsRng),
shares,
confirmation_nonces: {
@@ -194,7 +194,7 @@ fn serialize_transaction() {
}
for i in 0 .. 2 {
test_read_write(Transaction::InvalidDkgShare {
test_read_write(&Transaction::InvalidDkgShare {
attempt: random_u32(&mut OsRng),
accuser: frost::Participant::new(
u16::try_from(OsRng.next_u64() >> 48).unwrap().saturating_add(1),
@@ -213,7 +213,7 @@ fn serialize_transaction() {
});
}
test_read_write(Transaction::DkgConfirmed {
test_read_write(&Transaction::DkgConfirmed {
attempt: random_u32(&mut OsRng),
confirmation_share: {
let mut share = [0; 32];
@@ -226,20 +226,20 @@ fn serialize_transaction() {
{
let mut block = [0; 32];
OsRng.fill_bytes(&mut block);
test_read_write(Transaction::CosignSubstrateBlock(block));
test_read_write(&Transaction::CosignSubstrateBlock(block));
}
{
let mut block = [0; 32];
OsRng.fill_bytes(&mut block);
let batch = u32::try_from(OsRng.next_u64() >> 32).unwrap();
test_read_write(Transaction::Batch { block, batch });
test_read_write(&Transaction::Batch { block, batch });
}
test_read_write(Transaction::SubstrateBlock(OsRng.next_u64()));
test_read_write(&Transaction::SubstrateBlock(OsRng.next_u64()));
{
let batch = u32::try_from(OsRng.next_u64() >> 32).unwrap();
test_read_write(Transaction::SubstrateSign(random_sign_data(
test_read_write(&Transaction::SubstrateSign(random_sign_data(
&mut OsRng,
SubstrateSignableId::Batch(batch),
Label::Preprocess,
@@ -247,7 +247,7 @@ fn serialize_transaction() {
}
{
let batch = u32::try_from(OsRng.next_u64() >> 32).unwrap();
test_read_write(Transaction::SubstrateSign(random_sign_data(
test_read_write(&Transaction::SubstrateSign(random_sign_data(
&mut OsRng,
SubstrateSignableId::Batch(batch),
Label::Share,
@@ -257,12 +257,12 @@ fn serialize_transaction() {
{
let mut plan = [0; 32];
OsRng.fill_bytes(&mut plan);
test_read_write(Transaction::Sign(random_sign_data(&mut OsRng, plan, Label::Preprocess)));
test_read_write(&Transaction::Sign(random_sign_data(&mut OsRng, plan, Label::Preprocess)));
}
{
let mut plan = [0; 32];
OsRng.fill_bytes(&mut plan);
test_read_write(Transaction::Sign(random_sign_data(&mut OsRng, plan, Label::Share)));
test_read_write(&Transaction::Sign(random_sign_data(&mut OsRng, plan, Label::Share)));
}
{
@@ -270,7 +270,7 @@ fn serialize_transaction() {
OsRng.fill_bytes(&mut plan);
let mut tx_hash = vec![0; (OsRng.next_u64() % 64).try_into().unwrap()];
OsRng.fill_bytes(&mut tx_hash);
test_read_write(Transaction::SignCompleted {
test_read_write(&Transaction::SignCompleted {
plan,
tx_hash,
first_signer: random_signed_with_nonce(&mut OsRng, 2).signer,

View File

@@ -204,18 +204,18 @@ impl<
Accumulation::NotReady
}
async fn handle_data(
fn handle_data(
&mut self,
removed: &[<Ristretto as Ciphersuite>::G],
data_spec: &DataSpecification,
bytes: Vec<u8>,
bytes: &Vec<u8>,
signed: &Signed,
) -> Accumulation {
let genesis = self.spec.genesis();
let Some(curr_attempt) = AttemptDb::attempt(self.txn, genesis, data_spec.topic) else {
// Premature publication of a valid ID/publication of an invalid ID
self.fatal_slash(signed.signer.to_bytes(), "published data for ID without an attempt").await;
self.fatal_slash(signed.signer.to_bytes(), "published data for ID without an attempt");
return Accumulation::NotReady;
};
@@ -223,7 +223,7 @@ impl<
// This shouldn't be reachable since nonces were made inserted by the coordinator, yet it's a
// cheap check to leave in for safety
if DataDb::get(self.txn, genesis, data_spec, &signed.signer.to_bytes()).is_some() {
self.fatal_slash(signed.signer.to_bytes(), "published data multiple times").await;
self.fatal_slash(signed.signer.to_bytes(), "published data multiple times");
return Accumulation::NotReady;
}
@@ -239,12 +239,10 @@ impl<
}
// If the attempt is greater, this is a premature publication, full slash
if data_spec.attempt > curr_attempt {
self
.fatal_slash(
signed.signer.to_bytes(),
"published data with an attempt which hasn't started",
)
.await;
self.fatal_slash(
signed.signer.to_bytes(),
"published data with an attempt which hasn't started",
);
return Accumulation::NotReady;
}
@@ -254,10 +252,10 @@ impl<
// TODO: If this is shares, we need to check they are part of the selected signing set
// Accumulate this data
self.accumulate(removed, data_spec, signed.signer, &bytes)
self.accumulate(removed, data_spec, signed.signer, bytes)
}
async fn check_sign_data_len(
fn check_sign_data_len(
&mut self,
removed: &[<Ristretto as Ciphersuite>::G],
signer: <Ristretto as Ciphersuite>::G,
@@ -265,12 +263,10 @@ impl<
) -> Result<(), ()> {
let signer_i = self.spec.i(removed, signer).unwrap();
if len != usize::from(u16::from(signer_i.end) - u16::from(signer_i.start)) {
self
.fatal_slash(
signer.to_bytes(),
"signer published a distinct amount of sign data than they had shares",
)
.await;
self.fatal_slash(
signer.to_bytes(),
"signer published a distinct amount of sign data than they had shares",
);
Err(())?;
}
Ok(())
@@ -292,34 +288,28 @@ impl<
}
match tx {
Transaction::RemoveParticipantDueToDkg { attempt, participant } => {
self
.fatal_slash_with_participant_index(
&removed_as_of_dkg_attempt(self.txn, genesis, attempt).unwrap_or_else(|| {
panic!(
"removed a participant due to a provided transaction with an attempt not {}",
"locally handled?"
)
}),
participant,
"RemoveParticipantDueToDkg Provided TX",
)
.await
}
Transaction::RemoveParticipantDueToDkg { attempt, participant } => self
.fatal_slash_with_participant_index(
&removed_as_of_dkg_attempt(self.txn, genesis, attempt).unwrap_or_else(|| {
panic!(
"removed a participant due to a provided transaction with an attempt not {}",
"locally handled?"
)
}),
participant,
"RemoveParticipantDueToDkg Provided TX",
),
Transaction::DkgCommitments { attempt, commitments, signed } => {
let Some(removed) = removed_as_of_dkg_attempt(self.txn, genesis, attempt) else {
self
.fatal_slash(signed.signer.to_bytes(), "DkgCommitments with an unrecognized attempt")
.await;
self.fatal_slash(signed.signer.to_bytes(), "DkgCommitments with an unrecognized attempt");
return;
};
let Ok(()) = self.check_sign_data_len(&removed, signed.signer, commitments.len()).await
else {
let Ok(()) = self.check_sign_data_len(&removed, signed.signer, commitments.len()) else {
return;
};
let data_spec = DataSpecification { topic: Topic::Dkg, label: Label::Preprocess, attempt };
match self.handle_data(&removed, &data_spec, commitments.encode(), &signed).await {
match self.handle_data(&removed, &data_spec, &commitments.encode(), &signed) {
Accumulation::Ready(DataSet::Participating(mut commitments)) => {
log::info!("got all DkgCommitments for {}", hex::encode(genesis));
unflatten(self.spec, &removed, &mut commitments);
@@ -343,12 +333,10 @@ impl<
Transaction::DkgShares { attempt, mut shares, confirmation_nonces, signed } => {
let Some(removed) = removed_as_of_dkg_attempt(self.txn, genesis, attempt) else {
self
.fatal_slash(signed.signer.to_bytes(), "DkgShares with an unrecognized attempt")
.await;
self.fatal_slash(signed.signer.to_bytes(), "DkgShares with an unrecognized attempt");
return;
};
let Ok(()) = self.check_sign_data_len(&removed, signed.signer, shares.len()).await else {
let Ok(()) = self.check_sign_data_len(&removed, signed.signer, shares.len()) else {
return;
};
@@ -359,7 +347,7 @@ impl<
let sender_is_len = u16::from(sender_i.end) - u16::from(sender_i.start);
for shares in &shares {
if shares.len() != (usize::from(self.spec.n(&removed) - sender_is_len)) {
self.fatal_slash(signed.signer.to_bytes(), "invalid amount of DKG shares").await;
self.fatal_slash(signed.signer.to_bytes(), "invalid amount of DKG shares");
return;
}
}
@@ -419,7 +407,7 @@ impl<
let data_spec = DataSpecification { topic: Topic::Dkg, label: Label::Share, attempt };
let encoded_data = (confirmation_nonces.to_vec(), our_shares.encode()).encode();
match self.handle_data(&removed, &data_spec, encoded_data, &signed).await {
match self.handle_data(&removed, &data_spec, &encoded_data, &signed) {
Accumulation::Ready(DataSet::Participating(confirmation_nonces_and_shares)) => {
log::info!("got all DkgShares for {}", hex::encode(genesis));
@@ -479,34 +467,27 @@ impl<
Transaction::InvalidDkgShare { attempt, accuser, faulty, blame, signed } => {
let Some(removed) = removed_as_of_dkg_attempt(self.txn, genesis, attempt) else {
self
.fatal_slash(signed.signer.to_bytes(), "InvalidDkgShare with an unrecognized attempt")
.await;
.fatal_slash(signed.signer.to_bytes(), "InvalidDkgShare with an unrecognized attempt");
return;
};
let range = self.spec.i(&removed, signed.signer).unwrap();
if !range.contains(&accuser) {
self
.fatal_slash(
signed.signer.to_bytes(),
"accused with a Participant index which wasn't theirs",
)
.await;
self.fatal_slash(
signed.signer.to_bytes(),
"accused with a Participant index which wasn't theirs",
);
return;
}
if range.contains(&faulty) {
self
.fatal_slash(signed.signer.to_bytes(), "accused self of having an InvalidDkgShare")
.await;
self.fatal_slash(signed.signer.to_bytes(), "accused self of having an InvalidDkgShare");
return;
}
let Some(share) = DkgShare::get(self.txn, genesis, accuser.into(), faulty.into()) else {
self
.fatal_slash(
signed.signer.to_bytes(),
"InvalidDkgShare had a non-existent faulty participant",
)
.await;
self.fatal_slash(
signed.signer.to_bytes(),
"InvalidDkgShare had a non-existent faulty participant",
);
return;
};
self
@@ -526,15 +507,13 @@ impl<
Transaction::DkgConfirmed { attempt, confirmation_share, signed } => {
let Some(removed) = removed_as_of_dkg_attempt(self.txn, genesis, attempt) else {
self
.fatal_slash(signed.signer.to_bytes(), "DkgConfirmed with an unrecognized attempt")
.await;
self.fatal_slash(signed.signer.to_bytes(), "DkgConfirmed with an unrecognized attempt");
return;
};
let data_spec =
DataSpecification { topic: Topic::DkgConfirmation, label: Label::Share, attempt };
match self.handle_data(&removed, &data_spec, confirmation_share.to_vec(), &signed).await {
match self.handle_data(&removed, &data_spec, &confirmation_share.to_vec(), &signed) {
Accumulation::Ready(DataSet::Participating(shares)) => {
log::info!("got all DkgConfirmed for {}", hex::encode(genesis));
@@ -556,9 +535,7 @@ impl<
let sig = match confirmer.complete(preprocesses, &key_pair, shares) {
Ok(sig) => sig,
Err(p) => {
self
.fatal_slash_with_participant_index(&removed, p, "invalid DkgConfirmer share")
.await;
self.fatal_slash_with_participant_index(&removed, p, "invalid DkgConfirmer share");
return;
}
};
@@ -641,16 +618,14 @@ impl<
let Some(removed) =
crate::tributary::removed_as_of_set_keys(self.txn, self.spec.set(), genesis)
else {
self
.fatal_slash(
data.signed.signer.to_bytes(),
"signing despite not having set keys on substrate",
)
.await;
self.fatal_slash(
data.signed.signer.to_bytes(),
"signing despite not having set keys on substrate",
);
return;
};
let signer = data.signed.signer;
let Ok(()) = self.check_sign_data_len(&removed, signer, data.data.len()).await else {
let Ok(()) = self.check_sign_data_len(&removed, signer, data.data.len()) else {
return;
};
let expected_len = match data.label {
@@ -659,12 +634,10 @@ impl<
};
for data in &data.data {
if data.len() != expected_len {
self
.fatal_slash(
signer.to_bytes(),
"unexpected length data for substrate signing protocol",
)
.await;
self.fatal_slash(
signer.to_bytes(),
"unexpected length data for substrate signing protocol",
);
return;
}
}
@@ -675,7 +648,7 @@ impl<
attempt: data.attempt,
};
let Accumulation::Ready(DataSet::Participating(mut results)) =
self.handle_data(&removed, &data_spec, data.data.encode(), &data.signed).await
self.handle_data(&removed, &data_spec, &data.data.encode(), &data.signed)
else {
return;
};
@@ -703,16 +676,13 @@ impl<
let Some(removed) =
crate::tributary::removed_as_of_set_keys(self.txn, self.spec.set(), genesis)
else {
self
.fatal_slash(
data.signed.signer.to_bytes(),
"signing despite not having set keys on substrate",
)
.await;
self.fatal_slash(
data.signed.signer.to_bytes(),
"signing despite not having set keys on substrate",
);
return;
};
let Ok(()) = self.check_sign_data_len(&removed, data.signed.signer, data.data.len()).await
else {
let Ok(()) = self.check_sign_data_len(&removed, data.signed.signer, data.data.len()) else {
return;
};
@@ -722,7 +692,7 @@ impl<
attempt: data.attempt,
};
if let Accumulation::Ready(DataSet::Participating(mut results)) =
self.handle_data(&removed, &data_spec, data.data.encode(), &data.signed).await
self.handle_data(&removed, &data_spec, &data.data.encode(), &data.signed)
{
unflatten(self.spec, &removed, &mut results);
let id =
@@ -750,9 +720,7 @@ impl<
);
if AttemptDb::attempt(self.txn, genesis, Topic::Sign(plan)).is_none() {
self
.fatal_slash(first_signer.to_bytes(), "claimed an unrecognized plan was completed")
.await;
self.fatal_slash(first_signer.to_bytes(), "claimed an unrecognized plan was completed");
return;
};

View File

@@ -192,7 +192,7 @@ impl<
P: P2p,
> TributaryBlockHandler<'_, T, Pro, PST, PTT, RID, P>
{
pub async fn fatal_slash(&mut self, slashing: [u8; 32], reason: &str) {
pub fn fatal_slash(&mut self, slashing: [u8; 32], reason: &str) {
// TODO: If this fatal slash puts the remaining set below the threshold, spin
let genesis = self.spec.genesis();
@@ -209,7 +209,7 @@ impl<
// Tributary post-DKG
// https://github.com/serai-dex/serai/issues/426
pub async fn fatal_slash_with_participant_index(
pub fn fatal_slash_with_participant_index(
&mut self,
removed: &[<Ristretto as Ciphersuite>::G],
i: Participant,
@@ -227,7 +227,7 @@ impl<
}
let validator = validator.unwrap();
self.fatal_slash(validator.to_bytes(), reason).await;
self.fatal_slash(validator.to_bytes(), reason);
}
async fn handle<D: Db>(mut self) {
@@ -240,10 +240,9 @@ impl<
// Since the evidence is on the chain, it should already have been validated
// We can just punish the signer
let data = match ev {
Evidence::ConflictingMessages(first, second) => (first, Some(second)),
Evidence::ConflictingMessages(first, second) |
Evidence::ConflictingPrecommit(first, second) => (first, Some(second)),
Evidence::InvalidPrecommit(first) => (first, None),
Evidence::InvalidValidRound(first) => (first, None),
Evidence::InvalidPrecommit(first) | Evidence::InvalidValidRound(first) => (first, None),
};
let msgs = (
decode_signed_message::<TendermintNetwork<D, Transaction, P>>(&data.0).unwrap(),
@@ -259,9 +258,7 @@ impl<
// Since anything with evidence is fundamentally faulty behavior, not just temporal
// errors, mark the node as fatally slashed
self
.fatal_slash(msgs.0.msg.sender, &format!("invalid tendermint messages: {:?}", msgs))
.await;
self.fatal_slash(msgs.0.msg.sender, &format!("invalid tendermint messages: {msgs:?}"));
}
TributaryTransaction::Application(tx) => {
self.handle_application_tx(tx).await;
@@ -348,8 +345,7 @@ impl<
// Check if the cosigner has a signature from our set for this block/a newer one
let latest_cosign =
crate::cosign_evaluator::LatestCosign::get(self.txn, self.spec.set().network)
.map(|cosign| cosign.block_number)
.unwrap_or(0);
.map_or(0, |cosign| cosign.block_number);
if latest_cosign < block_number {
// Instruct the processor to start the next attempt
self

View File

@@ -184,7 +184,6 @@ impl<T: DbTxn, C: Encode> SigningProtocol<'_, T, C> {
}
fn complete_internal(
&mut self,
machine: AlgorithmSignatureMachine<Ristretto, Schnorrkel>,
shares: HashMap<Participant, Vec<u8>>,
) -> Result<[u8; 64], Participant> {
@@ -251,6 +250,8 @@ fn threshold_i_map_to_keys_and_musig_i_map(
(participants, map)
}
type DkgConfirmerSigningProtocol<'a, T> = SigningProtocol<'a, T, (&'static [u8; 12], u32)>;
pub(crate) struct DkgConfirmer<'a, T: DbTxn> {
key: &'a Zeroizing<<Ristretto as Ciphersuite>::F>,
spec: &'a TributarySpec,
@@ -271,7 +272,7 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
let removed = crate::tributary::removed_as_of_dkg_attempt(txn, spec.genesis(), attempt)?;
Some(DkgConfirmer { key, spec, removed, txn, attempt })
}
fn signing_protocol(&mut self) -> SigningProtocol<'_, T, (&'static [u8; 12], u32)> {
fn signing_protocol(&mut self) -> DkgConfirmerSigningProtocol<'_, T> {
let context = (b"DkgConfirmer", self.attempt);
SigningProtocol { key: self.key, spec: self.spec, txn: self.txn, context }
}
@@ -323,6 +324,6 @@ impl<T: DbTxn> DkgConfirmer<'_, T> {
.expect("trying to complete a machine which failed to preprocess")
.0;
self.signing_protocol().complete_internal(machine, shares)
DkgConfirmerSigningProtocol::<'_, T>::complete_internal(machine, shares)
}
}

View File

@@ -489,7 +489,7 @@ impl ReadWrite for Transaction {
writer.write_all(&u16::from(*faulty).to_le_bytes())?;
// Flattens Some(vec![]) to None on the expectation no actual blame will be 0-length
assert!(blame.as_ref().map(|blame| blame.len()).unwrap_or(1) != 0);
assert!(blame.as_ref().map_or(1, Vec::len) != 0);
let blame_len =
u16::try_from(blame.as_ref().unwrap_or(&vec![]).len()).expect("blame exceeded 64 KB");
writer.write_all(&blame_len.to_le_bytes())?;
@@ -547,15 +547,9 @@ impl TransactionTrait for Transaction {
match self {
Transaction::RemoveParticipantDueToDkg { .. } => TransactionKind::Provided("remove"),
Transaction::DkgCommitments { attempt, commitments: _, signed } => {
TransactionKind::Signed((b"dkg", attempt).encode(), signed)
}
Transaction::DkgShares { attempt, signed, .. } => {
TransactionKind::Signed((b"dkg", attempt).encode(), signed)
}
Transaction::InvalidDkgShare { attempt, signed, .. } => {
TransactionKind::Signed((b"dkg", attempt).encode(), signed)
}
Transaction::DkgCommitments { attempt, commitments: _, signed } |
Transaction::DkgShares { attempt, signed, .. } |
Transaction::InvalidDkgShare { attempt, signed, .. } |
Transaction::DkgConfirmed { attempt, signed, .. } => {
TransactionKind::Signed((b"dkg", attempt).encode(), signed)
}
@@ -625,8 +619,7 @@ impl Transaction {
Transaction::DkgCommitments { .. } => 0,
Transaction::DkgShares { .. } => 1,
Transaction::InvalidDkgShare { .. } => 2,
Transaction::DkgConfirmed { .. } => 2,
Transaction::InvalidDkgShare { .. } | Transaction::DkgConfirmed { .. } => 2,
Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"),
@@ -635,6 +628,7 @@ impl Transaction {
Transaction::SubstrateSign(data) => data.label.nonce(),
Transaction::Sign(data) => data.label.nonce(),
Transaction::SignCompleted { .. } => panic!("signing SignCompleted"),
};
@@ -643,9 +637,9 @@ impl Transaction {
match tx {
Transaction::RemoveParticipantDueToDkg { .. } => panic!("signing RemoveParticipant"),
Transaction::DkgCommitments { ref mut signed, .. } => signed,
Transaction::DkgShares { ref mut signed, .. } => signed,
Transaction::InvalidDkgShare { ref mut signed, .. } => signed,
Transaction::DkgCommitments { ref mut signed, .. } |
Transaction::DkgShares { ref mut signed, .. } |
Transaction::InvalidDkgShare { ref mut signed, .. } |
Transaction::DkgConfirmed { ref mut signed, .. } => signed,
Transaction::CosignSubstrateBlock(_) => panic!("signing CosignSubstrateBlock"),
@@ -655,6 +649,7 @@ impl Transaction {
Transaction::SubstrateSign(ref mut data) => &mut data.signed,
Transaction::Sign(ref mut data) => &mut data.signed,
Transaction::SignCompleted { .. } => panic!("signing SignCompleted"),
},
)

View File

@@ -174,7 +174,7 @@ impl<T: TransactionTrait> Block<T> {
last_block: [u8; 32],
mut locally_provided: HashMap<&'static str, VecDeque<T>>,
get_and_increment_nonce: &mut G,
schema: N::SignatureScheme,
schema: &N::SignatureScheme,
commit: impl Fn(u32) -> Option<Commit<N::SignatureScheme>>,
unsigned_in_chain: impl Fn([u8; 32]) -> bool,
provided_in_chain: impl Fn([u8; 32]) -> bool, // TODO: merge this with unsigned_on_chain?
@@ -217,7 +217,7 @@ impl<T: TransactionTrait> Block<T> {
Err(BlockError::ProvidedAlreadyIncluded)?;
}
if let Some(local) = locally_provided.get_mut(order).and_then(|deque| deque.pop_front()) {
if let Some(local) = locally_provided.get_mut(order).and_then(VecDeque::pop_front) {
// Since this was a provided TX, it must be an application TX
let Transaction::Application(tx) = tx else {
Err(BlockError::NonLocalProvided(txs.pop().unwrap()))?
@@ -250,12 +250,10 @@ impl<T: TransactionTrait> Block<T> {
last_tx_order = current_tx_order;
match tx {
Transaction::Tendermint(tx) => {
match verify_tendermint_tx::<N>(tx, schema.clone(), &commit) {
Ok(()) => {}
Err(e) => Err(BlockError::TransactionError(e))?,
}
}
Transaction::Tendermint(tx) => match verify_tendermint_tx::<N>(tx, schema, &commit) {
Ok(()) => {}
Err(e) => Err(BlockError::TransactionError(e))?,
},
Transaction::Application(tx) => {
match verify_transaction(tx, genesis, get_and_increment_nonce) {
Ok(()) => {}

View File

@@ -139,25 +139,23 @@ impl<D: Db, T: TransactionTrait> Blockchain<D, T> {
order: &str,
) -> bool {
let local_key = ProvidedTransactions::<D, T>::locally_provided_quantity_key(genesis, order);
let local =
db.get(local_key).map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap())).unwrap_or(0);
let local = db.get(local_key).map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap()));
let block_key =
ProvidedTransactions::<D, T>::block_provided_quantity_key(genesis, block, order);
let block =
db.get(block_key).map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap())).unwrap_or(0);
let block = db.get(block_key).map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap()));
local >= block
}
pub(crate) fn tip_from_db(db: &D, genesis: [u8; 32]) -> [u8; 32] {
db.get(Self::tip_key(genesis)).map(|bytes| bytes.try_into().unwrap()).unwrap_or(genesis)
db.get(Self::tip_key(genesis)).map_or(genesis, |bytes| bytes.try_into().unwrap())
}
pub(crate) fn add_transaction<N: Network>(
&mut self,
internal: bool,
tx: Transaction<T>,
schema: N::SignatureScheme,
schema: &N::SignatureScheme,
) -> Result<bool, TransactionError> {
let db = self.db.as_ref().unwrap();
let genesis = self.genesis;
@@ -177,8 +175,7 @@ impl<D: Db, T: TransactionTrait> Blockchain<D, T> {
if self.participants.contains(&signer) {
Some(
db.get(Self::next_nonce_key(&self.genesis, &signer, &order))
.map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap()))
.unwrap_or(0),
.map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap())),
)
} else {
None
@@ -211,15 +208,14 @@ impl<D: Db, T: TransactionTrait> Blockchain<D, T> {
.as_ref()
.unwrap()
.get(Self::next_nonce_key(&self.genesis, signer, order))
.map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap()))
.unwrap_or(0),
.map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap())),
)
} else {
None
}
}
pub(crate) fn build_block<N: Network>(&mut self, schema: N::SignatureScheme) -> Block<T> {
pub(crate) fn build_block<N: Network>(&mut self, schema: &N::SignatureScheme) -> Block<T> {
let block = Block::new(
self.tip,
self.provided.transactions.values().flatten().cloned().collect(),
@@ -233,7 +229,7 @@ impl<D: Db, T: TransactionTrait> Blockchain<D, T> {
pub(crate) fn verify_block<N: Network>(
&self,
block: &Block<T>,
schema: N::SignatureScheme,
schema: &N::SignatureScheme,
allow_non_local_provided: bool,
) -> Result<(), BlockError> {
let db = self.db.as_ref().unwrap();
@@ -258,8 +254,7 @@ impl<D: Db, T: TransactionTrait> Blockchain<D, T> {
let key = Self::next_nonce_key(&self.genesis, signer, order);
let next = txn
.get(&key)
.map(|next_nonce| u32::from_le_bytes(next_nonce.try_into().unwrap()))
.unwrap_or(0);
.map_or(0, |next_nonce| u32::from_le_bytes(next_nonce.try_into().unwrap()));
txn.put(key, (next + 1).to_le_bytes());
Some(next)
} else {
@@ -282,7 +277,7 @@ impl<D: Db, T: TransactionTrait> Blockchain<D, T> {
&mut self,
block: &Block<T>,
commit: Vec<u8>,
schema: N::SignatureScheme,
schema: &N::SignatureScheme,
) -> Result<(), BlockError> {
self.verify_block::<N>(block, schema, true)?;

View File

@@ -190,7 +190,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Tributary<D, T, P> {
start_time
};
let proposal = TendermintBlock(
blockchain.build_block::<TendermintNetwork<D, T, P>>(validators.clone()).serialize(),
blockchain.build_block::<TendermintNetwork<D, T, P>>(&validators).serialize(),
);
let blockchain = Arc::new(RwLock::new(blockchain));
@@ -273,7 +273,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Tributary<D, T, P> {
let res = self.network.blockchain.write().await.add_transaction::<TendermintNetwork<D, T, P>>(
true,
tx,
self.network.signature_scheme(),
&self.network.signature_scheme(),
);
if res == Ok(true) {
self.network.p2p.broadcast(self.genesis, to_broadcast).await;
@@ -344,7 +344,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Tributary<D, T, P> {
self.network.blockchain.write().await.add_transaction::<TendermintNetwork<D, T, P>>(
false,
tx,
self.network.signature_scheme(),
&self.network.signature_scheme(),
);
log::debug!("received transaction message. valid new transaction: {res:?}");
res == Ok(true)

View File

@@ -112,7 +112,7 @@ impl<D: Db, T: TransactionTrait> Mempool<D, T> {
blockchain_next_nonce: F,
internal: bool,
tx: Transaction<T>,
schema: N::SignatureScheme,
schema: &N::SignatureScheme,
unsigned_in_chain: impl Fn([u8; 32]) -> bool,
commit: impl Fn(u32) -> Option<Commit<N::SignatureScheme>>,
) -> Result<bool, TransactionError> {

View File

@@ -17,13 +17,7 @@ pub(crate) fn merkle(hash_args: &[[u8; 32]]) -> [u8; 32] {
[
b"branch_hash".as_ref(),
hashes[i].as_ref(),
hashes
.get(i + 1)
.map(|hash| {
let res: &[u8] = hash.as_ref();
res
})
.unwrap_or(zero.as_ref()),
hashes.get(i + 1).map_or(zero.as_ref(), AsRef::as_ref),
]
.concat(),
));
@@ -33,5 +27,5 @@ pub(crate) fn merkle(hash_args: &[[u8; 32]]) -> [u8; 32] {
hashes = interim;
}
hashes.first().copied().map(Into::into).unwrap_or(zero)
hashes.first().copied().map_or(zero, Into::into)
}

View File

@@ -103,17 +103,11 @@ impl<D: Db, T: Transaction> ProvidedTransactions<D, T> {
// get local and on-chain tx numbers
let local_key = Self::locally_provided_quantity_key(&self.genesis, order);
let mut local_quantity = self
.db
.get(&local_key)
.map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap()))
.unwrap_or(0);
let mut local_quantity =
self.db.get(&local_key).map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap()));
let on_chain_key = Self::on_chain_provided_quantity_key(&self.genesis, order);
let on_chain_quantity = self
.db
.get(on_chain_key)
.map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap()))
.unwrap_or(0);
let on_chain_quantity =
self.db.get(on_chain_key).map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap()));
let current_provided_key = self.current_provided_key();
@@ -158,7 +152,7 @@ impl<D: Db, T: Transaction> ProvidedTransactions<D, T> {
block: [u8; 32],
tx: [u8; 32],
) {
if let Some(next_tx) = self.transactions.get_mut(order).and_then(|queue| queue.pop_front()) {
if let Some(next_tx) = self.transactions.get_mut(order).and_then(VecDeque::pop_front) {
assert_eq!(next_tx.hash(), tx);
let current_provided_key = self.current_provided_key();
@@ -184,11 +178,8 @@ impl<D: Db, T: Transaction> ProvidedTransactions<D, T> {
// bump the on-chain tx number.
let on_chain_key = Self::on_chain_provided_quantity_key(&self.genesis, order);
let block_order_key = Self::block_provided_quantity_key(&self.genesis, &block, order);
let mut on_chain_quantity = self
.db
.get(&on_chain_key)
.map(|bytes| u32::from_le_bytes(bytes.try_into().unwrap()))
.unwrap_or(0);
let mut on_chain_quantity =
self.db.get(&on_chain_key).map_or(0, |bytes| u32::from_le_bytes(bytes.try_into().unwrap()));
let this_provided_id = on_chain_quantity;
txn.put(Self::on_chain_provided_key(&self.genesis, order, this_provided_id), tx);

View File

@@ -348,7 +348,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Network for TendermintNetwork<D, T, P>
if self.blockchain.write().await.add_transaction::<Self>(
true,
Transaction::Tendermint(tx),
self.signature_scheme(),
&self.signature_scheme(),
) == Ok(true)
{
self.p2p.broadcast(signer.genesis, to_broadcast).await;
@@ -362,7 +362,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Network for TendermintNetwork<D, T, P>
.blockchain
.read()
.await
.verify_block::<Self>(&block, self.signature_scheme(), false)
.verify_block::<Self>(&block, &self.signature_scheme(), false)
.map_err(|e| match e {
BlockError::NonLocalProvided(_) => TendermintBlockError::Temporal,
_ => {
@@ -398,7 +398,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Network for TendermintNetwork<D, T, P>
let block_res = self.blockchain.write().await.add_block::<Self>(
&block,
encoded_commit.clone(),
self.signature_scheme(),
&self.signature_scheme(),
);
match block_res {
Ok(()) => {
@@ -425,7 +425,7 @@ impl<D: Db, T: TransactionTrait, P: P2p> Network for TendermintNetwork<D, T, P>
*self.to_rebroadcast.write().await = vec![];
Some(TendermintBlock(
self.blockchain.write().await.build_block::<Self>(self.signature_scheme()).serialize(),
self.blockchain.write().await.build_block::<Self>(&self.signature_scheme()).serialize(),
))
}
}

View File

@@ -88,7 +88,7 @@ fn decode_and_verify_signed_message<N: Network>(
// re-implements an entire foreign library's checks for malicious behavior).
pub(crate) fn verify_tendermint_tx<N: Network>(
tx: &TendermintTx,
schema: N::SignatureScheme,
schema: &N::SignatureScheme,
commit: impl Fn(u32) -> Option<Commit<N::SignatureScheme>>,
) -> Result<(), TransactionError> {
tx.verify()?;
@@ -98,8 +98,8 @@ pub(crate) fn verify_tendermint_tx<N: Network>(
TendermintTx::SlashEvidence(ev) => {
match ev {
Evidence::ConflictingMessages(first, second) => {
let first = decode_and_verify_signed_message::<N>(first, &schema)?.msg;
let second = decode_and_verify_signed_message::<N>(second, &schema)?.msg;
let first = decode_and_verify_signed_message::<N>(first, schema)?.msg;
let second = decode_and_verify_signed_message::<N>(second, schema)?.msg;
// Make sure they're distinct messages, from the same sender, within the same block
if (first == second) || (first.sender != second.sender) || (first.block != second.block) {
@@ -112,8 +112,8 @@ pub(crate) fn verify_tendermint_tx<N: Network>(
}
}
Evidence::ConflictingPrecommit(first, second) => {
let first = decode_and_verify_signed_message::<N>(first, &schema)?.msg;
let second = decode_and_verify_signed_message::<N>(second, &schema)?.msg;
let first = decode_and_verify_signed_message::<N>(first, schema)?.msg;
let second = decode_and_verify_signed_message::<N>(second, schema)?.msg;
if (first.sender != second.sender) || (first.block != second.block) {
Err(TransactionError::InvalidContent)?;
@@ -136,7 +136,7 @@ pub(crate) fn verify_tendermint_tx<N: Network>(
Err(TransactionError::InvalidContent)?
}
Evidence::InvalidPrecommit(msg) => {
let msg = decode_and_verify_signed_message::<N>(msg, &schema)?.msg;
let msg = decode_and_verify_signed_message::<N>(msg, schema)?.msg;
let Data::Precommit(Some((id, sig))) = &msg.data else {
Err(TransactionError::InvalidContent)?
@@ -173,7 +173,7 @@ pub(crate) fn verify_tendermint_tx<N: Network>(
}
}
Evidence::InvalidValidRound(msg) => {
let msg = decode_and_verify_signed_message::<N>(msg, &schema)?.msg;
let msg = decode_and_verify_signed_message::<N>(msg, schema)?.msg;
let Data::Proposal(Some(vr), _) = &msg.data else {
Err(TransactionError::InvalidContent)?

View File

@@ -89,7 +89,7 @@ fn empty_block() {
LAST,
HashMap::new(),
&mut |_, _| None,
validators,
&validators,
commit,
unsigned_in_chain,
provided_in_chain,
@@ -129,7 +129,7 @@ fn duplicate_nonces() {
last_nonce += 1;
Some(res)
},
validators.clone(),
&validators,
commit,
unsigned_in_chain,
provided_in_chain,

View File

@@ -44,12 +44,12 @@ fn block_addition() {
let genesis = new_genesis();
let validators = Arc::new(Validators::new(genesis, vec![]).unwrap());
let (db, mut blockchain) = new_blockchain::<SignedTransaction>(genesis, &[]);
let block = blockchain.build_block::<N>(validators.clone());
let block = blockchain.build_block::<N>(&validators);
assert_eq!(block.header.parent, genesis);
assert_eq!(block.header.transactions, [0; 32]);
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
assert!(blockchain.add_block::<N>(&block, vec![], validators).is_ok());
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
assert!(blockchain.add_block::<N>(&block, vec![], &validators).is_ok());
assert_eq!(blockchain.tip(), block.hash());
assert_eq!(blockchain.block_number(), 1);
assert_eq!(
@@ -64,21 +64,21 @@ fn invalid_block() {
let validators = Arc::new(Validators::new(genesis, vec![]).unwrap());
let (_, mut blockchain) = new_blockchain::<SignedTransaction>(genesis, &[]);
let block = blockchain.build_block::<N>(validators.clone());
let block = blockchain.build_block::<N>(&validators);
// Mutate parent
{
#[allow(clippy::redundant_clone)] // False positive
let mut block = block.clone();
block.header.parent = Blake2s256::digest(block.header.parent).into();
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
}
// Mutate tranactions merkle
{
let mut block = block;
block.header.transactions = Blake2s256::digest(block.header.transactions).into();
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
}
let key = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
@@ -89,7 +89,7 @@ fn invalid_block() {
// Manually create the block to bypass build_block's checks
let block = Block::new(blockchain.tip(), vec![], vec![Transaction::Application(tx.clone())]);
assert_eq!(block.header.transactions, merkle(&[tx.hash()]));
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
}
// Run the rest of the tests with them as a participant
@@ -99,22 +99,22 @@ fn invalid_block() {
{
let block = Block::new(blockchain.tip(), vec![], vec![Transaction::Application(tx.clone())]);
assert_eq!(block.header.transactions, merkle(&[tx.hash()]));
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
}
{
// Add a valid transaction
let (_, mut blockchain) = new_blockchain(genesis, &[tx.1.signer]);
blockchain
.add_transaction::<N>(true, Transaction::Application(tx.clone()), validators.clone())
.add_transaction::<N>(true, Transaction::Application(tx.clone()), &validators)
.unwrap();
let mut block = blockchain.build_block::<N>(validators.clone());
let mut block = blockchain.build_block::<N>(&validators);
assert_eq!(block.header.transactions, merkle(&[tx.hash()]));
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
// And verify mutating the transactions merkle now causes a failure
block.header.transactions = merkle(&[]);
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
}
{
@@ -122,24 +122,22 @@ fn invalid_block() {
let tx = crate::tests::signed_transaction(&mut OsRng, genesis, &key, 5);
// Manually create the block to bypass build_block's checks
let block = Block::new(blockchain.tip(), vec![], vec![Transaction::Application(tx)]);
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
}
{
// Invalid signature
let (_, mut blockchain) = new_blockchain(genesis, &[tx.1.signer]);
blockchain
.add_transaction::<N>(true, Transaction::Application(tx), validators.clone())
.unwrap();
let mut block = blockchain.build_block::<N>(validators.clone());
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
blockchain.add_transaction::<N>(true, Transaction::Application(tx), &validators).unwrap();
let mut block = blockchain.build_block::<N>(&validators);
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
match &mut block.transactions[0] {
Transaction::Application(tx) => {
tx.1.signature.s += <Ristretto as Ciphersuite>::F::ONE;
}
_ => panic!("non-signed tx found"),
}
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
// Make sure this isn't because the merkle changed due to the transaction hash including the
// signature (which it explicitly isn't allowed to anyways)
@@ -166,12 +164,10 @@ fn signed_transaction() {
panic!("tendermint tx found");
};
let next_nonce = blockchain.next_nonce(&signer, &[]).unwrap();
blockchain
.add_transaction::<N>(true, Transaction::Application(tx), validators.clone())
.unwrap();
blockchain.add_transaction::<N>(true, Transaction::Application(tx), &validators).unwrap();
assert_eq!(next_nonce + 1, blockchain.next_nonce(&signer, &[]).unwrap());
}
let block = blockchain.build_block::<N>(validators.clone());
let block = blockchain.build_block::<N>(&validators);
assert_eq!(block, Block::new(blockchain.tip(), vec![], mempool.clone()));
assert_eq!(blockchain.tip(), tip);
assert_eq!(block.header.parent, tip);
@@ -185,8 +181,8 @@ fn signed_transaction() {
);
// Verify and add the block
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
assert!(blockchain.add_block::<N>(&block, vec![], validators.clone()).is_ok());
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
assert!(blockchain.add_block::<N>(&block, vec![], &validators).is_ok());
assert_eq!(blockchain.tip(), block.hash());
};
@@ -233,21 +229,21 @@ fn provided_transaction() {
{
// Non-provided transactions should fail verification because we don't have them locally.
let block = Block::new(blockchain.tip(), vec![tx.clone()], vec![]);
assert!(blockchain.verify_block::<N>(&block, validators.clone(), false).is_err());
assert!(blockchain.verify_block::<N>(&block, &validators, false).is_err());
// Provided transactions should pass verification
blockchain.provide_transaction(tx.clone()).unwrap();
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
// add_block should work for verified blocks
assert!(blockchain.add_block::<N>(&block, vec![], validators.clone()).is_ok());
assert!(blockchain.add_block::<N>(&block, vec![], &validators).is_ok());
let block = Block::new(blockchain.tip(), vec![tx.clone()], vec![]);
// The provided transaction should no longer considered provided but added to chain,
// causing this error
assert_eq!(
blockchain.verify_block::<N>(&block, validators.clone(), false),
blockchain.verify_block::<N>(&block, &validators, false),
Err(BlockError::ProvidedAlreadyIncluded)
);
}
@@ -262,11 +258,11 @@ fn provided_transaction() {
// add_block DOES NOT fail for unverified provided transactions if told to add them,
// since now we can have them later.
let block1 = Block::new(blockchain.tip(), vec![tx1.clone(), tx3.clone()], vec![]);
assert!(blockchain.add_block::<N>(&block1, vec![], validators.clone()).is_ok());
assert!(blockchain.add_block::<N>(&block1, vec![], &validators).is_ok());
// in fact, we can have many blocks that have provided txs that we don't have locally.
let block2 = Block::new(blockchain.tip(), vec![tx2.clone(), tx4.clone()], vec![]);
assert!(blockchain.add_block::<N>(&block2, vec![], validators.clone()).is_ok());
assert!(blockchain.add_block::<N>(&block2, vec![], &validators).is_ok());
// make sure we won't return ok for the block before we actually got the txs
let TransactionKind::Provided(order) = tx1.kind() else { panic!("tx wasn't provided") };
@@ -357,11 +353,9 @@ async fn tendermint_evidence_tx() {
let Transaction::Tendermint(tx) = tx else {
panic!("non-tendermint tx found");
};
blockchain
.add_transaction::<N>(true, Transaction::Tendermint(tx), validators.clone())
.unwrap();
blockchain.add_transaction::<N>(true, Transaction::Tendermint(tx), &validators).unwrap();
}
let block = blockchain.build_block::<N>(validators.clone());
let block = blockchain.build_block::<N>(&validators);
assert_eq!(blockchain.tip(), tip);
assert_eq!(block.header.parent, tip);
@@ -371,8 +365,8 @@ async fn tendermint_evidence_tx() {
}
// Verify and add the block
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
assert!(blockchain.add_block::<N>(&block, vec![], validators.clone()).is_ok());
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
assert!(blockchain.add_block::<N>(&block, vec![], &validators).is_ok());
assert_eq!(blockchain.tip(), block.hash());
};
@@ -467,7 +461,7 @@ async fn block_tx_ordering() {
let signed_tx = Transaction::Application(SignedTx::Signed(Box::new(
crate::tests::signed_transaction(&mut OsRng, genesis, &key, i),
)));
blockchain.add_transaction::<N>(true, signed_tx.clone(), validators.clone()).unwrap();
blockchain.add_transaction::<N>(true, signed_tx.clone(), &validators).unwrap();
mempool.push(signed_tx);
let unsigned_tx = Transaction::Tendermint(
@@ -477,7 +471,7 @@ async fn block_tx_ordering() {
)
.await,
);
blockchain.add_transaction::<N>(true, unsigned_tx.clone(), validators.clone()).unwrap();
blockchain.add_transaction::<N>(true, unsigned_tx.clone(), &validators).unwrap();
mempool.push(unsigned_tx);
let provided_tx =
@@ -485,7 +479,7 @@ async fn block_tx_ordering() {
blockchain.provide_transaction(provided_tx.clone()).unwrap();
provided_txs.push(provided_tx);
}
let block = blockchain.build_block::<N>(validators.clone());
let block = blockchain.build_block::<N>(&validators);
assert_eq!(blockchain.tip(), tip);
assert_eq!(block.header.parent, tip);
@@ -509,7 +503,7 @@ async fn block_tx_ordering() {
}
// should be a valid block
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap();
blockchain.verify_block::<N>(&block, &validators, false).unwrap();
// Unsigned before Provided
{
@@ -518,7 +512,7 @@ async fn block_tx_ordering() {
let unsigned = block.transactions.remove(128);
block.transactions.insert(0, unsigned);
assert_eq!(
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap_err(),
blockchain.verify_block::<N>(&block, &validators, false).unwrap_err(),
BlockError::WrongTransactionOrder
);
}
@@ -529,7 +523,7 @@ async fn block_tx_ordering() {
let signed = block.transactions.remove(256);
block.transactions.insert(0, signed);
assert_eq!(
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap_err(),
blockchain.verify_block::<N>(&block, &validators, false).unwrap_err(),
BlockError::WrongTransactionOrder
);
}
@@ -539,7 +533,7 @@ async fn block_tx_ordering() {
let mut block = block;
block.transactions.swap(128, 256);
assert_eq!(
blockchain.verify_block::<N>(&block, validators.clone(), false).unwrap_err(),
blockchain.verify_block::<N>(&block, &validators, false).unwrap_err(),
BlockError::WrongTransactionOrder
);
}

View File

@@ -47,7 +47,7 @@ async fn mempool_addition() {
&|_, _| Some(0),
true,
Transaction::Application(first_tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
)
@@ -63,7 +63,7 @@ async fn mempool_addition() {
&|_, _| None,
true,
Transaction::Tendermint(evidence_tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
)
@@ -78,7 +78,7 @@ async fn mempool_addition() {
&|_, _| Some(0),
true,
Transaction::Application(first_tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
),
@@ -89,7 +89,7 @@ async fn mempool_addition() {
&|_, _| None,
true,
Transaction::Tendermint(evidence_tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
),
@@ -103,7 +103,7 @@ async fn mempool_addition() {
&|_, _| Some(0),
true,
Transaction::Application(second_tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
),
@@ -115,7 +115,7 @@ async fn mempool_addition() {
&|_, _| Some(0),
true,
Transaction::Application(second_tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
),
@@ -133,7 +133,7 @@ async fn mempool_addition() {
&|_, _| Some(2),
true,
Transaction::Application(tx.clone()),
validators.clone(),
&validators,
unsigned_in_chain,
commit
)
@@ -173,7 +173,7 @@ fn too_many_mempool() {
&|_, _| Some(0),
false,
Transaction::Application(signed_transaction(&mut OsRng, genesis, &key, i)),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
)
@@ -190,7 +190,7 @@ fn too_many_mempool() {
&key,
ACCOUNT_MEMPOOL_LIMIT
)),
validators.clone(),
&validators,
unsigned_in_chain,
commit,
),

View File

@@ -57,13 +57,13 @@ async fn invalid_valid_round() {
// This should be invalid evidence if a valid valid round is specified
let (_, tx) = valid_round_tx(None).await;
assert!(verify_tendermint_tx::<N>(&tx, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
// If an invalid valid round is specified (>= current), this should be invalid evidence
let (mut signed, tx) = valid_round_tx(Some(RoundNumber(0))).await;
// should pass
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap();
// change the signature
let mut random_sig = [0u8; 64];
@@ -72,7 +72,7 @@ async fn invalid_valid_round() {
let tx = TendermintTx::SlashEvidence(Evidence::InvalidValidRound(signed.encode()));
// should fail
assert!(verify_tendermint_tx::<N>(&tx, validators, commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
}
#[tokio::test]
@@ -94,7 +94,7 @@ async fn invalid_precommit_signature() {
};
// Empty Precommit should fail.
assert!(verify_tendermint_tx::<N>(&precommit(None).await.1, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&precommit(None).await.1, &validators, commit).is_err());
// valid precommit signature should fail.
let block_id = [0x22u8; 32];
@@ -105,7 +105,7 @@ async fn invalid_precommit_signature() {
assert!(verify_tendermint_tx::<N>(
&precommit(Some((block_id, signer.clone().sign(&commit_msg).await))).await.1,
validators.clone(),
&validators,
commit
)
.is_err());
@@ -113,14 +113,14 @@ async fn invalid_precommit_signature() {
// any other signature can be used as evidence.
{
let (mut signed, tx) = precommit(Some((block_id, signer.sign(&[]).await))).await;
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap();
// So long as we can authenticate where it came from
let mut random_sig = [0u8; 64];
OsRng.fill_bytes(&mut random_sig);
signed.sig = random_sig;
let tx = TendermintTx::SlashEvidence(Evidence::InvalidPrecommit(signed.encode()));
assert!(verify_tendermint_tx::<N>(&tx, validators, commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
}
}
@@ -170,10 +170,10 @@ async fn evidence_with_prevote() {
// No prevote message alone should be valid as slash evidence at this time
for prevote in prevote(None).await {
assert!(verify_tendermint_tx::<N>(&prevote, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&prevote, &validators, commit).is_err());
}
for prevote in prevote(Some([0x22u8; 32])).await {
assert!(verify_tendermint_tx::<N>(&prevote, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&prevote, &validators, commit).is_err());
}
}
@@ -199,7 +199,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_1.encode(),
));
assert!(verify_tendermint_tx::<N>(&tx, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
// conflicting data should pass
let signed_2 = signed_for_b_r(0, 0, Data::Proposal(None, TendermintBlock(vec![0x22]))).await;
@@ -207,7 +207,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap();
// Except if it has a distinct round number, as we don't check cross-round conflicts
// (except for Precommit)
@@ -216,7 +216,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap_err();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap_err();
// Proposals for different block numbers should also fail as evidence
let signed_2 = signed_for_b_r(1, 0, Data::Proposal(None, TendermintBlock(vec![0x22]))).await;
@@ -224,7 +224,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap_err();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap_err();
}
// Prevote
@@ -235,7 +235,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_1.encode(),
));
assert!(verify_tendermint_tx::<N>(&tx, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
// conflicting data should pass
let signed_2 = signed_for_b_r(0, 0, Data::Prevote(Some([0x22; 32]))).await;
@@ -243,7 +243,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap();
// Except if it has a distinct round number, as we don't check cross-round conflicts
// (except for Precommit)
@@ -252,7 +252,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap_err();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap_err();
// Proposals for different block numbers should also fail as evidence
let signed_2 = signed_for_b_r(1, 0, Data::Prevote(Some([0x22; 32]))).await;
@@ -260,7 +260,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap_err();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap_err();
}
// Precommit
@@ -272,7 +272,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_1.encode(),
));
assert!(verify_tendermint_tx::<N>(&tx, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
// For precommit, the round number is ignored
let signed_2 = signed_for_b_r(0, 1, Data::Precommit(Some(([0x22; 32], sig)))).await;
@@ -280,7 +280,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
verify_tendermint_tx::<N>(&tx, validators.clone(), commit).unwrap();
verify_tendermint_tx::<N>(&tx, &validators, commit).unwrap();
// Yet the block number isn't
let signed_2 = signed_for_b_r(1, 0, Data::Precommit(Some(([0x22; 32], sig)))).await;
@@ -288,7 +288,7 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
assert!(verify_tendermint_tx::<N>(&tx, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
}
// msgs from different senders should fail
@@ -320,7 +320,7 @@ async fn conflicting_msgs_evidence_tx() {
let validators =
Arc::new(Validators::new(genesis, vec![(signer_pub, 1), (signer_pub_2, 1)]).unwrap());
assert!(verify_tendermint_tx::<N>(&tx, validators, commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
}
// msgs with different steps should fail
@@ -331,6 +331,6 @@ async fn conflicting_msgs_evidence_tx() {
signed_1.encode(),
signed_2.encode(),
));
assert!(verify_tendermint_tx::<N>(&tx, validators.clone(), commit).is_err());
assert!(verify_tendermint_tx::<N>(&tx, &validators, commit).is_err());
}
}

View File

@@ -191,8 +191,7 @@ pub(crate) fn verify_transaction<F: GAIN, T: Transaction>(
tx.verify()?;
match tx.kind() {
TransactionKind::Provided(_) => {}
TransactionKind::Unsigned => {}
TransactionKind::Provided(_) | TransactionKind::Unsigned => {}
TransactionKind::Signed(order, Signed { signer, nonce, signature }) => {
if let Some(next_nonce) = get_and_increment_nonce(signer, &order) {
if *nonce != next_nonce {

View File

@@ -543,8 +543,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
self.slash(sender, slash).await
}
Err(TendermintError::Temporal) => (),
Err(TendermintError::AlreadyHandled) => (),
Err(TendermintError::Temporal | TendermintError::AlreadyHandled) => (),
}
}
}
@@ -627,7 +626,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
// Uses a junk signature since message equality disregards the signature
if self.block.log.has_consensus(
msg.round,
Data::Precommit(Some((block.id(), self.signer.sign(&[]).await))),
&Data::Precommit(Some((block.id(), self.signer.sign(&[]).await))),
) {
// If msg.round is in the future, these Precommits won't have their inner signatures
// verified
@@ -714,7 +713,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
// of the round map
if (self.block.round().step == Step::Prevote) && matches!(msg.data, Data::Prevote(_)) {
let (participation, weight) =
self.block.log.message_instances(self.block.round().number, Data::Prevote(None));
self.block.log.message_instances(self.block.round().number, &Data::Prevote(None));
// 34-35
if participation >= self.weights.threshold() {
self.block.round_mut().set_timeout(Step::Prevote);
@@ -767,7 +766,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
// 23 and 29. If it's some, both are satisfied if they're for the same ID. If it's some
// with different IDs, the function on 22 rejects yet the function on 28 has one other
// condition
let locked = self.block.locked.as_ref().map(|(_, id)| id == &block.id()).unwrap_or(true);
let locked = self.block.locked.as_ref().map_or(true, |(_, id)| id == &block.id());
let mut vote = raw_vote.filter(|_| locked);
if let Some(vr) = vr {
@@ -780,7 +779,7 @@ impl<N: Network + 'static> TendermintMachine<N> {
))?;
}
if self.block.log.has_consensus(*vr, Data::Prevote(Some(block.id()))) {
if self.block.log.has_consensus(*vr, &Data::Prevote(Some(block.id()))) {
// Allow differing locked values if the proposal has a newer valid round
// This is the other condition described above
if let Some((locked_round, _)) = self.block.locked.as_ref() {
@@ -798,25 +797,18 @@ impl<N: Network + 'static> TendermintMachine<N> {
return Ok(None);
}
if self
.block
.valid
.as_ref()
.map(|(round, _)| round != &self.block.round().number)
.unwrap_or(true)
{
if self.block.valid.as_ref().map_or(true, |(round, _)| round != &self.block.round().number) {
// 36-43
// The run once condition is implemented above. Since valid will always be set by this, it
// not being set, or only being set historically, means this has yet to be run
if self.block.log.has_consensus(self.block.round().number, Data::Prevote(Some(block.id()))) {
if self.block.log.has_consensus(self.block.round().number, &Data::Prevote(Some(block.id()))) {
match self.network.validate(block).await {
Ok(()) => (),
// BlockError::Temporal is due to a temporal error we have, yet a supermajority of the
// network does not, Because we do not believe this block to be fatally invalid, and
// because a supermajority deems it valid, accept it.
Err(BlockError::Temporal) => (),
Ok(()) | Err(BlockError::Temporal) => (),
Err(BlockError::Fatal) => {
log::warn!(target: "tendermint", "Validator proposed a fatally invalid block");
// TODO: Produce evidence of this for the higher level code to decide what to do with

View File

@@ -64,14 +64,14 @@ impl<N: Network> MessageLog<N> {
// For a given round, return the participating weight for this step, and the weight agreeing with
// the data.
pub(crate) fn message_instances(&self, round: RoundNumber, data: DataFor<N>) -> (u64, u64) {
pub(crate) fn message_instances(&self, round: RoundNumber, data: &DataFor<N>) -> (u64, u64) {
let mut participating = 0;
let mut weight = 0;
for (participant, msgs) in &self.log[&round] {
if let Some(msg) = msgs.get(&data.step()) {
let validator_weight = self.weights.weight(*participant);
participating += validator_weight;
if data == msg.msg.data {
if data == &msg.msg.data {
weight += validator_weight;
}
}
@@ -102,7 +102,7 @@ impl<N: Network> MessageLog<N> {
}
// Check if consensus has been reached on a specific piece of data
pub(crate) fn has_consensus(&self, round: RoundNumber, data: DataFor<N>) -> bool {
pub(crate) fn has_consensus(&self, round: RoundNumber, data: &DataFor<N>) -> bool {
let (_, weight) = self.message_instances(round, data);
weight >= self.weights.threshold()
}