mirror of
https://github.com/serai-dex/serai.git
synced 2025-01-03 09:29:46 +00:00
Reduce coordinator tests contention re: cosign messages
This commit is contained in:
parent
4a6496a90b
commit
13b147cbf6
2 changed files with 158 additions and 162 deletions
|
@ -5,7 +5,10 @@ use std::{
|
||||||
time::Duration,
|
time::Duration,
|
||||||
};
|
};
|
||||||
|
|
||||||
use tokio::{task::AbortHandle, sync::Mutex as AsyncMutex};
|
use tokio::{
|
||||||
|
task::AbortHandle,
|
||||||
|
sync::{Mutex as AsyncMutex, mpsc},
|
||||||
|
};
|
||||||
|
|
||||||
use rand_core::{RngCore, OsRng};
|
use rand_core::{RngCore, OsRng};
|
||||||
|
|
||||||
|
@ -96,7 +99,6 @@ pub struct Handles {
|
||||||
pub(crate) message_queue: String,
|
pub(crate) message_queue: String,
|
||||||
}
|
}
|
||||||
|
|
||||||
#[derive(Clone)]
|
|
||||||
pub struct Processor {
|
pub struct Processor {
|
||||||
network: NetworkId,
|
network: NetworkId,
|
||||||
|
|
||||||
|
@ -104,7 +106,8 @@ pub struct Processor {
|
||||||
#[allow(unused)]
|
#[allow(unused)]
|
||||||
handles: Handles,
|
handles: Handles,
|
||||||
|
|
||||||
queue: Arc<AsyncMutex<(u64, u64, MessageQueue)>>,
|
msgs: mpsc::UnboundedReceiver<messages::CoordinatorMessage>,
|
||||||
|
queue_for_sending: MessageQueue,
|
||||||
abort_handle: Option<Arc<AbortHandle>>,
|
abort_handle: Option<Arc<AbortHandle>>,
|
||||||
|
|
||||||
substrate_key: Arc<AsyncMutex<Option<Zeroizing<<Ristretto as Ciphersuite>::F>>>>,
|
substrate_key: Arc<AsyncMutex<Option<Zeroizing<<Ristretto as Ciphersuite>::F>>>>,
|
||||||
|
@ -145,156 +148,173 @@ impl Processor {
|
||||||
// The Serai RPC may or may not be started
|
// The Serai RPC may or may not be started
|
||||||
// Assume it is and continue, so if it's a few seconds late, it's still within tolerance
|
// Assume it is and continue, so if it's a few seconds late, it's still within tolerance
|
||||||
|
|
||||||
|
// Create the queue
|
||||||
|
let mut queue = (
|
||||||
|
0,
|
||||||
|
Arc::new(MessageQueue::new(
|
||||||
|
Service::Processor(network),
|
||||||
|
message_queue_rpc.clone(),
|
||||||
|
Zeroizing::new(processor_key),
|
||||||
|
)),
|
||||||
|
);
|
||||||
|
|
||||||
|
let (msg_send, msg_recv) = mpsc::unbounded_channel();
|
||||||
|
|
||||||
|
let substrate_key = Arc::new(AsyncMutex::new(None));
|
||||||
let mut res = Processor {
|
let mut res = Processor {
|
||||||
network,
|
network,
|
||||||
|
|
||||||
serai_rpc,
|
serai_rpc,
|
||||||
handles,
|
handles,
|
||||||
|
|
||||||
queue: Arc::new(AsyncMutex::new((
|
queue_for_sending: MessageQueue::new(
|
||||||
0,
|
Service::Processor(network),
|
||||||
0,
|
message_queue_rpc,
|
||||||
MessageQueue::new(
|
Zeroizing::new(processor_key),
|
||||||
Service::Processor(network),
|
),
|
||||||
message_queue_rpc,
|
msgs: msg_recv,
|
||||||
Zeroizing::new(processor_key),
|
|
||||||
),
|
|
||||||
))),
|
|
||||||
abort_handle: None,
|
abort_handle: None,
|
||||||
|
|
||||||
substrate_key: Arc::new(AsyncMutex::new(None)),
|
substrate_key: substrate_key.clone(),
|
||||||
};
|
};
|
||||||
|
|
||||||
// Handle any cosigns which come up
|
// Spawn a task to handle cosigns and forward messages as appropriate
|
||||||
res.abort_handle = Some(Arc::new(
|
let abort_handle = tokio::spawn({
|
||||||
tokio::spawn({
|
async move {
|
||||||
let mut res = res.clone();
|
loop {
|
||||||
async move {
|
// Get new messages
|
||||||
loop {
|
let (next_recv_id, queue) = &mut queue;
|
||||||
tokio::task::yield_now().await;
|
let msg = queue.next(Service::Coordinator).await;
|
||||||
|
assert_eq!(msg.from, Service::Coordinator);
|
||||||
|
assert_eq!(msg.id, *next_recv_id);
|
||||||
|
queue.ack(Service::Coordinator, msg.id).await;
|
||||||
|
*next_recv_id += 1;
|
||||||
|
|
||||||
let msg = {
|
let msg_msg = borsh::from_slice(&msg.msg).unwrap();
|
||||||
let mut queue_lock = res.queue.lock().await;
|
|
||||||
let (_, next_recv_id, queue) = &mut *queue_lock;
|
|
||||||
let Ok(msg) =
|
|
||||||
tokio::time::timeout(Duration::from_secs(1), queue.next(Service::Coordinator))
|
|
||||||
.await
|
|
||||||
else {
|
|
||||||
continue;
|
|
||||||
};
|
|
||||||
assert_eq!(msg.from, Service::Coordinator);
|
|
||||||
assert_eq!(msg.id, *next_recv_id);
|
|
||||||
|
|
||||||
let msg_msg = borsh::from_slice(&msg.msg).unwrap();
|
// Remove any BatchReattempts clogging the pipe
|
||||||
// Remove any BatchReattempts clogging the pipe
|
// TODO: Set up a wrapper around serai-client so we aren't throwing this away yet
|
||||||
// TODO: Set up a wrapper around serai-client so we aren't throwing this away yet
|
// leave it for the tests
|
||||||
// leave it for the tests
|
if matches!(
|
||||||
if matches!(
|
msg_msg,
|
||||||
msg_msg,
|
messages::CoordinatorMessage::Coordinator(
|
||||||
messages::CoordinatorMessage::Coordinator(
|
messages::coordinator::CoordinatorMessage::BatchReattempt { .. }
|
||||||
messages::coordinator::CoordinatorMessage::BatchReattempt { .. }
|
)
|
||||||
)
|
) {
|
||||||
) {
|
continue;
|
||||||
queue.ack(Service::Coordinator, msg.id).await;
|
}
|
||||||
*next_recv_id += 1;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
if !is_cosign_message(&msg_msg) {
|
|
||||||
continue;
|
|
||||||
};
|
|
||||||
queue.ack(Service::Coordinator, msg.id).await;
|
|
||||||
*next_recv_id += 1;
|
|
||||||
msg_msg
|
|
||||||
};
|
|
||||||
|
|
||||||
struct CurrentCosign {
|
if !is_cosign_message(&msg_msg) {
|
||||||
block_number: u64,
|
msg_send.send(msg_msg).unwrap();
|
||||||
block: [u8; 32],
|
continue;
|
||||||
}
|
}
|
||||||
static CURRENT_COSIGN: OnceLock<AsyncMutex<Option<CurrentCosign>>> = OnceLock::new();
|
let msg = msg_msg;
|
||||||
let mut current_cosign =
|
|
||||||
CURRENT_COSIGN.get_or_init(|| AsyncMutex::new(None)).lock().await;
|
let send_message = |msg: ProcessorMessage| async move {
|
||||||
match msg {
|
queue
|
||||||
// If this is a CosignSubstrateBlock, reset the CurrentCosign
|
.queue(
|
||||||
// While technically, each processor should individually track the current cosign,
|
Metadata {
|
||||||
// this is fine for current testing purposes
|
from: Service::Processor(network),
|
||||||
CoordinatorMessage::Coordinator(
|
to: Service::Coordinator,
|
||||||
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock {
|
intent: msg.intent(),
|
||||||
id,
|
|
||||||
block_number,
|
|
||||||
},
|
},
|
||||||
) => {
|
borsh::to_vec(&msg).unwrap(),
|
||||||
let SubstrateSignId {
|
)
|
||||||
id: SubstrateSignableId::CosigningSubstrateBlock(block), ..
|
.await;
|
||||||
} = id
|
};
|
||||||
else {
|
|
||||||
panic!("CosignSubstrateBlock didn't have CosigningSubstrateBlock ID")
|
|
||||||
};
|
|
||||||
|
|
||||||
let new_cosign = CurrentCosign { block_number, block };
|
struct CurrentCosign {
|
||||||
if current_cosign.is_none() || (current_cosign.as_ref().unwrap().block != block) {
|
block_number: u64,
|
||||||
*current_cosign = Some(new_cosign);
|
block: [u8; 32],
|
||||||
|
}
|
||||||
|
static CURRENT_COSIGN: OnceLock<AsyncMutex<Option<CurrentCosign>>> = OnceLock::new();
|
||||||
|
let mut current_cosign =
|
||||||
|
CURRENT_COSIGN.get_or_init(|| AsyncMutex::new(None)).lock().await;
|
||||||
|
match msg {
|
||||||
|
// If this is a CosignSubstrateBlock, reset the CurrentCosign
|
||||||
|
// While technically, each processor should individually track the current cosign,
|
||||||
|
// this is fine for current testing purposes
|
||||||
|
CoordinatorMessage::Coordinator(
|
||||||
|
messages::coordinator::CoordinatorMessage::CosignSubstrateBlock { id, block_number },
|
||||||
|
) => {
|
||||||
|
let SubstrateSignId {
|
||||||
|
id: SubstrateSignableId::CosigningSubstrateBlock(block), ..
|
||||||
|
} = id
|
||||||
|
else {
|
||||||
|
panic!("CosignSubstrateBlock didn't have CosigningSubstrateBlock ID")
|
||||||
|
};
|
||||||
|
|
||||||
|
let new_cosign = CurrentCosign { block_number, block };
|
||||||
|
if current_cosign.is_none() || (current_cosign.as_ref().unwrap().block != block) {
|
||||||
|
*current_cosign = Some(new_cosign);
|
||||||
|
}
|
||||||
|
send_message(
|
||||||
|
messages::coordinator::ProcessorMessage::CosignPreprocess {
|
||||||
|
id: id.clone(),
|
||||||
|
preprocesses: vec![[raw_i; 64]],
|
||||||
}
|
}
|
||||||
res
|
.into(),
|
||||||
.send_message(messages::coordinator::ProcessorMessage::CosignPreprocess {
|
)
|
||||||
id: id.clone(),
|
.await;
|
||||||
preprocesses: vec![[raw_i; 64]],
|
|
||||||
})
|
|
||||||
.await;
|
|
||||||
}
|
|
||||||
CoordinatorMessage::Coordinator(
|
|
||||||
messages::coordinator::CoordinatorMessage::SubstratePreprocesses { id, .. },
|
|
||||||
) => {
|
|
||||||
// TODO: Assert the ID matches CURRENT_COSIGN
|
|
||||||
// TODO: Verify the received preprocesses
|
|
||||||
res
|
|
||||||
.send_message(messages::coordinator::ProcessorMessage::SubstrateShare {
|
|
||||||
id,
|
|
||||||
shares: vec![[raw_i; 32]],
|
|
||||||
})
|
|
||||||
.await;
|
|
||||||
}
|
|
||||||
CoordinatorMessage::Coordinator(
|
|
||||||
messages::coordinator::CoordinatorMessage::SubstrateShares { .. },
|
|
||||||
) => {
|
|
||||||
// TODO: Assert the ID matches CURRENT_COSIGN
|
|
||||||
// TODO: Verify the shares
|
|
||||||
|
|
||||||
let block_number = current_cosign.as_ref().unwrap().block_number;
|
|
||||||
let block = current_cosign.as_ref().unwrap().block;
|
|
||||||
|
|
||||||
let substrate_key = res.substrate_key.lock().await.clone().unwrap();
|
|
||||||
|
|
||||||
// Expand to a key pair as Schnorrkel expects
|
|
||||||
// It's the private key + 32-bytes of entropy for nonces + the public key
|
|
||||||
let mut schnorrkel_key_pair = [0; 96];
|
|
||||||
schnorrkel_key_pair[.. 32].copy_from_slice(&substrate_key.to_repr());
|
|
||||||
OsRng.fill_bytes(&mut schnorrkel_key_pair[32 .. 64]);
|
|
||||||
schnorrkel_key_pair[64 ..].copy_from_slice(
|
|
||||||
&(<Ristretto as Ciphersuite>::generator() * *substrate_key).to_bytes(),
|
|
||||||
);
|
|
||||||
let signature = Signature(
|
|
||||||
schnorrkel::keys::Keypair::from_bytes(&schnorrkel_key_pair)
|
|
||||||
.unwrap()
|
|
||||||
.sign_simple(b"substrate", &cosign_block_msg(block_number, block))
|
|
||||||
.to_bytes(),
|
|
||||||
);
|
|
||||||
|
|
||||||
res
|
|
||||||
.send_message(messages::coordinator::ProcessorMessage::CosignedBlock {
|
|
||||||
block_number,
|
|
||||||
block,
|
|
||||||
signature: signature.0.to_vec(),
|
|
||||||
})
|
|
||||||
.await;
|
|
||||||
}
|
|
||||||
_ => panic!("unexpected message passed is_cosign_message"),
|
|
||||||
}
|
}
|
||||||
|
CoordinatorMessage::Coordinator(
|
||||||
|
messages::coordinator::CoordinatorMessage::SubstratePreprocesses { id, .. },
|
||||||
|
) => {
|
||||||
|
// TODO: Assert the ID matches CURRENT_COSIGN
|
||||||
|
// TODO: Verify the received preprocesses
|
||||||
|
send_message(
|
||||||
|
messages::coordinator::ProcessorMessage::SubstrateShare {
|
||||||
|
id,
|
||||||
|
shares: vec![[raw_i; 32]],
|
||||||
|
}
|
||||||
|
.into(),
|
||||||
|
)
|
||||||
|
.await;
|
||||||
|
}
|
||||||
|
CoordinatorMessage::Coordinator(
|
||||||
|
messages::coordinator::CoordinatorMessage::SubstrateShares { .. },
|
||||||
|
) => {
|
||||||
|
// TODO: Assert the ID matches CURRENT_COSIGN
|
||||||
|
// TODO: Verify the shares
|
||||||
|
|
||||||
|
let block_number = current_cosign.as_ref().unwrap().block_number;
|
||||||
|
let block = current_cosign.as_ref().unwrap().block;
|
||||||
|
|
||||||
|
let substrate_key = substrate_key.lock().await.clone().unwrap();
|
||||||
|
|
||||||
|
// Expand to a key pair as Schnorrkel expects
|
||||||
|
// It's the private key + 32-bytes of entropy for nonces + the public key
|
||||||
|
let mut schnorrkel_key_pair = [0; 96];
|
||||||
|
schnorrkel_key_pair[.. 32].copy_from_slice(&substrate_key.to_repr());
|
||||||
|
OsRng.fill_bytes(&mut schnorrkel_key_pair[32 .. 64]);
|
||||||
|
schnorrkel_key_pair[64 ..].copy_from_slice(
|
||||||
|
&(<Ristretto as Ciphersuite>::generator() * *substrate_key).to_bytes(),
|
||||||
|
);
|
||||||
|
let signature = Signature(
|
||||||
|
schnorrkel::keys::Keypair::from_bytes(&schnorrkel_key_pair)
|
||||||
|
.unwrap()
|
||||||
|
.sign_simple(b"substrate", &cosign_block_msg(block_number, block))
|
||||||
|
.to_bytes(),
|
||||||
|
);
|
||||||
|
|
||||||
|
send_message(
|
||||||
|
messages::coordinator::ProcessorMessage::CosignedBlock {
|
||||||
|
block_number,
|
||||||
|
block,
|
||||||
|
signature: signature.0.to_vec(),
|
||||||
|
}
|
||||||
|
.into(),
|
||||||
|
)
|
||||||
|
.await;
|
||||||
|
}
|
||||||
|
_ => panic!("unexpected message passed is_cosign_message"),
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
})
|
}
|
||||||
.abort_handle(),
|
})
|
||||||
));
|
.abort_handle();
|
||||||
|
|
||||||
|
res.abort_handle = Some(Arc::new(abort_handle));
|
||||||
|
|
||||||
res
|
res
|
||||||
}
|
}
|
||||||
|
@ -307,9 +327,8 @@ impl Processor {
|
||||||
pub async fn send_message(&mut self, msg: impl Into<ProcessorMessage>) {
|
pub async fn send_message(&mut self, msg: impl Into<ProcessorMessage>) {
|
||||||
let msg: ProcessorMessage = msg.into();
|
let msg: ProcessorMessage = msg.into();
|
||||||
|
|
||||||
let mut queue_lock = self.queue.lock().await;
|
self
|
||||||
let (next_send_id, _, queue) = &mut *queue_lock;
|
.queue_for_sending
|
||||||
queue
|
|
||||||
.queue(
|
.queue(
|
||||||
Metadata {
|
Metadata {
|
||||||
from: Service::Processor(self.network),
|
from: Service::Processor(self.network),
|
||||||
|
@ -319,36 +338,13 @@ impl Processor {
|
||||||
borsh::to_vec(&msg).unwrap(),
|
borsh::to_vec(&msg).unwrap(),
|
||||||
)
|
)
|
||||||
.await;
|
.await;
|
||||||
*next_send_id += 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
async fn recv_message_inner(&mut self) -> CoordinatorMessage {
|
|
||||||
loop {
|
|
||||||
tokio::task::yield_now().await;
|
|
||||||
|
|
||||||
let mut queue_lock = self.queue.lock().await;
|
|
||||||
let (_, next_recv_id, queue) = &mut *queue_lock;
|
|
||||||
let msg = queue.next(Service::Coordinator).await;
|
|
||||||
assert_eq!(msg.from, Service::Coordinator);
|
|
||||||
assert_eq!(msg.id, *next_recv_id);
|
|
||||||
|
|
||||||
// If this is a cosign message, let the cosign task handle it
|
|
||||||
let msg_msg = borsh::from_slice(&msg.msg).unwrap();
|
|
||||||
if is_cosign_message(&msg_msg) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
queue.ack(Service::Coordinator, msg.id).await;
|
|
||||||
*next_recv_id += 1;
|
|
||||||
return msg_msg;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Receive a message from the coordinator as a processor.
|
/// Receive a message from the coordinator as a processor.
|
||||||
pub async fn recv_message(&mut self) -> CoordinatorMessage {
|
pub async fn recv_message(&mut self) -> CoordinatorMessage {
|
||||||
// Set a timeout of 20 minutes to allow effectively any protocol to occur without a fear of
|
// Set a timeout of 20 minutes to allow effectively any protocol to occur without a fear of
|
||||||
// an arbitrary timeout cutting it short
|
// an arbitrary timeout cutting it short
|
||||||
tokio::time::timeout(Duration::from_secs(20 * 60), self.recv_message_inner()).await.unwrap()
|
tokio::time::timeout(Duration::from_secs(20 * 60), self.msgs.recv()).await.unwrap().unwrap()
|
||||||
}
|
}
|
||||||
|
|
||||||
pub async fn set_substrate_key(
|
pub async fn set_substrate_key(
|
||||||
|
|
|
@ -245,7 +245,7 @@ pub async fn batch(
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
// Send the ack as expected, though it shouldn't trigger any observable behavior
|
// Send the ack as expected
|
||||||
processor
|
processor
|
||||||
.send_message(messages::ProcessorMessage::Coordinator(
|
.send_message(messages::ProcessorMessage::Coordinator(
|
||||||
messages::coordinator::ProcessorMessage::SubstrateBlockAck {
|
messages::coordinator::ProcessorMessage::SubstrateBlockAck {
|
||||||
|
|
Loading…
Reference in a new issue