mirror of
https://github.com/serai-dex/serai.git
synced 2024-12-22 19:49:22 +00:00
Integrate coordinator with MessageQueue and RocksDB
Also resolves a couple TODOs.
This commit is contained in:
parent
a05961974a
commit
a7c9c1ef55
12 changed files with 309 additions and 237 deletions
11
Cargo.lock
generated
11
Cargo.lock
generated
|
@ -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",
|
||||||
|
|
|
@ -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"] }
|
||||||
|
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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!()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -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]
|
||||||
|
|
|
@ -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
169
message-queue/src/client.rs
Normal 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}");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,2 +1,4 @@
|
||||||
mod messages;
|
mod messages;
|
||||||
pub use messages::*;
|
pub use messages::*;
|
||||||
|
|
||||||
|
pub mod client;
|
||||||
|
|
|
@ -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]
|
||||||
|
|
|
@ -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
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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")]
|
||||||
|
|
Loading…
Reference in a new issue