mirror of
https://github.com/serai-dex/serai.git
synced 2025-01-12 13:55:28 +00:00
Remove async-trait from processor/
Part of https://github.com/serai-dex/issues/607.
This commit is contained in:
parent
2c4c33e632
commit
e78236276a
29 changed files with 1481 additions and 1378 deletions
|
@ -17,7 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
|
|||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
zeroize = { version = "1", default-features = false, features = ["std"] }
|
||||
|
||||
hex = { version = "0.4", default-features = false, features = ["std"] }
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use core::future::Future;
|
||||
use std::sync::{LazyLock, Arc, Mutex};
|
||||
|
||||
use tokio::sync::mpsc;
|
||||
|
@ -169,24 +170,26 @@ impl Coordinator {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl signers::Coordinator for CoordinatorSend {
|
||||
type EphemeralError = ();
|
||||
|
||||
async fn send(
|
||||
fn send(
|
||||
&mut self,
|
||||
msg: messages::sign::ProcessorMessage,
|
||||
) -> Result<(), Self::EphemeralError> {
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move {
|
||||
self.send(&messages::ProcessorMessage::Sign(msg));
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
async fn publish_cosign(
|
||||
fn publish_cosign(
|
||||
&mut self,
|
||||
block_number: u64,
|
||||
block: [u8; 32],
|
||||
signature: Signature,
|
||||
) -> Result<(), Self::EphemeralError> {
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move {
|
||||
self.send(&messages::ProcessorMessage::Coordinator(
|
||||
messages::coordinator::ProcessorMessage::CosignedBlock {
|
||||
block_number,
|
||||
|
@ -196,26 +199,38 @@ impl signers::Coordinator for CoordinatorSend {
|
|||
));
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
async fn publish_batch(&mut self, batch: Batch) -> Result<(), Self::EphemeralError> {
|
||||
fn publish_batch(
|
||||
&mut self,
|
||||
batch: Batch,
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move {
|
||||
self.send(&messages::ProcessorMessage::Substrate(
|
||||
messages::substrate::ProcessorMessage::Batch { batch },
|
||||
));
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
async fn publish_signed_batch(&mut self, batch: SignedBatch) -> Result<(), Self::EphemeralError> {
|
||||
fn publish_signed_batch(
|
||||
&mut self,
|
||||
batch: SignedBatch,
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move {
|
||||
self.send(&messages::ProcessorMessage::Coordinator(
|
||||
messages::coordinator::ProcessorMessage::SignedBatch { batch },
|
||||
));
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
async fn publish_slash_report_signature(
|
||||
fn publish_slash_report_signature(
|
||||
&mut self,
|
||||
session: Session,
|
||||
signature: Signature,
|
||||
) -> Result<(), Self::EphemeralError> {
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move {
|
||||
self.send(&messages::ProcessorMessage::Coordinator(
|
||||
messages::coordinator::ProcessorMessage::SignedSlashReport {
|
||||
session,
|
||||
|
@ -225,3 +240,4 @@ impl signers::Coordinator for CoordinatorSend {
|
|||
Ok(())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
|
|||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
rand_core = { version = "0.6", default-features = false }
|
||||
|
||||
hex = { version = "0.4", default-features = false, features = ["std"] }
|
||||
|
|
|
@ -96,7 +96,6 @@ use serai_client::{
|
|||
*/
|
||||
|
||||
/*
|
||||
#[async_trait]
|
||||
impl TransactionTrait<Bitcoin> for Transaction {
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, network: &Bitcoin) -> u64 {
|
||||
|
@ -210,7 +209,6 @@ impl Bitcoin {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl Network for Bitcoin {
|
||||
// 2 inputs should be 2 * 230 = 460 weight units
|
||||
// The output should be ~36 bytes, or 144 weight units
|
||||
|
|
|
@ -31,7 +31,6 @@ impl<D: Db> fmt::Debug for Block<D> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db> primitives::Block for Block<D> {
|
||||
type Header = BlockHeader;
|
||||
|
||||
|
|
|
@ -1,3 +1,5 @@
|
|||
use core::future::Future;
|
||||
|
||||
use bitcoin_serai::rpc::{RpcError, Rpc as BRpc};
|
||||
|
||||
use serai_client::primitives::{NetworkId, Coin, Amount};
|
||||
|
@ -18,7 +20,6 @@ pub(crate) struct Rpc<D: Db> {
|
|||
pub(crate) rpc: BRpc,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db> ScannerFeed for Rpc<D> {
|
||||
const NETWORK: NetworkId = NetworkId::Bitcoin;
|
||||
// 6 confirmations is widely accepted as secure and shouldn't occur
|
||||
|
@ -32,11 +33,17 @@ impl<D: Db> ScannerFeed for Rpc<D> {
|
|||
|
||||
type EphemeralError = RpcError;
|
||||
|
||||
async fn latest_finalized_block_number(&self) -> Result<u64, Self::EphemeralError> {
|
||||
db::LatestBlockToYieldAsFinalized::get(&self.db).ok_or(RpcError::ConnectionError)
|
||||
fn latest_finalized_block_number(
|
||||
&self,
|
||||
) -> impl Send + Future<Output = Result<u64, Self::EphemeralError>> {
|
||||
async move { db::LatestBlockToYieldAsFinalized::get(&self.db).ok_or(RpcError::ConnectionError) }
|
||||
}
|
||||
|
||||
async fn time_of_block(&self, number: u64) -> Result<u64, Self::EphemeralError> {
|
||||
fn time_of_block(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> impl Send + Future<Output = Result<u64, Self::EphemeralError>> {
|
||||
async move {
|
||||
let number = usize::try_from(number).unwrap();
|
||||
|
||||
/*
|
||||
|
@ -49,7 +56,8 @@ impl<D: Db> ScannerFeed for Rpc<D> {
|
|||
const MEDIAN_TIMESPAN: usize = 11;
|
||||
let mut timestamps = Vec::with_capacity(MEDIAN_TIMESPAN);
|
||||
for i in number.saturating_sub(MEDIAN_TIMESPAN) .. number {
|
||||
timestamps.push(self.rpc.get_block(&self.rpc.get_block_hash(i).await?).await?.header.time);
|
||||
timestamps
|
||||
.push(self.rpc.get_block(&self.rpc.get_block_hash(i).await?).await?.header.time);
|
||||
}
|
||||
timestamps.sort();
|
||||
timestamps[timestamps.len() / 2]
|
||||
|
@ -72,32 +80,43 @@ impl<D: Db> ScannerFeed for Rpc<D> {
|
|||
the same median.
|
||||
|
||||
The median will never decrease however. The values pushed onto the window will always be
|
||||
greater than the median. If a value greater than the median is popped, the median will remain
|
||||
the same (due to the counterbalance of the pushed value). If a value less than the median is
|
||||
popped, the median will increase (either to another instance of the same value, yet one
|
||||
closer to the end of the repeating sequence, or to a higher value).
|
||||
greater than the median. If a value greater than the median is popped, the median will
|
||||
remain the same (due to the counterbalance of the pushed value). If a value less than the
|
||||
median is popped, the median will increase (either to another instance of the same value,
|
||||
yet one closer to the end of the repeating sequence, or to a higher value).
|
||||
*/
|
||||
Ok(median.into())
|
||||
}
|
||||
|
||||
async fn unchecked_block_header_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> Result<<Self::Block as primitives::Block>::Header, Self::EphemeralError> {
|
||||
Ok(BlockHeader(
|
||||
self.rpc.get_block(&self.rpc.get_block_hash(number.try_into().unwrap()).await?).await?.header,
|
||||
))
|
||||
}
|
||||
|
||||
async fn unchecked_block_by_number(
|
||||
fn unchecked_block_header_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> Result<Self::Block, Self::EphemeralError> {
|
||||
) -> impl Send
|
||||
+ Future<Output = Result<<Self::Block as primitives::Block>::Header, Self::EphemeralError>>
|
||||
{
|
||||
async move {
|
||||
Ok(BlockHeader(
|
||||
self
|
||||
.rpc
|
||||
.get_block(&self.rpc.get_block_hash(number.try_into().unwrap()).await?)
|
||||
.await?
|
||||
.header,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
fn unchecked_block_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> impl Send + Future<Output = Result<Self::Block, Self::EphemeralError>> {
|
||||
async move {
|
||||
Ok(Block(
|
||||
self.db.clone(),
|
||||
self.rpc.get_block(&self.rpc.get_block_hash(number.try_into().unwrap()).await?).await?,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
fn dust(coin: Coin) -> Amount {
|
||||
assert_eq!(coin, Coin::Bitcoin);
|
||||
|
@ -137,22 +156,26 @@ impl<D: Db> ScannerFeed for Rpc<D> {
|
|||
Amount(10_000)
|
||||
}
|
||||
|
||||
async fn cost_to_aggregate(
|
||||
fn cost_to_aggregate(
|
||||
&self,
|
||||
coin: Coin,
|
||||
_reference_block: &Self::Block,
|
||||
) -> Result<Amount, Self::EphemeralError> {
|
||||
) -> impl Send + Future<Output = Result<Amount, Self::EphemeralError>> {
|
||||
async move {
|
||||
assert_eq!(coin, Coin::Bitcoin);
|
||||
// TODO
|
||||
Ok(Amount(0))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db> TransactionPublisher<Transaction> for Rpc<D> {
|
||||
type EphemeralError = RpcError;
|
||||
|
||||
async fn publish(&self, tx: Transaction) -> Result<(), Self::EphemeralError> {
|
||||
self.rpc.send_raw_transaction(&tx.0).await.map(|_| ())
|
||||
fn publish(
|
||||
&self,
|
||||
tx: Transaction,
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move { self.rpc.send_raw_transaction(&tx.0).await.map(|_| ()) }
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,18 +1,4 @@
|
|||
/*
|
||||
We want to be able to return received outputs. We do that by iterating over the inputs to find an
|
||||
address format we recognize, then setting that address as the address to return to.
|
||||
|
||||
Since inputs only contain the script signatures, yet addresses are for script public keys, we
|
||||
need to pull up the output spent by an input and read the script public key from that. While we
|
||||
could use `txindex=1`, and an asynchronous call to the Bitcoin node, we:
|
||||
|
||||
1) Can maintain a much smaller index ourselves
|
||||
2) Don't want the asynchronous call (which would require the flow be async, allowed to
|
||||
potentially error, and more latent)
|
||||
3) Don't want to risk Bitcoin's `txindex` corruptions (frequently observed on testnet)
|
||||
|
||||
This task builds that index.
|
||||
*/
|
||||
use core::future::Future;
|
||||
|
||||
use bitcoin_serai::bitcoin::ScriptBuf;
|
||||
|
||||
|
@ -35,11 +21,26 @@ pub(crate) fn script_pubkey_for_on_chain_output(
|
|||
)
|
||||
}
|
||||
|
||||
/*
|
||||
We want to be able to return received outputs. We do that by iterating over the inputs to find an
|
||||
address format we recognize, then setting that address as the address to return to.
|
||||
|
||||
Since inputs only contain the script signatures, yet addresses are for script public keys, we
|
||||
need to pull up the output spent by an input and read the script public key from that. While we
|
||||
could use `txindex=1`, and an asynchronous call to the Bitcoin node, we:
|
||||
|
||||
1) Can maintain a much smaller index ourselves
|
||||
2) Don't want the asynchronous call (which would require the flow be async, allowed to
|
||||
potentially error, and more latent)
|
||||
3) Don't want to risk Bitcoin's `txindex` corruptions (frequently observed on testnet)
|
||||
|
||||
This task builds that index.
|
||||
*/
|
||||
pub(crate) struct TxIndexTask<D: Db>(pub(crate) Rpc<D>);
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db> ContinuallyRan for TxIndexTask<D> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let latest_block_number = self
|
||||
.0
|
||||
.rpc
|
||||
|
@ -60,8 +61,8 @@ impl<D: Db> ContinuallyRan for TxIndexTask<D> {
|
|||
blocks and panics on reorganization, this runs prior to the scanner and that index.
|
||||
|
||||
A reorganization of `CONFIRMATIONS` blocks is still an invariant. Even if that occurs, this
|
||||
saves the script public keys *by the transaction hash an output index*. Accordingly, it isn't
|
||||
invalidated on reorganization. The only risk would be if the new chain reorganized to
|
||||
saves the script public keys *by the transaction hash an output index*. Accordingly, it
|
||||
isn't invalidated on reorganization. The only risk would be if the new chain reorganized to
|
||||
include a transaction to Serai which we didn't index the parents of. If that happens, we'll
|
||||
panic when we scan the transaction, causing the invariant to be detected.
|
||||
*/
|
||||
|
@ -104,3 +105,4 @@ impl<D: Db> ContinuallyRan for TxIndexTask<D> {
|
|||
Ok(iterated)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,8 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
|
|||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
|
||||
const-hex = { version = "1", default-features = false }
|
||||
hex = { version = "0.4", default-features = false, features = ["std"] }
|
||||
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std"] }
|
||||
|
|
|
@ -17,7 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
|
|||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
rand_core = { version = "0.6", default-features = false }
|
||||
|
||||
hex = { version = "0.4", default-features = false, features = ["std"] }
|
||||
|
|
|
@ -24,7 +24,6 @@ impl primitives::BlockHeader for BlockHeader {
|
|||
#[derive(Clone, Debug)]
|
||||
pub(crate) struct Block(pub(crate) MBlock, Vec<Transaction>);
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl primitives::Block for Block {
|
||||
type Header = BlockHeader;
|
||||
|
||||
|
|
|
@ -1,3 +1,5 @@
|
|||
use core::future::Future;
|
||||
|
||||
use monero_wallet::rpc::{RpcError, Rpc as RpcTrait};
|
||||
use monero_simple_request_rpc::SimpleRequestRpc;
|
||||
|
||||
|
@ -16,7 +18,6 @@ pub(crate) struct Rpc {
|
|||
pub(crate) rpc: SimpleRequestRpc,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl ScannerFeed for Rpc {
|
||||
const NETWORK: NetworkId = NetworkId::Monero;
|
||||
// Outputs aren't spendable until 10 blocks later due to the 10-block lock
|
||||
|
@ -32,28 +33,44 @@ impl ScannerFeed for Rpc {
|
|||
|
||||
type EphemeralError = RpcError;
|
||||
|
||||
async fn latest_finalized_block_number(&self) -> Result<u64, Self::EphemeralError> {
|
||||
Ok(self.rpc.get_height().await?.checked_sub(1).expect("connected to an invalid Monero RPC").try_into().unwrap())
|
||||
fn latest_finalized_block_number(
|
||||
&self,
|
||||
) -> impl Send + Future<Output = Result<u64, Self::EphemeralError>> {
|
||||
async move {
|
||||
Ok(
|
||||
self
|
||||
.rpc
|
||||
.get_height()
|
||||
.await?
|
||||
.checked_sub(1)
|
||||
.expect("connected to an invalid Monero RPC")
|
||||
.try_into()
|
||||
.unwrap(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
async fn time_of_block(&self, number: u64) -> Result<u64, Self::EphemeralError> {
|
||||
todo!("TODO")
|
||||
}
|
||||
|
||||
async fn unchecked_block_header_by_number(
|
||||
fn time_of_block(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> Result<<Self::Block as primitives::Block>::Header, Self::EphemeralError> {
|
||||
Ok(BlockHeader(
|
||||
self.rpc.get_block_by_number(number.try_into().unwrap()).await?
|
||||
))
|
||||
) -> impl Send + Future<Output = Result<u64, Self::EphemeralError>> {
|
||||
async move{todo!("TODO")}
|
||||
}
|
||||
|
||||
async fn unchecked_block_by_number(
|
||||
fn unchecked_block_header_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> Result<Self::Block, Self::EphemeralError> {
|
||||
todo!("TODO")
|
||||
) -> impl Send
|
||||
+ Future<Output = Result<<Self::Block as primitives::Block>::Header, Self::EphemeralError>>
|
||||
{
|
||||
async move { Ok(BlockHeader(self.rpc.get_block_by_number(number.try_into().unwrap()).await?)) }
|
||||
}
|
||||
|
||||
fn unchecked_block_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> impl Send + Future<Output = Result<Self::Block, Self::EphemeralError>> {
|
||||
async move { todo!("TODO") }
|
||||
}
|
||||
|
||||
fn dust(coin: Coin) -> Amount {
|
||||
|
@ -62,22 +79,26 @@ impl ScannerFeed for Rpc {
|
|||
todo!("TODO")
|
||||
}
|
||||
|
||||
async fn cost_to_aggregate(
|
||||
fn cost_to_aggregate(
|
||||
&self,
|
||||
coin: Coin,
|
||||
_reference_block: &Self::Block,
|
||||
) -> Result<Amount, Self::EphemeralError> {
|
||||
) -> impl Send + Future<Output = Result<Amount, Self::EphemeralError>> {
|
||||
async move {
|
||||
assert_eq!(coin, Coin::Bitcoin);
|
||||
// TODO
|
||||
Ok(Amount(0))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl TransactionPublisher<Transaction> for Rpc {
|
||||
type EphemeralError = RpcError;
|
||||
|
||||
async fn publish(&self, tx: Transaction) -> Result<(), Self::EphemeralError> {
|
||||
self.rpc.publish_transaction(&tx.0).await
|
||||
fn publish(
|
||||
&self,
|
||||
tx: Transaction,
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>> {
|
||||
async move { self.rpc.publish_transaction(&tx.0).await }
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,8 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
|
|||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
|
||||
group = { version = "0.13", default-features = false }
|
||||
|
||||
serai-primitives = { path = "../../substrate/primitives", default-features = false, features = ["std"] }
|
||||
|
|
|
@ -22,7 +22,6 @@ pub trait BlockHeader: Send + Sync + Sized + Clone + Debug {
|
|||
/// necessary to literally define it as whatever the external network defines as a block. For
|
||||
/// external networks which finalize block(s), this block type should be a representation of all
|
||||
/// transactions within a period finalization (whether block or epoch).
|
||||
#[async_trait::async_trait]
|
||||
pub trait Block: Send + Sync + Sized + Clone + Debug {
|
||||
/// The type used for this block's header.
|
||||
type Header: BlockHeader;
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
use core::time::Duration;
|
||||
use core::{future::Future, time::Duration};
|
||||
use std::sync::Arc;
|
||||
|
||||
use tokio::sync::{mpsc, oneshot, Mutex};
|
||||
|
@ -78,8 +78,7 @@ impl TaskHandle {
|
|||
}
|
||||
|
||||
/// A task to be continually ran.
|
||||
#[async_trait::async_trait]
|
||||
pub trait ContinuallyRan: Sized {
|
||||
pub trait ContinuallyRan: Sized + Send {
|
||||
/// The amount of seconds before this task should be polled again.
|
||||
const DELAY_BETWEEN_ITERATIONS: u64 = 5;
|
||||
/// The maximum amount of seconds before this task should be run again.
|
||||
|
@ -91,10 +90,15 @@ pub trait ContinuallyRan: Sized {
|
|||
///
|
||||
/// If this returns `true`, all dependents of the task will immediately have a new iteration ran
|
||||
/// (without waiting for whatever timer they were already on).
|
||||
async fn run_iteration(&mut self) -> Result<bool, String>;
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>>;
|
||||
|
||||
/// Continually run the task.
|
||||
async fn continually_run(mut self, mut task: Task, dependents: Vec<TaskHandle>) {
|
||||
fn continually_run(
|
||||
mut self,
|
||||
mut task: Task,
|
||||
dependents: Vec<TaskHandle>,
|
||||
) -> impl Send + Future<Output = ()> {
|
||||
async move {
|
||||
// The default number of seconds to sleep before running the task again
|
||||
let default_sleep_before_next_task = Self::DELAY_BETWEEN_ITERATIONS;
|
||||
// The current number of seconds to sleep before running the task again
|
||||
|
@ -148,3 +152,4 @@ pub trait ContinuallyRan: Sized {
|
|||
task.closed.send(()).unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,9 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
|
|||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
# Macros
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
|
||||
# Encoders
|
||||
hex = { version = "0.4", default-features = false, features = ["std"] }
|
||||
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["std"] }
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
use core::marker::PhantomData;
|
||||
use core::{marker::PhantomData, future::Future};
|
||||
use std::collections::{HashSet, HashMap};
|
||||
|
||||
use group::GroupEncoding;
|
||||
|
@ -185,9 +185,9 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> EventualityTask<D, S, Sch> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTask<D, S, Sch> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
// Fetch the highest acknowledged block
|
||||
let Some(highest_acknowledged) = ScannerGlobalDb::<S>::highest_acknowledged_block(&self.db)
|
||||
else {
|
||||
|
@ -210,14 +210,14 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
|
||||
We can't know the processor has intaked all Burns it should have when we process block `b`.
|
||||
We solve this by executing a consensus protocol whenever a resolution for an Eventuality
|
||||
created to fulfill Burns occurs. Accordingly, we force ourselves to obtain synchrony on such
|
||||
blocks (and all preceding Burns).
|
||||
created to fulfill Burns occurs. Accordingly, we force ourselves to obtain synchrony on
|
||||
such blocks (and all preceding Burns).
|
||||
|
||||
This means we can only iterate up to the block currently pending acknowledgement.
|
||||
|
||||
We only know blocks will need acknowledgement *for sure* if they were scanned. The only other
|
||||
causes are key activation and retirement (both scheduled outside the scan window). This makes
|
||||
the exclusive upper bound the *next block to scan*.
|
||||
We only know blocks will need acknowledgement *for sure* if they were scanned. The only
|
||||
other causes are key activation and retirement (both scheduled outside the scan window).
|
||||
This makes the exclusive upper bound the *next block to scan*.
|
||||
*/
|
||||
let exclusive_upper_bound = {
|
||||
// Fetch the next to scan block
|
||||
|
@ -241,19 +241,20 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
/*
|
||||
If this block is notable *and* not acknowledged, break.
|
||||
|
||||
This is so if Burns queued prior to this block's acknowledgement caused any Eventualities
|
||||
(which may resolve this block), we have them. If it wasn't for that, it'd be so if this
|
||||
block's acknowledgement caused any Eventualities, we have them, though those would only
|
||||
potentially resolve in the next block (letting us scan this block without delay).
|
||||
This is so if Burns queued prior to this block's acknowledgement caused any
|
||||
Eventualities (which may resolve this block), we have them. If it wasn't for that, it'd
|
||||
be so if this block's acknowledgement caused any Eventualities, we have them, though
|
||||
those would only potentially resolve in the next block (letting us scan this block
|
||||
without delay).
|
||||
*/
|
||||
if b > highest_acknowledged {
|
||||
break;
|
||||
}
|
||||
|
||||
// Since this block is notable, ensure we've intaked all the Burns preceding it
|
||||
// We can know with certainty that the channel is fully populated at this time since we've
|
||||
// acknowledged a newer block (so we've handled the state up to this point and any new
|
||||
// state will be for the newer block)
|
||||
// We can know with certainty that the channel is fully populated at this time since
|
||||
// we've acknowledged a newer block (so we've handled the state up to this point and any
|
||||
// new state will be for the newer block)
|
||||
#[allow(unused_assignments)]
|
||||
{
|
||||
made_progress |= self.intake_burns().await?;
|
||||
|
@ -286,7 +287,8 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
|
||||
let completed_eventualities = {
|
||||
let mut eventualities = EventualityDb::<S>::eventualities(&txn, key.key);
|
||||
let completed_eventualities = block.check_for_eventuality_resolutions(&mut eventualities);
|
||||
let completed_eventualities =
|
||||
block.check_for_eventuality_resolutions(&mut eventualities);
|
||||
EventualityDb::<S>::set_eventualities(&mut txn, key.key, &eventualities);
|
||||
completed_eventualities
|
||||
};
|
||||
|
@ -322,8 +324,8 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
which can only be created while the multisig is actively handling `Burn`s (therefore
|
||||
ensuring this multisig cannot be kept alive ad-infinitum)
|
||||
|
||||
The commentary on Change outputs also applies to Branch/Forwarded. They'll presumably get
|
||||
ignored if not usable however.
|
||||
The commentary on Change outputs also applies to Branch/Forwarded. They'll presumably
|
||||
get ignored if not usable however.
|
||||
*/
|
||||
if key.stage == LifetimeStage::Finishing {
|
||||
non_external_outputs
|
||||
|
@ -334,8 +336,10 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
// cost to aggregate them to avoid some profitable spam attacks by malicious miners
|
||||
{
|
||||
// Fetch and cache the costs to aggregate as this call may be expensive
|
||||
let coins =
|
||||
non_external_outputs.iter().map(|output| output.balance().coin).collect::<HashSet<_>>();
|
||||
let coins = non_external_outputs
|
||||
.iter()
|
||||
.map(|output| output.balance().coin)
|
||||
.collect::<HashSet<_>>();
|
||||
let mut costs_to_aggregate = HashMap::new();
|
||||
for coin in coins {
|
||||
costs_to_aggregate.insert(
|
||||
|
@ -364,8 +368,8 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
continue;
|
||||
};
|
||||
let Some(forwarded) = eventuality.singular_spent_output() else {
|
||||
// This was a TX made by us, yet someone burned to the forwarding address as it doesn't
|
||||
// follow the structure of forwarding transactions
|
||||
// This was a TX made by us, yet someone burned to the forwarding address as it
|
||||
// doesn't follow the structure of forwarding transactions
|
||||
continue;
|
||||
};
|
||||
|
||||
|
@ -407,7 +411,8 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
let empty = {
|
||||
let a: core::slice::Iter<'_, OutputFor<S>> = scheduler_update.outputs.iter();
|
||||
let b: core::slice::Iter<'_, OutputFor<S>> = scheduler_update.forwards.iter();
|
||||
let c = scheduler_update.returns.iter().map(|output_to_return| &output_to_return.output);
|
||||
let c =
|
||||
scheduler_update.returns.iter().map(|output_to_return| &output_to_return.output);
|
||||
let mut all_outputs = a.chain(b).chain(c).peekable();
|
||||
|
||||
// If we received any output, sanity check this block is notable
|
||||
|
@ -433,10 +438,10 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
/*
|
||||
This uses the `keys_with_stages` for the current block, yet this block is notable.
|
||||
Accordingly, all future intaked Burns will use at least this block when determining
|
||||
what LifetimeStage a key is. That makes the LifetimeStage monotonically incremented. If
|
||||
this block wasn't notable, we'd potentially intake Burns with the LifetimeStage
|
||||
determined off an earlier block than this (enabling an earlier LifetimeStage to be used
|
||||
after a later one was already used).
|
||||
what LifetimeStage a key is. That makes the LifetimeStage monotonically incremented.
|
||||
If this block wasn't notable, we'd potentially intake Burns with the LifetimeStage
|
||||
determined off an earlier block than this (enabling an earlier LifetimeStage to be
|
||||
used after a later one was already used).
|
||||
*/
|
||||
let new_eventualities =
|
||||
Sch::update(&mut txn, &block, &keys_with_stages, scheduler_update);
|
||||
|
@ -499,3 +504,4 @@ impl<D: Db, S: ScannerFeed, Sch: Scheduler<S>> ContinuallyRan for EventualityTas
|
|||
Ok(made_progress)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,5 +1,6 @@
|
|||
use serai_db::{Get, DbTxn, Db};
|
||||
use core::future::Future;
|
||||
|
||||
use serai_db::{Get, DbTxn, Db};
|
||||
use primitives::{task::ContinuallyRan, BlockHeader};
|
||||
|
||||
use crate::ScannerFeed;
|
||||
|
@ -56,9 +57,9 @@ impl<D: Db, S: ScannerFeed> IndexTask<D, S> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, S: ScannerFeed> ContinuallyRan for IndexTask<D, S> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
// Fetch the latest finalized block
|
||||
let our_latest_finalized = IndexDb::latest_finalized_block(&self.db)
|
||||
.expect("IndexTask run before writing the start block");
|
||||
|
@ -111,3 +112,4 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for IndexTask<D, S> {
|
|||
Ok(our_latest_finalized != latest_finalized)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
#![doc = include_str!("../README.md")]
|
||||
#![deny(missing_docs)]
|
||||
|
||||
use core::{marker::PhantomData, fmt::Debug};
|
||||
use core::{marker::PhantomData, future::Future, fmt::Debug};
|
||||
use std::{io, collections::HashMap};
|
||||
|
||||
use group::GroupEncoding;
|
||||
|
@ -59,7 +59,6 @@ impl<B: Block> BlockExt for B {
|
|||
/// A feed usable to scan a blockchain.
|
||||
///
|
||||
/// This defines the primitive types used, along with various getters necessary for indexing.
|
||||
#[async_trait::async_trait]
|
||||
pub trait ScannerFeed: 'static + Send + Sync + Clone {
|
||||
/// The ID of the network being scanned for.
|
||||
const NETWORK: NetworkId;
|
||||
|
@ -110,38 +109,43 @@ pub trait ScannerFeed: 'static + Send + Sync + Clone {
|
|||
///
|
||||
/// The block number is its zero-indexed position within a linear view of the external network's
|
||||
/// consensus. The genesis block accordingly has block number 0.
|
||||
async fn latest_finalized_block_number(&self) -> Result<u64, Self::EphemeralError>;
|
||||
fn latest_finalized_block_number(
|
||||
&self,
|
||||
) -> impl Send + Future<Output = Result<u64, Self::EphemeralError>>;
|
||||
|
||||
/// Fetch the timestamp of a block (represented in seconds since the epoch).
|
||||
///
|
||||
/// This must be monotonically incrementing. Two blocks may share a timestamp.
|
||||
async fn time_of_block(&self, number: u64) -> Result<u64, Self::EphemeralError>;
|
||||
fn time_of_block(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> impl Send + Future<Output = Result<u64, Self::EphemeralError>>;
|
||||
|
||||
/// Fetch a block header by its number.
|
||||
///
|
||||
/// This does not check the returned BlockHeader is the header for the block we indexed.
|
||||
async fn unchecked_block_header_by_number(
|
||||
fn unchecked_block_header_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> Result<<Self::Block as Block>::Header, Self::EphemeralError>;
|
||||
) -> impl Send + Future<Output = Result<<Self::Block as Block>::Header, Self::EphemeralError>>;
|
||||
|
||||
/// Fetch a block by its number.
|
||||
///
|
||||
/// This does not check the returned Block is the block we indexed.
|
||||
async fn unchecked_block_by_number(
|
||||
fn unchecked_block_by_number(
|
||||
&self,
|
||||
number: u64,
|
||||
) -> Result<Self::Block, Self::EphemeralError>;
|
||||
) -> impl Send + Future<Output = Result<Self::Block, Self::EphemeralError>>;
|
||||
|
||||
/// Fetch a block by its number.
|
||||
///
|
||||
/// Panics if the block requested wasn't indexed.
|
||||
async fn block_by_number(
|
||||
fn block_by_number(
|
||||
&self,
|
||||
getter: &(impl Send + Sync + Get),
|
||||
number: u64,
|
||||
) -> Result<Self::Block, String> {
|
||||
let block = match self.unchecked_block_by_number(number).await {
|
||||
) -> impl Send + Future<Output = Result<Self::Block, String>> {
|
||||
async move {let block = match self.unchecked_block_by_number(number).await {
|
||||
Ok(block) => block,
|
||||
Err(e) => Err(format!("couldn't fetch block {number}: {e:?}"))?,
|
||||
};
|
||||
|
@ -159,7 +163,7 @@ pub trait ScannerFeed: 'static + Send + Sync + Clone {
|
|||
}
|
||||
}
|
||||
|
||||
Ok(block)
|
||||
Ok(block)}
|
||||
}
|
||||
|
||||
/// The dust threshold for the specified coin.
|
||||
|
@ -171,11 +175,11 @@ pub trait ScannerFeed: 'static + Send + Sync + Clone {
|
|||
/// The cost to aggregate an input as of the specified block.
|
||||
///
|
||||
/// This is defined as the transaction fee for a 2-input, 1-output transaction.
|
||||
async fn cost_to_aggregate(
|
||||
fn cost_to_aggregate(
|
||||
&self,
|
||||
coin: Coin,
|
||||
reference_block: &Self::Block,
|
||||
) -> Result<Amount, Self::EphemeralError>;
|
||||
) -> impl Send + Future<Output = Result<Amount, Self::EphemeralError>>;
|
||||
}
|
||||
|
||||
/// The key type for this ScannerFeed.
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
use core::marker::PhantomData;
|
||||
use core::{marker::PhantomData, future::Future};
|
||||
|
||||
use scale::Encode;
|
||||
use serai_db::{DbTxn, Db};
|
||||
|
@ -65,9 +65,9 @@ impl<D: Db, S: ScannerFeed> ReportTask<D, S> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, S: ScannerFeed> ContinuallyRan for ReportTask<D, S> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let highest_reportable = {
|
||||
// Fetch the next to scan block
|
||||
let next_to_scan = next_to_scan_for_outputs_block::<S>(&self.db)
|
||||
|
@ -92,7 +92,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ReportTask<D, S> {
|
|||
let mut txn = self.db.txn();
|
||||
|
||||
// Receive the InInstructions for this block
|
||||
// We always do this as we can't trivially tell if we should recv InInstructions before we do
|
||||
// We always do this as we can't trivially tell if we should recv InInstructions before we
|
||||
// do
|
||||
let InInstructionData {
|
||||
external_key_for_session_to_sign_batch,
|
||||
returnable_in_instructions: in_instructions,
|
||||
|
@ -108,10 +109,14 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ReportTask<D, S> {
|
|||
let mut batch_id = ReportDb::<S>::acquire_batch_id(&mut txn, b);
|
||||
|
||||
// start with empty batch
|
||||
let mut batches =
|
||||
vec![Batch { network, id: batch_id, block: BlockHash(block_hash), instructions: vec![] }];
|
||||
// We also track the return information for the InInstructions within a Batch in case they
|
||||
// error
|
||||
let mut batches = vec![Batch {
|
||||
network,
|
||||
id: batch_id,
|
||||
block: BlockHash(block_hash),
|
||||
instructions: vec![],
|
||||
}];
|
||||
// We also track the return information for the InInstructions within a Batch in case
|
||||
// they error
|
||||
let mut return_information = vec![vec![]];
|
||||
|
||||
for Returnable { return_address, in_instruction } in in_instructions {
|
||||
|
@ -175,3 +180,4 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ReportTask<D, S> {
|
|||
Ok(next_to_potentially_report <= highest_reportable)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use core::future::Future;
|
||||
use std::collections::HashMap;
|
||||
|
||||
use scale::Decode;
|
||||
|
@ -107,9 +108,9 @@ impl<D: Db, S: ScannerFeed> ScanTask<D, S> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
// Fetch the safe to scan block
|
||||
let latest_scannable =
|
||||
latest_scannable_block::<S>(&self.db).expect("ScanTask run before writing the start block");
|
||||
|
@ -172,25 +173,25 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
|
||||
/*
|
||||
The scan task runs ahead of time, obtaining ordering on the external network's blocks
|
||||
with relation to events on the Serai network. This is done via publishing a Batch which
|
||||
contains the InInstructions from External outputs. Accordingly, the scan process only
|
||||
has to yield External outputs.
|
||||
with relation to events on the Serai network. This is done via publishing a Batch
|
||||
which contains the InInstructions from External outputs. Accordingly, the scan
|
||||
process only has to yield External outputs.
|
||||
|
||||
It'd appear to make sense to scan for all outputs, and after scanning for all outputs,
|
||||
yield all outputs. The issue is we can't identify outputs we created here. We can only
|
||||
identify the outputs we receive and their *declared intention*.
|
||||
It'd appear to make sense to scan for all outputs, and after scanning for all
|
||||
outputs, yield all outputs. The issue is we can't identify outputs we created here.
|
||||
We can only identify the outputs we receive and their *declared intention*.
|
||||
|
||||
We only want to handle Change/Branch/Forwarded outputs we made ourselves. For
|
||||
Forwarded, the reasoning is obvious (retiring multisigs should only downsize, yet
|
||||
accepting new outputs solely because they claim to be Forwarded would increase the size
|
||||
of the multisig). For Change/Branch, it's because such outputs which aren't ours are
|
||||
pointless. They wouldn't hurt to accumulate though.
|
||||
accepting new outputs solely because they claim to be Forwarded would increase the
|
||||
size of the multisig). For Change/Branch, it's because such outputs which aren't ours
|
||||
are pointless. They wouldn't hurt to accumulate though.
|
||||
|
||||
The issue is they would hurt to accumulate. We want to filter outputs which are less
|
||||
than their cost to aggregate, a variable itself variable to the current blockchain. We
|
||||
can filter such outputs here, yet if we drop a Change output, we create an insolvency.
|
||||
We'd need to track the loss and offset it later. That means we can't filter such
|
||||
outputs, as we expect any Change output we make.
|
||||
than their cost to aggregate, a variable itself variable to the current blockchain.
|
||||
We can filter such outputs here, yet if we drop a Change output, we create an
|
||||
insolvency. We'd need to track the loss and offset it later. That means we can't
|
||||
filter such outputs, as we expect any Change output we make.
|
||||
|
||||
The issue is the Change outputs we don't make. Someone can create an output declaring
|
||||
to be Change, yet not actually Change. If we don't filter it, it'd be queued for
|
||||
|
@ -199,8 +200,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
The solution is to let the Eventuality task, which does know if we made an output or
|
||||
not (or rather, if a transaction is identical to a transaction which should exist
|
||||
regarding effects) decide to keep/yield the outputs which we should only keep if we
|
||||
made them (as Serai itself should not make worthless outputs, so we can assume they're
|
||||
worthwhile, and even if they're not economically, they are technically).
|
||||
made them (as Serai itself should not make worthless outputs, so we can assume
|
||||
they're worthwhile, and even if they're not economically, they are technically).
|
||||
|
||||
The alternative, we drop outputs here with a generic filter rule and then report back
|
||||
the insolvency created, still doesn't work as we'd only be creating an insolvency if
|
||||
|
@ -213,8 +214,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
// While we don't report these outputs, we still need consensus on this block and
|
||||
// accordingly still need to set it as notable
|
||||
let balance = output.balance();
|
||||
// We ensure it's over the dust limit to prevent people sending 1 satoshi from causing
|
||||
// an invocation of a consensus/signing protocol
|
||||
// We ensure it's over the dust limit to prevent people sending 1 satoshi from
|
||||
// causing an invocation of a consensus/signing protocol
|
||||
if balance.amount.0 >= S::dust(balance.coin).0 {
|
||||
ScannerGlobalDb::<S>::flag_notable_due_to_non_external_output(&mut txn, b);
|
||||
}
|
||||
|
@ -258,7 +259,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
in_instruction: InInstructionWithBalance { instruction, balance: balance_to_use },
|
||||
},
|
||||
(Some(address), None) => {
|
||||
// Since there was no instruction here, return this since we parsed a return address
|
||||
// Since there was no instruction here, return this since we parsed a return
|
||||
// address
|
||||
if key.stage != LifetimeStage::Finishing {
|
||||
scan_data.returns.push(Return { address, output });
|
||||
}
|
||||
|
@ -293,8 +295,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
// We should report External outputs only once forwarded, where they'll appear as
|
||||
// OutputType::Forwarded. We save them now for when they appear
|
||||
LifetimeStage::Forwarding => {
|
||||
// When the forwarded output appears, we can see which Plan it's associated with and
|
||||
// from there recover this output
|
||||
// When the forwarded output appears, we can see which Plan it's associated with
|
||||
// and from there recover this output
|
||||
scan_data.forwards.push(output_with_in_instruction);
|
||||
continue;
|
||||
}
|
||||
|
@ -326,8 +328,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
res
|
||||
});
|
||||
// Check we haven't prior reported an InInstruction for this output
|
||||
// This is a sanity check which is intended to prevent multiple instances of sriXYZ on-chain
|
||||
// due to a single output
|
||||
// This is a sanity check which is intended to prevent multiple instances of sriXYZ
|
||||
// on-chain due to a single output
|
||||
for (id, _) in &in_instructions {
|
||||
assert!(
|
||||
!ScanDb::<S>::prior_reported_in_instruction_for_output(&txn, id),
|
||||
|
@ -336,8 +338,10 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
ScanDb::<S>::reported_in_instruction_for_output(&mut txn, id);
|
||||
}
|
||||
// Reformat the InInstructions to just the InInstructions
|
||||
let in_instructions =
|
||||
in_instructions.into_iter().map(|(_id, in_instruction)| in_instruction).collect::<Vec<_>>();
|
||||
let in_instructions = in_instructions
|
||||
.into_iter()
|
||||
.map(|(_id, in_instruction)| in_instruction)
|
||||
.collect::<Vec<_>>();
|
||||
// Send the InInstructions to the report task
|
||||
// We need to also specify which key is responsible for signing the Batch for these, which
|
||||
// will always be the oldest key (as the new key signing the Batch signifies handover
|
||||
|
@ -362,3 +366,4 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for ScanTask<D, S> {
|
|||
Ok(next_to_scan <= latest_scannable)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
use core::marker::PhantomData;
|
||||
use core::{marker::PhantomData, future::Future};
|
||||
|
||||
use serai_db::{DbTxn, Db};
|
||||
|
||||
|
@ -52,9 +52,9 @@ impl<D: Db, S: ScannerFeed> SubstrateTask<D, S> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, S: ScannerFeed> ContinuallyRan for SubstrateTask<D, S> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let mut made_progress = false;
|
||||
loop {
|
||||
// Fetch the next action to handle
|
||||
|
@ -81,8 +81,13 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for SubstrateTask<D, S> {
|
|||
|
||||
{
|
||||
let external_key_for_session_to_sign_batch =
|
||||
report::take_external_key_for_session_to_sign_batch::<S>(&mut txn, batch_id).unwrap();
|
||||
AcknowledgedBatches::send(&mut txn, &external_key_for_session_to_sign_batch, batch_id);
|
||||
report::take_external_key_for_session_to_sign_batch::<S>(&mut txn, batch_id)
|
||||
.unwrap();
|
||||
AcknowledgedBatches::send(
|
||||
&mut txn,
|
||||
&external_key_for_session_to_sign_batch,
|
||||
batch_id,
|
||||
);
|
||||
}
|
||||
|
||||
// Mark we made progress and handle this
|
||||
|
@ -164,3 +169,4 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for SubstrateTask<D, S> {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -14,13 +14,12 @@ all-features = true
|
|||
rustdoc-args = ["--cfg", "docsrs"]
|
||||
|
||||
[package.metadata.cargo-machete]
|
||||
ignored = ["borsh", "scale"]
|
||||
ignored = ["borsh"]
|
||||
|
||||
[lints]
|
||||
workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait = { version = "0.1", default-features = false }
|
||||
rand_core = { version = "0.6", default-features = false }
|
||||
zeroize = { version = "1", default-features = false, features = ["std"] }
|
||||
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use core::future::Future;
|
||||
use std::collections::HashSet;
|
||||
|
||||
use ciphersuite::{group::GroupEncoding, Ristretto};
|
||||
|
@ -75,9 +76,9 @@ impl<D: Db, E: GroupEncoding> BatchSignerTask<D, E> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, E: Send + GroupEncoding> ContinuallyRan for BatchSignerTask<D, E> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let mut iterated = false;
|
||||
|
||||
// Check for new batches to sign
|
||||
|
@ -125,10 +126,10 @@ impl<D: Db, E: Send + GroupEncoding> ContinuallyRan for BatchSignerTask<D, E> {
|
|||
/*
|
||||
We may have yet to register this signing protocol.
|
||||
|
||||
While `BatchesToSign` is populated before `AcknowledgedBatches`, we could theoretically have
|
||||
`BatchesToSign` populated with a new batch _while iterating over `AcknowledgedBatches`_, and
|
||||
then have `AcknowledgedBatched` populated. In that edge case, we will see the
|
||||
acknowledgement notification before we see the transaction.
|
||||
While `BatchesToSign` is populated before `AcknowledgedBatches`, we could theoretically
|
||||
have `BatchesToSign` populated with a new batch _while iterating over
|
||||
`AcknowledgedBatches`_, and then have `AcknowledgedBatched` populated. In that edge case,
|
||||
we will see the acknowledgement notification before we see the transaction.
|
||||
|
||||
In such a case, we break (dropping the txn, re-queueing the acknowledgement notification).
|
||||
On the task's next iteration, we'll process the Batch from `BatchesToSign` and be
|
||||
|
@ -186,3 +187,4 @@ impl<D: Db, E: Send + GroupEncoding> ContinuallyRan for BatchSignerTask<D, E> {
|
|||
Ok(iterated)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,5 @@
|
|||
use core::future::Future;
|
||||
|
||||
use scale::Decode;
|
||||
use serai_db::{DbTxn, Db};
|
||||
|
||||
|
@ -19,9 +21,9 @@ impl<D: Db, C: Coordinator> CoordinatorTask<D, C> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let mut iterated = false;
|
||||
|
||||
for session in RegisteredKeys::get(&self.db).unwrap_or(vec![]) {
|
||||
|
@ -60,7 +62,8 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
|
|||
|
||||
loop {
|
||||
let mut txn = self.db.txn();
|
||||
let Some(msg) = SlashReportSignerToCoordinatorMessages::try_recv(&mut txn, session) else {
|
||||
let Some(msg) = SlashReportSignerToCoordinatorMessages::try_recv(&mut txn, session)
|
||||
else {
|
||||
break;
|
||||
};
|
||||
iterated = true;
|
||||
|
@ -76,7 +79,8 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
|
|||
|
||||
loop {
|
||||
let mut txn = self.db.txn();
|
||||
let Some(msg) = TransactionSignerToCoordinatorMessages::try_recv(&mut txn, session) else {
|
||||
let Some(msg) = TransactionSignerToCoordinatorMessages::try_recv(&mut txn, session)
|
||||
else {
|
||||
break;
|
||||
};
|
||||
iterated = true;
|
||||
|
@ -93,12 +97,17 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
|
|||
// Publish the cosigns from this session
|
||||
{
|
||||
let mut txn = self.db.txn();
|
||||
while let Some(((block_number, block_id), signature)) = Cosign::try_recv(&mut txn, session)
|
||||
while let Some(((block_number, block_id), signature)) =
|
||||
Cosign::try_recv(&mut txn, session)
|
||||
{
|
||||
iterated = true;
|
||||
self
|
||||
.coordinator
|
||||
.publish_cosign(block_number, block_id, <_>::decode(&mut signature.as_slice()).unwrap())
|
||||
.publish_cosign(
|
||||
block_number,
|
||||
block_id,
|
||||
<_>::decode(&mut signature.as_slice()).unwrap(),
|
||||
)
|
||||
.await
|
||||
.map_err(|e| format!("couldn't publish Cosign: {e:?}"))?;
|
||||
}
|
||||
|
@ -165,3 +174,4 @@ impl<D: Db, C: Coordinator> ContinuallyRan for CoordinatorTask<D, C> {
|
|||
Ok(iterated)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,5 @@
|
|||
use core::future::Future;
|
||||
|
||||
use ciphersuite::Ristretto;
|
||||
use frost::dkg::ThresholdKeys;
|
||||
|
||||
|
@ -48,9 +50,9 @@ impl<D: Db> CosignerTask<D> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db> ContinuallyRan for CosignerTask<D> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let mut iterated = false;
|
||||
|
||||
// Check the cosign to work on
|
||||
|
@ -120,3 +122,4 @@ impl<D: Db> ContinuallyRan for CosignerTask<D> {
|
|||
Ok(iterated)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
#![doc = include_str!("../README.md")]
|
||||
#![deny(missing_docs)]
|
||||
|
||||
use core::{fmt::Debug, marker::PhantomData};
|
||||
use core::{future::Future, fmt::Debug, marker::PhantomData};
|
||||
use std::collections::HashMap;
|
||||
|
||||
use zeroize::Zeroizing;
|
||||
|
@ -43,7 +43,6 @@ mod transaction;
|
|||
use transaction::TransactionSignerTask;
|
||||
|
||||
/// A connection to the Coordinator which messages can be published with.
|
||||
#[async_trait::async_trait]
|
||||
pub trait Coordinator: 'static + Send + Sync {
|
||||
/// An error encountered when interacting with a coordinator.
|
||||
///
|
||||
|
@ -52,32 +51,38 @@ pub trait Coordinator: 'static + Send + Sync {
|
|||
type EphemeralError: Debug;
|
||||
|
||||
/// Send a `messages::sign::ProcessorMessage`.
|
||||
async fn send(&mut self, message: ProcessorMessage) -> Result<(), Self::EphemeralError>;
|
||||
fn send(
|
||||
&mut self,
|
||||
message: ProcessorMessage,
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
|
||||
|
||||
/// Publish a cosign.
|
||||
async fn publish_cosign(
|
||||
fn publish_cosign(
|
||||
&mut self,
|
||||
block_number: u64,
|
||||
block_id: [u8; 32],
|
||||
signature: Signature,
|
||||
) -> Result<(), Self::EphemeralError>;
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
|
||||
|
||||
/// Publish a `Batch`.
|
||||
async fn publish_batch(&mut self, batch: Batch) -> Result<(), Self::EphemeralError>;
|
||||
fn publish_batch(&mut self, batch: Batch)
|
||||
-> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
|
||||
|
||||
/// Publish a `SignedBatch`.
|
||||
async fn publish_signed_batch(&mut self, batch: SignedBatch) -> Result<(), Self::EphemeralError>;
|
||||
fn publish_signed_batch(
|
||||
&mut self,
|
||||
batch: SignedBatch,
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
|
||||
|
||||
/// Publish a slash report's signature.
|
||||
async fn publish_slash_report_signature(
|
||||
fn publish_slash_report_signature(
|
||||
&mut self,
|
||||
session: Session,
|
||||
signature: Signature,
|
||||
) -> Result<(), Self::EphemeralError>;
|
||||
) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
|
||||
}
|
||||
|
||||
/// An object capable of publishing a transaction.
|
||||
#[async_trait::async_trait]
|
||||
pub trait TransactionPublisher<T: Transaction>: 'static + Send + Sync + Clone {
|
||||
/// An error encountered when publishing a transaction.
|
||||
///
|
||||
|
@ -92,7 +97,7 @@ pub trait TransactionPublisher<T: Transaction>: 'static + Send + Sync + Clone {
|
|||
///
|
||||
/// The transaction already being present in the mempool/on-chain MUST NOT be considered an
|
||||
/// error.
|
||||
async fn publish(&self, tx: T) -> Result<(), Self::EphemeralError>;
|
||||
fn publish(&self, tx: T) -> impl Send + Future<Output = Result<(), Self::EphemeralError>>;
|
||||
}
|
||||
|
||||
struct Tasks {
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
use core::marker::PhantomData;
|
||||
use core::{marker::PhantomData, future::Future};
|
||||
|
||||
use ciphersuite::Ristretto;
|
||||
use frost::dkg::ThresholdKeys;
|
||||
|
@ -51,9 +51,9 @@ impl<D: Db, S: ScannerFeed> SlashReportSignerTask<D, S> {
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, S: ScannerFeed> ContinuallyRan for SlashReportSignerTask<D, S> {
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
||||
async move {
|
||||
let mut iterated = false;
|
||||
|
||||
// Check for the slash report to sign
|
||||
|
@ -79,7 +79,8 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for SlashReportSignerTask<D, S> {
|
|||
}
|
||||
}
|
||||
let mut txn = self.db.txn();
|
||||
for msg in self.attempt_manager.register(VariantSignId::SlashReport(self.session), machines) {
|
||||
for msg in self.attempt_manager.register(VariantSignId::SlashReport(self.session), machines)
|
||||
{
|
||||
SlashReportSignerToCoordinatorMessages::send(&mut txn, self.session, &msg);
|
||||
}
|
||||
txn.commit();
|
||||
|
@ -118,3 +119,4 @@ impl<D: Db, S: ScannerFeed> ContinuallyRan for SlashReportSignerTask<D, S> {
|
|||
Ok(iterated)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use core::future::Future;
|
||||
use std::{
|
||||
collections::HashSet,
|
||||
time::{Duration, Instant},
|
||||
|
@ -88,11 +89,10 @@ impl<D: Db, ST: SignableTransaction, P: TransactionPublisher<TransactionFor<ST>>
|
|||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<D: Db, ST: SignableTransaction, P: TransactionPublisher<TransactionFor<ST>>> ContinuallyRan
|
||||
for TransactionSignerTask<D, ST, P>
|
||||
{
|
||||
async fn run_iteration(&mut self) -> Result<bool, String> {
|
||||
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {async{
|
||||
let mut iterated = false;
|
||||
|
||||
// Check for new transactions to sign
|
||||
|
@ -233,3 +233,4 @@ impl<D: Db, ST: SignableTransaction, P: TransactionPublisher<TransactionFor<ST>>
|
|||
Ok(iterated)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -71,7 +71,7 @@ pub async fn test_scanner<N: Network>(
|
|||
let block_id = block.id();
|
||||
|
||||
// Verify the Scanner picked them up
|
||||
let verify_event = |mut scanner: ScannerHandle<N, MemDb>| async {
|
||||
let verify_event = |mut scanner: ScannerHandle<N, MemDb>| async move {
|
||||
let outputs =
|
||||
match timeout(Duration::from_secs(30), scanner.events.recv()).await.unwrap().unwrap() {
|
||||
ScannerEvent::Block { is_retirement_block, block, outputs } => {
|
||||
|
|
Loading…
Reference in a new issue