Txn for handling a processor message

handle_processor_messages function added to remove a very large block of nested
code.

MainDb cleaned to never be instantiated.
This commit is contained in:
Luke Parker
2023-09-27 00:00:31 -04:00
parent 2e0f8138e2
commit 086458d041
2 changed files with 335 additions and 358 deletions

View File

@@ -1,3 +1,5 @@
use core::marker::PhantomData;
use scale::{Encode, Decode};
use serai_client::{primitives::NetworkId, in_instructions::primitives::SignedBatch};
@@ -6,12 +8,8 @@ pub use serai_db::*;
use crate::tributary::TributarySpec;
#[derive(Debug)]
pub struct MainDb<'a, D: Db>(&'a mut D);
impl<'a, D: Db> MainDb<'a, D> {
pub fn new(db: &'a mut D) -> Self {
Self(db)
}
pub struct MainDb<D: Db>(PhantomData<D>);
impl<D: Db> MainDb<D> {
fn main_key(dst: &'static [u8], key: impl AsRef<[u8]>) -> Vec<u8> {
D::key(b"coordinator_main", dst, key)
}
@@ -29,8 +27,8 @@ impl<'a, D: Db> MainDb<'a, D> {
fn acive_tributaries_key() -> Vec<u8> {
Self::main_key(b"active_tributaries", [])
}
pub fn active_tributaries(&self) -> (Vec<u8>, Vec<TributarySpec>) {
let bytes = self.0.get(Self::acive_tributaries_key()).unwrap_or(vec![]);
pub fn active_tributaries<G: Get>(getter: &G) -> (Vec<u8>, Vec<TributarySpec>) {
let bytes = getter.get(Self::acive_tributaries_key()).unwrap_or(vec![]);
let mut bytes_ref: &[u8] = bytes.as_ref();
let mut tributaries = vec![];
@@ -40,9 +38,9 @@ impl<'a, D: Db> MainDb<'a, D> {
(bytes, tributaries)
}
pub fn add_active_tributary(&mut self, spec: &TributarySpec) {
pub fn add_active_tributary(txn: &mut D::Transaction<'_>, spec: &TributarySpec) {
let key = Self::acive_tributaries_key();
let (mut existing_bytes, existing) = self.active_tributaries();
let (mut existing_bytes, existing) = Self::active_tributaries(txn);
for tributary in &existing {
if tributary == spec {
return;
@@ -50,9 +48,7 @@ impl<'a, D: Db> MainDb<'a, D> {
}
spec.write(&mut existing_bytes).unwrap();
let mut txn = self.0.txn();
txn.put(key, existing_bytes);
txn.commit();
}
fn first_preprocess_key(id: [u8; 32]) -> Vec<u8> {
@@ -73,14 +69,11 @@ impl<'a, D: Db> MainDb<'a, D> {
fn batch_key(network: NetworkId, id: u32) -> Vec<u8> {
Self::main_key(b"batch", (network, id).encode())
}
pub fn save_batch(&mut self, batch: SignedBatch) {
let mut txn = self.0.txn();
pub fn save_batch(txn: &mut D::Transaction<'_>, batch: SignedBatch) {
txn.put(Self::batch_key(batch.batch.network, batch.batch.id), batch.encode());
txn.commit();
}
pub fn batch(&self, network: NetworkId, id: u32) -> Option<SignedBatch> {
self
.0
pub fn batch<G: Get>(getter: &G, network: NetworkId, id: u32) -> Option<SignedBatch> {
getter
.get(Self::batch_key(network, id))
.map(|batch| SignedBatch::decode(&mut batch.as_ref()).unwrap())
}

View File

@@ -99,7 +99,7 @@ async fn add_tributary<D: Db, Pro: Processors, P: P2p>(
spec.n(),
spec
.i(Ristretto::generator() * key.deref())
.expect("adding a tribtuary for a set we aren't in set for"),
.expect("adding a tributary for a set we aren't in set for"),
)
.unwrap(),
},
@@ -173,7 +173,9 @@ pub async fn scan_substrate<D: Db, Pro: Processors>(
log::info!("creating new tributary for {:?}", spec.set());
// Save it to the database
MainDb::new(db).add_active_tributary(&spec);
let mut txn = db.txn();
MainDb::<D>::add_active_tributary(&mut txn, &spec);
txn.commit();
// If we reboot before this is read, the fact it was saved to the database means it'll be
// handled on reboot
@@ -513,51 +515,34 @@ pub async fn publish_signed_transaction<D: Db, P: P2p>(
}
}
pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
db: D,
async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
mut db: D,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: Arc<Serai>,
mut processors: Pro,
mut new_tributary: broadcast::Receiver<ActiveTributary<D, P>>,
tributary: ActiveTributary<D, P>,
mut recv: mpsc::UnboundedReceiver<processors::Message>,
) {
let db_clone = db.clone(); // Enables cloning the DB while we have a txn
let pub_key = Ristretto::generator() * key.deref();
let channels = Arc::new(RwLock::new(HashMap::new()));
tokio::spawn({
let db = db.clone();
let processors = processors.clone();
let channels = channels.clone();
async move {
loop {
let channels = channels.clone();
let ActiveTributary { spec, tributary } = new_tributary.recv().await.unwrap();
let ActiveTributary { spec, tributary } = tributary;
let genesis = spec.genesis();
tokio::spawn({
let mut db = db.clone();
let key = key.clone();
let serai = serai.clone();
let mut processors = processors.clone();
async move {
let (send, mut recv) = mpsc::unbounded_channel();
// TODO: Support multisig rotation (not per-Tributary yet per-network?)
channels.write().await.insert(spec.set().network, send);
loop {
let msg: processors::Message = recv.recv().await.unwrap();
if !MainDb::<D>::handled_message(&db, msg.id) {
let mut txn = db.txn();
// TODO: We probably want to NOP here, not panic?
// TODO: We do have to track produced Batches in order to ensure their integrity
let my_i =
spec.i(pub_key).expect("processor message for network we aren't a validator in");
let my_i = spec.i(pub_key).expect("processor message for network we aren't a validator in");
let tx = match msg.msg.clone() {
ProcessorMessage::KeyGen(inner_msg) => match inner_msg {
key_gen::ProcessorMessage::Commitments { id, commitments } => {
Some(Transaction::DkgCommitments(
id.attempt,
commitments,
Transaction::empty_signed(),
))
Some(Transaction::DkgCommitments(id.attempt, commitments, Transaction::empty_signed()))
}
key_gen::ProcessorMessage::Shares { id, mut shares } => {
// Create a MuSig-based machine to inform Substrate of this key generation
@@ -570,9 +555,7 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
continue;
}
tx_shares.push(
shares
.remove(&i)
.expect("processor didn't send share for another validator"),
shares.remove(&i).expect("processor didn't send share for another validator"),
);
}
@@ -583,19 +566,15 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
signed: Transaction::empty_signed(),
})
}
key_gen::ProcessorMessage::GeneratedKeyPair {
id,
substrate_key,
network_key,
} => {
key_gen::ProcessorMessage::GeneratedKeyPair { id, substrate_key, network_key } => {
assert_eq!(
id.set.network, msg.network,
"processor claimed to be a different network than it was for GeneratedKeyPair",
);
// TODO: Also check the other KeyGenId fields
// Tell the Tributary the key pair, get back the share for the MuSig signature
let mut txn = db.txn();
// Tell the Tributary the key pair, get back the share for the MuSig
// signature
let share = crate::tributary::generated_key_pair::<D>(
&mut txn,
&key,
@@ -603,14 +582,11 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
&(Public(substrate_key), network_key.try_into().unwrap()),
id.attempt,
);
txn.commit();
match share {
Ok(share) => Some(Transaction::DkgConfirmed(
id.attempt,
share,
Transaction::empty_signed(),
)),
Ok(share) => {
Some(Transaction::DkgConfirmed(id.attempt, share, Transaction::empty_signed()))
}
Err(p) => {
todo!("participant {p:?} sent invalid DKG confirmation preprocesses")
}
@@ -620,9 +596,7 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
ProcessorMessage::Sign(msg) => match msg {
sign::ProcessorMessage::Preprocess { id, preprocess } => {
if id.attempt == 0 {
let mut txn = db.txn();
MainDb::<D>::save_first_preprocess(&mut txn, id.id, preprocess);
txn.commit();
None
} else {
@@ -634,14 +608,12 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
}))
}
}
sign::ProcessorMessage::Share { id, share } => {
Some(Transaction::SignShare(SignData {
sign::ProcessorMessage::Share { id, share } => Some(Transaction::SignShare(SignData {
plan: id.id,
attempt: id.attempt,
data: share,
signed: Transaction::empty_signed(),
}))
}
})),
sign::ProcessorMessage::Completed { key: _, id, tx } => {
let r = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
#[allow(non_snake_case)]
@@ -669,12 +641,8 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
"processor claimed to be a different network than it was for SubstrateBlockAck",
);
// Safe to use its own txn since this is static and just needs to be written
// before we provide SubstrateBlock
let mut txn = db.txn();
// TODO: This needs to be scoped per multisig
TributaryDb::<D>::set_plan_ids(&mut txn, genesis, block, &plans);
txn.commit();
Some(Transaction::SubstrateBlock(block))
}
@@ -685,16 +653,10 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
id.attempt,
hex::encode(block),
);
// If this is the first attempt instance, wait until we synchronize around the
// batch first
// If this is the first attempt instance, wait until we synchronize around
// the batch first
if id.attempt == 0 {
// Save the preprocess to disk so we can publish it later
// This is fine to use its own TX since it's static and just needs to be
// written before this message finishes it handling (or with this message's
// finished handling)
let mut txn = db.txn();
MainDb::<D>::save_first_preprocess(&mut txn, id.id, preprocess);
txn.commit();
Some(Transaction::Batch(block.0, id.id))
} else {
@@ -725,16 +687,16 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
// batches
// Save this batch to the disk
MainDb::new(&mut db).save_batch(batch);
MainDb::<D>::save_batch(&mut txn, batch);
/*
Use a dedicated task to publish batches due to the latency potentially
incurred.
This does not guarantee the batch has actually been published when the
message is `ack`ed to message-queue. Accordingly, if we reboot, these batches
would be dropped (as we wouldn't see the `Update` again, triggering our
re-attempt to publish).
message is `ack`ed to message-queue. Accordingly, if we reboot, these
batches would be dropped (as we wouldn't see the `Update` again, triggering
our re-attempt to publish).
The solution to this is to have the task try not to publish the batch which
caused it to be spawned, yet all saved batches which have yet to published.
@@ -742,7 +704,7 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
yet these aren't notably complex.
*/
tokio::spawn({
let mut db = db.clone();
let db = db_clone.clone();
let serai = serai.clone();
let network = msg.network;
async move {
@@ -766,9 +728,8 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
let Ok(latest_block) = serai.get_latest_block().await else {
continue;
};
let Ok(last) = serai
.get_last_batch_for_network(latest_block.hash(), network)
.await
let Ok(last) =
serai.get_last_batch_for_network(latest_block.hash(), network).await
else {
continue;
};
@@ -777,7 +738,7 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
};
// If we have this batch, attempt to publish it
MainDb::new(&mut db).batch(network, next)
MainDb::<D>::batch(&db, network, next)
} {
let id = batch.batch.id;
let block = batch.batch.block;
@@ -786,13 +747,10 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
// This publish may fail if this transactions already exists in the
// mempool, which is possible, or if this batch was already executed
// on-chain
// Either case will have eventual resolution and be handled by the above
// check on if this batch should execute
// Either case will have eventual resolution and be handled by the
// above check on if this batch should execute
if serai.publish(&tx).await.is_ok() {
log::info!(
"published batch {network:?} {id} (block {})",
hex::encode(block)
);
log::info!("published batch {network:?} {id} (block {})", hex::encode(block));
}
}
}
@@ -804,6 +762,8 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
};
// If this created a transaction, publish it
// TODO: This block may be fired multiple times, with the Tributary maintaining its
// own txns. How safe is that?
if let Some(mut tx) = tx {
log::trace!("processor message effected transaction {}", hex::encode(tx.hash()));
@@ -823,21 +783,17 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
tributary.add_transaction(tx).await;
}
TransactionKind::Signed(_) => {
log::trace!(
"getting next nonce for Tributary TX in response to processor message"
);
log::trace!("getting next nonce for Tributary TX in response to processor message");
let nonce = loop {
let Some(nonce) = NonceDecider::<D>::nonce(&db, genesis, &tx)
.expect("signed TX didn't have nonce")
let Some(nonce) =
NonceDecider::<D>::nonce(&txn, genesis, &tx).expect("signed TX didn't have nonce")
else {
// This can be None if:
// 1) We scanned the relevant transaction(s) in a Tributary block
// 2) The processor was sent a message and responded
// 3) The Tributary TXN has yet to be committed
log::warn!(
"nonce has yet to be saved for processor-instigated transaction"
);
log::warn!("nonce has yet to be saved for processor-instigated transaction");
sleep(Duration::from_millis(100)).await;
continue;
};
@@ -850,19 +806,51 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
}
}
// TODO: Consider a global txn for this message?
let mut txn = db.txn();
MainDb::<'static, D>::save_handled_message(&mut txn, msg.id);
MainDb::<D>::save_handled_message(&mut txn, msg.id);
txn.commit();
}
processors.ack(msg).await;
}
}
});
pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
db: D,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: Arc<Serai>,
mut processors: Pro,
mut new_tributary: broadcast::Receiver<ActiveTributary<D, P>>,
) {
let channels = Arc::new(RwLock::new(HashMap::new()));
// Listen to new tributary events
tokio::spawn({
let db = db.clone();
let processors = processors.clone();
let channels = channels.clone();
async move {
loop {
let channels = channels.clone();
let tributary = new_tributary.recv().await.unwrap();
let (send, recv) = mpsc::unbounded_channel();
// TODO: Support multisig rotation (not per-Tributary yet per-network?)
channels.write().await.insert(tributary.spec.set().network, send);
// For each new tributary, spawn a dedicated task to handle its messages from the processor
// TODO: Redo per network, not per tributary
tokio::spawn(handle_processor_messages(
db.clone(),
key.clone(),
serai.clone(),
processors.clone(),
tributary,
recv,
));
}
}
});
// Dispatch task
let mut last_msg = None;
loop {
// TODO: We dispatch this to an async task per-processor, yet we don't move to the next message
@@ -871,10 +859,6 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
// Alternatively, a peek method with local delineation of handled messages would work.
let msg = processors.recv().await;
if MainDb::<'static, D>::handled_message(&db, msg.id) {
processors.ack(msg).await;
continue;
}
if last_msg == Some(msg.id) {
sleep(Duration::from_secs(1)).await;
continue;
@@ -892,7 +876,7 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
}
pub async fn run<D: Db, Pro: Processors, P: P2p>(
mut raw_db: D,
raw_db: D,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
p2p: P,
processors: Pro,
@@ -902,7 +886,7 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
let (new_tributary_spec_send, mut new_tributary_spec_recv) = mpsc::unbounded_channel();
// Reload active tributaries from the database
for spec in MainDb::new(&mut raw_db).active_tributaries().1 {
for spec in MainDb::<D>::active_tributaries(&raw_db).1 {
new_tributary_spec_send.send(spec).unwrap();
}
@@ -975,13 +959,13 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
let key = key.clone();
let tributaries = tributaries.clone();
async move {
// SubstrateBlockAck is fired before Preprocess, creating a race between Tributary ack
// of the SubstrateBlock and the sending of all Preprocesses
// The transactions for these are fired before the preprocesses are actually
// received/saved, creating a race between Tributary ack and the availability of all
// Preprocesses
// This waits until the necessary preprocess is available
let get_preprocess = |raw_db, id| async move {
loop {
let Some(preprocess) = MainDb::<D>::first_preprocess(raw_db, id) else {
assert_eq!(id_type, RecognizedIdType::Plan);
sleep(Duration::from_millis(100)).await;
continue;
};