mirror of
https://github.com/serai-dex/serai.git
synced 2024-12-26 21:50:26 +00:00
ca69f97fef
* Design and document a multisig rotation flow * Make Scanner::eventualities a HashMap so it's per-key * Don't drop eventualities, always follow through on them Technical improvements made along the way. * Start creating an isolate object to manage multisigs, which doesn't require being a signer Removes key from SubstrateBlock. * Move Scanner/Scheduler under multisigs * Move Batch construction into MultisigManager * Clarify "should" in Multisig Rotation docs * Add block_number to MultisigManager, as it controls the scanner * Move sign_plans into MultisigManager Removes ThresholdKeys from prepare_send. * Make SubstrateMutable an alias for MultisigManager * Rewrite Multisig Rotation The prior scheme had an exploit possible where funds were sent to the old multisig, then burnt on Serai to send from the new multisig, locking liquidity for 6 hours. While a fee could be applied to stragglers, to make this attack unprofitable, the newly described scheme avoids all this. * Add mini mini is a miniature version of Serai, emphasizing Serai's nature as a collection of independent clocks. The intended use is to identify race conditions and prove protocols are comprehensive regarding when certain clocks tick. This uses loom, a prior candidate for evaluating the processor/coordinator as free of race conditions (#361). * Use mini to prove a race condition in the current multisig rotation docs, and prove safety of alternatives Technically, the prior commit had mini prove the race condition. The docs currently say the activation block of the new multisig is the block after the next Batch's. If the two next Batches had already entered the mempool, prior to set_keys being called, the second next Batch would be expected to contain the new key's data yet fail to as the key wasn't public when the Batch was actually created. The naive solution is to create a Batch, publish it, wait until it's included, and only then scan the next block. This sets a bound of `Batch publication time < block time`. Optimistically, we can publish a Batch in 24s while our shortest block time is 2m. Accordingly, we should be fine with the naive solution which doesn't take advantage of throughput. #333 may significantly change latency however and require an algorithm whose throughput exceeds the rate of blocks created. In order to re-introduce parallelization, enabling throughput, we need to define a safe range of blocks to scan without Serai ordering the first one. mini demonstrates safety of scanning n blocks Serai hasn't acknowledged, so long as the first is scanned before block n+1 is (shifting the n-block window). The docs will be updated next, to reflect this. * Fix Multisig Rotation I believe this is finally good enough to be final. 1) Fixes the race condition present in the prior document, as demonstrated by mini. `Batch`s for block `n` and `n+1`, may have been in the mempool when a multisig's activation block was set to `n`. This would cause a potentially distinct `Batch` for `n+1`, despite `n+1` already having a signed `Batch`. 2) Tightens when UIs should use the new multisig to prevent eclipse attacks, and protection against `Batch` publication delays. 3) Removes liquidity fragmentation by tightening flow/handling of latency. 4) Several clarifications and documentation of reasoning. 5) Correction of "prior multisig" to "all prior multisigs" regarding historical verification, with explanation why. * Clarify terminology in mini Synchronizes it from my original thoughts on potential schema to the design actually created. * Remove most of processor's README for a reference to docs/processor This does drop some misc commentary, though none too beneficial. The section on scanning, deemed sufficiently beneficial, has been moved to a document and expanded on. * Update scanner TODOs in line with new docs * Correct documentation on Bitcoin::Block::time, and Block::time * Make the scanner in MultisigManager no longer public * Always send ConfirmKeyPair, regardless of if in-set * Cargo.lock changes from a prior commit * Add a policy document on defining a Canonical Chain I accidentally committed a version of this with a few headers earlier, and this is a proper version. * Competent MultisigManager::new * Update processor's comments * Add mini to copied files * Re-organize Scanner per multisig rotation document * Add RUST_LOG trace targets to e2e tests * Have the scanner wait once it gets too far ahead Also bug fixes. * Add activation blocks to the scanner * Split received outputs into existing/new in MultisigManager * Select the proper scheduler * Schedule multisig activation as detailed in documentation * Have the Coordinator assert if multiple `Batch`s occur within a block While the processor used to have ack_up_to_block, enabling skips in the block acked, support for this was removed while reworking it for multiple multisigs. It should happen extremely infrequently. While it would still be beneficial to have, if multiple `Batch`s could occur within a block (with the complexity here not being worth adding that ban as a policy), multiple `Batch`s were blocked for DoS reasons. * Schedule payments to the proper multisig * Correct >= to < * Use the new multisig's key for change on schedule * Don't report External TXs to prior multisig once deprecated * Forward from the old multisig to the new one at all opportunities * Move unfulfilled payments in queue from prior to new multisig * Create MultisigsDb, splitting it out of MainDb Drops the call to finish_signing from the Signer. While this will cause endless re-attempts, the Signer will still consider them completed and drop them, making this an O(n) cost at boot even if we did nothing from here. The MultisigManager should call finish_signing once the Scanner completes the Eventuality. * Don't check Scanner-emitted completions, trust they are completions Prevents needing to use async code to mark the completion and creates a fault-free model. The current model, on fault, would cause a lack of marked completion in the signer. * Fix a possible panic in the processor A shorter-chain reorg could cause this assert to trip. It's fixed by de-duplicating the data, as the assertion checked consistency. Without the potential for inconsistency, it's unnecessary. * Document why an existing TODO isn't valid * Change when we drop payments for being to the change address The earlier timing prevents creating Plans solely to the branch address, causing the payments to be dropped, and the TX to become an effective aggregation TX. * Extensively document solutions to Eventualities being potentially created after having already scanned their resolutions * When closing, drop External/Branch outputs which don't cause progress * Properly decide if Change outputs should be forward or not when closing This completes all code needed to make the old multisig have a finite lifetime. * Commentary on forwarding schemes * Provide a 1 block window, with liquidity fragmentation risks, due to latency On Bitcoin, this will be 10 minutes for the relevant Batch to be confirmed. On Monero, 2 minutes. On Ethereum, ~6 minutes. Also updates the Multisig Rotation document with the new forwarding plan. * Implement transaction forwarding from old multisig to new multisig Identifies a fault where Branch outputs which shouldn't be dropped may be, if another output fulfills their next step. Locking Branch fulfillment down to only Branch outputs is not done in this commit, but will be in the next. * Only let Branch outputs fulfill branches * Update TODOs * Move the location of handling signer events to avoid a race condition * Avoid a deadlock by using a RwLock on a single txn instead of two txns * Move Batch ID out of the Scanner * Increase from one block of latency on new keys activation to two For Monero, this offered just two minutes when our latency to publish a Batch is around a minute already. This does increase the time our liquidity can be fragmented by up to 20 minutes (Bitcoin), yet it's a stupid attack only possible once a week (when we rotate). Prioritizing normal users' transactions not being subject to forwarding is more important here. Ideally, we'd not do +2 blocks yet plus `time`, such as +10 minutes, making this agnostic of the underlying network's block scheduling. This is a complexity not worth it. * Split MultisigManager::substrate_block into multiple functions * Further tweaks to substrate_block * Acquire a lock on all Scanner operations after calling ack_block Gives time to call register_eventuality and initiate signing. * Merge sign_plans into substrate_block Also ensure the Scanner's lock isn't prematurely released. * Use a HashMap to pass to-be-forwarded instructions, not the DB * Successfully determine in ClosingExisting * Move from 2 blocks of latency when rotating to 10 minutes Superior as noted in 6d07af92ce10cfd74c17eb3400368b0150eb36d7, now trivial to implement thanks to prior commit. * Add note justifying measuring time in blocks when rotating * Implement delaying of outputs received early to the new multisig per specification * Documentation on why Branch outputs don't have the race condition concerns Change do Also ensures 6 hours is at least N::CONFIRMATIONS, for sanity purposes. * Remove TODO re: sanity checking Eventualities We sanity check the Plan the Eventuality is derived from, and the Eventuality is handled moments later (in the same file, with a clear call path). There's no reason to add such APIs to Eventualities for a sanity check given that. * Add TODO(now) for TODOs which must be done in this branch Also deprecates a pair of TODOs to TODO2, and accepts the flow of the Signer having the Eventuality. * Correct errors in potential/future flow descriptions * Accept having a single Plan Vec Per the following code consuming it, there's no benefit to bifurcating it by key. * Only issue sign_transaction on boot for the proper signer * Only set keys when participating in their construction * Misc progress Only send SubstrateBlockAck when we have a signer, as it's only used to tell the Tributary of what Plans are being signed in response to this block. Only immediately sets substrate_signer if session is 0. On boot, doesn't panic if we don't have an active key (as we wouldn't if only joining the next multisig). Continues. * Correctly detect and set retirement block Modifies the retirement block from first block meeting requirements to block CONFIRMATIONS after. Adds an ack flow to the Scanner's Confirmed event and Block event to accomplish this, which may deadlock at this time (will be fixed shortly). Removes an invalid await (after a point declared unsafe to use await) from MultisigsManager::next_event. * Remove deadlock in multisig_completed and document alternative The alternative is simpler, albeit less efficient. There's no reason to adopt it now, yet perhaps if it benefits modeling? * Handle the final step of retirement, dropping the old key and setting new to existing * Remove TODO about emitting a Block on every step If we emit on NewAsChange, we lose the purpose of the NewAsChange period. The only concern is if we reach ClosingExisting, and nothing has happened, then all coins will still be in the old multisig until something finally does. This isn't a problem worth solving, as it's latency under exceptional dead time. * Add TODO about potentially not emitting a Block event for the reitrement block * Restore accidentally deleted CI file * Pair of slight tweaks * Add missing if statement * Disable an assertion when testing One of the test flows currently abuses the Scanner in a way triggering it.
647 lines
24 KiB
Rust
647 lines
24 KiB
Rust
use std::{sync::RwLock, time::Duration, collections::HashMap};
|
|
|
|
use zeroize::{Zeroize, Zeroizing};
|
|
|
|
use transcript::{Transcript, RecommendedTranscript};
|
|
use ciphersuite::{group::GroupEncoding, Ciphersuite};
|
|
|
|
use log::{info, warn};
|
|
use tokio::time::sleep;
|
|
|
|
use serai_client::{
|
|
primitives::{BlockHash, NetworkId},
|
|
validator_sets::primitives::{ValidatorSet, KeyPair},
|
|
};
|
|
|
|
use messages::{CoordinatorMessage, ProcessorMessage};
|
|
|
|
use serai_env as env;
|
|
|
|
use message_queue::{Service, client::MessageQueue};
|
|
|
|
mod plan;
|
|
pub use plan::*;
|
|
|
|
mod networks;
|
|
use networks::{PostFeeBranch, Block, Network, get_latest_block_number, get_block};
|
|
#[cfg(feature = "bitcoin")]
|
|
use networks::Bitcoin;
|
|
#[cfg(feature = "monero")]
|
|
use networks::Monero;
|
|
|
|
mod additional_key;
|
|
pub use additional_key::additional_key;
|
|
|
|
mod db;
|
|
pub use db::*;
|
|
|
|
mod coordinator;
|
|
pub use coordinator::*;
|
|
|
|
mod key_gen;
|
|
use key_gen::{KeyConfirmed, KeyGen};
|
|
|
|
mod signer;
|
|
use signer::{SignerEvent, Signer};
|
|
|
|
mod substrate_signer;
|
|
use substrate_signer::{SubstrateSignerEvent, SubstrateSigner};
|
|
|
|
mod multisigs;
|
|
use multisigs::{MultisigEvent, MultisigManager};
|
|
|
|
#[cfg(test)]
|
|
mod tests;
|
|
|
|
// Items which are mutably borrowed by Tributary.
|
|
// Any exceptions to this have to be carefully monitored in order to ensure consistency isn't
|
|
// violated.
|
|
struct TributaryMutable<N: Network, D: Db> {
|
|
// The following are actually mutably borrowed by Substrate as well.
|
|
// - Substrate triggers key gens, and determines which to use.
|
|
// - SubstrateBlock events cause scheduling which causes signing.
|
|
//
|
|
// This is still considered Tributary-mutable as most mutation (preprocesses/shares) happens by
|
|
// the Tributary.
|
|
//
|
|
// Creation of tasks is by Substrate, yet this is safe since the mutable borrow is transferred to
|
|
// Tributary.
|
|
//
|
|
// Tributary stops mutating a key gen attempt before Substrate is made aware of it, ensuring
|
|
// Tributary drops its mutable borrow before Substrate acquires it. Tributary will maintain a
|
|
// mutable borrow on the *key gen task*, yet the finalization code can successfully run for any
|
|
// attempt.
|
|
//
|
|
// The only other note is how the scanner may cause a signer task to be dropped, effectively
|
|
// invalidating the Tributary's mutable borrow. The signer is coded to allow for attempted usage
|
|
// of a dropped task.
|
|
key_gen: KeyGen<N, D>,
|
|
signers: HashMap<Vec<u8>, Signer<N, D>>,
|
|
|
|
// This is also mutably borrowed by the Scanner.
|
|
// The Scanner starts new sign tasks.
|
|
// The Tributary mutates already-created signed tasks, potentially completing them.
|
|
// Substrate may mark tasks as completed, invalidating any existing mutable borrows.
|
|
// The safety of this follows as written above.
|
|
|
|
// There should only be one SubstrateSigner at a time (see #277)
|
|
substrate_signer: Option<SubstrateSigner<D>>,
|
|
}
|
|
|
|
// Items which are mutably borrowed by Substrate.
|
|
// Any exceptions to this have to be carefully monitored in order to ensure consistency isn't
|
|
// violated.
|
|
|
|
/*
|
|
The MultisigManager contains the Scanner and Schedulers.
|
|
|
|
The scanner is expected to autonomously operate, scanning blocks as they appear. When a block is
|
|
sufficiently confirmed, the scanner causes the Substrate signer to sign a batch. It itself only
|
|
mutates its list of finalized blocks, to protect against re-orgs, and its in-memory state though.
|
|
|
|
Disk mutations to the scan-state only happens once the relevant `Batch` is included on Substrate.
|
|
It can't be mutated as soon as the `Batch` is signed as we need to know the order of `Batch`s
|
|
relevant to `Burn`s.
|
|
|
|
Schedulers take in new outputs, confirmed in `Batch`s, and outbound payments, triggered by
|
|
`Burn`s.
|
|
|
|
Substrate also decides when to move to a new multisig, hence why this entire object is
|
|
Substate-mutable.
|
|
|
|
Since MultisigManager should always be verifiable, and the Tributary is temporal, MultisigManager
|
|
being entirely SubstrateMutable shows proper data pipe-lining.
|
|
*/
|
|
|
|
type SubstrateMutable<N, D> = MultisigManager<D, N>;
|
|
|
|
async fn handle_coordinator_msg<D: Db, N: Network, Co: Coordinator>(
|
|
txn: &mut D::Transaction<'_>,
|
|
network: &N,
|
|
coordinator: &mut Co,
|
|
tributary_mutable: &mut TributaryMutable<N, D>,
|
|
substrate_mutable: &mut SubstrateMutable<N, D>,
|
|
msg: &Message,
|
|
) {
|
|
// If this message expects a higher block number than we have, halt until synced
|
|
async fn wait<N: Network, D: Db>(
|
|
txn: &D::Transaction<'_>,
|
|
substrate_mutable: &SubstrateMutable<N, D>,
|
|
block_hash: &BlockHash,
|
|
) {
|
|
let mut needed_hash = <N::Block as Block<N>>::Id::default();
|
|
needed_hash.as_mut().copy_from_slice(&block_hash.0);
|
|
|
|
loop {
|
|
// Ensure our scanner has scanned this block, which means our daemon has this block at
|
|
// a sufficient depth
|
|
if substrate_mutable.block_number(txn, &needed_hash).await.is_none() {
|
|
warn!(
|
|
"node is desynced. we haven't scanned {} which should happen after {} confirms",
|
|
hex::encode(&needed_hash),
|
|
N::CONFIRMATIONS,
|
|
);
|
|
sleep(Duration::from_secs(10)).await;
|
|
continue;
|
|
};
|
|
break;
|
|
}
|
|
|
|
// TODO2: Sanity check we got an AckBlock (or this is the AckBlock) for the block in question
|
|
|
|
/*
|
|
let synced = |context: &SubstrateContext, key| -> Result<(), ()> {
|
|
// Check that we've synced this block and can actually operate on it ourselves
|
|
let latest = scanner.latest_scanned(key);
|
|
if usize::try_from(context.network_latest_finalized_block).unwrap() < latest {
|
|
log::warn!(
|
|
"external network node disconnected/desynced from rest of the network. \
|
|
our block: {latest:?}, network's acknowledged: {}",
|
|
context.network_latest_finalized_block,
|
|
);
|
|
Err(())?;
|
|
}
|
|
Ok(())
|
|
};
|
|
*/
|
|
}
|
|
|
|
if let Some(required) = msg.msg.required_block() {
|
|
// wait only reads from, it doesn't mutate, substrate_mutable
|
|
wait(txn, substrate_mutable, &required).await;
|
|
}
|
|
|
|
async fn activate_key<N: Network, D: Db>(
|
|
network: &N,
|
|
substrate_mutable: &mut SubstrateMutable<N, D>,
|
|
tributary_mutable: &mut TributaryMutable<N, D>,
|
|
txn: &mut D::Transaction<'_>,
|
|
set: ValidatorSet,
|
|
key_pair: KeyPair,
|
|
activation_number: usize,
|
|
) {
|
|
info!("activating {set:?}'s keys at {activation_number}");
|
|
|
|
let network_key = <N as Network>::Curve::read_G::<&[u8]>(&mut key_pair.1.as_ref())
|
|
.expect("Substrate finalized invalid point as a network's key");
|
|
|
|
if tributary_mutable.key_gen.in_set(&set) {
|
|
// See TributaryMutable's struct definition for why this block is safe
|
|
let KeyConfirmed { substrate_keys, network_keys } =
|
|
tributary_mutable.key_gen.confirm(txn, set, key_pair.clone()).await;
|
|
if set.session.0 == 0 {
|
|
tributary_mutable.substrate_signer = Some(SubstrateSigner::new(N::NETWORK, substrate_keys));
|
|
}
|
|
tributary_mutable
|
|
.signers
|
|
.insert(key_pair.1.into(), Signer::new(network.clone(), network_keys));
|
|
}
|
|
|
|
substrate_mutable.add_key(txn, activation_number, network_key).await;
|
|
}
|
|
|
|
match msg.msg.clone() {
|
|
CoordinatorMessage::KeyGen(msg) => {
|
|
coordinator
|
|
.send(ProcessorMessage::KeyGen(tributary_mutable.key_gen.handle(txn, msg).await))
|
|
.await;
|
|
}
|
|
|
|
CoordinatorMessage::Sign(msg) => {
|
|
tributary_mutable
|
|
.signers
|
|
.get_mut(msg.key())
|
|
.expect("coordinator told us to sign with a signer we don't have")
|
|
.handle(txn, msg)
|
|
.await;
|
|
}
|
|
|
|
CoordinatorMessage::Coordinator(msg) => {
|
|
tributary_mutable
|
|
.substrate_signer
|
|
.as_mut()
|
|
.expect(
|
|
"coordinator told us to sign a batch when we don't have a Substrate signer at this time",
|
|
)
|
|
.handle(txn, msg)
|
|
.await;
|
|
}
|
|
|
|
CoordinatorMessage::Substrate(msg) => {
|
|
match msg {
|
|
messages::substrate::CoordinatorMessage::ConfirmKeyPair { context, set, key_pair } => {
|
|
// This is the first key pair for this network so no block has been finalized yet
|
|
// TODO: Write documentation for this in docs/
|
|
// TODO: Use an Option instead of a magic?
|
|
if context.network_latest_finalized_block.0 == [0; 32] {
|
|
assert!(tributary_mutable.signers.is_empty());
|
|
assert!(tributary_mutable.substrate_signer.is_none());
|
|
// We can't check this as existing is no longer pub
|
|
// assert!(substrate_mutable.existing.as_ref().is_none());
|
|
|
|
// Wait until a network's block's time exceeds Serai's time
|
|
|
|
// If the latest block number is 10, then the block indexed by 1 has 10 confirms
|
|
// 10 + 1 - 10 = 1
|
|
let mut block_i;
|
|
while {
|
|
block_i =
|
|
(get_latest_block_number(network).await + 1).saturating_sub(N::CONFIRMATIONS);
|
|
get_block(network, block_i).await.time() < context.serai_time
|
|
} {
|
|
info!(
|
|
"serai confirmed the first key pair for a set. {} {}",
|
|
"we're waiting for a network's finalized block's time to exceed unix time ",
|
|
context.serai_time,
|
|
);
|
|
sleep(Duration::from_secs(5)).await;
|
|
}
|
|
|
|
// Find the first block to do so
|
|
let mut earliest = block_i;
|
|
// earliest > 0 prevents a panic if Serai creates keys before the genesis block
|
|
// which... should be impossible
|
|
// Yet a prevented panic is a prevented panic
|
|
while (earliest > 0) &&
|
|
(get_block(network, earliest - 1).await.time() >= context.serai_time)
|
|
{
|
|
earliest -= 1;
|
|
}
|
|
|
|
// Use this as the activation block
|
|
let activation_number = earliest;
|
|
|
|
activate_key(
|
|
network,
|
|
substrate_mutable,
|
|
tributary_mutable,
|
|
txn,
|
|
set,
|
|
key_pair,
|
|
activation_number,
|
|
)
|
|
.await;
|
|
} else {
|
|
let mut block_before_queue_block = <N::Block as Block<N>>::Id::default();
|
|
block_before_queue_block
|
|
.as_mut()
|
|
.copy_from_slice(&context.network_latest_finalized_block.0);
|
|
// We can't set these keys for activation until we know their queue block, which we
|
|
// won't until the next Batch is confirmed
|
|
// Set this variable so when we get the next Batch event, we can handle it
|
|
MainDb::<N, D>::set_pending_activation(txn, block_before_queue_block, set, key_pair);
|
|
}
|
|
}
|
|
|
|
messages::substrate::CoordinatorMessage::SubstrateBlock {
|
|
context,
|
|
network: network_id,
|
|
block: substrate_block,
|
|
burns,
|
|
batches,
|
|
} => {
|
|
assert_eq!(network_id, N::NETWORK, "coordinator sent us data for another network");
|
|
|
|
if let Some((block, set, key_pair)) = MainDb::<N, D>::pending_activation(txn) {
|
|
// Only run if this is a Batch belonging to a distinct block
|
|
if context.network_latest_finalized_block.as_ref() != block.as_ref() {
|
|
let mut queue_block = <N::Block as Block<N>>::Id::default();
|
|
queue_block.as_mut().copy_from_slice(context.network_latest_finalized_block.as_ref());
|
|
|
|
let activation_number = substrate_mutable
|
|
.block_number(txn, &queue_block)
|
|
.await
|
|
.expect("KeyConfirmed from context we haven't synced") +
|
|
N::CONFIRMATIONS;
|
|
|
|
activate_key(
|
|
network,
|
|
substrate_mutable,
|
|
tributary_mutable,
|
|
txn,
|
|
set,
|
|
key_pair,
|
|
activation_number,
|
|
)
|
|
.await;
|
|
|
|
MainDb::<N, D>::clear_pending_activation(txn);
|
|
}
|
|
}
|
|
|
|
// Since this block was acknowledged, we no longer have to sign the batches for it
|
|
if let Some(substrate_signer) = tributary_mutable.substrate_signer.as_mut() {
|
|
for batch_id in batches {
|
|
substrate_signer.batch_signed(txn, batch_id);
|
|
}
|
|
}
|
|
|
|
let (acquired_lock, to_sign) =
|
|
substrate_mutable.substrate_block(txn, network, context, burns).await;
|
|
|
|
// Send SubstrateBlockAck, with relevant plan IDs, before we trigger the signing of these
|
|
// plans
|
|
if !tributary_mutable.signers.is_empty() {
|
|
coordinator
|
|
.send(messages::ProcessorMessage::Coordinator(
|
|
messages::coordinator::ProcessorMessage::SubstrateBlockAck {
|
|
network: N::NETWORK,
|
|
block: substrate_block,
|
|
plans: to_sign.iter().map(|signable| signable.1).collect(),
|
|
},
|
|
))
|
|
.await;
|
|
}
|
|
|
|
// See commentary in TributaryMutable for why this is safe
|
|
let signers = &mut tributary_mutable.signers;
|
|
for (key, id, tx, eventuality) in to_sign {
|
|
if let Some(signer) = signers.get_mut(key.to_bytes().as_ref()) {
|
|
signer.sign_transaction(txn, id, tx, eventuality).await;
|
|
}
|
|
}
|
|
|
|
// This is not premature, even if this block had multiple `Batch`s created, as the first
|
|
// `Batch` alone will trigger all Plans/Eventualities/Signs
|
|
if acquired_lock {
|
|
substrate_mutable.release_scanner_lock().await;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
async fn boot<N: Network, D: Db>(
|
|
raw_db: &mut D,
|
|
network: &N,
|
|
) -> (MainDb<N, D>, TributaryMutable<N, D>, SubstrateMutable<N, D>) {
|
|
let mut entropy_transcript = {
|
|
let entropy = Zeroizing::new(env::var("ENTROPY").expect("entropy wasn't specified"));
|
|
if entropy.len() != 64 {
|
|
panic!("entropy isn't the right length");
|
|
}
|
|
let mut bytes =
|
|
Zeroizing::new(hex::decode(entropy).map_err(|_| ()).expect("entropy wasn't hex-formatted"));
|
|
if bytes.len() != 32 {
|
|
bytes.zeroize();
|
|
panic!("entropy wasn't 32 bytes");
|
|
}
|
|
let mut entropy = Zeroizing::new([0; 32]);
|
|
let entropy_mut: &mut [u8] = entropy.as_mut();
|
|
entropy_mut.copy_from_slice(bytes.as_ref());
|
|
|
|
let mut transcript = RecommendedTranscript::new(b"Serai Processor Entropy");
|
|
transcript.append_message(b"entropy", entropy);
|
|
transcript
|
|
};
|
|
|
|
// TODO: Save a hash of the entropy to the DB and make sure the entropy didn't change
|
|
|
|
let mut entropy = |label| {
|
|
let mut challenge = entropy_transcript.challenge(label);
|
|
let mut res = Zeroizing::new([0; 32]);
|
|
let res_mut: &mut [u8] = res.as_mut();
|
|
res_mut.copy_from_slice(&challenge[.. 32]);
|
|
challenge.zeroize();
|
|
res
|
|
};
|
|
|
|
// We don't need to re-issue GenerateKey orders because the coordinator is expected to
|
|
// schedule/notify us of new attempts
|
|
// TODO: Is this above comment still true? Not at all due to the planned lack of DKG timeouts?
|
|
let key_gen = KeyGen::<N, _>::new(raw_db.clone(), entropy(b"key-gen_entropy"));
|
|
|
|
let (multisig_manager, current_keys, actively_signing) =
|
|
MultisigManager::new(raw_db, network).await;
|
|
|
|
let mut substrate_signer = None;
|
|
let mut signers = HashMap::new();
|
|
|
|
let main_db = MainDb::<N, _>::new(raw_db.clone());
|
|
|
|
for (i, key) in current_keys.iter().enumerate() {
|
|
let Some((substrate_keys, network_keys)) = key_gen.keys(key) else { continue };
|
|
let network_key = network_keys.group_key();
|
|
|
|
// If this is the oldest key, load the SubstrateSigner for it as the active SubstrateSigner
|
|
// The new key only takes responsibility once the old key is fully deprecated
|
|
//
|
|
// We don't have to load any state for this since the Scanner will re-fire any events
|
|
// necessary, only no longer scanning old blocks once Substrate acks them
|
|
if i == 0 {
|
|
substrate_signer = Some(SubstrateSigner::new(N::NETWORK, substrate_keys));
|
|
}
|
|
|
|
// The Scanner re-fires events as needed for substrate_signer yet not signer
|
|
// This is due to the transactions which we start signing from due to a block not being
|
|
// guaranteed to be signed before we stop scanning the block on reboot
|
|
// We could simplify the Signer flow by delaying when it acks a block, yet that'd:
|
|
// 1) Increase the startup time
|
|
// 2) Cause re-emission of Batch events, which we'd need to check the safety of
|
|
// (TODO: Do anyways?)
|
|
// 3) Violate the attempt counter (TODO: Is this already being violated?)
|
|
let mut signer = Signer::new(network.clone(), network_keys);
|
|
|
|
// Sign any TXs being actively signed
|
|
let key = key.to_bytes();
|
|
for (plan, tx, eventuality) in &actively_signing {
|
|
if plan.key == network_key {
|
|
let mut txn = raw_db.txn();
|
|
signer.sign_transaction(&mut txn, plan.id(), tx.clone(), eventuality.clone()).await;
|
|
// This should only have re-writes of existing data
|
|
drop(txn);
|
|
}
|
|
}
|
|
|
|
signers.insert(key.as_ref().to_vec(), signer);
|
|
}
|
|
|
|
(main_db, TributaryMutable { key_gen, substrate_signer, signers }, multisig_manager)
|
|
}
|
|
|
|
#[allow(clippy::await_holding_lock)] // Needed for txn, unfortunately can't be down-scoped
|
|
async fn run<N: Network, D: Db, Co: Coordinator>(mut raw_db: D, network: N, mut coordinator: Co) {
|
|
// We currently expect a contextless bidirectional mapping between these two values
|
|
// (which is that any value of A can be interpreted as B and vice versa)
|
|
// While we can write a contextual mapping, we have yet to do so
|
|
// This check ensures no network which doesn't have a bidirectional mapping is defined
|
|
assert_eq!(<N::Block as Block<N>>::Id::default().as_ref().len(), BlockHash([0u8; 32]).0.len());
|
|
|
|
let (main_db, mut tributary_mutable, mut substrate_mutable) = boot(&mut raw_db, &network).await;
|
|
|
|
// We can't load this from the DB as we can't guarantee atomic increments with the ack function
|
|
let mut last_coordinator_msg = None;
|
|
|
|
loop {
|
|
// The following select uses this txn in both branches, hence why needing a RwLock to pass it
|
|
// around is needed
|
|
let txn = RwLock::new(raw_db.txn());
|
|
|
|
let mut outer_msg = None;
|
|
|
|
tokio::select! {
|
|
// This blocks the entire processor until it finishes handling this message
|
|
// KeyGen specifically may take a notable amount of processing time
|
|
// While that shouldn't be an issue in practice, as after processing an attempt it'll handle
|
|
// the other messages in the queue, it may be beneficial to parallelize these
|
|
// They could likely be parallelized by type (KeyGen, Sign, Substrate) without issue
|
|
msg = coordinator.recv() => {
|
|
let mut txn = txn.write().unwrap();
|
|
let txn = &mut txn;
|
|
|
|
assert_eq!(msg.id, (last_coordinator_msg.unwrap_or(msg.id - 1) + 1));
|
|
last_coordinator_msg = Some(msg.id);
|
|
|
|
// Only handle this if we haven't already
|
|
if !main_db.handled_message(msg.id) {
|
|
MainDb::<N, D>::handle_message(txn, msg.id);
|
|
|
|
// This is isolated to better think about how its ordered, or rather, about how the other
|
|
// cases aren't ordered
|
|
//
|
|
// While the coordinator messages are ordered, they're not deterministically ordered
|
|
// Tributary-caused messages are deterministically ordered, and Substrate-caused messages
|
|
// are deterministically-ordered, yet they're both shoved into a singular queue
|
|
// The order at which they're shoved in together isn't deterministic
|
|
//
|
|
// This is safe so long as Tributary and Substrate messages don't both expect mutable
|
|
// references over the same data
|
|
handle_coordinator_msg(
|
|
&mut **txn,
|
|
&network,
|
|
&mut coordinator,
|
|
&mut tributary_mutable,
|
|
&mut substrate_mutable,
|
|
&msg,
|
|
).await;
|
|
}
|
|
|
|
outer_msg = Some(msg);
|
|
},
|
|
|
|
msg = substrate_mutable.next_event(&txn) => {
|
|
let mut txn = txn.write().unwrap();
|
|
let txn = &mut txn;
|
|
match msg {
|
|
MultisigEvent::Batches(retired_key_new_key, batches) => {
|
|
// Start signing this batch
|
|
for batch in batches {
|
|
info!("created batch {} ({} instructions)", batch.id, batch.instructions.len());
|
|
|
|
if let Some(substrate_signer) = tributary_mutable.substrate_signer.as_mut() {
|
|
substrate_signer.sign(txn, batch).await;
|
|
}
|
|
}
|
|
|
|
if let Some((retired_key, new_key)) = retired_key_new_key {
|
|
// Safe to mutate since all signing operations are done and no more will be added
|
|
tributary_mutable.signers.remove(retired_key.to_bytes().as_ref());
|
|
tributary_mutable.substrate_signer.take();
|
|
if let Some((substrate_keys, _)) = tributary_mutable.key_gen.keys(&new_key) {
|
|
tributary_mutable.substrate_signer =
|
|
Some(SubstrateSigner::new(N::NETWORK, substrate_keys));
|
|
}
|
|
}
|
|
},
|
|
MultisigEvent::Completed(key, id, tx) => {
|
|
if let Some(signer) = tributary_mutable.signers.get_mut(&key) {
|
|
signer.completed(txn, id, tx);
|
|
}
|
|
}
|
|
}
|
|
},
|
|
}
|
|
|
|
// Check if the signers have events
|
|
// The signers will only have events after the above select executes, so having no timeout on
|
|
// the above is fine
|
|
// TODO: Have the Signers return these events, allowing removing these channels?
|
|
for (key, signer) in tributary_mutable.signers.iter_mut() {
|
|
while let Some(msg) = signer.events.pop_front() {
|
|
match msg {
|
|
SignerEvent::ProcessorMessage(msg) => {
|
|
coordinator.send(ProcessorMessage::Sign(msg)).await;
|
|
}
|
|
|
|
SignerEvent::SignedTransaction { id, tx } => {
|
|
coordinator
|
|
.send(ProcessorMessage::Sign(messages::sign::ProcessorMessage::Completed {
|
|
key: key.clone(),
|
|
id,
|
|
tx: tx.as_ref().to_vec(),
|
|
}))
|
|
.await;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if let Some(signer) = tributary_mutable.substrate_signer.as_mut() {
|
|
while let Some(msg) = signer.events.pop_front() {
|
|
match msg {
|
|
SubstrateSignerEvent::ProcessorMessage(msg) => {
|
|
coordinator.send(ProcessorMessage::Coordinator(msg)).await;
|
|
}
|
|
SubstrateSignerEvent::SignedBatch(batch) => {
|
|
coordinator
|
|
.send(ProcessorMessage::Substrate(messages::substrate::ProcessorMessage::Update {
|
|
batch,
|
|
}))
|
|
.await;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
txn.into_inner().unwrap().commit();
|
|
if let Some(msg) = outer_msg {
|
|
coordinator.ack(msg).await;
|
|
}
|
|
}
|
|
}
|
|
|
|
#[tokio::main]
|
|
async fn main() {
|
|
// Override the panic handler with one which will panic if any tokio task panics
|
|
{
|
|
let existing = std::panic::take_hook();
|
|
std::panic::set_hook(Box::new(move |panic| {
|
|
existing(panic);
|
|
const MSG: &str = "exiting the process due to a task panicking";
|
|
println!("{MSG}");
|
|
log::error!("{MSG}");
|
|
std::process::exit(1);
|
|
}));
|
|
}
|
|
|
|
if std::env::var("RUST_LOG").is_err() {
|
|
std::env::set_var("RUST_LOG", serai_env::var("RUST_LOG").unwrap_or_else(|| "info".to_string()));
|
|
}
|
|
env_logger::init();
|
|
|
|
let db = serai_db::new_rocksdb(&env::var("DB_PATH").expect("path to DB wasn't specified"));
|
|
|
|
// Network configuration
|
|
let url = {
|
|
let login = env::var("NETWORK_RPC_LOGIN").expect("network RPC login wasn't specified");
|
|
let hostname = env::var("NETWORK_RPC_HOSTNAME").expect("network RPC hostname wasn't specified");
|
|
let port = env::var("NETWORK_RPC_PORT").expect("network port domain wasn't specified");
|
|
"http://".to_string() + &login + "@" + &hostname + ":" + &port
|
|
};
|
|
let network_id = match env::var("NETWORK").expect("network wasn't specified").as_str() {
|
|
"bitcoin" => NetworkId::Bitcoin,
|
|
"monero" => NetworkId::Monero,
|
|
_ => panic!("unrecognized network"),
|
|
};
|
|
|
|
let coordinator = MessageQueue::from_env(Service::Processor(network_id));
|
|
|
|
match network_id {
|
|
#[cfg(feature = "bitcoin")]
|
|
NetworkId::Bitcoin => run(db, Bitcoin::new(url).await, coordinator).await,
|
|
#[cfg(feature = "monero")]
|
|
NetworkId::Monero => run(db, Monero::new(url), coordinator).await,
|
|
_ => panic!("spawning a processor for an unsupported network"),
|
|
}
|
|
}
|