Add a cosigning protocol to ensure finalizations are unique (#433)

* Add a function to deterministically decide which Serai blocks should be co-signed

Has a 5 minute latency between co-signs, also used as the maximal latency
before a co-sign is started.

* Get all active tributaries we're in at a specific block

* Add and route CosignSubstrateBlock, a new provided TX

* Split queued cosigns per network

* Rename BatchSignId to SubstrateSignId

* Add SubstrateSignableId, a meta-type for either Batch or Block, and modularize around it

* Handle the CosignSubstrateBlock provided TX

* Revert substrate_signer.rs to develop (and patch to still work)

Due to SubstrateSigner moving when the prior multisig closes, yet cosigning
occurring with the most recent key, a single SubstrateSigner can be reused.
We could manage multiple SubstrateSigners, yet considering the much lower
specifications for cosigning, I'd rather treat it distinctly.

* Route cosigning through the processor

* Add note to rename SubstrateSigner post-PR

I don't want to do so now in order to preserve the diff's clarity.

* Implement cosign evaluation into the coordinator

* Get tests to compile

* Bug fixes, mark blocks without cosigners available as cosigned

* Correct the ID Batch preprocesses are saved under, add log statements

* Create a dedicated function to handle cosigns

* Correct the flow around Batch verification/queueing

Verifying `Batch`s could stall when a `Batch` was signed before its
predecessors/before the block it's contained in was cosigned (the latter being
inevitable as we can't sign a block containing a signed batch before signing
the batch).

Now, Batch verification happens on a distinct async task in order to not block
the handling of processor messages. This task is the sole caller of verify in
order to ensure last_verified_batch isn't unexpectedly mutated.

When the processor message handler needs to access it, or needs to queue a
Batch, it associates the DB TXN with a lock preventing the other task from
doing so.

This lock, as currently implemented, is a poor and inefficient design. It
should be modified to the pattern used for cosign management. Additionally, a
new primitive of a DB-backed channel may be immensely valuable.

Fixes a standing potential deadlock and a deadlock introduced with the
cosigning protocol.

* Working full-stack tests

After the last commit, this only required extending a timeout.

* Replace "co-sign" with "cosign" to make finding text easier

* Update the coordinator tests to support cosigning

* Inline prior_batch calculation to prevent panic on rotation

Noticed when doing a final review of the branch.
This commit is contained in:
Luke Parker
2023-11-15 16:57:21 -05:00
committed by GitHub
parent 79e4cce2f6
commit 96f1d26f7a
29 changed files with 1900 additions and 348 deletions

View File

