mirror of
https://github.com/serai-dex/serai.git
synced 2025-12-08 20:29:23 +00:00
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:
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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)]
|
||||
|
||||
@@ -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> {
|
||||
|
||||
Reference in New Issue
Block a user