Satisfy Scheduler for Bitcoin

This commit is contained in:
Luke Parker
2024-09-11 00:01:40 -04:00
parent ba3a6f9e91
commit 017aab2258
13 changed files with 245 additions and 357 deletions

View File

@@ -45,6 +45,8 @@ messages = { package = "serai-processor-messages", path = "../messages" }
primitives = { package = "serai-processor-primitives", path = "../primitives" }
scheduler = { package = "serai-processor-scheduler-primitives", path = "../scheduler/primitives" }
scanner = { package = "serai-processor-scanner", path = "../scanner" }
utxo-scheduler = { package = "serai-processor-utxo-scheduler-primitives", path = "../scheduler/utxo/primitives" }
transaction-chaining-scheduler = { package = "serai-processor-transaction-chaining-scheduler", path = "../scheduler/utxo/transaction-chaining" }
message-queue = { package = "serai-message-queue", path = "../../message-queue" }

View File

@@ -9,15 +9,14 @@ static ALLOCATOR: zalloc::ZeroizingAlloc<std::alloc::System> =
// Internal utilities for scanning transactions
mod scan;
// Output trait satisfaction
// Primitive trait satisfactions
mod output;
// Transaction/SignableTransaction/Eventuality trait satisfaction
mod transaction;
// Block trait satisfaction
mod block;
// ScannerFeed trait satisfaction
// App-logic trait satisfactions
mod scanner_feed;
mod scheduler;
pub(crate) fn hash_bytes(hash: bitcoin_serai::bitcoin::hashes::sha256d::Hash) -> [u8; 32] {
use bitcoin_serai::bitcoin::hashes::Hash;
@@ -28,21 +27,6 @@ pub(crate) fn hash_bytes(hash: bitcoin_serai::bitcoin::hashes::sha256d::Hash) ->
}
/*
use std::{sync::LazyLock, time::Duration, io, collections::HashMap};
use async_trait::async_trait;
use scale::{Encode, Decode};
use ciphersuite::group::ff::PrimeField;
use k256::{ProjectivePoint, Scalar};
use frost::{
curve::{Curve, Secp256k1},
ThresholdKeys,
};
use tokio::time::sleep;
use bitcoin_serai::{
bitcoin::{
hashes::Hash as HashTrait,
@@ -111,19 +95,6 @@ impl TransactionTrait<Bitcoin> for Transaction {
#[async_trait]
impl BlockTrait<Bitcoin> for Block {
type Id = [u8; 32];
fn id(&self) -> Self::Id {
let mut hash = *self.block_hash().as_raw_hash().as_byte_array();
hash.reverse();
hash
}
fn parent(&self) -> Self::Id {
let mut hash = *self.header.prev_blockhash.as_raw_hash().as_byte_array();
hash.reverse();
hash
}
async fn time(&self, rpc: &Bitcoin) -> u64 {
// Use the network median time defined in BIP-0113 since the in-block time isn't guaranteed to
// be monotonic
@@ -152,51 +123,6 @@ impl BlockTrait<Bitcoin> for Block {
}
}
const KEY_DST: &[u8] = b"Serai Bitcoin Output Offset";
static BRANCH_OFFSET: OnceLock<Scalar> = OnceLock::new();
static CHANGE_OFFSET: OnceLock<Scalar> = OnceLock::new();
static FORWARD_OFFSET: OnceLock<Scalar> = OnceLock::new();
// Always construct the full scanner in order to ensure there's no collisions
fn scanner(
key: ProjectivePoint,
) -> (Scanner, HashMap<OutputType, Scalar>, HashMap<Vec<u8>, OutputType>) {
let mut scanner = Scanner::new(key).unwrap();
let mut offsets = HashMap::from([(OutputType::External, Scalar::ZERO)]);
let zero = Scalar::ZERO.to_repr();
let zero_ref: &[u8] = zero.as_ref();
let mut kinds = HashMap::from([(zero_ref.to_vec(), OutputType::External)]);
let mut register = |kind, offset| {
let offset = scanner.register_offset(offset).expect("offset collision");
offsets.insert(kind, offset);
let offset = offset.to_repr();
let offset_ref: &[u8] = offset.as_ref();
kinds.insert(offset_ref.to_vec(), kind);
};
register(
OutputType::Branch,
*BRANCH_OFFSET.get_or_init(|| Secp256k1::hash_to_F(KEY_DST, b"branch")),
);
register(
OutputType::Change,
*CHANGE_OFFSET.get_or_init(|| Secp256k1::hash_to_F(KEY_DST, b"change")),
);
register(
OutputType::Forwarded,
*FORWARD_OFFSET.get_or_init(|| Secp256k1::hash_to_F(KEY_DST, b"forward")),
);
(scanner, offsets, kinds)
}
#[derive(Clone, Debug)]
pub struct Bitcoin {
pub(crate) rpc: Rpc,
}
// Shim required for testing/debugging purposes due to generic arguments also necessitating trait
// bounds
impl PartialEq for Bitcoin {
@@ -355,20 +281,6 @@ impl Bitcoin {
}
}
// Bitcoin has a max weight of 400,000 (MAX_STANDARD_TX_WEIGHT)
// A non-SegWit TX will have 4 weight units per byte, leaving a max size of 100,000 bytes
// While our inputs are entirely SegWit, such fine tuning is not necessary and could create
// issues in the future (if the size decreases or we misevaluate it)
// It also offers a minimal amount of benefit when we are able to logarithmically accumulate
// inputs
// For 128-byte inputs (36-byte output specification, 64-byte signature, whatever overhead) and
// 64-byte outputs (40-byte script, 8-byte amount, whatever overhead), they together take up 192
// bytes
// 100,000 / 192 = 520
// 520 * 192 leaves 160 bytes of overhead for the transaction structure itself
const MAX_INPUTS: usize = 520;
const MAX_OUTPUTS: usize = 520;
fn address_from_key(key: ProjectivePoint) -> Address {
Address::new(
p2tr_script_buf(key).expect("creating address from key which isn't properly tweaked"),
@@ -378,59 +290,8 @@ fn address_from_key(key: ProjectivePoint) -> Address {
#[async_trait]
impl Network for Bitcoin {
type Curve = Secp256k1;
type Transaction = Transaction;
type Block = Block;
type Output = Output;
type SignableTransaction = SignableTransaction;
type Eventuality = Eventuality;
type TransactionMachine = TransactionMachine;
type Scheduler = Scheduler<Bitcoin>;
type Address = Address;
const NETWORK: NetworkId = NetworkId::Bitcoin;
const ID: &'static str = "Bitcoin";
const ESTIMATED_BLOCK_TIME_IN_SECONDS: usize = 600;
const CONFIRMATIONS: usize = 6;
/*
A Taproot input is:
- 36 bytes for the OutPoint
- 0 bytes for the script (+1 byte for the length)
- 4 bytes for the sequence
Per https://developer.bitcoin.org/reference/transactions.html#raw-transaction-format
There's also:
- 1 byte for the witness length
- 1 byte for the signature length
- 64 bytes for the signature
which have the SegWit discount.
(4 * (36 + 1 + 4)) + (1 + 1 + 64) = 164 + 66 = 230 weight units
230 ceil div 4 = 57 vbytes
Bitcoin defines multiple minimum feerate constants *per kilo-vbyte*. Currently, these are:
- 1000 sat/kilo-vbyte for a transaction to be relayed
- Each output's value must exceed the fee of the TX spending it at 3000 sat/kilo-vbyte
The DUST constant needs to be determined by the latter.
Since these are solely relay rules, and may be raised, we require all outputs be spendable
under a 5000 sat/kilo-vbyte fee rate.
5000 sat/kilo-vbyte = 5 sat/vbyte
5 * 57 = 285 sats/spent-output
Even if an output took 100 bytes (it should be just ~29-43), taking 400 weight units, adding
100 vbytes, tripling the transaction size, then the sats/tx would be < 1000.
Increase by an order of magnitude, in order to ensure this is actually worth our time, and we
get 10,000 satoshis.
*/
const DUST: u64 = 10_000;
// 2 inputs should be 2 * 230 = 460 weight units
// The output should be ~36 bytes, or 144 weight units
// The overhead should be ~20 bytes at most, or 80 weight units
@@ -467,195 +328,6 @@ impl Network for Bitcoin {
Some(address_from_key(key + (ProjectivePoint::GENERATOR * offsets[&OutputType::Forwarded])))
}
async fn get_latest_block_number(&self) -> Result<usize, NetworkError> {
self.rpc.get_latest_block_number().await.map_err(|_| NetworkError::ConnectionError)
}
async fn get_block(&self, number: usize) -> Result<Self::Block, NetworkError> {
let block_hash =
self.rpc.get_block_hash(number).await.map_err(|_| NetworkError::ConnectionError)?;
self.rpc.get_block(&block_hash).await.map_err(|_| NetworkError::ConnectionError)
}
async fn get_outputs(&self, block: &Self::Block, key: ProjectivePoint) -> Vec<Output> {
let (scanner, _, kinds) = scanner(key);
let mut outputs = vec![];
// Skip the coinbase transaction which is burdened by maturity
for tx in &block.txdata[1 ..] {
for output in scanner.scan_transaction(tx) {
let offset_repr = output.offset().to_repr();
let offset_repr_ref: &[u8] = offset_repr.as_ref();
let kind = kinds[offset_repr_ref];
let output = Output { kind, presumed_origin: None, output, data: vec![] };
assert_eq!(output.tx_id(), tx.id());
outputs.push(output);
}
if outputs.is_empty() {
continue;
}
// populate the outputs with the origin and data
let presumed_origin = {
// This may identify the P2WSH output *embedding the InInstruction* as the origin, which
// would be a bit trickier to spend that a traditional output...
// There's no risk of the InInstruction going missing as it'd already be on-chain though
// We *could* parse out the script *without the InInstruction prefix* and declare that the
// origin
// TODO
let spent_output = {
let input = &tx.input[0];
let mut spent_tx = input.previous_output.txid.as_raw_hash().to_byte_array();
spent_tx.reverse();
let mut tx;
while {
tx = self.rpc.get_transaction(&spent_tx).await;
tx.is_err()
} {
log::error!("couldn't get transaction from bitcoin node: {tx:?}");
sleep(Duration::from_secs(5)).await;
}
tx.unwrap().output.swap_remove(usize::try_from(input.previous_output.vout).unwrap())
};
Address::new(spent_output.script_pubkey)
};
let data = Self::extract_serai_data(tx);
for output in &mut outputs {
if output.kind == OutputType::External {
output.data.clone_from(&data);
}
output.presumed_origin.clone_from(&presumed_origin);
}
}
outputs
}
async fn get_eventuality_completions(
&self,
eventualities: &mut EventualitiesTracker<Eventuality>,
block: &Self::Block,
) -> HashMap<[u8; 32], (usize, [u8; 32], Transaction)> {
let mut res = HashMap::new();
if eventualities.map.is_empty() {
return res;
}
fn check_block(
eventualities: &mut EventualitiesTracker<Eventuality>,
block: &Block,
res: &mut HashMap<[u8; 32], (usize, [u8; 32], Transaction)>,
) {
for tx in &block.txdata[1 ..] {
if let Some((plan, _)) = eventualities.map.remove(tx.id().as_slice()) {
res.insert(plan, (eventualities.block_number, tx.id(), tx.clone()));
}
}
eventualities.block_number += 1;
}
let this_block_hash = block.id();
let this_block_num = (async {
loop {
match self.rpc.get_block_number(&this_block_hash).await {
Ok(number) => return number,
Err(e) => {
log::error!("couldn't get the block number for {}: {}", hex::encode(this_block_hash), e)
}
}
sleep(Duration::from_secs(60)).await;
}
})
.await;
for block_num in (eventualities.block_number + 1) .. this_block_num {
let block = {
let mut block;
while {
block = self.get_block(block_num).await;
block.is_err()
} {
log::error!("couldn't get block {}: {}", block_num, block.err().unwrap());
sleep(Duration::from_secs(60)).await;
}
block.unwrap()
};
check_block(eventualities, &block, &mut res);
}
// Also check the current block
check_block(eventualities, block, &mut res);
assert_eq!(eventualities.block_number, this_block_num);
res
}
async fn needed_fee(
&self,
block_number: usize,
inputs: &[Output],
payments: &[Payment<Self>],
change: &Option<Address>,
) -> Result<Option<u64>, NetworkError> {
Ok(
self
.make_signable_transaction(block_number, inputs, payments, change, true)
.await?
.map(|signable| signable.needed_fee()),
)
}
async fn signable_transaction(
&self,
block_number: usize,
_plan_id: &[u8; 32],
_key: ProjectivePoint,
inputs: &[Output],
payments: &[Payment<Self>],
change: &Option<Address>,
(): &(),
) -> Result<Option<(Self::SignableTransaction, Self::Eventuality)>, NetworkError> {
Ok(self.make_signable_transaction(block_number, inputs, payments, change, false).await?.map(
|signable| {
let eventuality = Eventuality(signable.txid());
(SignableTransaction { actual: signable }, eventuality)
},
))
}
async fn attempt_sign(
&self,
keys: ThresholdKeys<Self::Curve>,
transaction: Self::SignableTransaction,
) -> Result<Self::TransactionMachine, NetworkError> {
Ok(transaction.actual.clone().multisig(&keys).expect("used the wrong keys"))
}
async fn publish_completion(&self, tx: &Transaction) -> Result<(), NetworkError> {
match self.rpc.send_raw_transaction(tx).await {
Ok(_) => (),
Err(RpcError::ConnectionError) => Err(NetworkError::ConnectionError)?,
// TODO: Distinguish already in pool vs double spend (other signing attempt succeeded) vs
// invalid transaction
Err(e) => panic!("failed to publish TX {}: {e}", tx.compute_txid()),
}
Ok(())
}
async fn confirm_completion(
&self,
eventuality: &Self::Eventuality,
_: &EmptyClaim,
) -> Result<Option<Transaction>, NetworkError> {
Ok(Some(
self.rpc.get_transaction(&eventuality.0).await.map_err(|_| NetworkError::ConnectionError)?,
))
}
#[cfg(test)]
async fn get_block_number(&self, id: &[u8; 32]) -> usize {
self.rpc.get_block_number(id).await.unwrap()

View File

@@ -47,7 +47,7 @@ impl AsMut<[u8]> for OutputId {
pub(crate) struct Output {
kind: OutputType,
presumed_origin: Option<Address>,
output: WalletOutput,
pub(crate) output: WalletOutput,
data: Vec<u8>,
}

View File

@@ -46,7 +46,39 @@ impl ScannerFeed for Rpc {
fn dust(coin: Coin) -> Amount {
assert_eq!(coin, Coin::Bitcoin);
// 10,000 satoshis, or $5 if 1 BTC = 50,000 USD
/*
A Taproot input is:
- 36 bytes for the OutPoint
- 0 bytes for the script (+1 byte for the length)
- 4 bytes for the sequence
Per https://developer.bitcoin.org/reference/transactions.html#raw-transaction-format
There's also:
- 1 byte for the witness length
- 1 byte for the signature length
- 64 bytes for the signature
which have the SegWit discount.
(4 * (36 + 1 + 4)) + (1 + 1 + 64) = 164 + 66 = 230 weight units
230 ceil div 4 = 57 vbytes
Bitcoin defines multiple minimum feerate constants *per kilo-vbyte*. Currently, these are:
- 1000 sat/kilo-vbyte for a transaction to be relayed
- Each output's value must exceed the fee of the TX spending it at 3000 sat/kilo-vbyte
The DUST constant needs to be determined by the latter.
Since these are solely relay rules, and may be raised, we require all outputs be spendable
under a 5000 sat/kilo-vbyte fee rate.
5000 sat/kilo-vbyte = 5 sat/vbyte
5 * 57 = 285 sats/spent-output
Even if an output took 100 bytes (it should be just ~29-43), taking 400 weight units, adding
100 vbytes, tripling the transaction size, then the sats/tx would be < 1000.
Increase by an order of magnitude, in order to ensure this is actually worth our time, and we
get 10,000 satoshis. This is $5 if 1 BTC = 50,000 USD.
*/
Amount(10_000)
}

View File

@@ -0,0 +1,177 @@
use ciphersuite::{Ciphersuite, Secp256k1};
use bitcoin_serai::{
bitcoin::ScriptBuf,
wallet::{TransactionError, SignableTransaction as BSignableTransaction, p2tr_script_buf},
};
use serai_client::{
primitives::{Coin, Amount},
networks::bitcoin::Address,
};
use primitives::{OutputType, ReceivedOutput, Payment};
use scanner::{KeyFor, AddressFor, OutputFor, BlockFor};
use utxo_scheduler::{PlannedTransaction, TransactionPlanner};
use transaction_chaining_scheduler::{EffectedReceivedOutputs, Scheduler as GenericScheduler};
use crate::{
scan::{offsets_for_key, scanner},
output::Output,
transaction::{SignableTransaction, Eventuality},
scanner_feed::Rpc,
};
fn address_from_serai_key(key: <Secp256k1 as Ciphersuite>::G, kind: OutputType) -> Address {
let offset = <Secp256k1 as Ciphersuite>::G::GENERATOR * offsets_for_key(key)[&kind];
Address::new(
p2tr_script_buf(key + offset)
.expect("creating address from Serai key which wasn't properly tweaked"),
)
.expect("couldn't create Serai-representable address for P2TR script")
}
fn signable_transaction(
fee_per_vbyte: u64,
inputs: Vec<OutputFor<Rpc>>,
payments: Vec<Payment<AddressFor<Rpc>>>,
change: Option<KeyFor<Rpc>>,
) -> Result<(SignableTransaction, BSignableTransaction), TransactionError> {
assert!(inputs.len() < Planner::MAX_INPUTS);
assert!((payments.len() + usize::from(u8::from(change.is_some()))) < Planner::MAX_OUTPUTS);
let inputs = inputs.into_iter().map(|input| input.output).collect::<Vec<_>>();
let payments = payments
.into_iter()
.map(|payment| {
(payment.address().clone(), {
let balance = payment.balance();
assert_eq!(balance.coin, Coin::Bitcoin);
balance.amount.0
})
})
.collect::<Vec<_>>();
let change = change.map(Planner::change_address);
// TODO: ACP output
BSignableTransaction::new(
inputs.clone(),
&payments
.iter()
.cloned()
.map(|(address, amount)| (ScriptBuf::from(address), amount))
.collect::<Vec<_>>(),
change.clone().map(ScriptBuf::from),
None,
fee_per_vbyte,
)
.map(|bst| (SignableTransaction { inputs, payments, change, fee_per_vbyte }, bst))
}
pub(crate) struct Planner;
impl TransactionPlanner<Rpc, EffectedReceivedOutputs<Rpc>> for Planner {
type FeeRate = u64;
type SignableTransaction = SignableTransaction;
/*
Bitcoin has a max weight of 400,000 (MAX_STANDARD_TX_WEIGHT).
A non-SegWit TX will have 4 weight units per byte, leaving a max size of 100,000 bytes. While
our inputs are entirely SegWit, such fine tuning is not necessary and could create issues in
the future (if the size decreases or we misevaluate it). It also offers a minimal amount of
benefit when we are able to logarithmically accumulate inputs/fulfill payments.
For 128-byte inputs (36-byte output specification, 64-byte signature, whatever overhead) and
64-byte outputs (40-byte script, 8-byte amount, whatever overhead), they together take up 192
bytes.
100,000 / 192 = 520
520 * 192 leaves 160 bytes of overhead for the transaction structure itself.
*/
const MAX_INPUTS: usize = 520;
// We always reserve one output to create an anyone-can-spend output enabling anyone to use CPFP
// to unstick any transactions which had too low of a fee.
const MAX_OUTPUTS: usize = 519;
fn fee_rate(block: &BlockFor<Rpc>, coin: Coin) -> Self::FeeRate {
assert_eq!(coin, Coin::Bitcoin);
// TODO
1
}
fn branch_address(key: KeyFor<Rpc>) -> AddressFor<Rpc> {
address_from_serai_key(key, OutputType::Branch)
}
fn change_address(key: KeyFor<Rpc>) -> AddressFor<Rpc> {
address_from_serai_key(key, OutputType::Change)
}
fn forwarding_address(key: KeyFor<Rpc>) -> AddressFor<Rpc> {
address_from_serai_key(key, OutputType::Forwarded)
}
fn calculate_fee(
fee_rate: Self::FeeRate,
inputs: Vec<OutputFor<Rpc>>,
payments: Vec<Payment<AddressFor<Rpc>>>,
change: Option<KeyFor<Rpc>>,
) -> Amount {
match signable_transaction(fee_rate, inputs, payments, change) {
Ok(tx) => Amount(tx.1.needed_fee()),
Err(
TransactionError::NoInputs | TransactionError::NoOutputs | TransactionError::DustPayment,
) => panic!("malformed arguments to calculate_fee"),
// No data, we have a minimum fee rate, we checked the amount of inputs/outputs
Err(
TransactionError::TooMuchData |
TransactionError::TooLowFee |
TransactionError::TooLargeTransaction,
) => unreachable!(),
Err(TransactionError::NotEnoughFunds { fee, .. }) => Amount(fee),
}
}
fn plan(
fee_rate: Self::FeeRate,
inputs: Vec<OutputFor<Rpc>>,
payments: Vec<Payment<AddressFor<Rpc>>>,
change: Option<KeyFor<Rpc>>,
) -> PlannedTransaction<Rpc, Self::SignableTransaction, EffectedReceivedOutputs<Rpc>> {
let key = inputs.first().unwrap().key();
for input in &inputs {
assert_eq!(key, input.key());
}
let singular_spent_output = (inputs.len() == 1).then(|| inputs[0].id());
match signable_transaction(fee_rate, inputs, payments, change) {
Ok(tx) => PlannedTransaction {
signable: tx.0,
eventuality: Eventuality { txid: tx.1.txid(), singular_spent_output },
auxilliary: EffectedReceivedOutputs({
let tx = tx.1.transaction();
let scanner = scanner(key);
let mut res = vec![];
for output in scanner.scan_transaction(tx) {
res.push(Output::new(key, tx, output));
}
res
}),
},
Err(
TransactionError::NoInputs | TransactionError::NoOutputs | TransactionError::DustPayment,
) => panic!("malformed arguments to plan"),
// No data, we have a minimum fee rate, we checked the amount of inputs/outputs
Err(
TransactionError::TooMuchData |
TransactionError::TooLowFee |
TransactionError::TooLargeTransaction,
) => unreachable!(),
Err(TransactionError::NotEnoughFunds { .. }) => {
panic!("plan called for a transaction without enough funds")
}
}
}
}
pub(crate) type Scheduler = GenericScheduler<Rpc, Planner>;

View File

@@ -48,11 +48,10 @@ impl scheduler::Transaction for Transaction {
#[derive(Clone, Debug)]
pub(crate) struct SignableTransaction {
inputs: Vec<ReceivedOutput>,
payments: Vec<(Address, u64)>,
change: Option<Address>,
data: Option<Vec<u8>>,
fee_per_vbyte: u64,
pub(crate) inputs: Vec<ReceivedOutput>,
pub(crate) payments: Vec<(Address, u64)>,
pub(crate) change: Option<Address>,
pub(crate) fee_per_vbyte: u64,
}
impl SignableTransaction {
@@ -66,7 +65,7 @@ impl SignableTransaction {
.map(|(address, amount)| (ScriptBuf::from(address), amount))
.collect::<Vec<_>>(),
self.change.map(ScriptBuf::from),
self.data,
None,
self.fee_per_vbyte,
)
}
@@ -111,10 +110,9 @@ impl scheduler::SignableTransaction for SignableTransaction {
let payments = <_>::deserialize_reader(reader)?;
let change = <_>::deserialize_reader(reader)?;
let data = <_>::deserialize_reader(reader)?;
let fee_per_vbyte = <_>::deserialize_reader(reader)?;
Ok(Self { inputs, payments, change, data, fee_per_vbyte })
Ok(Self { inputs, payments, change, fee_per_vbyte })
}
fn write(&self, writer: &mut impl io::Write) -> io::Result<()> {
writer.write_all(&u32::try_from(self.inputs.len()).unwrap().to_le_bytes())?;
@@ -124,7 +122,6 @@ impl scheduler::SignableTransaction for SignableTransaction {
self.payments.serialize(writer)?;
self.change.serialize(writer)?;
self.data.serialize(writer)?;
self.fee_per_vbyte.serialize(writer)?;
Ok(())
@@ -141,8 +138,8 @@ impl scheduler::SignableTransaction for SignableTransaction {
#[derive(Clone, PartialEq, Eq, Debug, BorshSerialize, BorshDeserialize)]
pub(crate) struct Eventuality {
txid: [u8; 32],
singular_spent_output: Option<OutputId>,
pub(crate) txid: [u8; 32],
pub(crate) singular_spent_output: Option<OutputId>,
}
impl primitives::Eventuality for Eventuality {

View File

@@ -17,8 +17,6 @@ rustdoc-args = ["--cfg", "docsrs"]
workspace = true
[dependencies]
async-trait = { version = "0.1", default-features = false }
borsh = { version = "1", default-features = false, features = ["std", "derive", "de_strict_order"] }
serai-primitives = { path = "../../../../substrate/primitives", default-features = false, features = ["std"] }

View File

@@ -39,7 +39,6 @@ pub struct AmortizePlannedTransaction<S: ScannerFeed, ST: SignableTransaction, A
}
/// An object able to plan a transaction.
#[async_trait::async_trait]
pub trait TransactionPlanner<S: ScannerFeed, A>: 'static + Send + Sync {
/// The type representing a fee rate to use for transactions.
type FeeRate: Clone + Copy;

View File

@@ -203,6 +203,9 @@ impl<S: ScannerFeed, P: TransactionPlanner<S, ()>> Scheduler<S, P> {
// Fetch the operating costs/outputs
let mut operating_costs = Db::<S>::operating_costs(txn, coin).0;
let outputs = Db::<S>::outputs(txn, key, coin).unwrap();
if outputs.is_empty() {
continue;
}
// Fetch the fulfillable payments
let payments = Self::fulfillable_payments(

View File

@@ -23,7 +23,7 @@ mod db;
use db::Db;
/// The outputs which will be effected by a PlannedTransaction and received by Serai.
pub struct EffectedReceivedOutputs<S: ScannerFeed>(Vec<OutputFor<S>>);
pub struct EffectedReceivedOutputs<S: ScannerFeed>(pub Vec<OutputFor<S>>);
/// A scheduler of transactions for networks premised on the UTXO model which support
/// transaction chaining.
@@ -179,6 +179,9 @@ impl<S: ScannerFeed, P: TransactionPlanner<S, EffectedReceivedOutputs<S>>> Sched
// Fetch the operating costs/outputs
let mut operating_costs = Db::<S>::operating_costs(txn, coin).0;
let outputs = Db::<S>::outputs(txn, key, coin).unwrap();
if outputs.is_empty() {
continue;
}
// Fetch the fulfillable payments
let payments = Self::fulfillable_payments(