@@ -1,6 +1,6 @@
use core::ops::Deref;
use std::{
sync::Arc,
sync::{OnceLock, Arc},
time::Duration,
collections::{VecDeque, HashSet, HashMap},
};
@@ -18,6 +18,7 @@ use frost::Participant;
use serai_db::{DbTxn, Db};
use serai_env as env;
use scale::Encode;
use serai_client::{
primitives::NetworkId,
validator_sets::primitives::{Session, ValidatorSet},
@@ -27,7 +28,7 @@ use serai_client::{
use message_queue::{Service, client::MessageQueue};
use tokio::{
sync::{RwLock, mpsc, broadcast},
sync::{Mutex, RwLock, mpsc, broadcast},
time::sleep,
};
@@ -46,13 +47,20 @@ use db::MainDb;
mod p2p;
pub use p2p::*;
use processor_messages::{key_gen, sign, coordinator, ProcessorMessage};
use processor_messages::{
key_gen, sign,
coordinator::{self, SubstrateSignableId},
ProcessorMessage,
};
pub mod processors;
use processors::Processors;
mod substrate;
use substrate::SubstrateDb;
use substrate::{CosignTransactions, SubstrateDb};
mod cosign_evaluator;
use cosign_evaluator::CosignEvaluator;
#[cfg(test)]
pub mod tests;
@@ -162,10 +170,16 @@ async fn publish_signed_transaction<D: Db, P: P2p>(
}
}
// TODO: Find a better pattern for this
static HANDOVER_VERIFY_QUEUE_LOCK: OnceLock<Mutex<()>> = OnceLock::new();
#[allow(clippy::too_many_arguments)]
async fn handle_processor_message<D: Db, P: P2p>(
db: &mut D,
key: &Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: &Serai,
p2p: &P,
cosign_channel: &mpsc::UnboundedSender<CosignedBlock>,
tributaries: &HashMap<Session, ActiveTributary<D, P>>,
network: NetworkId,
msg: &processors::Message,
@@ -174,6 +188,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
return true;
}
let _hvq_lock = HANDOVER_VERIFY_QUEUE_LOCK.get_or_init(|| Mutex::new(())).lock().await;
let mut txn = db.txn();
let mut relevant_tributary = match &msg.msg {
@@ -270,12 +285,29 @@ async fn handle_processor_message<D: Db, P: P2p>(
coordinator::ProcessorMessage::InvalidParticipant { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::CosignPreprocess { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::BatchShare { id, .. } => {
coordinator::ProcessorMessage::SubstrateShare { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::CosignedBlock { block, signature } => {
let cosigned_block = CosignedBlock {
network,
block: *block,
signature: {
let mut arr = [0; 64];
arr.copy_from_slice(signature);
arr
},
};
cosign_channel.send(cosigned_block).unwrap();
P2p::broadcast(p2p, P2pMessageKind::CosignedBlock, cosigned_block.encode()).await;
None
}
},
// These don't return a relevant Tributary as there's no Tributary with action expected
ProcessorMessage::Substrate(inner_msg) => match inner_msg {
@@ -284,20 +316,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
batch.network, msg.network,
"processor sent us a batch for a different network than it was for",
);
let this_batch_id = batch.id;
MainDb::<D>::save_expected_batch(&mut txn, batch);
// Re-define batch
// We can't drop it, yet it shouldn't be accidentally used in the following block
#[allow(clippy::let_unit_value, unused_variables)]
let batch = ();
// This won't be complete, as this call is when a `Batch` message is received, which
// will be before we get a `SignedBatch`
// It is, however, incremental
// When we need a complete version, we use another call, continuously called as-needed
substrate::verify_published_batches::<D>(&mut txn, msg.network, this_batch_id).await;
None
}
// If this is a new Batch, immediately publish it (if we can)
@@ -323,8 +342,6 @@ async fn handle_processor_message<D: Db, P: P2p>(
next += 1;
}
let start_id = batches.front().map(|batch| batch.batch.id);
let last_id = batches.back().map(|batch| batch.batch.id);
while let Some(batch) = batches.pop_front() {
// If this Batch should no longer be published, continue
if substrate::get_expected_next_batch(serai, network).await > batch.batch.id {
@@ -357,40 +374,8 @@ async fn handle_processor_message<D: Db, P: P2p>(
sleep(Duration::from_secs(5)).await;
}
}
// Verify the `Batch`s we just published
if let Some(last_id) = last_id {
loop {
let verified =
substrate::verify_published_batches::<D>(&mut txn, msg.network, last_id).await;
if verified == Some(last_id) {
break;
}
}
}
// Check if any of these `Batch`s were a handover `Batch`
// If so, we need to publish any delayed `Batch` provided transactions
let mut relevant = None;
if let Some(start_id) = start_id {
let last_id = last_id.unwrap();
for batch in start_id .. last_id {
if let Some(set) = MainDb::<D>::is_handover_batch(&txn, msg.network, batch) {
// relevant may already be Some. This is a safe over-write, as we don't need to
// be concerned for handovers of Tributaries which have completed their handovers
// While this does bypass the checks that Tributary would've performed at the
// time, if we ever actually participate in a handover, we will verify *all*
// prior `Batch`s, including the ones which would've been explicitly verified
// then
//
// We should only declare this session relevant if it's relevant to us
// We only set handover `Batch`s when we're trying to produce said `Batch`, so this
// would be a `Batch` we were involved in the production of
// Accordingly, iy's relevant
relevant = Some(set.session);
}
}
}
relevant
None
}
},
};
@@ -598,10 +583,18 @@ async fn handle_processor_message<D: Db, P: P2p>(
// slash) and censor transactions (yet don't explicitly ban)
vec![]
}
coordinator::ProcessorMessage::CosignPreprocess { id, preprocesses } => {
vec![Transaction::SubstratePreprocess(SignData {
plan: id.id,
attempt: id.attempt,
data: preprocesses,
signed: Transaction::empty_signed(),
})]
}
coordinator::ProcessorMessage::BatchPreprocess { id, block, preprocesses } => {
log::info!(
"informed of batch (sign ID {}, attempt {}) for block {}",
hex::encode(id.id),
hex::encode(id.id.encode()),
id.attempt,
hex::encode(block),
);
@@ -613,69 +606,79 @@ async fn handle_processor_message<D: Db, P: P2p>(
&mut txn,
spec.set().network,
RecognizedIdType::Batch,
&id.id,
&{
let SubstrateSignableId::Batch(id) = id.id else {
panic!("BatchPreprocess SubstrateSignableId wasn't Batch")
};
id.encode()
},
preprocesses,
);
let intended = Transaction::Batch(
block.0,
match id.id {
SubstrateSignableId::Batch(id) => id,
_ => panic!("BatchPreprocess did not contain Batch ID"),
},
);
// If this is the new key's first Batch, only create this TX once we verify all
// all prior published `Batch`s
// TODO: This assumes BatchPreprocess is immediately after Batch
// Ensure that assumption
let last_received = MainDb::<D>::last_received_batch(&txn, msg.network).unwrap();
let handover_batch = MainDb::<D>::handover_batch(&txn, spec.set());
if handover_batch.is_none() {
MainDb::<D>::set_handover_batch(&mut txn, spec.set(), last_received);
if last_received != 0 {
// Decrease by 1, to get the ID of the Batch prior to this Batch
let prior_sets_last_batch = last_received - 1;
// TODO: If we're looping here, we're not handling the messages we need to in order
// to create the Batch we're looking for
// Don't have the processor yield the handover batch untill the batch before is
// acknowledged on-chain?
loop {
let successfully_verified = substrate::verify_published_batches::<D>(
&mut txn,
msg.network,
prior_sets_last_batch,
)
.await;
if successfully_verified == Some(prior_sets_last_batch) {
break;
let mut queue = false;
if let Some(handover_batch) = handover_batch {
// There is a race condition here. We may verify all `Batch`s from the prior set,
// start signing the handover `Batch` `n`, start signing `n+1`, have `n+1` signed
// before `n` (or at the same time), yet then the prior set forges a malicious
// `Batch` `n`.
//
// The malicious `Batch` `n` would be publishable to Serai, as Serai can't
// distinguish what's intended to be a handover `Batch`, yet then anyone could
// publish the new set's `n+1`, causing their acceptance of the handover.
//
// To fix this, if this is after the handover `Batch` and we have yet to verify
// publication of the handover `Batch`, don't yet yield the provided.
if last_received > handover_batch {
if let Some(last_verified) = MainDb::<D>::last_verified_batch(&txn, msg.network) {
if last_verified < handover_batch {
queue = true;
}
sleep(Duration::from_secs(5)).await;
} else {
queue = true;
}
}
} else {
MainDb::<D>::set_handover_batch(&mut txn, spec.set(), last_received);
// If this isn't the first batch, meaning we do have to verify all prior batches, and
// the prior Batch hasn't been verified yet...
if (last_received != 0) &&
MainDb::<D>::last_verified_batch(&txn, msg.network)
.map(|last_verified| last_verified < (last_received - 1))
.unwrap_or(true)
{
// Withhold this TX until we verify all prior `Batch`s
queue = true;
}
}
// There is a race condition here. We may verify all `Batch`s from the prior set,
// start signing the handover `Batch` `n`, start signing `n+1`, have `n+1` signed
// before `n` (or at the same time), yet then the prior set forges a malicious
// `Batch` `n`.
//
// The malicious `Batch` `n` would be publishable to Serai, as Serai can't
// distinguish what's intended to be a handover `Batch`, yet then anyone could
// publish the new set's `n+1`, causing their acceptance of the handover.
//
// To fix this, if this is after the handover `Batch` and we have yet to verify
// publication of the handover `Batch`, don't yet yield the provided.
let handover_batch = MainDb::<D>::handover_batch(&txn, spec.set()).unwrap();
let intended = Transaction::Batch(block.0, id.id);
let mut res = vec![intended.clone()];
if last_received > handover_batch {
if let Some(last_verified) = MainDb::<D>::last_verified_batch(&txn, msg.network) {
if last_verified < handover_batch {
res = vec![];
}
} else {
res = vec![];
}
}
if res.is_empty() {
if queue {
MainDb::<D>::queue_batch(&mut txn, spec.set(), intended);
vec![]
} else {
// Because this is post-verification of the handover batch, take all queued `Batch`s
// now to ensure we don't provide this before an already queued Batch
// This *may* be an unreachable case due to how last_verified_batch is set, yet it
// doesn't hurt to have as a defensive pattern
let mut res = MainDb::<D>::take_queued_batches(&mut txn, spec.set());
res.push(intended);
res
}
res
} else {
vec![Transaction::BatchPreprocess(SignData {
vec![Transaction::SubstratePreprocess(SignData {
plan: id.id,
attempt: id.attempt,
data: preprocesses,
@@ -683,24 +686,19 @@ async fn handle_processor_message<D: Db, P: P2p>(
})]
}
}
coordinator::ProcessorMessage::BatchShare { id, shares } => {
vec![Transaction::BatchShare(SignData {
coordinator::ProcessorMessage::SubstrateShare { id, shares } => {
vec![Transaction::SubstrateShare(SignData {
plan: id.id,
attempt: id.attempt,
data: shares.into_iter().map(|share| share.to_vec()).collect(),
signed: Transaction::empty_signed(),
})]
}
coordinator::ProcessorMessage::CosignedBlock { .. } => unreachable!(),
},
ProcessorMessage::Substrate(inner_msg) => match inner_msg {
processor_messages::substrate::ProcessorMessage::Batch { .. } => unreachable!(),
processor_messages::substrate::ProcessorMessage::SignedBatch { .. } => {
// We only reach here if this SignedBatch triggered the publication of a handover
// Batch
// Since the handover `Batch` was successfully published and verified, we no longer
// have to worry about the above n+1 attack
MainDb::<D>::take_queued_batches(&mut txn, spec.set())
}
processor_messages::substrate::ProcessorMessage::SignedBatch { .. } => unreachable!(),
},
};
@@ -766,11 +764,14 @@ async fn handle_processor_message<D: Db, P: P2p>(
true
}
#[allow(clippy::too_many_arguments)]
async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
mut db: D,
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: Arc<Serai>,
mut processors: Pro,
p2p: P,
cosign_channel: mpsc::UnboundedSender<CosignedBlock>,
network: NetworkId,
mut tributary_event: mpsc::UnboundedReceiver<TributaryEvent<D, P>>,
) {
@@ -794,10 +795,154 @@ async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
}
// TODO: Check this ID is sane (last handled ID or expected next ID)
let msg = processors.recv(network).await;
if handle_processor_message(&mut db, &key, &serai, &tributaries, network, &msg).await {
let Ok(msg) = tokio::time::timeout(Duration::from_secs(1), processors.recv(network)).await
else {
continue;
};
log::trace!("entering handle_processor_message for {:?}", network);
if handle_processor_message(
&mut db,
&key,
&serai,
&p2p,
&cosign_channel,
&tributaries,
network,
&msg,
)
.await
{
processors.ack(msg).await;
}
log::trace!("exited handle_processor_message for {:?}", network);
}
}
#[allow(clippy::too_many_arguments)]
async fn handle_cosigns_and_batch_publication<D: Db, P: P2p>(
mut db: D,
network: NetworkId,
mut tributary_event: mpsc::UnboundedReceiver<TributaryEvent<D, P>>,
) {
let mut tributaries = HashMap::new();
'outer: loop {
// TODO: Create a better async flow for this, as this does still hammer this task
tokio::task::yield_now().await;
match tributary_event.try_recv() {
Ok(event) => match event {
TributaryEvent::NewTributary(tributary) => {
let set = tributary.spec.set();
assert_eq!(set.network, network);
tributaries.insert(set.session, tributary);
}
TributaryEvent::TributaryRetired(set) => {
tributaries.remove(&set.session);
}
},
Err(mpsc::error::TryRecvError::Empty) => {}
Err(mpsc::error::TryRecvError::Disconnected) => {
panic!("handle_processor_messages tributary_event sender closed")
}
}
// Handle pending cosigns
while let Some((session, block, hash)) = CosignTransactions::peek_cosign(&db, network) {
let Some(ActiveTributary { spec, tributary }) = tributaries.get(&session) else {
log::warn!("didn't yet have tributary we're supposed to cosign with");
break;
};
log::info!(
"{network:?} {session:?} cosigning block #{block} (hash {}...)",
hex::encode(&hash[.. 8])
);
let tx = Transaction::CosignSubstrateBlock(hash);
let res = tributary.provide_transaction(tx.clone()).await;
if !(res.is_ok() || (res == Err(ProvidedError::AlreadyProvided))) {
if res == Err(ProvidedError::LocalMismatchesOnChain) {
// Spin, since this is a crit for this Tributary
loop {
log::error!(
"{}. tributary: {}, provided: {:?}",
"tributary added distinct CosignSubstrateBlock",
hex::encode(spec.genesis()),
&tx,
);
sleep(Duration::from_secs(60)).await;
}
}
panic!("provided an invalid CosignSubstrateBlock: {res:?}");
}
CosignTransactions::take_cosign(db.txn(), network);
}
// Verify any publifshed `Batch`s
{
let _hvq_lock = HANDOVER_VERIFY_QUEUE_LOCK.get_or_init(|| Mutex::new(())).lock().await;
let mut txn = db.txn();
let mut to_publish = vec![];
let start_id = MainDb::<D>::last_verified_batch(&txn, network)
.map(|already_verified| already_verified + 1)
.unwrap_or(0);
if let Some(last_id) =
substrate::verify_published_batches::<D>(&mut txn, network, u32::MAX).await
{
// Check if any of these `Batch`s were a handover `Batch` or the `Batch` before a handover
// `Batch`
// If so, we need to publish queued provided `Batch` transactions
for batch in start_id ..= last_id {
let is_pre_handover = MainDb::<D>::is_handover_batch(&txn, network, batch + 1);
if let Some(set) = is_pre_handover {
let mut queued = MainDb::<D>::take_queued_batches(&mut txn, set);
// is_handover_batch is only set for handover `Batch`s we're participating in, making
// this safe
if queued.is_empty() {
panic!("knew the next Batch was a handover yet didn't queue it");
}
// Only publish the handover Batch
to_publish.push((set.session, queued.remove(0)));
// Re-queue the remaining batches
for remaining in queued {
MainDb::<D>::queue_batch(&mut txn, set, remaining);
}
}
let is_handover = MainDb::<D>::is_handover_batch(&txn, network, batch);
if let Some(set) = is_handover {
for queued in MainDb::<D>::take_queued_batches(&mut txn, set) {
to_publish.push((set.session, queued));
}
}
}
}
for (session, tx) in to_publish {
let Some(ActiveTributary { spec, tributary }) = tributaries.get(&session) else {
log::warn!("didn't yet have tributary we're supposed to provide a queued Batch for");
// Safe since this will drop the txn updating the most recently queued batch
continue 'outer;
};
let res = tributary.provide_transaction(tx.clone()).await;
if !(res.is_ok() || (res == Err(ProvidedError::AlreadyProvided))) {
if res == Err(ProvidedError::LocalMismatchesOnChain) {
// Spin, since this is a crit for this Tributary
loop {
log::error!(
"{}. tributary: {}, provided: {:?}",
"tributary added distinct Batch",
hex::encode(spec.genesis()),
&tx,
);
sleep(Duration::from_secs(60)).await;
}
}
panic!("provided an invalid Batch: {res:?}");
}
}
txn.commit();
}
}
}
@@ -806,6 +951,8 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
key: Zeroizing<<Ristretto as Ciphersuite>::F>,
serai: Arc<Serai>,
processors: Pro,
p2p: P,
cosign_channel: mpsc::UnboundedSender<CosignedBlock>,
mut tributary_event: broadcast::Receiver<TributaryEvent<D, P>>,
) {
let mut channels = HashMap::new();
@@ -813,26 +960,34 @@ pub async fn handle_processors<D: Db, Pro: Processors, P: P2p>(
if network == NetworkId::Serai {
continue;
}
let (send, recv) = mpsc::unbounded_channel();
let (processor_send, processor_recv) = mpsc::unbounded_channel();
tokio::spawn(handle_processor_messages(
db.clone(),
key.clone(),
serai.clone(),
processors.clone(),
p2p.clone(),
cosign_channel.clone(),
network,
recv,
processor_recv,
));
channels.insert(network, send);
let (cosign_send, cosign_recv) = mpsc::unbounded_channel();
tokio::spawn(handle_cosigns_and_batch_publication(db.clone(), network, cosign_recv));
channels.insert(network, (processor_send, cosign_send));
}
// Listen to new tributary events
loop {
match tributary_event.recv().await.unwrap() {
TributaryEvent::NewTributary(tributary) => channels[&tributary.spec.set().network]
.send(TributaryEvent::NewTributary(tributary))
.unwrap(),
TributaryEvent::NewTributary(tributary) => {
let (c1, c2) = &channels[&tributary.spec.set().network];
c1.send(TributaryEvent::NewTributary(tributary.clone())).unwrap();
c2.send(TributaryEvent::NewTributary(tributary)).unwrap();
}
TributaryEvent::TributaryRetired(set) => {
channels[&set.network].send(TributaryEvent::TributaryRetired(set)).unwrap()
let (c1, c2) = &channels[&set.network];
c1.send(TributaryEvent::TributaryRetired(set)).unwrap();
c2.send(TributaryEvent::TributaryRetired(set)).unwrap();
}
};
}
@@ -944,6 +1099,7 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
});
move |set: ValidatorSet, genesis, id_type, id: Vec<u8>, nonce| {
log::debug!("recognized ID {:?} {}", id_type, hex::encode(&id));
let mut raw_db = raw_db.clone();
let key = key.clone();
let tributaries = tributaries.clone();
@@ -956,6 +1112,7 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
loop {
let Some(preprocess) = MainDb::<D>::first_preprocess(raw_db, set.network, id_type, id)
else {
log::warn!("waiting for preprocess for recognized ID");
sleep(Duration::from_millis(100)).await;
continue;
};
@@ -964,9 +1121,9 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
};
let mut tx = match id_type {
RecognizedIdType::Batch => Transaction::BatchPreprocess(SignData {
RecognizedIdType::Batch => Transaction::SubstratePreprocess(SignData {
data: get_preprocess(&raw_db, id_type, &id).await,
plan: id.try_into().unwrap(),
plan: SubstrateSignableId::Batch(id.as_slice().try_into().unwrap()),
attempt: 0,
signed: Transaction::empty_signed(),
}),
@@ -1029,11 +1186,27 @@ pub async fn run<D: Db, Pro: Processors, P: P2p>(
// in a while (presumably because we're behind)
tokio::spawn(p2p::heartbeat_tributaries_task(p2p.clone(), tributary_event_listener_3));
// Create the Cosign evaluator
let cosign_channel = CosignEvaluator::new(raw_db.clone(), p2p.clone(), serai.clone());
// Handle P2P messages
tokio::spawn(p2p::handle_p2p_task(p2p, tributary_event_listener_4));
tokio::spawn(p2p::handle_p2p_task(
p2p.clone(),
cosign_channel.clone(),
tributary_event_listener_4,
));
// Handle all messages from processors
handle_processors(raw_db, key, serai, processors, tributary_event_listener_5).await;
handle_processors(
raw_db,
key,
serai,
processors,
p2p,
cosign_channel,
tributary_event_listener_5,
)
.await;
}
#[tokio::main]