Use dedicated Queues for each from-to pair

Prevents one Processor's message from halting the entire pipeline.
This commit is contained in:
Luke Parker
2023-09-27 12:20:57 -04:00
parent 269db1c4be
commit 40b7bc59d0
12 changed files with 142 additions and 125 deletions

View File

@@ -15,14 +15,14 @@ impl<D: Db> MainDb<D> {
D::key(b"coordinator_main", dst, key)
}
fn handled_message_key(id: u64) -> Vec<u8> {
Self::main_key(b"handled_message", id.to_le_bytes())
fn handled_message_key(network: NetworkId, id: u64) -> Vec<u8> {
Self::main_key(b"handled_message", (network, id).encode())
}
pub fn save_handled_message(txn: &mut D::Transaction<'_>, id: u64) {
txn.put(Self::handled_message_key(id), []);
pub fn save_handled_message(txn: &mut D::Transaction<'_>, network: NetworkId, id: u64) {
txn.put(Self::handled_message_key(network, id), []);
}
pub fn handled_message<G: Get>(getter: &G, id: u64) -> bool {
getter.get(Self::handled_message_key(id)).is_some()
pub fn handled_message<G: Get>(getter: &G, network: NetworkId, id: u64) -> bool {
getter.get(Self::handled_message_key(network, id)).is_some()
}
fn acive_tributaries_key() -> Vec<u8> {

View File

@@ -533,19 +533,20 @@ async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: Arc<Serai>,
mut processors: Pro,
tributary: ActiveTributary<D, P>,
mut recv: mpsc::UnboundedReceiver<processors::Message>,
network: NetworkId,
mut recv: mpsc::UnboundedReceiver<ActiveTributary<D, P>>,
) {
let mut db_clone = db.clone(); // Enables cloning the DB while we have a txn
let pub_key = Ristretto::generator() * key.deref();
let ActiveTributary { spec, tributary } = tributary;
let ActiveTributary { spec, tributary } = recv.recv().await.unwrap();
let genesis = spec.genesis();
loop {
let msg: processors::Message = recv.recv().await.unwrap();
// TODO: Check this ID is sane (last handled ID or expected next ID)
let msg = processors.recv(network).await;
if !MainDb::<D>::handled_message(&db, msg.id) {
if !MainDb::<D>::handled_message(&db, msg.network, msg.id) {
let mut txn = db.txn();
// TODO: We probably want to NOP here, not panic?
@@ -817,7 +818,7 @@ async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
}
}
MainDb::<D>::save_handled_message(&mut txn, msg.id);
MainDb::<D>::save_handled_message(&mut txn, msg.network, msg.id);
txn.commit();
}
@@ -829,61 +830,27 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
db: D,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: Arc<Serai>,
mut processors: Pro,
processors: Pro,
mut new_tributary: broadcast::Receiver<ActiveTributary<D, P>>,
) {
let channels = Arc::new(RwLock::new(HashMap::new()));
let mut channels = HashMap::new();
for network in [NetworkId::Bitcoin, NetworkId::Ethereum, NetworkId::Monero] {
let (send, recv) = mpsc::unbounded_channel();
tokio::spawn(handle_processor_messages(
db.clone(),
key.clone(),
serai.clone(),
processors.clone(),
network,
recv,
));
channels.insert(network, send);
}
// Listen to new tributary events
tokio::spawn({
let db = db.clone();
let processors = processors.clone();
let channels = channels.clone();
async move {
loop {
let channels = channels.clone();
let tributary = new_tributary.recv().await.unwrap();
let (send, recv) = mpsc::unbounded_channel();
// TODO: Support multisig rotation (not per-Tributary yet per-network?)
channels.write().await.insert(tributary.spec.set().network, send);
// For each new tributary, spawn a dedicated task to handle its messages from the processor
// TODO: Redo per network, not per tributary
tokio::spawn(handle_processor_messages(
db.clone(),
key.clone(),
serai.clone(),
processors.clone(),
tributary,
recv,
));
}
}
});
// Dispatch task
let mut last_msg = None;
loop {
// TODO: We dispatch this to an async task per-processor, yet we don't move to the next message
// yet as all processor messages are shoved into a global queue.
// Modify message-queue to offer per-sender queues, not per-receiver.
// Alternatively, a peek method with local delineation of handled messages would work.
let msg = processors.recv().await;
// TODO: Check this ID is sane (last handled ID or expected next ID)
if last_msg == Some(msg.id) {
sleep(Duration::from_secs(1)).await;
continue;
}
last_msg = Some(msg.id);
// TODO: Race conditions with above tributary availability?
// TODO: How does this hold up to multisig rotation?
if let Some(channel) = channels.read().await.get(&msg.network) {
channel.send(msg).unwrap();
} else {
log::warn!("received processor message for network we don't have a channel for");
}
let tributary = new_tributary.recv().await.unwrap();
channels[&tributary.spec.set().network].send(tributary).unwrap();
}
}

View File

@@ -15,7 +15,7 @@ pub struct Message {
#[async_trait::async_trait]
pub trait Processors: 'static + Send + Sync + Clone {
async fn send(&self, network: NetworkId, msg: CoordinatorMessage);
async fn recv(&mut self) -> Message;
async fn recv(&mut self, network: NetworkId) -> Message;
async fn ack(&mut self, msg: Message);
}
@@ -27,13 +27,10 @@ impl Processors for Arc<MessageQueue> {
let msg = serde_json::to_string(&msg).unwrap();
self.queue(metadata, msg.into_bytes()).await;
}
async fn recv(&mut self) -> Message {
let msg = self.next().await;
async fn recv(&mut self, network: NetworkId) -> Message {
let msg = self.next(Service::Processor(network)).await;
assert_eq!(msg.from, Service::Processor(network));
let network = match msg.from {
Service::Processor(network) => network,
Service::Coordinator => panic!("coordinator received coordinator message"),
};
let id = msg.id;
// Deserialize it into a ProcessorMessage
@@ -43,6 +40,6 @@ impl Processors for Arc<MessageQueue> {
return Message { id, network, msg };
}
async fn ack(&mut self, msg: Message) {
MessageQueue::ack(self, msg.id).await
MessageQueue::ack(self, Service::Processor(msg.network), msg.id).await
}
}

View File

@@ -35,7 +35,7 @@ impl Processors for MemProcessors {
let processor = processors.entry(network).or_insert_with(VecDeque::new);
processor.push_back(msg);
}
async fn recv(&mut self) -> Message {
async fn recv(&mut self, _: NetworkId) -> Message {
todo!()
}
async fn ack(&mut self, _: Message) {