Integrate coordinator with MessageQueue and RocksDB

Also resolves a couple TODOs.
This commit is contained in:
Luke Parker 2023-07-18 01:53:51 -04:00
parent a05961974a
commit a7c9c1ef55
No known key found for this signature in database
12 changed files with 309 additions and 237 deletions

11
Cargo.lock generated
View file

@ -8572,11 +8572,14 @@ dependencies = [
"modular-frost", "modular-frost",
"parity-scale-codec", "parity-scale-codec",
"rand_core 0.6.4", "rand_core 0.6.4",
"rocksdb",
"schnorr-signatures", "schnorr-signatures",
"schnorrkel",
"serai-client", "serai-client",
"serai-db", "serai-db",
"serai-env",
"serai-message-queue",
"serai-processor-messages", "serai-processor-messages",
"serde_json",
"sp-application-crypto", "sp-application-crypto",
"tokio", "tokio",
"tributary-chain", "tributary-chain",
@ -8636,7 +8639,8 @@ dependencies = [
"hex", "hex",
"jsonrpsee", "jsonrpsee",
"lazy_static", "lazy_static",
"log", "rand_core 0.6.4",
"reqwest",
"rocksdb", "rocksdb",
"schnorr-signatures", "schnorr-signatures",
"serai-db", "serai-db",
@ -8645,6 +8649,7 @@ dependencies = [
"serde", "serde",
"serde_json", "serde_json",
"tokio", "tokio",
"zeroize",
] ]
[[package]] [[package]]
@ -8738,9 +8743,7 @@ dependencies = [
"parity-scale-codec", "parity-scale-codec",
"rand_chacha 0.3.1", "rand_chacha 0.3.1",
"rand_core 0.6.4", "rand_core 0.6.4",
"reqwest",
"rocksdb", "rocksdb",
"schnorr-signatures",
"secp256k1", "secp256k1",
"serai-client", "serai-client",
"serai-db", "serai-db",

View file

@ -29,17 +29,20 @@ frost = { package = "modular-frost", path = "../crypto/frost" }
scale = { package = "parity-scale-codec", version = "3", features = ["derive"] } scale = { package = "parity-scale-codec", version = "3", features = ["derive"] }
schnorrkel = "0.10"
sp-application-crypto = { git = "https://github.com/serai-dex/substrate", default-features = false } sp-application-crypto = { git = "https://github.com/serai-dex/substrate", default-features = false }
rocksdb = "0.21"
serai-db = { path = "../common/db" } serai-db = { path = "../common/db" }
serai-env = { path = "../common/env" }
processor-messages = { package = "serai-processor-messages", path = "../processor/messages" } processor-messages = { package = "serai-processor-messages", path = "../processor/messages" }
message-queue = { package = "serai-message-queue", path = "../message-queue" }
tributary = { package = "tributary-chain", path = "./tributary" } tributary = { package = "tributary-chain", path = "./tributary" }
serai-client = { path = "../substrate/client", features = ["serai"] } serai-client = { path = "../substrate/client", features = ["serai"] }
hex = "0.4" hex = "0.4"
serde_json = "1"
log = "0.4" log = "0.4"
tokio = { version = "1", features = ["full"] } tokio = { version = "1", features = ["full"] }

View file

@ -14,10 +14,13 @@ use rand_core::OsRng;
use ciphersuite::{group::ff::Field, Ciphersuite, Ristretto}; use ciphersuite::{group::ff::Field, Ciphersuite, Ristretto};
use serai_db::{DbTxn, Db, MemDb}; use serai_db::{DbTxn, Db};
use serai_env as env;
use serai_client::{Public, Signature, Serai}; use serai_client::{Public, Signature, Serai};
use message_queue::{Service, client::MessageQueue};
use tokio::{ use tokio::{
sync::{ sync::{
mpsc::{self, UnboundedSender}, mpsc::{self, UnboundedSender},
@ -322,7 +325,7 @@ pub async fn handle_p2p<D: Db, P: P2p>(
// connection // connection
// In order to reduce congestion though, we should at least check if we take value from // In order to reduce congestion though, we should at least check if we take value from
// this message before running spawn // this message before running spawn
// TODO // TODO2
tokio::spawn({ tokio::spawn({
let tributaries = tributaries.clone(); let tributaries = tributaries.clone();
async move { async move {
@ -398,7 +401,7 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
key_gen::ProcessorMessage::GeneratedKeyPair { id, substrate_key, coin_key } => { key_gen::ProcessorMessage::GeneratedKeyPair { id, substrate_key, coin_key } => {
assert_eq!( assert_eq!(
id.set.network, msg.network, id.set.network, msg.network,
"processor claimed to be a different network than it was for SubstrateBlockAck", "processor claimed to be a different network than it was for GeneratedKeyPair",
); );
// TODO: Also check the other KeyGenId fields // TODO: Also check the other KeyGenId fields
@ -415,13 +418,15 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
Signature([0; 64]), // TODO Signature([0; 64]), // TODO
); );
match serai.publish(&tx).await { loop {
Ok(hash) => { match serai.publish(&tx).await {
log::info!("voted on key pair for {:?} in TX {}", id.set, hex::encode(hash)) Ok(hash) => {
} log::info!("voted on key pair for {:?} in TX {}", id.set, hex::encode(hash))
Err(e) => { }
log::error!("couldn't connect to Serai node to publish vote TX: {:?}", e); Err(e) => {
todo!(); // TODO log::error!("couldn't connect to Serai node to publish vote TX: {:?}", e);
tokio::time::sleep(Duration::from_secs(10)).await;
}
} }
} }
@ -507,19 +512,22 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
// TODO: Check this key's key pair's substrate key is authorized to publish batches // TODO: Check this key's key pair's substrate key is authorized to publish batches
// TODO: Check the batch ID is an atomic increment // TODO: Check the batch ID is an atomic increment
match serai.publish(&Serai::execute_batch(batch.clone())).await { loop {
Ok(hash) => { match serai.publish(&Serai::execute_batch(batch.clone())).await {
log::info!( Ok(hash) => {
"executed batch {:?} {} (block {}) in TX {}", log::info!(
batch.batch.network, "executed batch {:?} {} (block {}) in TX {}",
batch.batch.id, batch.batch.network,
hex::encode(batch.batch.block), batch.batch.id,
hex::encode(hash), hex::encode(batch.batch.block),
) hex::encode(hash),
} );
Err(e) => { break;
log::error!("couldn't connect to Serai node to publish batch TX: {:?}", e); }
todo!(); // TODO Err(e) => {
log::error!("couldn't connect to Serai node to publish batch TX: {:?}", e);
tokio::time::sleep(Duration::from_secs(10)).await;
}
} }
} }
@ -661,12 +669,17 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
#[tokio::main] #[tokio::main]
async fn main() { async fn main() {
let db = MemDb::new(); // TODO let db = Arc::new(
rocksdb::TransactionDB::<rocksdb::SingleThreaded>::open_default(
env::var("DB_PATH").expect("path to DB wasn't specified"),
)
.unwrap(),
);
let key = Zeroizing::new(<Ristretto as Ciphersuite>::F::ZERO); // TODO let key = Zeroizing::new(<Ristretto as Ciphersuite>::F::ZERO); // TODO
let p2p = LocalP2p::new(1).swap_remove(0); // TODO let p2p = LocalP2p::new(1).swap_remove(0); // TODO
let processors = processors::MemProcessors::new(); // TODO let processors = Arc::new(MessageQueue::new(Service::Coordinator));
let serai = || async { let serai = || async {
loop { loop {

View file

@ -1,14 +1,10 @@
use std::{ use std::sync::Arc;
sync::Arc,
collections::{VecDeque, HashMap},
};
use tokio::sync::RwLock;
use serai_client::primitives::NetworkId; use serai_client::primitives::NetworkId;
use processor_messages::{ProcessorMessage, CoordinatorMessage}; use processor_messages::{ProcessorMessage, CoordinatorMessage};
use message_queue::{Service, Metadata, client::MessageQueue};
#[derive(Clone, PartialEq, Eq, Debug)] #[derive(Clone, PartialEq, Eq, Debug)]
pub struct Message { pub struct Message {
pub id: u64, pub id: u64,
@ -23,27 +19,31 @@ pub trait Processors: 'static + Send + Sync + Clone {
async fn ack(&mut self, msg: Message); async fn ack(&mut self, msg: Message);
} }
// TODO: Move this to tests
#[derive(Clone)]
pub struct MemProcessors(pub Arc<RwLock<HashMap<NetworkId, VecDeque<CoordinatorMessage>>>>);
impl MemProcessors {
#[allow(clippy::new_without_default)]
pub fn new() -> MemProcessors {
MemProcessors(Arc::new(RwLock::new(HashMap::new())))
}
}
#[async_trait::async_trait] #[async_trait::async_trait]
impl Processors for MemProcessors { impl Processors for Arc<MessageQueue> {
async fn send(&self, network: NetworkId, msg: CoordinatorMessage) { async fn send(&self, network: NetworkId, msg: CoordinatorMessage) {
let mut processors = self.0.write().await; let metadata =
let processor = processors.entry(network).or_insert_with(VecDeque::new); Metadata { from: self.service, to: Service::Processor(network), intent: msg.intent() };
processor.push_back(msg); let msg = serde_json::to_string(&msg).unwrap();
self.queue(metadata, msg.into_bytes()).await;
} }
async fn recv(&mut self) -> Message { async fn recv(&mut self) -> Message {
todo!() // TODO: Use a proper expected next ID
let msg = self.next(0).await;
let network = match msg.from {
Service::Processor(network) => network,
Service::Coordinator => panic!("coordinator sent coordinator message"),
};
let id = msg.id;
// Deserialize it into a ProcessorMessage
let msg: ProcessorMessage =
serde_json::from_slice(&msg.msg).expect("message wasn't a JSON-encoded ProcessorMessage");
return Message { id, network, msg };
} }
async fn ack(&mut self, _: Message) { async fn ack(&mut self, msg: Message) {
todo!() MessageQueue::ack(self, msg.id).await
} }
} }

View file

@ -1 +1,38 @@
use std::{
sync::Arc,
collections::{VecDeque, HashMap},
};
use serai_client::primitives::NetworkId;
use processor_messages::CoordinatorMessage;
use tokio::sync::RwLock;
use crate::processors::{Message, Processors};
pub mod tributary; pub mod tributary;
#[derive(Clone)]
pub struct MemProcessors(pub Arc<RwLock<HashMap<NetworkId, VecDeque<CoordinatorMessage>>>>);
impl MemProcessors {
#[allow(clippy::new_without_default)]
pub fn new() -> MemProcessors {
MemProcessors(Arc::new(RwLock::new(HashMap::new())))
}
}
#[async_trait::async_trait]
impl Processors for MemProcessors {
async fn send(&self, network: NetworkId, msg: CoordinatorMessage) {
let mut processors = self.0.write().await;
let processor = processors.entry(network).or_insert_with(VecDeque::new);
processor.push_back(msg);
}
async fn recv(&mut self) -> Message {
todo!()
}
async fn ack(&mut self, _: Message) {
todo!()
}
}

View file

@ -19,10 +19,12 @@ use processor_messages::{
use tributary::{Transaction as TransactionTrait, Tributary}; use tributary::{Transaction as TransactionTrait, Tributary};
use crate::{ use crate::{
processors::MemProcessors,
LocalP2p, LocalP2p,
tributary::{TributaryDb, Transaction, TributarySpec, scanner::handle_new_blocks}, tributary::{TributaryDb, Transaction, TributarySpec, scanner::handle_new_blocks},
tests::tributary::{new_keys, new_spec, new_tributaries, run_tributaries, wait_for_tx_inclusion}, tests::{
MemProcessors,
tributary::{new_keys, new_spec, new_tributaries, run_tributaries, wait_for_tx_inclusion},
},
}; };
#[tokio::test] #[tokio::test]

View file

@ -14,6 +14,7 @@ all-features = true
rustdoc-args = ["--cfg", "docsrs"] rustdoc-args = ["--cfg", "docsrs"]
[dependencies] [dependencies]
# Macros # Macros
lazy_static = "1" lazy_static = "1"
serde = { version = "1", features = ["derive"] } serde = { version = "1", features = ["derive"] }
@ -23,13 +24,16 @@ hex = "0.4"
bincode = "1" bincode = "1"
serde_json = "1" serde_json = "1"
# Libs
zeroize = "1"
rand_core = "0.6"
# Cryptography # Cryptography
transcript = { package = "flexible-transcript", path = "../crypto/transcript", features = ["recommended"] } transcript = { package = "flexible-transcript", path = "../crypto/transcript", features = ["recommended"] }
ciphersuite = { path = "../crypto/ciphersuite", features = ["ristretto"] } ciphersuite = { path = "../crypto/ciphersuite", features = ["ristretto"] }
schnorr-signatures = { path = "../crypto/schnorr" } schnorr-signatures = { path = "../crypto/schnorr" }
# Application # Application
log = "0.4"
tokio = { version = "1", features = ["full"] } tokio = { version = "1", features = ["full"] }
serai-db = { path = "../common/db", features = ["rocksdb"] } serai-db = { path = "../common/db", features = ["rocksdb"] }
@ -40,3 +44,4 @@ serai-env = { path = "../common/env" }
serai-primitives = { path = "../substrate/primitives" } serai-primitives = { path = "../substrate/primitives" }
jsonrpsee = { version = "0.16", features = ["server"] } jsonrpsee = { version = "0.16", features = ["server"] }
reqwest = { version = "0.11", features = ["json"] }

169
message-queue/src/client.rs Normal file
View file

@ -0,0 +1,169 @@
use core::ops::Deref;
use zeroize::{Zeroize, Zeroizing};
use rand_core::OsRng;
use ciphersuite::{
group::ff::{Field, PrimeField},
Ciphersuite, Ristretto,
};
use schnorr_signatures::SchnorrSignature;
use serde::{Serialize, Deserialize};
use reqwest::Client;
use serai_env as env;
use crate::{Service, Metadata, QueuedMessage, message_challenge, ack_challenge};
pub struct MessageQueue {
pub service: Service,
priv_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
pub_key: <Ristretto as Ciphersuite>::G,
client: Client,
url: String,
}
impl MessageQueue {
pub fn new(service: Service) -> MessageQueue {
let url = env::var("MESSAGE_QUEUE_RPC").expect("message-queue RPC wasn't specified");
let priv_key: Zeroizing<<Ristretto as Ciphersuite>::F> = {
let key_str =
Zeroizing::new(env::var("MESSAGE_QUEUE_KEY").expect("message-queue key wasn't specified"));
let key_bytes = Zeroizing::new(
hex::decode(&key_str).expect("invalid message-queue key specified (wasn't hex)"),
);
let mut bytes = <<Ristretto as Ciphersuite>::F as PrimeField>::Repr::default();
bytes.copy_from_slice(&key_bytes);
let key = Zeroizing::new(
Option::from(<<Ristretto as Ciphersuite>::F as PrimeField>::from_repr(bytes))
.expect("invalid message-queue key specified"),
);
bytes.zeroize();
key
};
MessageQueue {
service,
pub_key: Ristretto::generator() * priv_key.deref(),
priv_key,
client: Client::new(),
url,
}
}
async fn json_call(&self, method: &'static str, params: serde_json::Value) -> serde_json::Value {
#[derive(Clone, PartialEq, Eq, Debug, Serialize, Deserialize)]
struct JsonRpcRequest {
version: &'static str,
method: &'static str,
params: serde_json::Value,
id: u64,
}
let res = loop {
// Make the request
if let Ok(req) = self
.client
.post(&self.url)
.json(&JsonRpcRequest { version: "2.0", method, params: params.clone(), id: 0 })
.send()
.await
{
// Get the response
if let Ok(res) = req.text().await {
break res;
}
}
// Sleep 5s before trying again
tokio::time::sleep(core::time::Duration::from_secs(5)).await;
};
let json =
serde_json::from_str::<serde_json::Value>(&res).expect("message-queue returned invalid JSON");
if json.get("result").is_none() {
panic!("call failed: {json}");
}
json
}
pub async fn queue(&self, metadata: Metadata, msg: Vec<u8>) {
// TODO: Should this use OsRng? Deterministic or deterministic + random may be better.
let nonce = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
let nonce_pub = Ristretto::generator() * nonce.deref();
let sig = SchnorrSignature::<Ristretto>::sign(
&self.priv_key,
nonce,
message_challenge(
metadata.from,
self.pub_key,
metadata.to,
&metadata.intent,
&msg,
nonce_pub,
),
)
.serialize();
let json = self.json_call("queue", serde_json::json!([metadata, msg, sig])).await;
if json.get("result") != Some(&serde_json::Value::Bool(true)) {
panic!("failed to queue message: {json}");
}
}
pub async fn next(&self, expected: u64) -> QueuedMessage {
loop {
let json = self.json_call("next", serde_json::json!([self.service, expected])).await;
// Convert from a Value to a type via reserialization
let msg: Option<QueuedMessage> = serde_json::from_str(
&serde_json::to_string(
&json.get("result").expect("successful JSON RPC call didn't have result"),
)
.unwrap(),
)
.expect("next didn't return an Option<QueuedMessage>");
// If there wasn't a message, check again in 5s
let Some(msg) = msg else {
tokio::time::sleep(core::time::Duration::from_secs(5)).await;
continue;
};
// Verify the message
// Verify the sender is sane
if matches!(self.service, Service::Processor(_)) {
assert_eq!(msg.from, Service::Coordinator, "non-coordinator sent processor message");
} else {
assert!(
matches!(msg.from, Service::Processor(_)),
"non-processor sent coordinator message"
);
}
// TODO: Verify the sender's signature
// TODO: Check the ID is sane
return msg;
}
}
pub async fn ack(&self, id: u64) {
// TODO: Should this use OsRng? Deterministic or deterministic + random may be better.
let nonce = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
let nonce_pub = Ristretto::generator() * nonce.deref();
let sig = SchnorrSignature::<Ristretto>::sign(
&self.priv_key,
nonce,
ack_challenge(self.service, self.pub_key, id, nonce_pub),
)
.serialize();
let json = self.json_call("ack", serde_json::json!([id, sig])).await;
if json.get("result") != Some(&serde_json::Value::Bool(true)) {
panic!("failed to ack message {id}: {json}");
}
}
}

View file

@ -1,2 +1,4 @@
mod messages; mod messages;
pub use messages::*; pub use messages::*;
pub mod client;

View file

@ -33,7 +33,6 @@ serde_json = "1"
# Cryptography # Cryptography
ciphersuite = { path = "../crypto/ciphersuite", features = ["ristretto"] } ciphersuite = { path = "../crypto/ciphersuite", features = ["ristretto"] }
schnorr = { package = "schnorr-signatures", path = "../crypto/schnorr" }
transcript = { package = "flexible-transcript", path = "../crypto/transcript" } transcript = { package = "flexible-transcript", path = "../crypto/transcript" }
frost = { package = "modular-frost", path = "../crypto/frost", features = ["ristretto"] } frost = { package = "modular-frost", path = "../crypto/frost", features = ["ristretto"] }
@ -62,7 +61,6 @@ serai-client = { path = "../substrate/client", default-features = false }
messages = { package = "serai-processor-messages", path = "./messages" } messages = { package = "serai-processor-messages", path = "./messages" }
reqwest = "0.11"
message-queue = { package = "serai-message-queue", path = "../message-queue" } message-queue = { package = "serai-message-queue", path = "../message-queue" }
[dev-dependencies] [dev-dependencies]

View file

@ -1,18 +1,6 @@
use core::ops::Deref;
use zeroize::Zeroizing;
use rand_core::OsRng;
use ciphersuite::{group::ff::Field, Ciphersuite, Ristretto};
use schnorr::SchnorrSignature;
use serde::{Serialize, Deserialize};
use messages::{ProcessorMessage, CoordinatorMessage}; use messages::{ProcessorMessage, CoordinatorMessage};
use serai_client::primitives::NetworkId; use message_queue::{Service, Metadata, client::MessageQueue};
use message_queue::{Service, Metadata, QueuedMessage, message_challenge, ack_challenge};
use reqwest::Client;
#[derive(Clone, PartialEq, Eq, Debug)] #[derive(Clone, PartialEq, Eq, Debug)]
pub struct Message { pub struct Message {
@ -27,156 +15,29 @@ pub trait Coordinator {
async fn ack(&mut self, msg: Message); async fn ack(&mut self, msg: Message);
} }
pub struct MessageQueue {
network: NetworkId,
priv_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
pub_key: <Ristretto as Ciphersuite>::G,
client: Client,
message_queue_url: String,
}
impl MessageQueue {
pub fn new(
message_queue_url: String,
network: NetworkId,
priv_key: Zeroizing<<Ristretto as Ciphersuite>::F>,
) -> MessageQueue {
MessageQueue {
network,
pub_key: Ristretto::generator() * priv_key.deref(),
priv_key,
client: Client::new(),
message_queue_url,
}
}
async fn json_call(&self, method: &'static str, params: serde_json::Value) -> serde_json::Value {
#[derive(Clone, PartialEq, Eq, Debug, Serialize, Deserialize)]
struct JsonRpcRequest {
version: &'static str,
method: &'static str,
params: serde_json::Value,
id: u64,
}
let res = loop {
// Make the request
if let Ok(req) = self
.client
.post(&self.message_queue_url)
.json(&JsonRpcRequest { version: "2.0", method, params: params.clone(), id: 0 })
.send()
.await
{
// Get the response
if let Ok(res) = req.text().await {
break res;
}
}
// Sleep 5s before trying again
tokio::time::sleep(core::time::Duration::from_secs(5)).await;
};
let json =
serde_json::from_str::<serde_json::Value>(&res).expect("message-queue returned invalid JSON");
if json.get("result").is_none() {
panic!("call failed: {json}");
}
json
}
async fn queue(&self, metadata: Metadata, msg: Vec<u8>, sig: Vec<u8>) {
let json = self.json_call("queue", serde_json::json!([metadata, msg, sig])).await;
if json.get("result") != Some(&serde_json::Value::Bool(true)) {
panic!("failed to queue message: {json}");
}
}
async fn next(&self) -> Message {
loop {
// TODO: Use a proper expected next ID
let json =
self.json_call("next", serde_json::json!([Service::Processor(self.network), 0])).await;
// Convert from a Value to a type via reserialization
let msg: Option<QueuedMessage> = serde_json::from_str(
&serde_json::to_string(
&json.get("result").expect("successful JSON RPC call didn't have result"),
)
.unwrap(),
)
.expect("next didn't return an Option<QueuedMessage>");
// If there wasn't a message, check again in 5s
let Some(msg) = msg else {
tokio::time::sleep(core::time::Duration::from_secs(5)).await;
continue;
};
// Verify the message
assert_eq!(msg.from, Service::Coordinator, "non-coordinator sent us message");
// TODO: Verify the coordinator's signature
// TODO: Check the ID is sane
let id = msg.id;
// Deserialize it into a CoordinatorMessage
let msg: CoordinatorMessage =
serde_json::from_slice(&msg.msg).expect("message wasn't a JSON-encoded CoordinatorMessage");
return Message { id, msg };
}
}
async fn ack(&self, id: u64, sig: Vec<u8>) {
let json = self.json_call("ack", serde_json::json!([id, sig])).await;
if json.get("result") != Some(&serde_json::Value::Bool(true)) {
panic!("failed to ack message {id}: {json}");
}
}
}
#[async_trait::async_trait] #[async_trait::async_trait]
impl Coordinator for MessageQueue { impl Coordinator for MessageQueue {
async fn send(&mut self, msg: ProcessorMessage) { async fn send(&mut self, msg: ProcessorMessage) {
let metadata = Metadata { let metadata = Metadata { from: self.service, to: Service::Coordinator, intent: msg.intent() };
from: Service::Processor(self.network),
to: Service::Coordinator,
intent: msg.intent(),
};
let msg = serde_json::to_string(&msg).unwrap(); let msg = serde_json::to_string(&msg).unwrap();
// TODO: Should this use OsRng? Deterministic or deterministic + random may be better. self.queue(metadata, msg.into_bytes()).await;
let nonce = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
let nonce_pub = Ristretto::generator() * nonce.deref();
let sig = SchnorrSignature::<Ristretto>::sign(
&self.priv_key,
nonce,
message_challenge(
metadata.from,
self.pub_key,
metadata.to,
&metadata.intent,
msg.as_bytes(),
nonce_pub,
),
);
self.queue(metadata, msg.into_bytes(), sig.serialize()).await;
} }
async fn recv(&mut self) -> Message { async fn recv(&mut self) -> Message {
self.next().await // TODO: Use a proper expected next ID
let msg = self.next(0).await;
let id = msg.id;
// Deserialize it into a CoordinatorMessage
let msg: CoordinatorMessage =
serde_json::from_slice(&msg.msg).expect("message wasn't a JSON-encoded CoordinatorMessage");
return Message { id, msg };
} }
async fn ack(&mut self, msg: Message) { async fn ack(&mut self, msg: Message) {
// TODO: Should this use OsRng? Deterministic or deterministic + random may be better. MessageQueue::ack(self, msg.id).await
let nonce = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
let nonce_pub = Ristretto::generator() * nonce.deref();
let sig = SchnorrSignature::<Ristretto>::sign(
&self.priv_key,
nonce,
ack_challenge(Service::Processor(self.network), self.pub_key, msg.id, nonce_pub),
);
MessageQueue::ack(self, msg.id, sig.serialize()).await
} }
} }

View file

@ -7,10 +7,7 @@ use std::{
use zeroize::{Zeroize, Zeroizing}; use zeroize::{Zeroize, Zeroizing};
use transcript::{Transcript, RecommendedTranscript}; use transcript::{Transcript, RecommendedTranscript};
use ciphersuite::{ use ciphersuite::group::GroupEncoding;
group::{ff::PrimeField, GroupEncoding},
Ristretto,
};
use frost::{curve::Ciphersuite, ThresholdKeys}; use frost::{curve::Ciphersuite, ThresholdKeys};
use log::{info, warn, error}; use log::{info, warn, error};
@ -30,6 +27,8 @@ use messages::{SubstrateContext, CoordinatorMessage, ProcessorMessage};
use serai_env as env; use serai_env as env;
use message_queue::{Service, client::MessageQueue};
mod plan; mod plan;
pub use plan::*; pub use plan::*;
@ -733,27 +732,7 @@ async fn main() {
_ => panic!("unrecognized network"), _ => panic!("unrecognized network"),
}; };
// Coordinator configuration let coordinator = MessageQueue::new(Service::Processor(network_id));
let priv_key = {
let key_str =
Zeroizing::new(env::var("MESSAGE_QUEUE_KEY").expect("message-queue key wasn't specified"));
let key_bytes = Zeroizing::new(
hex::decode(&key_str).expect("invalid message-queue key specified (wasn't hex)"),
);
let mut bytes = <<Ristretto as Ciphersuite>::F as PrimeField>::Repr::default();
bytes.copy_from_slice(&key_bytes);
let key = Zeroizing::new(
Option::from(<<Ristretto as Ciphersuite>::F as PrimeField>::from_repr(bytes))
.expect("invalid message-queue key specified"),
);
bytes.zeroize();
key
};
let coordinator = MessageQueue::new(
env::var("MESSAGE_QUEUE_RPC").expect("message-queue RPC wasn't specified"),
network_id,
priv_key,
);
match network_id { match network_id {
#[cfg(feature = "bitcoin")] #[cfg(feature = "bitcoin")]