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

@ -49,6 +49,7 @@ jobs:
-p serai-processor-utxo-scheduler \
-p serai-processor-transaction-chaining-scheduler \
-p serai-processor-signers \
-p serai-processor-bin \
-p serai-bitcoin-processor \
-p serai-ethereum-processor \
-p serai-monero-processor \

37
Cargo.lock generated
View file

@ -8131,6 +8131,7 @@ dependencies = [
"dkg",
"env_logger",
"flexible-transcript",
"hex",
"log",
"modular-frost",
"parity-scale-codec",
@ -8140,6 +8141,7 @@ dependencies = [
"serai-db",
"serai-env",
"serai-message-queue",
"serai-processor-bin",
"serai-processor-key-gen",
"serai-processor-messages",
"serai-processor-primitives",
@ -8150,6 +8152,7 @@ dependencies = [
"serai-processor-utxo-scheduler-primitives",
"tokio",
"zalloc",
"zeroize",
]
[[package]]
@ -8635,6 +8638,40 @@ dependencies = [
"zeroize",
]
[[package]]
name = "serai-processor-bin"
version = "0.1.0"
dependencies = [
"async-trait",
"bitcoin-serai",
"borsh",
"ciphersuite",
"dkg",
"env_logger",
"flexible-transcript",
"hex",
"log",
"modular-frost",
"parity-scale-codec",
"rand_core",
"secp256k1",
"serai-client",
"serai-db",
"serai-env",
"serai-message-queue",
"serai-processor-key-gen",
"serai-processor-messages",
"serai-processor-primitives",
"serai-processor-scanner",
"serai-processor-scheduler-primitives",
"serai-processor-signers",
"serai-processor-transaction-chaining-scheduler",
"serai-processor-utxo-scheduler-primitives",
"tokio",
"zalloc",
"zeroize",
]
[[package]]
name = "serai-processor-frost-attempt-manager"
version = "0.1.0"

View file

@ -83,6 +83,7 @@ members = [
"processor/scheduler/utxo/transaction-chaining",
"processor/signers",
"processor/bin",
"processor/bitcoin",
"processor/ethereum",
"processor/monero",

View file

@ -72,6 +72,9 @@ pub(crate) fn queue_message(
// Assert one, and only one of these, is the coordinator
assert!(matches!(meta.from, Service::Coordinator) ^ matches!(meta.to, Service::Coordinator));
// Lock the queue
let queue_lock = QUEUES.read().unwrap()[&(meta.from, meta.to)].write().unwrap();
// Verify (from, to, intent) hasn't been prior seen
fn key(domain: &'static [u8], key: impl AsRef<[u8]>) -> Vec<u8> {
[&[u8::try_from(domain.len()).unwrap()], domain, key.as_ref()].concat()
@ -93,7 +96,7 @@ pub(crate) fn queue_message(
DbTxn::put(&mut txn, intent_key, []);
// Queue it
let id = QUEUES.read().unwrap()[&(meta.from, meta.to)].write().unwrap().queue_message(
let id = queue_lock.queue_message(
&mut txn,
QueuedMessage {
from: meta.from,

60
processor/bin/Cargo.toml Normal file
View file

@ -0,0 +1,60 @@
[package]
name = "serai-processor-bin"
version = "0.1.0"
description = "Framework for Serai processor binaries"
license = "AGPL-3.0-only"
repository = "https://github.com/serai-dex/serai/tree/develop/processor/bin"
authors = ["Luke Parker <lukeparker5132@gmail.com>"]
keywords = []
edition = "2021"
publish = false
[package.metadata.docs.rs]
all-features = true
rustdoc-args = ["--cfg", "docsrs"]
[lints]
workspace = true
[dependencies]
async-trait = { version = "0.1", default-features = false }
zeroize = { version = "1", default-features = false, features = ["std"] }
rand_core = { version = "0.6", default-features = false }
hex = { version = "0.4", default-features = false, features = ["std"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
transcript = { package = "flexible-transcript", path = "../../crypto/transcript", default-features = false, features = ["std", "recommended"] }
ciphersuite = { path = "../../crypto/ciphersuite", default-features = false, features = ["std", "secp256k1"] }
dkg = { path = "../../crypto/dkg", default-features = false, features = ["std", "evrf-secp256k1"] }
frost = { package = "modular-frost", path = "../../crypto/frost", default-features = false }
secp256k1 = { version = "0.29", default-features = false, features = ["std", "global-context", "rand-std"] }
bitcoin-serai = { path = "../../networks/bitcoin", default-features = false, features = ["std"] }
log = { version = "0.4", default-features = false, features = ["std"] }
env_logger = { version = "0.10", default-features = false, features = ["humantime"] }
tokio = { version = "1", default-features = false, features = ["rt-multi-thread", "sync", "time", "macros"] }
zalloc = { path = "../../common/zalloc" }
serai-db = { path = "../../common/db" }
serai-env = { path = "../../common/env" }
serai-client = { path = "../../substrate/client", default-features = false, features = ["bitcoin"] }
messages = { package = "serai-processor-messages", path = "../messages" }
key-gen = { package = "serai-processor-key-gen", path = "../key-gen" }
primitives = { package = "serai-processor-primitives", path = "../primitives" }
scheduler = { package = "serai-processor-scheduler-primitives", path = "../scheduler/primitives" }
scanner = { package = "serai-processor-scanner", path = "../scanner" }
utxo-scheduler = { package = "serai-processor-utxo-scheduler-primitives", path = "../scheduler/utxo/primitives" }
transaction-chaining-scheduler = { package = "serai-processor-transaction-chaining-scheduler", path = "../scheduler/utxo/transaction-chaining" }
signers = { package = "serai-processor-signers", path = "../signers" }
message-queue = { package = "serai-message-queue", path = "../../message-queue" }
[features]
parity-db = ["serai-db/parity-db"]
rocksdb = ["serai-db/rocksdb"]

15
processor/bin/LICENSE Normal file
View file

@ -0,0 +1,15 @@
AGPL-3.0-only license
Copyright (c) 2022-2024 Luke Parker
This program is free software: you can redistribute it and/or modify
it under the terms of the GNU Affero General Public License Version 3 as
published by the Free Software Foundation.
This program is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
GNU Affero General Public License for more details.
You should have received a copy of the GNU Affero General Public License
along with this program. If not, see <http://www.gnu.org/licenses/>.

3
processor/bin/README.md Normal file
View file

@ -0,0 +1,3 @@
# Serai Processor Bin
The framework for Serai processor binaries, common to the Serai processors.

View file

@ -0,0 +1,196 @@
use std::sync::Arc;
use tokio::sync::mpsc;
use scale::Encode;
use serai_client::{
primitives::{NetworkId, Signature},
validator_sets::primitives::Session,
in_instructions::primitives::{Batch, SignedBatch},
};
use serai_env as env;
use serai_db::{Get, DbTxn, Db, create_db, db_channel};
use message_queue::{Service, Metadata, client::MessageQueue};
create_db! {
ProcessorBinCoordinator {
SavedMessages: () -> u64,
}
}
db_channel! {
ProcessorBinCoordinator {
CoordinatorMessages: () -> Vec<u8>
}
}
async fn send(service: Service, queue: &MessageQueue, msg: messages::ProcessorMessage) {
let metadata = Metadata { from: service, to: Service::Coordinator, intent: msg.intent() };
let msg = borsh::to_vec(&msg).unwrap();
queue.queue(metadata, msg).await;
}
pub(crate) struct Coordinator {
new_message: mpsc::UnboundedReceiver<()>,
service: Service,
message_queue: Arc<MessageQueue>,
}
pub(crate) struct CoordinatorSend(Service, Arc<MessageQueue>);
impl Coordinator {
pub(crate) fn new(mut db: crate::Db) -> Self {
let (new_message_send, new_message_recv) = mpsc::unbounded_channel();
let network_id = match env::var("NETWORK").expect("network wasn't specified").as_str() {
"bitcoin" => NetworkId::Bitcoin,
"ethereum" => NetworkId::Ethereum,
"monero" => NetworkId::Monero,
_ => panic!("unrecognized network"),
};
let service = Service::Processor(network_id);
let message_queue = Arc::new(MessageQueue::from_env(service));
// Spawn a task to move messages from the message-queue to our database so we can achieve
// atomicity. This is the only place we read/ack messages from
tokio::spawn({
let message_queue = message_queue.clone();
async move {
loop {
let msg = message_queue.next(Service::Coordinator).await;
let prior_msg = msg.id.checked_sub(1);
let saved_messages = SavedMessages::get(&db);
/*
This should either be:
A) The message after the message we just saved (as normal)
B) The message we just saved (if we rebooted and failed to ack it)
*/
assert!((saved_messages == prior_msg) || (saved_messages == Some(msg.id)));
if saved_messages < Some(msg.id) {
let mut txn = db.txn();
CoordinatorMessages::send(&mut txn, &msg.msg);
SavedMessages::set(&mut txn, &msg.id);
txn.commit();
}
// Acknowledge this message
message_queue.ack(Service::Coordinator, msg.id).await;
// Fire that there's a new message
new_message_send.send(()).expect("failed to tell the Coordinator there's a new message");
}
}
});
Coordinator { new_message: new_message_recv, service, message_queue }
}
pub(crate) fn coordinator_send(&self) -> CoordinatorSend {
CoordinatorSend(self.service, self.message_queue.clone())
}
/// 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.
pub(crate) async fn next_message(
&mut self,
txn: &mut impl DbTxn,
) -> messages::CoordinatorMessage {
loop {
match CoordinatorMessages::try_recv(txn) {
Some(msg) => {
return borsh::from_slice(&msg)
.expect("message wasn't a borsh-encoded CoordinatorMessage")
}
None => {
let _ =
tokio::time::timeout(core::time::Duration::from_secs(60), self.new_message.recv())
.await;
}
}
}
}
#[allow(clippy::unused_async)]
pub(crate) async fn send_message(&mut self, msg: messages::ProcessorMessage) {
send(self.service, &self.message_queue, msg).await
}
}
#[async_trait::async_trait]
impl signers::Coordinator for CoordinatorSend {
type EphemeralError = ();
async fn send(
&mut self,
msg: messages::sign::ProcessorMessage,
) -> Result<(), Self::EphemeralError> {
// TODO: Use a fallible send for these methods
send(self.0, &self.1, messages::ProcessorMessage::Sign(msg)).await;
Ok(())
}
async fn publish_cosign(
&mut self,
block_number: u64,
block: [u8; 32],
signature: Signature,
) -> Result<(), Self::EphemeralError> {
send(
self.0,
&self.1,
messages::ProcessorMessage::Coordinator(
messages::coordinator::ProcessorMessage::CosignedBlock {
block_number,
block,
signature: signature.encode(),
},
),
)
.await;
Ok(())
}
async fn publish_batch(&mut self, batch: Batch) -> Result<(), Self::EphemeralError> {
send(
self.0,
&self.1,
messages::ProcessorMessage::Substrate(messages::substrate::ProcessorMessage::Batch { batch }),
)
.await;
Ok(())
}
async fn publish_signed_batch(&mut self, batch: SignedBatch) -> Result<(), Self::EphemeralError> {
send(
self.0,
&self.1,
messages::ProcessorMessage::Coordinator(
messages::coordinator::ProcessorMessage::SignedBatch { batch },
),
)
.await;
Ok(())
}
async fn publish_slash_report_signature(
&mut self,
session: Session,
signature: Signature,
) -> Result<(), Self::EphemeralError> {
send(
self.0,
&self.1,
messages::ProcessorMessage::Coordinator(
messages::coordinator::ProcessorMessage::SignedSlashReport {
session,
signature: signature.encode(),
},
),
)
.await;
Ok(())
}
}

293
processor/bin/src/lib.rs Normal file
View file

@ -0,0 +1,293 @@
use core::cmp::Ordering;
use zeroize::{Zeroize, Zeroizing};
use ciphersuite::{
group::{ff::PrimeField, GroupEncoding},
Ciphersuite, Ristretto,
};
use dkg::evrf::EvrfCurve;
use serai_client::validator_sets::primitives::Session;
use serai_env as env;
use serai_db::{Get, DbTxn, Db as DbTrait, create_db, db_channel};
use primitives::EncodableG;
use ::key_gen::{KeyGenParams, KeyGen};
use scheduler::SignableTransaction;
use scanner::{ScannerFeed, Scanner, KeyFor, Scheduler};
use signers::{TransactionPublisher, Signers};
mod coordinator;
use coordinator::Coordinator;
create_db! {
ProcessorBin {
ExternalKeyForSessionForSigners: <K: GroupEncoding>(session: Session) -> EncodableG<K>,
}
}
db_channel! {
ProcessorBin {
KeyToActivate: <K: GroupEncoding>() -> EncodableG<K>
}
}
/// The type used for the database.
#[cfg(all(feature = "parity-db", not(feature = "rocksdb")))]
pub type Db = serai_db::ParityDb;
/// The type used for the database.
#[cfg(feature = "rocksdb")]
pub type Db = serai_db::RocksDB;
/// Initialize the processor.
///
/// Yields the database.
#[allow(unused_variables, unreachable_code)]
pub fn init() -> Db {
// 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();
#[cfg(all(feature = "parity-db", not(feature = "rocksdb")))]
let db =
serai_db::new_parity_db(&serai_env::var("DB_PATH").expect("path to DB wasn't specified"));
#[cfg(feature = "rocksdb")]
let db = serai_db::new_rocksdb(&serai_env::var("DB_PATH").expect("path to DB wasn't specified"));
db
}
/// THe URL for the external network's node.
pub fn url() -> String {
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
}
fn key_gen<K: KeyGenParams>() -> KeyGen<K> {
fn read_key_from_env<C: Ciphersuite>(label: &'static str) -> Zeroizing<C::F> {
let key_hex =
Zeroizing::new(env::var(label).unwrap_or_else(|| panic!("{label} wasn't provided")));
let bytes = Zeroizing::new(
hex::decode(key_hex).unwrap_or_else(|_| panic!("{label} wasn't a valid hex string")),
);
let mut repr = <C::F as PrimeField>::Repr::default();
if repr.as_ref().len() != bytes.len() {
panic!("{label} wasn't the correct length");
}
repr.as_mut().copy_from_slice(bytes.as_slice());
let res = Zeroizing::new(
Option::from(<C::F as PrimeField>::from_repr(repr))
.unwrap_or_else(|| panic!("{label} wasn't a valid scalar")),
);
repr.as_mut().zeroize();
res
}
KeyGen::new(
read_key_from_env::<<Ristretto as EvrfCurve>::EmbeddedCurve>("SUBSTRATE_EVRF_KEY"),
read_key_from_env::<<K::ExternalNetworkCiphersuite as EvrfCurve>::EmbeddedCurve>(
"NETWORK_EVRF_KEY",
),
)
}
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;
}
}
/// The main loop of a Processor, interacting with the Coordinator.
pub async fn coordinator_loop<
S: ScannerFeed,
K: KeyGenParams<ExternalNetworkCiphersuite: Ciphersuite<G = KeyFor<S>>>,
Sch: Scheduler<
S,
SignableTransaction: SignableTransaction<Ciphersuite = K::ExternalNetworkCiphersuite>,
>,
P: TransactionPublisher<<Sch::SignableTransaction as SignableTransaction>::Transaction>,
>(
mut db: Db,
feed: S,
publisher: P,
) {
let mut coordinator = Coordinator::new(db.clone());
let mut key_gen = key_gen::<K>();
let mut scanner = Scanner::new::<Sch>(db.clone(), feed.clone()).await;
let mut signers =
Signers::<Db, S, Sch, P>::new(db.clone(), coordinator.coordinator_send(), publisher);
loop {
let db_clone = db.clone();
let mut txn = db.txn();
let msg = coordinator.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)
}
coordinator.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) = KeyGen::<K>::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(K::decode_key(key_pair.1.as_ref()).expect("invalid key set on serai"));
// Queue the key to be activated upon the next Batch
KeyToActivate::<KeyFor<S>>::send(txn, &key);
// Set the external key, as needed by the signers
ExternalKeyForSessionForSigners::<KeyFor<S>>::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::initialize::<Sch>(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 = ExternalKeyForSessionForSigners::<KeyFor<S>>::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 = KeyToActivate::<KeyFor<S>>::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();
}
}
}

View file

@ -18,8 +18,10 @@ workspace = true
[dependencies]
async-trait = { version = "0.1", default-features = false }
zeroize = { version = "1", default-features = false, features = ["std"] }
rand_core = { version = "0.6", default-features = false }
hex = { version = "0.4", default-features = false, features = ["std"] }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std"] }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
@ -51,8 +53,10 @@ utxo-scheduler = { package = "serai-processor-utxo-scheduler-primitives", path =
transaction-chaining-scheduler = { package = "serai-processor-transaction-chaining-scheduler", path = "../scheduler/utxo/transaction-chaining" }
signers = { package = "serai-processor-signers", path = "../signers" }
bin = { package = "serai-processor-bin", path = "../bin" }
message-queue = { package = "serai-message-queue", path = "../../message-queue" }
[features]
parity-db = ["serai-db/parity-db"]
rocksdb = ["serai-db/rocksdb"]
parity-db = ["bin/parity-db"]
rocksdb = ["bin/rocksdb"]

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> {

View file

@ -19,7 +19,7 @@ pub(crate) struct Params<P: KeyGenParams> {
pub(crate) substrate_evrf_public_keys:
Vec<<<Ristretto as EvrfCurve>::EmbeddedCurve as Ciphersuite>::G>,
pub(crate) network_evrf_public_keys:
Vec<<<P::ExternalNetworkCurve as EvrfCurve>::EmbeddedCurve as Ciphersuite>::G>,
Vec<<<P::ExternalNetworkCiphersuite as EvrfCurve>::EmbeddedCurve as Ciphersuite>::G>,
}
#[derive(BorshSerialize, BorshDeserialize)]
@ -93,9 +93,9 @@ impl<P: KeyGenParams> KeyGenDb<P> {
.network_evrf_public_keys
.into_iter()
.map(|key| {
<<P::ExternalNetworkCurve as EvrfCurve>::EmbeddedCurve as Ciphersuite>::read_G::<&[u8]>(
&mut key.as_ref(),
)
<<P::ExternalNetworkCiphersuite as EvrfCurve>::EmbeddedCurve as Ciphersuite>::read_G::<
&[u8],
>(&mut key.as_ref())
.unwrap()
})
.collect(),
@ -118,7 +118,7 @@ impl<P: KeyGenParams> KeyGenDb<P> {
txn: &mut impl DbTxn,
session: Session,
substrate_keys: &[ThresholdKeys<Ristretto>],
network_keys: &[ThresholdKeys<P::ExternalNetworkCurve>],
network_keys: &[ThresholdKeys<P::ExternalNetworkCiphersuite>],
) {
assert_eq!(substrate_keys.len(), network_keys.len());
@ -134,7 +134,8 @@ impl<P: KeyGenParams> KeyGenDb<P> {
pub(crate) fn key_shares(
getter: &impl Get,
session: Session,
) -> Option<(Vec<ThresholdKeys<Ristretto>>, Vec<ThresholdKeys<P::ExternalNetworkCurve>>)> {
) -> Option<(Vec<ThresholdKeys<Ristretto>>, Vec<ThresholdKeys<P::ExternalNetworkCiphersuite>>)>
{
let keys = _db::KeyShares::get(getter, &session)?;
let mut keys: &[u8] = keys.as_ref();

View file

@ -34,27 +34,29 @@ pub trait KeyGenParams {
const ID: &'static str;
/// The curve used for the external network.
type ExternalNetworkCurve: EvrfCurve<
type ExternalNetworkCiphersuite: EvrfCurve<
EmbeddedCurve: Ciphersuite<
G: ec_divisors::DivisorCurve<FieldElement = <Self::ExternalNetworkCurve as Ciphersuite>::F>,
G: ec_divisors::DivisorCurve<
FieldElement = <Self::ExternalNetworkCiphersuite as Ciphersuite>::F,
>,
>,
>;
/// Tweaks keys as necessary/beneficial.
fn tweak_keys(keys: &mut ThresholdKeys<Self::ExternalNetworkCurve>);
fn tweak_keys(keys: &mut ThresholdKeys<Self::ExternalNetworkCiphersuite>);
/// Encode keys as optimal.
///
/// A default implementation is provided which calls the traditional `to_bytes`.
fn encode_key(key: <Self::ExternalNetworkCurve as Ciphersuite>::G) -> Vec<u8> {
fn encode_key(key: <Self::ExternalNetworkCiphersuite as Ciphersuite>::G) -> Vec<u8> {
key.to_bytes().as_ref().to_vec()
}
/// Decode keys from their optimal encoding.
///
/// A default implementation is provided which calls the traditional `from_bytes`.
fn decode_key(mut key: &[u8]) -> Option<<Self::ExternalNetworkCurve as Ciphersuite>::G> {
let res = <Self::ExternalNetworkCurve as Ciphersuite>::read_G(&mut key).ok()?;
fn decode_key(mut key: &[u8]) -> Option<<Self::ExternalNetworkCiphersuite as Ciphersuite>::G> {
let res = <Self::ExternalNetworkCiphersuite as Ciphersuite>::read_G(&mut key).ok()?;
if !key.is_empty() {
None?;
}
@ -143,7 +145,7 @@ pub struct KeyGen<P: KeyGenParams> {
substrate_evrf_private_key:
Zeroizing<<<Ristretto as EvrfCurve>::EmbeddedCurve as Ciphersuite>::F>,
network_evrf_private_key:
Zeroizing<<<P::ExternalNetworkCurve as EvrfCurve>::EmbeddedCurve as Ciphersuite>::F>,
Zeroizing<<<P::ExternalNetworkCiphersuite as EvrfCurve>::EmbeddedCurve as Ciphersuite>::F>,
}
impl<P: KeyGenParams> KeyGen<P> {
@ -154,7 +156,7 @@ impl<P: KeyGenParams> KeyGen<P> {
<<Ristretto as EvrfCurve>::EmbeddedCurve as Ciphersuite>::F,
>,
network_evrf_private_key: Zeroizing<
<<P::ExternalNetworkCurve as EvrfCurve>::EmbeddedCurve as Ciphersuite>::F,
<<P::ExternalNetworkCiphersuite as EvrfCurve>::EmbeddedCurve as Ciphersuite>::F,
>,
) -> KeyGen<P> {
KeyGen { substrate_evrf_private_key, network_evrf_private_key }
@ -165,7 +167,8 @@ impl<P: KeyGenParams> KeyGen<P> {
pub fn key_shares(
getter: &impl Get,
session: Session,
) -> Option<(Vec<ThresholdKeys<Ristretto>>, Vec<ThresholdKeys<P::ExternalNetworkCurve>>)> {
) -> Option<(Vec<ThresholdKeys<Ristretto>>, Vec<ThresholdKeys<P::ExternalNetworkCiphersuite>>)>
{
// This is safe, despite not having a txn, since it's a static value
// It doesn't change over time/in relation to other operations
// It is solely set or unset
@ -198,7 +201,7 @@ impl<P: KeyGenParams> KeyGen<P> {
let network_evrf_public_keys =
evrf_public_keys.into_iter().map(|(_, key)| key).collect::<Vec<_>>();
let (network_evrf_public_keys, additional_faulty) =
coerce_keys::<P::ExternalNetworkCurve>(&network_evrf_public_keys);
coerce_keys::<P::ExternalNetworkCiphersuite>(&network_evrf_public_keys);
faulty.extend(additional_faulty);
// Participate for both Substrate and the network
@ -228,7 +231,7 @@ impl<P: KeyGenParams> KeyGen<P> {
&self.substrate_evrf_private_key,
&mut participation,
);
participate::<P::ExternalNetworkCurve>(
participate::<P::ExternalNetworkCiphersuite>(
context::<P>(session, NETWORK_KEY_CONTEXT),
threshold,
&network_evrf_public_keys,
@ -283,7 +286,7 @@ impl<P: KeyGenParams> KeyGen<P> {
};
let len_at_network_participation_start_pos = participation.len();
let Ok(network_participation) =
Participation::<P::ExternalNetworkCurve>::read(&mut participation, n)
Participation::<P::ExternalNetworkCiphersuite>::read(&mut participation, n)
else {
return blame;
};
@ -317,7 +320,7 @@ impl<P: KeyGenParams> KeyGen<P> {
}
}
match EvrfDkg::<P::ExternalNetworkCurve>::verify(
match EvrfDkg::<P::ExternalNetworkCiphersuite>::verify(
&mut OsRng,
generators(),
context::<P>(session, NETWORK_KEY_CONTEXT),
@ -490,7 +493,7 @@ impl<P: KeyGenParams> KeyGen<P> {
Err(blames) => return blames,
};
let network_dkg = match verify_dkg::<P, P::ExternalNetworkCurve>(
let network_dkg = match verify_dkg::<P, P::ExternalNetworkCiphersuite>(
txn,
session,
false,

View file

@ -70,6 +70,8 @@ impl<S: ScannerFeed> OutputWithInInstruction<S> {
create_db!(
ScannerGlobal {
StartBlock: () -> u64,
QueuedKey: <K: Encode>(key: K) -> (),
ActiveKeys: <K: Borshy>() -> Vec<SeraiKeyDbEntry<K>>,
@ -106,8 +108,11 @@ create_db!(
pub(crate) struct ScannerGlobalDb<S: ScannerFeed>(PhantomData<S>);
impl<S: ScannerFeed> ScannerGlobalDb<S> {
pub(crate) fn has_any_key_been_queued(getter: &impl Get) -> bool {
ActiveKeys::<EncodableG<KeyFor<S>>>::get(getter).is_some()
pub(crate) fn start_block(getter: &impl Get) -> Option<u64> {
StartBlock::get(getter)
}
pub(crate) fn set_start_block(txn: &mut impl DbTxn, block: u64) {
StartBlock::set(txn, &block)
}
/// Queue a key.

View file

@ -344,17 +344,10 @@ impl<S: ScannerFeed> Scanner<S> {
/// Create a new scanner.
///
/// This will begin its execution, spawning several asynchronous tasks.
pub async fn new<Sch: Scheduler<S>>(
mut db: impl Db,
feed: S,
start_block: u64,
start_key: KeyFor<S>,
) -> Self {
if !ScannerGlobalDb::<S>::has_any_key_been_queued(&db) {
let mut txn = db.txn();
ScannerGlobalDb::<S>::queue_key(&mut txn, start_block, start_key);
txn.commit();
}
///
/// This will return None if the Scanner was never initialized.
pub async fn new<Sch: Scheduler<S>>(db: impl Db, feed: S) -> Option<Self> {
let start_block = ScannerGlobalDb::<S>::start_block(&db)?;
let index_task = index::IndexTask::new(db.clone(), feed.clone(), start_block).await;
let scan_task = scan::ScanTask::new(db.clone(), feed.clone(), start_block);
@ -381,7 +374,28 @@ impl<S: ScannerFeed> Scanner<S> {
// window its allowed to scan
tokio::spawn(eventuality_task.continually_run(eventuality_task_def, vec![scan_handle]));
Self { substrate_handle, _S: PhantomData }
Some(Self { substrate_handle, _S: PhantomData })
}
/// Initialize the scanner.
///
/// This will begin its execution, spawning several asynchronous tasks.
///
/// This passes through to `Scanner::new` if prior called.
pub async fn initialize<Sch: Scheduler<S>>(
mut db: impl Db,
feed: S,
start_block: u64,
start_key: KeyFor<S>,
) -> Self {
if ScannerGlobalDb::<S>::start_block(&db).is_none() {
let mut txn = db.txn();
ScannerGlobalDb::<S>::set_start_block(&mut txn, start_block);
ScannerGlobalDb::<S>::queue_key(&mut txn, start_block, start_key);
txn.commit();
}
Self::new::<Sch>(db, feed).await.unwrap()
}
/// Acknowledge a Batch having been published on Serai.

View file

@ -114,6 +114,7 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
self
.coordinator
.publish_slash_report_signature(
session,
<_>::decode(&mut slash_report_signature.as_slice()).unwrap(),
)
.await

View file

@ -71,6 +71,7 @@ pub trait Coordinator: 'static + Send + Sync {
/// Publish a slash report's signature.
async fn publish_slash_report_signature(
&mut self,
session: Session,
signature: Signature,
) -> Result<(), Self::EphemeralError>;
}

View file

@ -1,43 +0,0 @@
use messages::{ProcessorMessage, CoordinatorMessage};
use message_queue::{Service, Metadata, client::MessageQueue};
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct Message {
pub id: u64,
pub msg: CoordinatorMessage,
}
#[async_trait::async_trait]
pub trait Coordinator {
async fn send(&mut self, msg: impl Send + Into<ProcessorMessage>);
async fn recv(&mut self) -> Message;
async fn ack(&mut self, msg: Message);
}
#[async_trait::async_trait]
impl Coordinator for MessageQueue {
async fn send(&mut self, msg: impl Send + Into<ProcessorMessage>) {
let msg: ProcessorMessage = msg.into();
let metadata = Metadata { from: self.service, to: Service::Coordinator, intent: msg.intent() };
let msg = borsh::to_vec(&msg).unwrap();
self.queue(metadata, msg).await;
}
async fn recv(&mut self) -> Message {
let msg = self.next(Service::Coordinator).await;
let id = msg.id;
// Deserialize it into a CoordinatorMessage
let msg: CoordinatorMessage =
borsh::from_slice(&msg.msg).expect("message wasn't a borsh-encoded CoordinatorMessage");
return Message { id, msg };
}
async fn ack(&mut self, msg: Message) {
MessageQueue::ack(self, Service::Coordinator, msg.id).await
}
}

View file

@ -1,43 +0,0 @@
use std::io::Read;
use scale::{Encode, Decode};
use serai_client::validator_sets::primitives::{Session, KeyPair};
pub use serai_db::*;
use crate::networks::{Block, Network};
create_db!(
MainDb {
HandledMessageDb: (id: u64) -> (),
PendingActivationsDb: () -> Vec<u8>
}
);
impl PendingActivationsDb {
pub fn pending_activation<N: Network>(
getter: &impl Get,
) -> Option<(<N::Block as Block<N>>::Id, Session, KeyPair)> {
if let Some(bytes) = Self::get(getter) {
if !bytes.is_empty() {
let mut slice = bytes.as_slice();
let (session, key_pair) = <(Session, KeyPair)>::decode(&mut slice).unwrap();
let mut block_before_queue_block = <N::Block as Block<N>>::Id::default();
slice.read_exact(block_before_queue_block.as_mut()).unwrap();
assert!(slice.is_empty());
return Some((block_before_queue_block, session, key_pair));
}
}
None
}
pub fn set_pending_activation<N: Network>(
txn: &mut impl DbTxn,
block_before_queue_block: &<N::Block as Block<N>>::Id,
session: Session,
key_pair: KeyPair,
) {
let mut buf = (session, key_pair).encode();
buf.extend(block_before_queue_block.as_ref());
Self::set(txn, &buf);
}
}

View file

@ -60,263 +60,9 @@ async fn handle_coordinator_msg<D: Db, N: Network, Co: Coordinator>(
}
}
async fn boot<N: Network, D: Db, Co: Coordinator>(
raw_db: &mut D,
network: &N,
coordinator: &mut Co,
) -> (D, TributaryMutable<N, D>, SubstrateMutable<N, D>) {
fn read_key_from_env<C: Ciphersuite>(label: &'static str) -> Zeroizing<C::F> {
let key_hex =
Zeroizing::new(env::var(label).unwrap_or_else(|| panic!("{label} wasn't provided")));
let bytes = Zeroizing::new(
hex::decode(key_hex).unwrap_or_else(|_| panic!("{label} wasn't a valid hex string")),
);
let mut repr = <C::F as PrimeField>::Repr::default();
if repr.as_ref().len() != bytes.len() {
panic!("{label} wasn't the correct length");
}
repr.as_mut().copy_from_slice(bytes.as_slice());
let res = Zeroizing::new(
Option::from(<C::F as PrimeField>::from_repr(repr))
.unwrap_or_else(|| panic!("{label} wasn't a valid scalar")),
);
repr.as_mut().zeroize();
res
}
let key_gen = KeyGen::<N, _>::new(
raw_db.clone(),
read_key_from_env::<<Ristretto as EvrfCurve>::EmbeddedCurve>("SUBSTRATE_EVRF_KEY"),
read_key_from_env::<<N::Curve as EvrfCurve>::EmbeddedCurve>("NETWORK_EVRF_KEY"),
);
let (multisig_manager, current_keys, actively_signing) =
MultisigManager::new(raw_db, network).await;
let mut batch_signer = None;
let mut signers = HashMap::new();
for (i, key) in current_keys.iter().enumerate() {
let Some((session, (substrate_keys, network_keys))) = key_gen.keys(key) else { continue };
let network_key = network_keys[0].group_key();
// If this is the oldest key, load the BatchSigner for it as the active BatchSigner
// 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 {
batch_signer = Some(BatchSigner::new(N::NETWORK, session, substrate_keys));
}
// The Scanner re-fires events as needed for batch_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(), session, network_keys);
// Sign any TXs being actively signed
for (plan, tx, eventuality) in &actively_signing {
if plan.key == network_key {
let mut txn = raw_db.txn();
if let Some(msg) =
signer.sign_transaction(&mut txn, plan.id(), tx.clone(), eventuality).await
{
coordinator.send(msg).await;
}
// This should only have re-writes of existing data
drop(txn);
}
}
signers.insert(session, signer);
}
// Spawn a task to rebroadcast signed TXs yet to be mined into a finalized block
// This hedges against being dropped due to full mempools, temporarily too low of a fee...
tokio::spawn(Signer::<N, D>::rebroadcast_task(raw_db.clone(), network.clone()));
(
raw_db.clone(),
TributaryMutable { key_gen, batch_signer, cosigner: None, slash_report_signer: None, 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, &mut coordinator).await;
// We can't load this from the DB as we can't guarantee atomic increments with the ack function
// TODO: Load with a slight tolerance
let mut last_coordinator_msg = None;
loop {
let mut txn = raw_db.txn();
log::trace!("new db txn in run");
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 potentially be parallelized by type (KeyGen, Sign, Substrate) without issue
msg = coordinator.recv() => {
if let Some(last_coordinator_msg) = last_coordinator_msg {
assert_eq!(msg.id, last_coordinator_msg + 1);
}
last_coordinator_msg = Some(msg.id);
// Only handle this if we haven't already
if HandledMessageDb::get(&main_db, msg.id).is_none() {
HandledMessageDb::set(&mut 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);
},
scanner_event = substrate_mutable.next_scanner_event() => {
let msg = substrate_mutable.scanner_event_to_multisig_event(
&mut txn,
&network,
scanner_event
).await;
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());
// The coordinator expects BatchPreprocess to immediately follow Batch
coordinator.send(
messages::substrate::ProcessorMessage::Batch { batch: batch.clone() }
).await;
if let Some(batch_signer) = tributary_mutable.batch_signer.as_mut() {
if let Some(msg) = batch_signer.sign(&mut txn, batch) {
coordinator.send(msg).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
if let Some(retired_session) = SessionDb::get(&txn, retired_key.to_bytes().as_ref()) {
tributary_mutable.signers.remove(&retired_session);
}
tributary_mutable.batch_signer.take();
let keys = tributary_mutable.key_gen.keys(&new_key);
if let Some((session, (substrate_keys, _))) = keys {
tributary_mutable.batch_signer =
Some(BatchSigner::new(N::NETWORK, session, substrate_keys));
}
}
},
MultisigEvent::Completed(key, id, tx) => {
if let Some(session) = SessionDb::get(&txn, &key) {
let signer = tributary_mutable.signers.get_mut(&session).unwrap();
if let Some(msg) = signer.completed(&mut txn, id, &tx) {
coordinator.send(msg).await;
}
}
}
}
},
}
txn.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();
#[allow(unused_variables, unreachable_code)]
let db = {
#[cfg(all(feature = "parity-db", feature = "rocksdb"))]
panic!("built with parity-db and rocksdb");
#[cfg(all(feature = "parity-db", not(feature = "rocksdb")))]
let db =
serai_db::new_parity_db(&serai_env::var("DB_PATH").expect("path to DB wasn't specified"));
#[cfg(feature = "rocksdb")]
let db =
serai_db::new_rocksdb(&serai_env::var("DB_PATH").expect("path to DB wasn't specified"));
db
};
// 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,
"ethereum" => NetworkId::Ethereum,
"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 = "ethereum")]
NetworkId::Ethereum => {
let relayer_hostname = env::var("ETHEREUM_RELAYER_HOSTNAME")
@ -327,8 +73,5 @@ async fn main() {
let relayer_url = relayer_hostname + ":" + &relayer_port;
run(db.clone(), Ethereum::new(db, url, relayer_url).await, coordinator).await
}
#[cfg(feature = "monero")]
NetworkId::Monero => run(db, Monero::new(url).await, coordinator).await,
_ => panic!("spawning a processor for an unsupported network"),
}
}