serai-processor-bin

Moves the coordinator loop out of serai-bitcoin-processor, completing it.

Fixes a potential race condition in the message-queue regarding multiple
sockets sending messages at once.
This commit is contained in:
Luke Parker
2024-09-11 18:56:23 -04:00
parent fcd5fb85df
commit b6811f9015
22 changed files with 705 additions and 594 deletions

View File

@@ -7,22 +7,22 @@ pub(crate) struct KeyGenParams;
impl key_gen::KeyGenParams for KeyGenParams {
const ID: &'static str = "Bitcoin";
type ExternalNetworkCurve = Secp256k1;
type ExternalNetworkCiphersuite = Secp256k1;
fn tweak_keys(keys: &mut ThresholdKeys<Self::ExternalNetworkCurve>) {
fn tweak_keys(keys: &mut ThresholdKeys<Self::ExternalNetworkCiphersuite>) {
*keys = bitcoin_serai::wallet::tweak_keys(keys);
// Also create a scanner to assert these keys, and all expected paths, are usable
scanner(keys.group_key());
}
fn encode_key(key: <Self::ExternalNetworkCurve as Ciphersuite>::G) -> Vec<u8> {
fn encode_key(key: <Self::ExternalNetworkCiphersuite as Ciphersuite>::G) -> Vec<u8> {
let key = key.to_bytes();
let key: &[u8] = key.as_ref();
// Skip the parity encoding as we know this key is even
key[1 ..].to_vec()
}
fn decode_key(key: &[u8]) -> Option<<Self::ExternalNetworkCurve as Ciphersuite>::G> {
fn decode_key(key: &[u8]) -> Option<<Self::ExternalNetworkCiphersuite as Ciphersuite>::G> {
x_coord_to_even_point(key)
}
}

View File

@@ -6,16 +6,9 @@
static ALLOCATOR: zalloc::ZeroizingAlloc<std::alloc::System> =
zalloc::ZeroizingAlloc(std::alloc::System);
use core::cmp::Ordering;
use bitcoin_serai::rpc::Rpc as BRpc;
use ciphersuite::Ciphersuite;
use serai_client::validator_sets::primitives::Session;
use serai_db::{DbTxn, Db};
use ::primitives::EncodableG;
use ::key_gen::KeyGenParams as KeyGenParamsTrait;
use scanner::{ScannerFeed, Scanner};
use ::primitives::task::{Task, ContinuallyRan};
mod primitives;
pub(crate) use crate::primitives::*;
@@ -34,6 +27,7 @@ use scheduler::Scheduler;
// Our custom code for Bitcoin
mod db;
mod txindex;
use txindex::TxIndexTask;
pub(crate) fn hash_bytes(hash: bitcoin_serai::bitcoin::hashes::sha256d::Hash) -> [u8; 32] {
use bitcoin_serai::bitcoin::hashes::Hash;
@@ -43,204 +37,29 @@ pub(crate) fn hash_bytes(hash: bitcoin_serai::bitcoin::hashes::sha256d::Hash) ->
res
}
async fn first_block_after_time<S: ScannerFeed>(feed: &S, serai_time: u64) -> u64 {
async fn first_block_after_time_iteration<S: ScannerFeed>(
feed: &S,
serai_time: u64,
) -> Result<Option<u64>, S::EphemeralError> {
let latest = feed.latest_finalized_block_number().await?;
let latest_time = feed.time_of_block(latest).await?;
if latest_time < serai_time {
tokio::time::sleep(core::time::Duration::from_secs(serai_time - latest_time)).await;
return Ok(None);
}
// A finalized block has a time greater than or equal to the time we want to start at
// Find the first such block with a binary search
// start_search and end_search are inclusive
let mut start_search = 0;
let mut end_search = latest;
while start_search != end_search {
// This on purposely chooses the earlier block in the case two blocks are both in the middle
let to_check = start_search + ((end_search - start_search) / 2);
let block_time = feed.time_of_block(to_check).await?;
match block_time.cmp(&serai_time) {
Ordering::Less => {
start_search = to_check + 1;
assert!(start_search <= end_search);
}
Ordering::Equal | Ordering::Greater => {
// This holds true since we pick the earlier block upon an even search distance
// If it didn't, this would cause an infinite loop
assert!(to_check < end_search);
end_search = to_check;
}
}
}
Ok(Some(start_search))
}
loop {
match first_block_after_time_iteration(feed, serai_time).await {
Ok(Some(block)) => return block,
Ok(None) => {
log::info!("waiting for block to activate at (a block with timestamp >= {serai_time})");
}
Err(e) => {
log::error!("couldn't find the first block Serai should scan due to an RPC error: {e:?}");
}
}
tokio::time::sleep(core::time::Duration::from_secs(5)).await;
}
}
/// Fetch the next message from the Coordinator.
///
/// This message is guaranteed to have never been handled before, where handling is defined as
/// this `txn` being committed.
async fn next_message(_txn: &mut impl DbTxn) -> messages::CoordinatorMessage {
todo!("TODO")
}
async fn send_message(_msg: messages::ProcessorMessage) {
todo!("TODO")
}
async fn coordinator_loop<D: Db>(
mut db: D,
feed: Rpc<D>,
mut key_gen: ::key_gen::KeyGen<KeyGenParams>,
mut signers: signers::Signers<D, Rpc<D>, Scheduler<D>, Rpc<D>>,
mut scanner: Option<scanner::Scanner<Rpc<D>>>,
) {
loop {
let db_clone = db.clone();
let mut txn = db.txn();
let msg = next_message(&mut txn).await;
let mut txn = Some(txn);
match msg {
messages::CoordinatorMessage::KeyGen(msg) => {
let txn = txn.as_mut().unwrap();
let mut new_key = None;
// This is a computationally expensive call yet it happens infrequently
for msg in key_gen.handle(txn, msg) {
if let messages::key_gen::ProcessorMessage::GeneratedKeyPair { session, .. } = &msg {
new_key = Some(*session)
}
send_message(messages::ProcessorMessage::KeyGen(msg)).await;
}
// If we were yielded a key, register it in the signers
if let Some(session) = new_key {
let (substrate_keys, network_keys) =
::key_gen::KeyGen::<KeyGenParams>::key_shares(txn, session)
.expect("generated key pair yet couldn't get key shares");
signers.register_keys(txn, session, substrate_keys, network_keys);
}
}
// These are cheap calls which are fine to be here in this loop
messages::CoordinatorMessage::Sign(msg) => {
let txn = txn.as_mut().unwrap();
signers.queue_message(txn, &msg)
}
messages::CoordinatorMessage::Coordinator(
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock {
session,
block_number,
block,
},
) => {
let txn = txn.take().unwrap();
signers.cosign_block(txn, session, block_number, block)
}
messages::CoordinatorMessage::Coordinator(
messages::coordinator::CoordinatorMessage::SignSlashReport { session, report },
) => {
let txn = txn.take().unwrap();
signers.sign_slash_report(txn, session, &report)
}
messages::CoordinatorMessage::Substrate(msg) => match msg {
messages::substrate::CoordinatorMessage::SetKeys { serai_time, session, key_pair } => {
let txn = txn.as_mut().unwrap();
let key = EncodableG(
KeyGenParams::decode_key(key_pair.1.as_ref()).expect("invalid key set on serai"),
);
// Queue the key to be activated upon the next Batch
db::KeyToActivate::<
<<KeyGenParams as ::key_gen::KeyGenParams>::ExternalNetworkCurve as Ciphersuite>::G,
>::send(txn, &key);
// Set the external key, as needed by the signers
db::ExternalKeyForSessionForSigners::<
<<KeyGenParams as ::key_gen::KeyGenParams>::ExternalNetworkCurve as Ciphersuite>::G,
>::set(txn, session, &key);
// This is presumed extremely expensive, potentially blocking for several minutes, yet
// only happens for the very first set of keys
if session == Session(0) {
assert!(scanner.is_none());
let start_block = first_block_after_time(&feed, serai_time).await;
scanner =
Some(Scanner::new::<Scheduler<D>>(db_clone, feed.clone(), start_block, key.0).await);
}
}
messages::substrate::CoordinatorMessage::SlashesReported { session } => {
let txn = txn.as_mut().unwrap();
// Since this session had its slashes reported, it has finished all its signature
// protocols and has been fully retired. We retire it from the signers accordingly
let key = db::ExternalKeyForSessionForSigners::<
<<KeyGenParams as ::key_gen::KeyGenParams>::ExternalNetworkCurve as Ciphersuite>::G,
>::take(txn, session)
.unwrap()
.0;
// This is a cheap call
signers.retire_session(txn, session, &key)
}
messages::substrate::CoordinatorMessage::BlockWithBatchAcknowledgement {
block: _,
batch_id,
in_instruction_succeededs,
burns,
} => {
let mut txn = txn.take().unwrap();
let scanner = scanner.as_mut().unwrap();
let key_to_activate = db::KeyToActivate::<
<<KeyGenParams as ::key_gen::KeyGenParams>::ExternalNetworkCurve as Ciphersuite>::G,
>::try_recv(&mut txn)
.map(|key| key.0);
// This is a cheap call as it internally just queues this to be done later
scanner.acknowledge_batch(
txn,
batch_id,
in_instruction_succeededs,
burns,
key_to_activate,
)
}
messages::substrate::CoordinatorMessage::BlockWithoutBatchAcknowledgement {
block: _,
burns,
} => {
let txn = txn.take().unwrap();
let scanner = scanner.as_mut().unwrap();
// This is a cheap call as it internally just queues this to be done later
scanner.queue_burns(txn, burns)
}
},
};
// If the txn wasn't already consumed and committed, commit it
if let Some(txn) = txn {
txn.commit();
}
}
}
#[tokio::main]
async fn main() {}
async fn main() {
let db = bin::init();
let feed = Rpc {
db: db.clone(),
rpc: loop {
match BRpc::new(bin::url()).await {
Ok(rpc) => break rpc,
Err(e) => {
log::error!("couldn't connect to the Bitcoin node: {e:?}");
tokio::time::sleep(core::time::Duration::from_secs(5)).await;
}
}
},
};
let (index_task, index_handle) = Task::new();
tokio::spawn(TxIndexTask(feed.clone()).continually_run(index_task, vec![]));
core::mem::forget(index_handle);
bin::coordinator_loop::<_, KeyGenParams, Scheduler<_>, Rpc<bin::Db>>(db, feed.clone(), feed)
.await;
}
/*
use bitcoin_serai::{
@@ -278,9 +97,6 @@ use serai_client::{
*/
/*
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
pub(crate) struct Fee(u64);
#[async_trait]
impl TransactionTrait<Bitcoin> for Transaction {
#[cfg(test)]

View File

@@ -35,7 +35,7 @@ pub(crate) fn script_pubkey_for_on_chain_output(
)
}
pub(crate) struct TxIndexTask<D: Db>(Rpc<D>);
pub(crate) struct TxIndexTask<D: Db>(pub(crate) Rpc<D>);
#[async_trait::async_trait]
impl<D: Db> ContinuallyRan for TxIndexTask<D> {