Reattempts (#483)

* Schedule re-attempts and add a (not filled out) match statement to actually execute them

A comment explains the methodology. To copy it here:

"""
This is because we *always* re-attempt any protocol which had participation. That doesn't
mean we *should* re-attempt this protocol.

The alternatives were:
1) Note on-chain we completed a protocol, halting re-attempts upon 34%.
2) Vote on-chain to re-attempt a protocol.

This schema doesn't have any additional messages upon the success case (whereas
alternative #1 does) and doesn't have overhead (as alternative #2 does, sending votes and
then preprocesses. This only sends preprocesses).
"""

Any signing protocol which reaches sufficient participation will be
re-attempted until it no longer does.

* Have the Substrate scanner track DKG removals/completions for the Tributary code

* Don't keep trying to publish a participant removal if we've already set keys

* Pad out the re-attempt match a bit more

* Have CosignEvaluator reload from the DB

* Correctly schedule cosign re-attempts

* Actuall spawn new DKG removal attempts

* Use u32 for Batch ID in SubstrateSignableId, finish Batch re-attempt routing

The batch ID was an opaque [u8; 5] which also included the network, yet that's
redundant and unhelpful.

* Clarify a pair of TODOs in the coordinator

* Remove old TODO

* Final comment cleanup

* Correct usage of TARGET_BLOCK_TIME in reattempt scheduler

It's in ms and I assumed it was in s.

* Have coordinator tests drop BatchReattempts which aren't relevant yet may exist

* Bug fix and pointless oddity removal

We scheduled a re-attempt upon receiving 2/3rds of preprocesses and upon
receiving 2/3rds of shares, so any signing protocol could cause two re-attempts
(not one more).

The coordinator tests randomly generated the Batch ID since it was prior an
opaque byte array. While that didn't break the test, it was pointless and did
make the already-succeeded check before re-attempting impossible to hit.

* Add log statements, correct dead-lock in coordinator tests

* Increase pessimistic timeout on recv_message to compensate for tighter best-case timeouts

* Further bump timeout by a minute

AFAICT, GH failed by just a few seconds.

This also is worst-case in a single instance, making it fine to be decently long.

* Further further bump timeout due to lack of distinct error
This commit is contained in:
Luke Parker
2023-12-12 12:28:53 -05:00
committed by GitHub
parent b297b79f07
commit 6a172825aa
17 changed files with 437 additions and 191 deletions

View File

@@ -1,4 +1,4 @@
use core::{marker::PhantomData, future::Future, time::Duration};
use core::{marker::PhantomData, ops::Deref, future::Future, time::Duration};
use std::sync::Arc;
use rand_core::OsRng;
@@ -15,6 +15,8 @@ use serai_client::{validator_sets::primitives::ValidatorSet, Serai};
use serai_db::DbTxn;
use processor_messages::coordinator::SubstrateSignableId;
use tributary::{
TransactionKind, Transaction as TributaryTransaction, TransactionError, Block, TributaryReader,
tendermint::{
@@ -26,10 +28,8 @@ use tributary::{
use crate::{
Db,
processors::Processors,
tributary::{
TributarySpec, Label, SignData, Transaction, Topic, AttemptDb, LastHandledBlock,
FatallySlashed, DkgCompleted, signing_protocol::DkgRemoval,
},
substrate::BatchInstructionsHashDb,
tributary::{*, signing_protocol::*},
P2p,
};
@@ -74,6 +74,7 @@ pub enum PstTxType {
#[async_trait::async_trait]
pub trait PSTTrait {
// TODO: Diversify publish_set_keys, publish_remove_participant, then remove PstTxType
async fn publish_serai_tx(
&self,
set: ValidatorSet,
@@ -125,13 +126,31 @@ pub struct TributaryBlockHandler<
pub publish_tributary_tx: &'a PTT,
pub spec: &'a TributarySpec,
block: Block<Transaction>,
pub block_number: u32,
_p2p: PhantomData<P>,
}
impl<T: DbTxn, Pro: Processors, PST: PSTTrait, PTT: PTTTrait, RID: RIDTrait, P: P2p>
TributaryBlockHandler<'_, T, Pro, PST, PTT, RID, P>
{
async fn dkg_removal_attempt(&mut self, removing: [u8; 32], attempt: u32) {
let preprocess =
(DkgRemoval { spec: self.spec, key: self.our_key, txn: self.txn, removing, attempt })
.preprocess();
let mut tx = Transaction::DkgRemoval(SignData {
plan: removing,
attempt,
label: Label::Preprocess,
data: vec![preprocess.to_vec()],
signed: Transaction::empty_signed(),
});
tx.sign(&mut OsRng, self.spec.genesis(), self.our_key);
self.publish_tributary_tx.publish_tributary_tx(tx).await;
}
pub async 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();
log::warn!("fatally slashing {}. reason: {}", hex::encode(slashing), reason);
@@ -144,23 +163,7 @@ impl<T: DbTxn, Pro: Processors, PST: PSTTrait, PTT: PTTTrait, RID: RIDTrait, P:
// If during a DKG, remove the participant
if DkgCompleted::get(self.txn, genesis).is_none() {
AttemptDb::recognize_topic(self.txn, genesis, Topic::DkgRemoval(slashing));
let preprocess = (DkgRemoval {
spec: self.spec,
key: self.our_key,
txn: self.txn,
removing: slashing,
attempt: 0,
})
.preprocess();
let mut tx = Transaction::DkgRemoval(SignData {
plan: slashing,
attempt: 0,
label: Label::Preprocess,
data: vec![preprocess.to_vec()],
signed: Transaction::empty_signed(),
});
tx.sign(&mut OsRng, genesis, self.our_key);
self.publish_tributary_tx.publish_tributary_tx(tx).await;
self.dkg_removal_attempt(slashing, 0).await;
}
}
@@ -223,7 +226,135 @@ impl<T: DbTxn, Pro: Processors, PST: PSTTrait, PTT: PTTTrait, RID: RIDTrait, P:
}
}
// TODO: Trigger any necessary re-attempts
let genesis = self.spec.genesis();
for topic in ReattemptDb::take(self.txn, genesis, self.block_number) {
let attempt = AttemptDb::start_next_attempt(self.txn, genesis, topic);
log::info!("re-attempting {topic:?} with attempt {attempt}");
/*
All of these have the same common flow:
1) Check if this re-attempt is actually needed
2) If so, dispatch whatever events as needed
This is because we *always* re-attempt any protocol which had participation. That doesn't
mean we *should* re-attempt this protocol.
The alternatives were:
1) Note on-chain we completed a protocol, halting re-attempts upon 34%.
2) Vote on-chain to re-attempt a protocol.
This schema doesn't have any additional messages upon the success case (whereas
alternative #1 does) and doesn't have overhead (as alternative #2 does, sending votes and
then preprocesses. This only sends preprocesses).
*/
match topic {
Topic::Dkg => {
if DkgCompleted::get(self.txn, genesis).is_none() {
// Since it wasn't completed, instruct the processor to start the next attempt
let id =
processor_messages::key_gen::KeyGenId { session: self.spec.set().session, attempt };
let our_i = self.spec.i(Ristretto::generator() * self.our_key.deref()).unwrap();
// TODO: Handle removed parties (modify n/i to accept list of removed)
// TODO: Don't fatal slash, yet don't include, parties who have been offline so long as
// we still meet the needed threshold. We'd need a complete DKG protocol we then remove
// the offline participants from. publishing the DKG protocol completed without them.
let params =
frost::ThresholdParams::new(self.spec.t(), self.spec.n(), our_i.start).unwrap();
let shares = u16::from(our_i.end) - u16::from(our_i.start);
self
.processors
.send(
self.spec.set().network,
processor_messages::key_gen::CoordinatorMessage::GenerateKey { id, params, shares },
)
.await;
}
}
Topic::DkgConfirmation => {
panic!("re-attempting DkgConfirmation when we should be re-attempting the Dkg")
}
Topic::DkgRemoval(removing) => {
if DkgCompleted::get(self.txn, genesis).is_none() &&
LocallyDkgRemoved::get(self.txn, genesis, removing).is_none() &&
SeraiDkgCompleted::get(self.txn, self.spec.set()).is_none() &&
SeraiDkgRemoval::get(self.txn, self.spec.set(), removing).is_none()
{
// Since it wasn't completed, attempt a new DkgRemoval
self.dkg_removal_attempt(removing, attempt).await;
}
}
Topic::SubstrateSign(inner_id) => {
let id = processor_messages::coordinator::SubstrateSignId {
session: self.spec.set().session,
id: inner_id,
attempt,
};
match inner_id {
SubstrateSignableId::CosigningSubstrateBlock(block) => {
let block_number = SeraiBlockNumber::get(self.txn, block)
.expect("couldn't get the block number for prior attempted cosign");
// 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);
if latest_cosign < block_number {
// Instruct the processor to start the next attempt
self
.processors
.send(
self.spec.set().network,
processor_messages::coordinator::CoordinatorMessage::CosignSubstrateBlock {
id,
block_number,
},
)
.await;
}
}
SubstrateSignableId::Batch(batch) => {
// If the Batch hasn't appeared on-chain...
if BatchInstructionsHashDb::get(self.txn, self.spec.set().network, batch).is_none() {
// Instruct the processor to start the next attempt
// The processor won't continue if it's already signed a Batch
// Prior checking if the Batch is on-chain just may reduce the non-participating
// 33% from publishing their re-attempt messages
self
.processors
.send(
self.spec.set().network,
processor_messages::coordinator::CoordinatorMessage::BatchReattempt { id },
)
.await;
}
}
}
}
Topic::Sign(id) => {
// Instruct the processor to start the next attempt
// If it has already noted a completion, it won't send a preprocess and will simply drop
// the re-attempt message
self
.processors
.send(
self.spec.set().network,
processor_messages::sign::CoordinatorMessage::Reattempt {
id: processor_messages::sign::SignId {
session: self.spec.set().session,
id,
attempt,
},
},
)
.await;
}
}
}
}
}
@@ -247,8 +378,10 @@ pub(crate) async fn handle_new_blocks<
) {
let genesis = tributary.genesis();
let mut last_block = LastHandledBlock::get(db, genesis).unwrap_or(genesis);
let mut block_number = TributaryBlockNumber::get(db, last_block).unwrap_or(0);
while let Some(next) = tributary.block_after(&last_block) {
let block = tributary.block(&next).unwrap();
block_number += 1;
// Make sure we have all of the provided transactions for this block
for tx in &block.transactions {
@@ -264,6 +397,7 @@ pub(crate) async fn handle_new_blocks<
}
let mut txn = db.txn();
TributaryBlockNumber::set(&mut txn, next, &block_number);
(TributaryBlockHandler {
txn: &mut txn,
spec,
@@ -273,6 +407,7 @@ pub(crate) async fn handle_new_blocks<
publish_serai_tx,
publish_tributary_tx,
block,
block_number,
_p2p: PhantomData::<P>,
})
.handle::<D>()
@@ -368,6 +503,15 @@ pub(crate) async fn scan_tributaries_task<
}
}
PstTxType::RemoveParticipant(removed) => {
if let Ok(Some(_)) = serai.keys(spec.set()).await {
log::info!(
"keys were set before we {} {:?}",
"personally could publish the removal for",
hex::encode(removed)
);
break;
}
if let Ok(Some(participants)) =
serai.participants(spec.set().network).await
{