mirror of
https://github.com/serai-dex/serai.git
synced 2025-12-08 20:29:23 +00:00
Replace "coin" with "network"
The Processor's coins folder referred to the networks it could process, as did its Coin trait. This, and other similar cases throughout the codebase, have now been corrected. Also corrects dated documentation for a key pair is confirmed under the validator-sets pallet.
This commit is contained in:
635
processor/src/networks/bitcoin.rs
Normal file
635
processor/src/networks/bitcoin.rs
Normal file
@@ -0,0 +1,635 @@
|
||||
use std::{time::Duration, io, collections::HashMap};
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use transcript::RecommendedTranscript;
|
||||
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,
|
||||
consensus::{Encodable, Decodable},
|
||||
script::Instruction,
|
||||
OutPoint, Transaction, Block, Network as BitcoinNetwork,
|
||||
},
|
||||
wallet::{
|
||||
tweak_keys, address, ReceivedOutput, Scanner, TransactionError,
|
||||
SignableTransaction as BSignableTransaction, TransactionMachine,
|
||||
},
|
||||
rpc::{RpcError, Rpc},
|
||||
};
|
||||
|
||||
#[cfg(test)]
|
||||
use bitcoin_serai::bitcoin::{
|
||||
secp256k1::{SECP256K1, SecretKey, Message},
|
||||
PrivateKey, PublicKey,
|
||||
sighash::{EcdsaSighashType, SighashCache},
|
||||
script::{PushBytesBuf, Builder},
|
||||
absolute::LockTime,
|
||||
Sequence, Script, Witness, TxIn, TxOut, Address as BAddress,
|
||||
};
|
||||
|
||||
use serai_client::{
|
||||
primitives::{MAX_DATA_LEN, Coin as SeraiCoin, NetworkId, Amount, Balance},
|
||||
networks::bitcoin::Address,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
networks::{
|
||||
NetworkError, Block as BlockTrait, OutputType, Output as OutputTrait,
|
||||
Transaction as TransactionTrait, Eventuality, EventualitiesTracker, PostFeeBranch, Network,
|
||||
drop_branches, amortize_fee,
|
||||
},
|
||||
Plan,
|
||||
};
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct OutputId(pub [u8; 36]);
|
||||
impl Default for OutputId {
|
||||
fn default() -> Self {
|
||||
Self([0; 36])
|
||||
}
|
||||
}
|
||||
impl AsRef<[u8]> for OutputId {
|
||||
fn as_ref(&self) -> &[u8] {
|
||||
self.0.as_ref()
|
||||
}
|
||||
}
|
||||
impl AsMut<[u8]> for OutputId {
|
||||
fn as_mut(&mut self) -> &mut [u8] {
|
||||
self.0.as_mut()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct Output {
|
||||
kind: OutputType,
|
||||
output: ReceivedOutput,
|
||||
data: Vec<u8>,
|
||||
}
|
||||
|
||||
impl OutputTrait for Output {
|
||||
type Id = OutputId;
|
||||
|
||||
fn kind(&self) -> OutputType {
|
||||
self.kind
|
||||
}
|
||||
|
||||
fn id(&self) -> Self::Id {
|
||||
let mut res = OutputId::default();
|
||||
self.output.outpoint().consensus_encode(&mut res.as_mut()).unwrap();
|
||||
debug_assert_eq!(
|
||||
{
|
||||
let mut outpoint = vec![];
|
||||
self.output.outpoint().consensus_encode(&mut outpoint).unwrap();
|
||||
outpoint
|
||||
},
|
||||
res.as_ref().to_vec()
|
||||
);
|
||||
res
|
||||
}
|
||||
|
||||
fn balance(&self) -> Balance {
|
||||
Balance { coin: SeraiCoin::Bitcoin, amount: Amount(self.output.value()) }
|
||||
}
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
&self.data
|
||||
}
|
||||
|
||||
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
|
||||
self.kind.write(writer)?;
|
||||
self.output.write(writer)?;
|
||||
writer.write_all(&u16::try_from(self.data.len()).unwrap().to_le_bytes())?;
|
||||
writer.write_all(&self.data)
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
Ok(Output {
|
||||
kind: OutputType::read(reader)?,
|
||||
output: ReceivedOutput::read(reader)?,
|
||||
data: {
|
||||
let mut data_len = [0; 2];
|
||||
reader.read_exact(&mut data_len)?;
|
||||
|
||||
let mut data = vec![0; usize::from(u16::from_le_bytes(data_len))];
|
||||
reader.read_exact(&mut data)?;
|
||||
data
|
||||
},
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
|
||||
pub struct Fee(u64);
|
||||
|
||||
#[async_trait]
|
||||
impl TransactionTrait<Bitcoin> for Transaction {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> Self::Id {
|
||||
let mut hash = *self.txid().as_raw_hash().as_byte_array();
|
||||
hash.reverse();
|
||||
hash
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
let mut buf = vec![];
|
||||
self.consensus_encode(&mut buf).unwrap();
|
||||
buf
|
||||
}
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, network: &Bitcoin) -> u64 {
|
||||
let mut value = 0;
|
||||
for input in &self.input {
|
||||
let output = input.previous_output;
|
||||
let mut hash = *output.txid.as_raw_hash().as_byte_array();
|
||||
hash.reverse();
|
||||
value += network.rpc.get_transaction(&hash).await.unwrap().output
|
||||
[usize::try_from(output.vout).unwrap()]
|
||||
.value;
|
||||
}
|
||||
for output in &self.output {
|
||||
value -= output.value;
|
||||
}
|
||||
value
|
||||
}
|
||||
}
|
||||
|
||||
impl Eventuality for OutPoint {
|
||||
fn lookup(&self) -> Vec<u8> {
|
||||
self.serialize()
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
OutPoint::consensus_decode(reader)
|
||||
.map_err(|_| io::Error::new(io::ErrorKind::Other, "couldn't decode outpoint as eventuality"))
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
let mut buf = Vec::with_capacity(36);
|
||||
self.consensus_encode(&mut buf).unwrap();
|
||||
buf
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct SignableTransaction {
|
||||
keys: ThresholdKeys<Secp256k1>,
|
||||
transcript: RecommendedTranscript,
|
||||
actual: BSignableTransaction,
|
||||
}
|
||||
impl PartialEq for SignableTransaction {
|
||||
fn eq(&self, other: &SignableTransaction) -> bool {
|
||||
self.actual == other.actual
|
||||
}
|
||||
}
|
||||
impl Eq for SignableTransaction {}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
fn time(&self) -> u64 {
|
||||
self.header.time.into()
|
||||
}
|
||||
|
||||
fn median_fee(&self) -> Fee {
|
||||
// TODO
|
||||
Fee(20)
|
||||
}
|
||||
}
|
||||
|
||||
const KEY_DST: &[u8] = b"Bitcoin Key";
|
||||
lazy_static::lazy_static! {
|
||||
static ref BRANCH_OFFSET: Scalar = Secp256k1::hash_to_F(KEY_DST, b"branch");
|
||||
static ref CHANGE_OFFSET: Scalar = Secp256k1::hash_to_F(KEY_DST, b"change");
|
||||
}
|
||||
|
||||
// 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);
|
||||
register(OutputType::Change, *CHANGE_OFFSET);
|
||||
|
||||
(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 {
|
||||
fn eq(&self, _: &Self) -> bool {
|
||||
true
|
||||
}
|
||||
}
|
||||
impl Eq for Bitcoin {}
|
||||
|
||||
impl Bitcoin {
|
||||
pub async fn new(url: String) -> Bitcoin {
|
||||
Bitcoin { rpc: Rpc::new(url).await.expect("couldn't create a Bitcoin RPC") }
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub async fn fresh_chain(&self) {
|
||||
if self.rpc.get_latest_block_number().await.unwrap() > 0 {
|
||||
self
|
||||
.rpc
|
||||
.rpc_call(
|
||||
"invalidateblock",
|
||||
serde_json::json!([hex::encode(self.rpc.get_block_hash(1).await.unwrap())]),
|
||||
)
|
||||
.await
|
||||
.unwrap()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl Network for Bitcoin {
|
||||
type Curve = Secp256k1;
|
||||
|
||||
type Fee = Fee;
|
||||
type Transaction = Transaction;
|
||||
type Block = Block;
|
||||
|
||||
type Output = Output;
|
||||
type SignableTransaction = SignableTransaction;
|
||||
// Valid given an honest multisig, as assumed
|
||||
// Only the multisig can spend this output and the multisig, if spending this output, will
|
||||
// always create a specific plan
|
||||
type Eventuality = OutPoint;
|
||||
type TransactionMachine = TransactionMachine;
|
||||
|
||||
type Address = Address;
|
||||
|
||||
const NETWORK: NetworkId = NetworkId::Bitcoin;
|
||||
const ID: &'static str = "Bitcoin";
|
||||
const CONFIRMATIONS: usize = 6;
|
||||
|
||||
// 0.0001 BTC, 10,000 satoshis
|
||||
#[allow(clippy::inconsistent_digit_grouping)]
|
||||
const DUST: u64 = 1_00_000_000 / 10_000;
|
||||
|
||||
// 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 mis-evaluate it)
|
||||
// It also offers a minimal amount of benefit when we are able to logarithmically accumulate
|
||||
// inputs
|
||||
// For 128-byte inputs (40-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 tweak_keys(keys: &mut ThresholdKeys<Self::Curve>) {
|
||||
*keys = tweak_keys(keys);
|
||||
// Also create a scanner to assert these keys, and all expected paths, are usable
|
||||
scanner(keys.group_key());
|
||||
}
|
||||
|
||||
fn address(key: ProjectivePoint) -> Address {
|
||||
Address(address(BitcoinNetwork::Bitcoin, key).unwrap())
|
||||
}
|
||||
|
||||
fn branch_address(key: ProjectivePoint) -> Self::Address {
|
||||
let (_, offsets, _) = scanner(key);
|
||||
Self::address(key + (ProjectivePoint::GENERATOR * offsets[&OutputType::Branch]))
|
||||
}
|
||||
|
||||
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,
|
||||
) -> Result<Vec<Self::Output>, NetworkError> {
|
||||
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 mut data = if kind == OutputType::External {
|
||||
(|| {
|
||||
for output in &tx.output {
|
||||
if output.script_pubkey.is_op_return() {
|
||||
match output.script_pubkey.instructions_minimal().last() {
|
||||
Some(Ok(Instruction::PushBytes(data))) => return data.as_bytes().to_vec(),
|
||||
_ => continue,
|
||||
}
|
||||
}
|
||||
}
|
||||
vec![]
|
||||
})()
|
||||
} else {
|
||||
vec![]
|
||||
};
|
||||
data.truncate(MAX_DATA_LEN.try_into().unwrap());
|
||||
|
||||
outputs.push(Output { kind, output, data })
|
||||
}
|
||||
}
|
||||
|
||||
Ok(outputs)
|
||||
}
|
||||
|
||||
async fn get_eventuality_completions(
|
||||
&self,
|
||||
eventualities: &mut EventualitiesTracker<OutPoint>,
|
||||
block: &Self::Block,
|
||||
) -> HashMap<[u8; 32], [u8; 32]> {
|
||||
let mut res = HashMap::new();
|
||||
if eventualities.map.is_empty() {
|
||||
return res;
|
||||
}
|
||||
|
||||
async fn check_block(
|
||||
eventualities: &mut EventualitiesTracker<OutPoint>,
|
||||
block: &Block,
|
||||
res: &mut HashMap<[u8; 32], [u8; 32]>,
|
||||
) {
|
||||
for tx in &block.txdata[1 ..] {
|
||||
let input = &tx.input[0].previous_output;
|
||||
if let Some((plan, eventuality)) = eventualities.map.remove(&input.serialize()) {
|
||||
assert_eq!(input, &eventuality);
|
||||
res.insert(plan, tx.id());
|
||||
}
|
||||
}
|
||||
|
||||
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).await;
|
||||
}
|
||||
|
||||
// Also check the current block
|
||||
check_block(eventualities, block, &mut res).await;
|
||||
assert_eq!(eventualities.block_number, this_block_num);
|
||||
|
||||
res
|
||||
}
|
||||
|
||||
async fn prepare_send(
|
||||
&self,
|
||||
keys: ThresholdKeys<Secp256k1>,
|
||||
_: usize,
|
||||
mut plan: Plan<Self>,
|
||||
fee: Fee,
|
||||
) -> Result<(Option<(SignableTransaction, Self::Eventuality)>, Vec<PostFeeBranch>), NetworkError>
|
||||
{
|
||||
let signable = |plan: &Plan<Self>, tx_fee: Option<_>| {
|
||||
let mut payments = vec![];
|
||||
for payment in &plan.payments {
|
||||
// If we're solely estimating the fee, don't specify the actual amount
|
||||
// This won't affect the fee calculation yet will ensure we don't hit a not enough funds
|
||||
// error
|
||||
payments.push((
|
||||
payment.address.0.clone(),
|
||||
if tx_fee.is_none() { Self::DUST } else { payment.amount },
|
||||
));
|
||||
}
|
||||
|
||||
match BSignableTransaction::new(
|
||||
plan.inputs.iter().map(|input| input.output.clone()).collect(),
|
||||
&payments,
|
||||
plan.change.map(|key| {
|
||||
let (_, offsets, _) = scanner(key);
|
||||
Self::address(key + (ProjectivePoint::GENERATOR * offsets[&OutputType::Change])).0
|
||||
}),
|
||||
None,
|
||||
fee.0,
|
||||
) {
|
||||
Ok(signable) => Some(signable),
|
||||
Err(TransactionError::NoInputs) => {
|
||||
panic!("trying to create a bitcoin transaction without inputs")
|
||||
}
|
||||
// No outputs left and the change isn't worth enough
|
||||
Err(TransactionError::NoOutputs) => None,
|
||||
Err(TransactionError::TooMuchData) => panic!("too much data despite not specifying data"),
|
||||
Err(TransactionError::NotEnoughFunds) => {
|
||||
if tx_fee.is_none() {
|
||||
// Mot even enough funds to pay the fee
|
||||
None
|
||||
} else {
|
||||
panic!("not enough funds for bitcoin TX despite amortizing the fee")
|
||||
}
|
||||
}
|
||||
// amortize_fee removes payments which fall below the dust threshold
|
||||
Err(TransactionError::DustPayment) => panic!("dust payment despite removing dust"),
|
||||
Err(TransactionError::TooLargeTransaction) => {
|
||||
panic!("created a too large transaction despite limiting inputs/outputs")
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
let tx_fee = match signable(&plan, None) {
|
||||
Some(tx) => tx.needed_fee(),
|
||||
None => return Ok((None, drop_branches(&plan))),
|
||||
};
|
||||
|
||||
let branch_outputs = amortize_fee(&mut plan, tx_fee);
|
||||
|
||||
Ok((
|
||||
Some((
|
||||
SignableTransaction {
|
||||
keys,
|
||||
transcript: plan.transcript(),
|
||||
actual: signable(&plan, Some(tx_fee)).unwrap(),
|
||||
},
|
||||
*plan.inputs[0].output.outpoint(),
|
||||
)),
|
||||
branch_outputs,
|
||||
))
|
||||
}
|
||||
|
||||
async fn attempt_send(
|
||||
&self,
|
||||
transaction: Self::SignableTransaction,
|
||||
) -> Result<Self::TransactionMachine, NetworkError> {
|
||||
Ok(
|
||||
transaction
|
||||
.actual
|
||||
.clone()
|
||||
.multisig(transaction.keys.clone(), transaction.transcript)
|
||||
.expect("used the wrong keys"),
|
||||
)
|
||||
}
|
||||
|
||||
async fn publish_transaction(&self, tx: &Self::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.txid()),
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn get_transaction(&self, id: &[u8; 32]) -> Result<Transaction, NetworkError> {
|
||||
self.rpc.get_transaction(id).await.map_err(|_| NetworkError::ConnectionError)
|
||||
}
|
||||
|
||||
fn confirm_completion(&self, eventuality: &OutPoint, tx: &Transaction) -> bool {
|
||||
eventuality == &tx.input[0].previous_output
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_block_number(&self, id: &[u8; 32]) -> usize {
|
||||
self.rpc.get_block_number(id).await.unwrap()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_fee(&self) -> Self::Fee {
|
||||
Fee(1)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn mine_block(&self) {
|
||||
self
|
||||
.rpc
|
||||
.rpc_call::<Vec<String>>(
|
||||
"generatetoaddress",
|
||||
serde_json::json!([1, BAddress::p2sh(Script::empty(), BitcoinNetwork::Regtest).unwrap()]),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn test_send(&self, address: Self::Address) -> Block {
|
||||
let secret_key = SecretKey::new(&mut rand_core::OsRng);
|
||||
let private_key = PrivateKey::new(secret_key, BitcoinNetwork::Regtest);
|
||||
let public_key = PublicKey::from_private_key(SECP256K1, &private_key);
|
||||
let main_addr = BAddress::p2pkh(&public_key, BitcoinNetwork::Regtest);
|
||||
|
||||
let new_block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
self
|
||||
.rpc
|
||||
.rpc_call::<Vec<String>>("generatetoaddress", serde_json::json!([1, main_addr]))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
for _ in 0 .. 100 {
|
||||
self.mine_block().await;
|
||||
}
|
||||
|
||||
let tx = self.get_block(new_block).await.unwrap().txdata.swap_remove(0);
|
||||
let mut tx = Transaction {
|
||||
version: 2,
|
||||
lock_time: LockTime::ZERO,
|
||||
input: vec![TxIn {
|
||||
previous_output: OutPoint { txid: tx.txid(), vout: 0 },
|
||||
script_sig: Script::empty().into(),
|
||||
sequence: Sequence(u32::MAX),
|
||||
witness: Witness::default(),
|
||||
}],
|
||||
output: vec![TxOut {
|
||||
value: tx.output[0].value - 10000,
|
||||
script_pubkey: address.0.script_pubkey(),
|
||||
}],
|
||||
};
|
||||
|
||||
let mut der = SECP256K1
|
||||
.sign_ecdsa_low_r(
|
||||
&Message::from(
|
||||
SighashCache::new(&tx)
|
||||
.legacy_signature_hash(0, &main_addr.script_pubkey(), EcdsaSighashType::All.to_u32())
|
||||
.unwrap()
|
||||
.to_raw_hash(),
|
||||
),
|
||||
&private_key.inner,
|
||||
)
|
||||
.serialize_der()
|
||||
.to_vec();
|
||||
der.push(1);
|
||||
tx.input[0].script_sig = Builder::new()
|
||||
.push_slice(PushBytesBuf::try_from(der).unwrap())
|
||||
.push_key(&public_key)
|
||||
.into_script();
|
||||
|
||||
let block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
self.rpc.send_raw_transaction(&tx).await.unwrap();
|
||||
for _ in 0 .. Self::CONFIRMATIONS {
|
||||
self.mine_block().await;
|
||||
}
|
||||
self.get_block(block).await.unwrap()
|
||||
}
|
||||
}
|
||||
394
processor/src/networks/mod.rs
Normal file
394
processor/src/networks/mod.rs
Normal file
@@ -0,0 +1,394 @@
|
||||
use core::fmt::Debug;
|
||||
use std::{io, collections::HashMap};
|
||||
|
||||
use async_trait::async_trait;
|
||||
use thiserror::Error;
|
||||
|
||||
use frost::{
|
||||
curve::{Ciphersuite, Curve},
|
||||
ThresholdKeys,
|
||||
sign::PreprocessMachine,
|
||||
};
|
||||
|
||||
use serai_client::primitives::{NetworkId, Balance};
|
||||
|
||||
#[cfg(feature = "bitcoin")]
|
||||
pub mod bitcoin;
|
||||
#[cfg(feature = "bitcoin")]
|
||||
pub use self::bitcoin::Bitcoin;
|
||||
|
||||
#[cfg(feature = "monero")]
|
||||
pub mod monero;
|
||||
#[cfg(feature = "monero")]
|
||||
pub use monero::Monero;
|
||||
|
||||
use crate::{Payment, Plan};
|
||||
|
||||
#[derive(Clone, Copy, Error, Debug)]
|
||||
pub enum NetworkError {
|
||||
#[error("failed to connect to network daemon")]
|
||||
ConnectionError,
|
||||
}
|
||||
|
||||
pub trait Id:
|
||||
Send + Sync + Clone + Default + PartialEq + AsRef<[u8]> + AsMut<[u8]> + Debug
|
||||
{
|
||||
}
|
||||
impl<I: Send + Sync + Clone + Default + PartialEq + AsRef<[u8]> + AsMut<[u8]> + Debug> Id for I {}
|
||||
|
||||
#[derive(Clone, Copy, PartialEq, Eq, Hash, Debug)]
|
||||
pub enum OutputType {
|
||||
// Needs to be processed/sent up to Substrate
|
||||
External,
|
||||
|
||||
// Given a known output set, and a known series of outbound transactions, we should be able to
|
||||
// form a completely deterministic schedule S. The issue is when S has TXs which spend prior TXs
|
||||
// in S (which is needed for our logarithmic scheduling). In order to have the descendant TX, say
|
||||
// S[1], build off S[0], we need to observe when S[0] is included on-chain.
|
||||
//
|
||||
// We cannot.
|
||||
//
|
||||
// Monero (and other privacy coins) do not expose their UTXO graphs. Even if we know how to
|
||||
// create S[0], and the actual payment info behind it, we cannot observe it on the blockchain
|
||||
// unless we participated in creating it. Locking the entire schedule, when we cannot sign for
|
||||
// the entire schedule at once, to a single signing set isn't feasible.
|
||||
//
|
||||
// While any member of the active signing set can provide data enabling other signers to
|
||||
// participate, it's several KB of data which we then have to code communication for.
|
||||
// The other option is to simply not observe S[0]. Instead, observe a TX with an identical output
|
||||
// to the one in S[0] we intended to use for S[1]. It's either from S[0], or Eve, a malicious
|
||||
// actor, has sent us a forged TX which is... equally as usable? so who cares?
|
||||
//
|
||||
// The only issue is if we have multiple outputs on-chain with identical amounts and purposes.
|
||||
// Accordingly, when the scheduler makes a plan for when a specific output is available, it
|
||||
// shouldn't write that plan. It should *push* that plan to a queue of plans to perform when
|
||||
// instances of that output occur.
|
||||
Branch,
|
||||
|
||||
// Should be added to the available UTXO pool with no further action
|
||||
Change,
|
||||
}
|
||||
|
||||
impl OutputType {
|
||||
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
|
||||
writer.write_all(&[match self {
|
||||
OutputType::External => 0,
|
||||
OutputType::Branch => 1,
|
||||
OutputType::Change => 2,
|
||||
}])
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
let mut byte = [0; 1];
|
||||
reader.read_exact(&mut byte)?;
|
||||
Ok(match byte[0] {
|
||||
0 => OutputType::External,
|
||||
1 => OutputType::Branch,
|
||||
2 => OutputType::Change,
|
||||
_ => Err(io::Error::new(io::ErrorKind::Other, "invalid OutputType"))?,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
pub trait Output: Send + Sync + Sized + Clone + PartialEq + Eq + Debug {
|
||||
type Id: 'static + Id;
|
||||
|
||||
fn kind(&self) -> OutputType;
|
||||
|
||||
fn id(&self) -> Self::Id;
|
||||
|
||||
fn balance(&self) -> Balance;
|
||||
fn amount(&self) -> u64 {
|
||||
self.balance().amount.0
|
||||
}
|
||||
fn data(&self) -> &[u8];
|
||||
|
||||
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()>;
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self>;
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
pub trait Transaction<N: Network>: Send + Sync + Sized + Clone + Debug {
|
||||
type Id: 'static + Id;
|
||||
fn id(&self) -> Self::Id;
|
||||
fn serialize(&self) -> Vec<u8>;
|
||||
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, network: &N) -> u64;
|
||||
}
|
||||
|
||||
pub trait Eventuality: Send + Sync + Clone + Debug {
|
||||
fn lookup(&self) -> Vec<u8>;
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self>;
|
||||
fn serialize(&self) -> Vec<u8>;
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct EventualitiesTracker<E: Eventuality> {
|
||||
// Lookup property (input, nonce, TX extra...) -> (plan ID, eventuality)
|
||||
map: HashMap<Vec<u8>, ([u8; 32], E)>,
|
||||
// Block number we've scanned these eventualities too
|
||||
block_number: usize,
|
||||
}
|
||||
|
||||
impl<E: Eventuality> EventualitiesTracker<E> {
|
||||
pub fn new() -> Self {
|
||||
EventualitiesTracker { map: HashMap::new(), block_number: usize::MAX }
|
||||
}
|
||||
|
||||
pub fn register(&mut self, block_number: usize, id: [u8; 32], eventuality: E) {
|
||||
log::info!("registering eventuality for {}", hex::encode(id));
|
||||
|
||||
let lookup = eventuality.lookup();
|
||||
if self.map.contains_key(&lookup) {
|
||||
panic!("registering an eventuality multiple times or lookup collision");
|
||||
}
|
||||
self.map.insert(lookup, (id, eventuality));
|
||||
// If our self tracker already went past this block number, set it back
|
||||
self.block_number = self.block_number.min(block_number);
|
||||
}
|
||||
|
||||
pub fn drop(&mut self, id: [u8; 32]) {
|
||||
// O(n) due to the lack of a reverse lookup
|
||||
let mut found_key = None;
|
||||
for (key, value) in &self.map {
|
||||
if value.0 == id {
|
||||
found_key = Some(key.clone());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if let Some(key) = found_key {
|
||||
self.map.remove(&key);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<E: Eventuality> Default for EventualitiesTracker<E> {
|
||||
fn default() -> Self {
|
||||
Self::new()
|
||||
}
|
||||
}
|
||||
|
||||
pub trait Block<N: Network>: Send + Sync + Sized + Clone + Debug {
|
||||
// This is currently bounded to being 32-bytes.
|
||||
type Id: 'static + Id;
|
||||
fn id(&self) -> Self::Id;
|
||||
fn parent(&self) -> Self::Id;
|
||||
fn time(&self) -> u64;
|
||||
fn median_fee(&self) -> N::Fee;
|
||||
}
|
||||
|
||||
// The post-fee value of an expected branch.
|
||||
pub struct PostFeeBranch {
|
||||
pub expected: u64,
|
||||
pub actual: Option<u64>,
|
||||
}
|
||||
|
||||
// Return the PostFeeBranches needed when dropping a transaction
|
||||
pub fn drop_branches<N: Network>(plan: &Plan<N>) -> Vec<PostFeeBranch> {
|
||||
let mut branch_outputs = vec![];
|
||||
for payment in &plan.payments {
|
||||
if payment.address == N::branch_address(plan.key) {
|
||||
branch_outputs.push(PostFeeBranch { expected: payment.amount, actual: None });
|
||||
}
|
||||
}
|
||||
branch_outputs
|
||||
}
|
||||
|
||||
// Amortize a fee over the plan's payments
|
||||
pub fn amortize_fee<N: Network>(plan: &mut Plan<N>, tx_fee: u64) -> Vec<PostFeeBranch> {
|
||||
// No payments to amortize over
|
||||
if plan.payments.is_empty() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
let original_outputs = plan.payments.iter().map(|payment| payment.amount).sum::<u64>();
|
||||
|
||||
// Amortize the transaction fee across outputs
|
||||
let mut payments_len = u64::try_from(plan.payments.len()).unwrap();
|
||||
// Use a formula which will round up
|
||||
let per_output_fee = |payments| (tx_fee + (payments - 1)) / payments;
|
||||
|
||||
let post_fee = |payment: &Payment<N>, per_output_fee| {
|
||||
let mut post_fee = payment.amount.checked_sub(per_output_fee);
|
||||
// If this is under our dust threshold, drop it
|
||||
if let Some(amount) = post_fee {
|
||||
if amount < N::DUST {
|
||||
post_fee = None;
|
||||
}
|
||||
}
|
||||
post_fee
|
||||
};
|
||||
|
||||
// If we drop outputs for being less than the fee, we won't successfully reduce the amount spent
|
||||
// (dropping a 800 output due to a 1000 fee leaves 200 we still have to deduct)
|
||||
// Do initial runs until the amount of output we will drop is known
|
||||
while {
|
||||
let last = payments_len;
|
||||
payments_len = u64::try_from(
|
||||
plan
|
||||
.payments
|
||||
.iter()
|
||||
.filter(|payment| post_fee(payment, per_output_fee(payments_len)).is_some())
|
||||
.count(),
|
||||
)
|
||||
.unwrap();
|
||||
last != payments_len
|
||||
} {}
|
||||
|
||||
// Now that we know how many outputs will survive, calculate the actual per_output_fee
|
||||
let per_output_fee = per_output_fee(payments_len);
|
||||
let mut branch_outputs = vec![];
|
||||
for payment in plan.payments.iter_mut() {
|
||||
let post_fee = post_fee(payment, per_output_fee);
|
||||
// Note the branch output, if this is one
|
||||
if payment.address == N::branch_address(plan.key) {
|
||||
branch_outputs.push(PostFeeBranch { expected: payment.amount, actual: post_fee });
|
||||
}
|
||||
payment.amount = post_fee.unwrap_or(0);
|
||||
}
|
||||
// Drop payments now worth 0
|
||||
plan.payments = plan.payments.drain(..).filter(|payment| payment.amount != 0).collect();
|
||||
|
||||
// Sanity check the fee wa successfully amortized
|
||||
let new_outputs = plan.payments.iter().map(|payment| payment.amount).sum::<u64>();
|
||||
assert!((new_outputs + tx_fee) <= original_outputs);
|
||||
|
||||
branch_outputs
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
pub trait Network: 'static + Send + Sync + Clone + PartialEq + Eq + Debug {
|
||||
/// The elliptic curve used for this network.
|
||||
type Curve: Curve;
|
||||
|
||||
/// The type representing the fee for this network.
|
||||
// This should likely be a u64, wrapped in a type which implements appropriate fee logic.
|
||||
type Fee: Copy;
|
||||
|
||||
/// The type representing the transaction for this network.
|
||||
type Transaction: Transaction<Self>;
|
||||
/// The type representing the block for this network.
|
||||
type Block: Block<Self>;
|
||||
|
||||
/// The type containing all information on a scanned output.
|
||||
// This is almost certainly distinct from the network's native output type.
|
||||
type Output: Output;
|
||||
/// The type containing all information on a planned transaction, waiting to be signed.
|
||||
type SignableTransaction: Send + Sync + Clone + Debug;
|
||||
/// The type containing all information to check if a plan was completed.
|
||||
type Eventuality: Eventuality;
|
||||
/// The FROST machine to sign a transaction.
|
||||
type TransactionMachine: PreprocessMachine<Signature = Self::Transaction>;
|
||||
|
||||
/// The type representing an address.
|
||||
// This should NOT be a String, yet a tailored type representing an efficient binary encoding,
|
||||
// as detailed in the integration documentation.
|
||||
type Address: Send
|
||||
+ Sync
|
||||
+ Clone
|
||||
+ PartialEq
|
||||
+ Eq
|
||||
+ Debug
|
||||
+ ToString
|
||||
+ TryInto<Vec<u8>>
|
||||
+ TryFrom<Vec<u8>>;
|
||||
|
||||
/// Network ID for this network.
|
||||
const NETWORK: NetworkId;
|
||||
/// String ID for this network.
|
||||
const ID: &'static str;
|
||||
/// The amount of confirmations required to consider a block 'final'.
|
||||
const CONFIRMATIONS: usize;
|
||||
/// The maximum amount of inputs which will fit in a TX.
|
||||
/// This should be equal to MAX_OUTPUTS unless one is specifically limited.
|
||||
/// A TX with MAX_INPUTS and MAX_OUTPUTS must not exceed the max size.
|
||||
const MAX_INPUTS: usize;
|
||||
/// The maximum amount of outputs which will fit in a TX.
|
||||
/// This should be equal to MAX_INPUTS unless one is specifically limited.
|
||||
/// A TX with MAX_INPUTS and MAX_OUTPUTS must not exceed the max size.
|
||||
const MAX_OUTPUTS: usize;
|
||||
|
||||
/// Minimum output value which will be handled.
|
||||
const DUST: u64;
|
||||
|
||||
/// Tweak keys for this network.
|
||||
fn tweak_keys(key: &mut ThresholdKeys<Self::Curve>);
|
||||
|
||||
/// Address for the given group key to receive external coins to.
|
||||
fn address(key: <Self::Curve as Ciphersuite>::G) -> Self::Address;
|
||||
/// Address for the given group key to use for scheduled branches.
|
||||
// This is purely used for debugging purposes. Any output may be used to execute a branch.
|
||||
fn branch_address(key: <Self::Curve as Ciphersuite>::G) -> Self::Address;
|
||||
|
||||
/// Get the latest block's number.
|
||||
async fn get_latest_block_number(&self) -> Result<usize, NetworkError>;
|
||||
/// Get a block by its number.
|
||||
async fn get_block(&self, number: usize) -> Result<Self::Block, NetworkError>;
|
||||
/// Get the outputs within a block for a specific key.
|
||||
async fn get_outputs(
|
||||
&self,
|
||||
block: &Self::Block,
|
||||
key: <Self::Curve as Ciphersuite>::G,
|
||||
) -> Result<Vec<Self::Output>, NetworkError>;
|
||||
|
||||
/// Get the registered eventualities completed within this block, and any prior blocks which
|
||||
/// registered eventualities may have been completed in.
|
||||
async fn get_eventuality_completions(
|
||||
&self,
|
||||
eventualities: &mut EventualitiesTracker<Self::Eventuality>,
|
||||
block: &Self::Block,
|
||||
) -> HashMap<[u8; 32], <Self::Transaction as Transaction<Self>>::Id>;
|
||||
|
||||
/// Prepare a SignableTransaction for a transaction.
|
||||
/// Returns None for the transaction if the SignableTransaction was dropped due to lack of value.
|
||||
#[rustfmt::skip]
|
||||
async fn prepare_send(
|
||||
&self,
|
||||
keys: ThresholdKeys<Self::Curve>,
|
||||
block_number: usize,
|
||||
plan: Plan<Self>,
|
||||
fee: Self::Fee,
|
||||
) -> Result<
|
||||
(Option<(Self::SignableTransaction, Self::Eventuality)>, Vec<PostFeeBranch>),
|
||||
NetworkError
|
||||
>;
|
||||
|
||||
/// Attempt to sign a SignableTransaction.
|
||||
async fn attempt_send(
|
||||
&self,
|
||||
transaction: Self::SignableTransaction,
|
||||
) -> Result<Self::TransactionMachine, NetworkError>;
|
||||
|
||||
/// Publish a transaction.
|
||||
async fn publish_transaction(&self, tx: &Self::Transaction) -> Result<(), NetworkError>;
|
||||
|
||||
/// Get a transaction by its ID.
|
||||
async fn get_transaction(
|
||||
&self,
|
||||
id: &<Self::Transaction as Transaction<Self>>::Id,
|
||||
) -> Result<Self::Transaction, NetworkError>;
|
||||
|
||||
/// Confirm a plan was completed by the specified transaction.
|
||||
// This is allowed to take shortcuts.
|
||||
// This may assume an honest multisig, solely checking the inputs specified were spent.
|
||||
// This may solely check the outputs are equivalent *so long as it's locked to the plan ID*.
|
||||
fn confirm_completion(&self, eventuality: &Self::Eventuality, tx: &Self::Transaction) -> bool;
|
||||
|
||||
/// Get a block's number by its ID.
|
||||
#[cfg(test)]
|
||||
async fn get_block_number(&self, id: &<Self::Block as Block<Self>>::Id) -> usize;
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_fee(&self) -> Self::Fee;
|
||||
|
||||
#[cfg(test)]
|
||||
async fn mine_block(&self);
|
||||
|
||||
/// Sends to the specified address.
|
||||
/// Additionally mines enough blocks so that the TX is past the confirmation depth.
|
||||
#[cfg(test)]
|
||||
async fn test_send(&self, key: Self::Address) -> Self::Block;
|
||||
}
|
||||
643
processor/src/networks/monero.rs
Normal file
643
processor/src/networks/monero.rs
Normal file
@@ -0,0 +1,643 @@
|
||||
use std::{time::Duration, collections::HashMap, io};
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use zeroize::Zeroizing;
|
||||
|
||||
use rand_core::SeedableRng;
|
||||
use rand_chacha::ChaCha20Rng;
|
||||
|
||||
use transcript::{Transcript, RecommendedTranscript};
|
||||
|
||||
use ciphersuite::group::{ff::Field, Group};
|
||||
use dalek_ff_group::{Scalar, EdwardsPoint};
|
||||
use frost::{curve::Ed25519, ThresholdKeys};
|
||||
|
||||
use monero_serai::{
|
||||
Protocol,
|
||||
transaction::Transaction,
|
||||
block::Block,
|
||||
rpc::{RpcError, HttpRpc, Rpc},
|
||||
wallet::{
|
||||
ViewPair, Scanner,
|
||||
address::{Network as MoneroNetwork, SubaddressIndex, AddressSpec},
|
||||
Fee, SpendableOutput, Change, Decoys, TransactionError,
|
||||
SignableTransaction as MSignableTransaction, Eventuality, TransactionMachine,
|
||||
},
|
||||
};
|
||||
|
||||
use tokio::time::sleep;
|
||||
|
||||
pub use serai_client::{
|
||||
primitives::{MAX_DATA_LEN, Coin as SeraiCoin, NetworkId, Amount, Balance},
|
||||
networks::monero::Address,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
Payment, Plan, additional_key,
|
||||
networks::{
|
||||
NetworkError, Block as BlockTrait, OutputType, Output as OutputTrait,
|
||||
Transaction as TransactionTrait, Eventuality as EventualityTrait, EventualitiesTracker,
|
||||
PostFeeBranch, Network, drop_branches, amortize_fee,
|
||||
},
|
||||
};
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct Output(SpendableOutput, Vec<u8>);
|
||||
|
||||
const EXTERNAL_SUBADDRESS: Option<SubaddressIndex> = SubaddressIndex::new(0, 0);
|
||||
const BRANCH_SUBADDRESS: Option<SubaddressIndex> = SubaddressIndex::new(1, 0);
|
||||
const CHANGE_SUBADDRESS: Option<SubaddressIndex> = SubaddressIndex::new(2, 0);
|
||||
|
||||
impl OutputTrait for Output {
|
||||
// While we could use (tx, o), using the key ensures we won't be susceptible to the burning bug.
|
||||
// While we already are immune, thanks to using featured address, this doesn't hurt and is
|
||||
// technically more efficient.
|
||||
type Id = [u8; 32];
|
||||
|
||||
fn kind(&self) -> OutputType {
|
||||
match self.0.output.metadata.subaddress {
|
||||
EXTERNAL_SUBADDRESS => OutputType::External,
|
||||
BRANCH_SUBADDRESS => OutputType::Branch,
|
||||
CHANGE_SUBADDRESS => OutputType::Change,
|
||||
_ => panic!("unrecognized address was scanned for"),
|
||||
}
|
||||
}
|
||||
|
||||
fn id(&self) -> Self::Id {
|
||||
self.0.output.data.key.compress().to_bytes()
|
||||
}
|
||||
|
||||
fn balance(&self) -> Balance {
|
||||
Balance { coin: SeraiCoin::Monero, amount: Amount(self.0.commitment().amount) }
|
||||
}
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
&self.1
|
||||
}
|
||||
|
||||
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
|
||||
self.0.write(writer)?;
|
||||
writer.write_all(&u16::try_from(self.1.len()).unwrap().to_le_bytes())?;
|
||||
writer.write_all(&self.1)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
let output = SpendableOutput::read(reader)?;
|
||||
|
||||
let mut data_len = [0; 2];
|
||||
reader.read_exact(&mut data_len)?;
|
||||
|
||||
let mut data = vec![0; usize::from(u16::from_le_bytes(data_len))];
|
||||
reader.read_exact(&mut data)?;
|
||||
|
||||
Ok(Output(output, data))
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl TransactionTrait<Monero> for Transaction {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> Self::Id {
|
||||
self.hash()
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
self.serialize()
|
||||
}
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, _: &Monero) -> u64 {
|
||||
self.rct_signatures.base.fee
|
||||
}
|
||||
}
|
||||
|
||||
impl EventualityTrait for Eventuality {
|
||||
// Use the TX extra to look up potential matches
|
||||
// While anyone can forge this, a transaction with distinct outputs won't actually match
|
||||
// Extra includess the one time keys which are derived from the plan ID, so a collision here is a
|
||||
// hash collision
|
||||
fn lookup(&self) -> Vec<u8> {
|
||||
self.extra().to_vec()
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
Eventuality::read(reader)
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
self.serialize()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct SignableTransaction {
|
||||
keys: ThresholdKeys<Ed25519>,
|
||||
transcript: RecommendedTranscript,
|
||||
actual: MSignableTransaction,
|
||||
}
|
||||
|
||||
impl BlockTrait<Monero> for Block {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> Self::Id {
|
||||
self.hash()
|
||||
}
|
||||
|
||||
fn parent(&self) -> Self::Id {
|
||||
self.header.previous
|
||||
}
|
||||
|
||||
fn time(&self) -> u64 {
|
||||
self.header.timestamp
|
||||
}
|
||||
|
||||
fn median_fee(&self) -> Fee {
|
||||
// TODO
|
||||
Fee { per_weight: 10000000, mask: 10000 }
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct Monero {
|
||||
rpc: Rpc<HttpRpc>,
|
||||
}
|
||||
// Shim required for testing/debugging purposes due to generic arguments also necessitating trait
|
||||
// bounds
|
||||
impl PartialEq for Monero {
|
||||
fn eq(&self, _: &Self) -> bool {
|
||||
true
|
||||
}
|
||||
}
|
||||
impl Eq for Monero {}
|
||||
|
||||
impl Monero {
|
||||
pub fn new(url: String) -> Monero {
|
||||
Monero { rpc: HttpRpc::new(url).unwrap() }
|
||||
}
|
||||
|
||||
fn view_pair(spend: EdwardsPoint) -> ViewPair {
|
||||
ViewPair::new(spend.0, Zeroizing::new(additional_key::<Monero>(0).0))
|
||||
}
|
||||
|
||||
fn address_internal(spend: EdwardsPoint, subaddress: Option<SubaddressIndex>) -> Address {
|
||||
Address::new(Self::view_pair(spend).address(
|
||||
MoneroNetwork::Mainnet,
|
||||
AddressSpec::Featured { subaddress, payment_id: None, guaranteed: true },
|
||||
))
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
fn scanner(spend: EdwardsPoint) -> Scanner {
|
||||
let mut scanner = Scanner::from_view(Self::view_pair(spend), None);
|
||||
debug_assert!(EXTERNAL_SUBADDRESS.is_none());
|
||||
scanner.register_subaddress(BRANCH_SUBADDRESS.unwrap());
|
||||
scanner.register_subaddress(CHANGE_SUBADDRESS.unwrap());
|
||||
scanner
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn test_view_pair() -> ViewPair {
|
||||
ViewPair::new(*EdwardsPoint::generator(), Zeroizing::new(Scalar::ONE.0))
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn test_scanner() -> Scanner {
|
||||
Scanner::from_view(Self::test_view_pair(), Some(std::collections::HashSet::new()))
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn test_address() -> Address {
|
||||
Address::new(Self::test_view_pair().address(MoneroNetwork::Mainnet, AddressSpec::Standard))
|
||||
.unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl Network for Monero {
|
||||
type Curve = Ed25519;
|
||||
|
||||
type Fee = Fee;
|
||||
type Transaction = Transaction;
|
||||
type Block = Block;
|
||||
|
||||
type Output = Output;
|
||||
type SignableTransaction = SignableTransaction;
|
||||
type Eventuality = Eventuality;
|
||||
type TransactionMachine = TransactionMachine;
|
||||
|
||||
type Address = Address;
|
||||
|
||||
const NETWORK: NetworkId = NetworkId::Monero;
|
||||
const ID: &'static str = "Monero";
|
||||
const CONFIRMATIONS: usize = 10;
|
||||
|
||||
// wallet2 will not create a transaction larger than 100kb, and Monero won't relay a transaction
|
||||
// larger than 150kb. This fits within the 100kb mark
|
||||
// Technically, it can be ~124, yet a small bit of buffer is appreciated
|
||||
// TODO: Test creating a TX this big
|
||||
const MAX_INPUTS: usize = 120;
|
||||
const MAX_OUTPUTS: usize = 16;
|
||||
|
||||
// 0.01 XMR
|
||||
const DUST: u64 = 10000000000;
|
||||
|
||||
// Monero doesn't require/benefit from tweaking
|
||||
fn tweak_keys(_: &mut ThresholdKeys<Self::Curve>) {}
|
||||
|
||||
fn address(key: EdwardsPoint) -> Self::Address {
|
||||
Self::address_internal(key, EXTERNAL_SUBADDRESS)
|
||||
}
|
||||
|
||||
fn branch_address(key: EdwardsPoint) -> Self::Address {
|
||||
Self::address_internal(key, BRANCH_SUBADDRESS)
|
||||
}
|
||||
|
||||
async fn get_latest_block_number(&self) -> Result<usize, NetworkError> {
|
||||
// Monero defines height as chain length, so subtract 1 for block number
|
||||
Ok(self.rpc.get_height().await.map_err(|_| NetworkError::ConnectionError)? - 1)
|
||||
}
|
||||
|
||||
async fn get_block(&self, number: usize) -> Result<Self::Block, NetworkError> {
|
||||
Ok(
|
||||
self
|
||||
.rpc
|
||||
.get_block(
|
||||
self.rpc.get_block_hash(number).await.map_err(|_| NetworkError::ConnectionError)?,
|
||||
)
|
||||
.await
|
||||
.map_err(|_| NetworkError::ConnectionError)?,
|
||||
)
|
||||
}
|
||||
|
||||
async fn get_outputs(
|
||||
&self,
|
||||
block: &Block,
|
||||
key: EdwardsPoint,
|
||||
) -> Result<Vec<Self::Output>, NetworkError> {
|
||||
let mut txs = Self::scanner(key)
|
||||
.scan(&self.rpc, block)
|
||||
.await
|
||||
.map_err(|_| NetworkError::ConnectionError)?
|
||||
.iter()
|
||||
.filter_map(|outputs| Some(outputs.not_locked()).filter(|outputs| !outputs.is_empty()))
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
// This should be pointless as we shouldn't be able to scan for any other subaddress
|
||||
// This just ensures nothing invalid makes it through
|
||||
for tx_outputs in &txs {
|
||||
for output in tx_outputs {
|
||||
assert!([EXTERNAL_SUBADDRESS, BRANCH_SUBADDRESS, CHANGE_SUBADDRESS]
|
||||
.contains(&output.output.metadata.subaddress));
|
||||
}
|
||||
}
|
||||
|
||||
let mut outputs = Vec::with_capacity(txs.len());
|
||||
for mut tx_outputs in txs.drain(..) {
|
||||
for output in tx_outputs.drain(..) {
|
||||
let mut data = output.arbitrary_data().get(0).cloned().unwrap_or(vec![]);
|
||||
|
||||
// The Output serialization code above uses u16 to represent length
|
||||
data.truncate(u16::MAX.into());
|
||||
// Monero data segments should be <= 255 already, and MAX_DATA_LEN is currently 512
|
||||
// This just allows either Monero to change, or MAX_DATA_LEN to change, without introducing
|
||||
// complicationso
|
||||
data.truncate(MAX_DATA_LEN.try_into().unwrap());
|
||||
|
||||
outputs.push(Output(output, data));
|
||||
}
|
||||
}
|
||||
|
||||
Ok(outputs)
|
||||
}
|
||||
|
||||
async fn get_eventuality_completions(
|
||||
&self,
|
||||
eventualities: &mut EventualitiesTracker<Eventuality>,
|
||||
block: &Block,
|
||||
) -> HashMap<[u8; 32], [u8; 32]> {
|
||||
let mut res = HashMap::new();
|
||||
if eventualities.map.is_empty() {
|
||||
return res;
|
||||
}
|
||||
|
||||
async fn check_block(
|
||||
network: &Monero,
|
||||
eventualities: &mut EventualitiesTracker<Eventuality>,
|
||||
block: &Block,
|
||||
res: &mut HashMap<[u8; 32], [u8; 32]>,
|
||||
) {
|
||||
for hash in &block.txs {
|
||||
let tx = {
|
||||
let mut tx;
|
||||
while {
|
||||
tx = network.get_transaction(hash).await;
|
||||
tx.is_err()
|
||||
} {
|
||||
log::error!("couldn't get transaction {}: {}", hex::encode(hash), tx.err().unwrap());
|
||||
sleep(Duration::from_secs(60)).await;
|
||||
}
|
||||
tx.unwrap()
|
||||
};
|
||||
|
||||
if let Some((_, eventuality)) = eventualities.map.get(&tx.prefix.extra) {
|
||||
if eventuality.matches(&tx) {
|
||||
res.insert(eventualities.map.remove(&tx.prefix.extra).unwrap().0, tx.hash());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
eventualities.block_number += 1;
|
||||
assert_eq!(eventualities.block_number, block.number());
|
||||
}
|
||||
|
||||
for block_num in (eventualities.block_number + 1) .. block.number() {
|
||||
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(self, eventualities, &block, &mut res).await;
|
||||
}
|
||||
|
||||
// Also check the current block
|
||||
check_block(self, eventualities, block, &mut res).await;
|
||||
assert_eq!(eventualities.block_number, block.number());
|
||||
|
||||
res
|
||||
}
|
||||
|
||||
async fn prepare_send(
|
||||
&self,
|
||||
keys: ThresholdKeys<Ed25519>,
|
||||
block_number: usize,
|
||||
mut plan: Plan<Self>,
|
||||
fee: Fee,
|
||||
) -> Result<(Option<(SignableTransaction, Eventuality)>, Vec<PostFeeBranch>), NetworkError> {
|
||||
// Sanity check this has at least one output planned
|
||||
assert!((!plan.payments.is_empty()) || plan.change.is_some());
|
||||
|
||||
// Get the protocol for the specified block number
|
||||
// For now, this should just be v16, the latest deployed protocol, since there's no upcoming
|
||||
// hard fork to be mindful of
|
||||
let get_protocol = || Protocol::v16;
|
||||
|
||||
#[cfg(not(test))]
|
||||
let protocol = get_protocol();
|
||||
// If this is a test, we won't be using a mainnet node and need a distinct protocol
|
||||
// determination
|
||||
// Just use whatever the node expects
|
||||
#[cfg(test)]
|
||||
let protocol = self.rpc.get_protocol().await.unwrap();
|
||||
|
||||
// Hedge against the above codegen failing by having an always included runtime check
|
||||
if !cfg!(test) {
|
||||
assert_eq!(protocol, get_protocol());
|
||||
}
|
||||
|
||||
// Check a fork hasn't occurred which this processor hasn't been updated for
|
||||
assert_eq!(protocol, self.rpc.get_protocol().await.map_err(|_| NetworkError::ConnectionError)?);
|
||||
|
||||
let spendable_outputs = plan.inputs.iter().cloned().map(|input| input.0).collect::<Vec<_>>();
|
||||
|
||||
let mut transcript = plan.transcript();
|
||||
|
||||
// All signers need to select the same decoys
|
||||
// All signers use the same height and a seeded RNG to make sure they do so.
|
||||
let decoys = Decoys::select(
|
||||
&mut ChaCha20Rng::from_seed(transcript.rng_seed(b"decoys")),
|
||||
&self.rpc,
|
||||
protocol.ring_len(),
|
||||
block_number + 1,
|
||||
&spendable_outputs,
|
||||
)
|
||||
.await
|
||||
.map_err(|_| NetworkError::ConnectionError)
|
||||
.unwrap();
|
||||
|
||||
let inputs = spendable_outputs.into_iter().zip(decoys.into_iter()).collect::<Vec<_>>();
|
||||
|
||||
let signable = |mut plan: Plan<Self>, tx_fee: Option<_>| {
|
||||
// Monero requires at least two outputs
|
||||
// If we only have one output planned, add a dummy payment
|
||||
let outputs = plan.payments.len() + usize::from(u8::from(plan.change.is_some()));
|
||||
if outputs == 0 {
|
||||
return Ok(None);
|
||||
} else if outputs == 1 {
|
||||
plan.payments.push(Payment {
|
||||
address: Address::new(
|
||||
ViewPair::new(EdwardsPoint::generator().0, Zeroizing::new(Scalar::ONE.0))
|
||||
.address(MoneroNetwork::Mainnet, AddressSpec::Standard),
|
||||
)
|
||||
.unwrap(),
|
||||
amount: 0,
|
||||
data: None,
|
||||
});
|
||||
}
|
||||
|
||||
let mut payments = vec![];
|
||||
for payment in &plan.payments {
|
||||
// If we're solely estimating the fee, don't actually specify an amount
|
||||
// This won't affect the fee calculation yet will ensure we don't hit an out of funds error
|
||||
payments.push((
|
||||
payment.address.clone().into(),
|
||||
if tx_fee.is_none() { 0 } else { payment.amount },
|
||||
));
|
||||
}
|
||||
|
||||
match MSignableTransaction::new(
|
||||
protocol,
|
||||
// Use the plan ID as the r_seed
|
||||
// This perfectly binds the plan while simultaneously allowing verifying the plan was
|
||||
// executed with no additional communication
|
||||
Some(Zeroizing::new(plan.id())),
|
||||
inputs.clone(),
|
||||
payments,
|
||||
plan.change.map(|key| {
|
||||
Change::fingerprintable(Self::address_internal(key, CHANGE_SUBADDRESS).into())
|
||||
}),
|
||||
vec![],
|
||||
fee,
|
||||
) {
|
||||
Ok(signable) => Ok(Some(signable)),
|
||||
Err(e) => match e {
|
||||
TransactionError::MultiplePaymentIds => {
|
||||
panic!("multiple payment IDs despite not supporting integrated addresses");
|
||||
}
|
||||
TransactionError::NoInputs |
|
||||
TransactionError::NoOutputs |
|
||||
TransactionError::InvalidDecoyQuantity |
|
||||
TransactionError::NoChange |
|
||||
TransactionError::TooManyOutputs |
|
||||
TransactionError::TooMuchData |
|
||||
TransactionError::TooLargeTransaction |
|
||||
TransactionError::WrongPrivateKey => {
|
||||
panic!("created an Monero invalid transaction: {e}");
|
||||
}
|
||||
TransactionError::ClsagError(_) |
|
||||
TransactionError::InvalidTransaction(_) |
|
||||
TransactionError::FrostError(_) => {
|
||||
panic!("supposedly unreachable (at this time) Monero error: {e}");
|
||||
}
|
||||
TransactionError::NotEnoughFunds { inputs, outputs, fee } => {
|
||||
if let Some(tx_fee) = tx_fee {
|
||||
panic!(
|
||||
"{}. in: {inputs}, out: {outputs}, fee: {fee}, prior estimated fee: {tx_fee}",
|
||||
"didn't have enough funds for a Monero TX",
|
||||
);
|
||||
} else {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
TransactionError::RpcError(e) => {
|
||||
log::error!("RpcError when preparing transaction: {e:?}");
|
||||
Err(NetworkError::ConnectionError)
|
||||
}
|
||||
},
|
||||
}
|
||||
};
|
||||
|
||||
let tx_fee = match signable(plan.clone(), None)? {
|
||||
Some(tx) => tx.fee(),
|
||||
None => return Ok((None, drop_branches(&plan))),
|
||||
};
|
||||
|
||||
let branch_outputs = amortize_fee(&mut plan, tx_fee);
|
||||
|
||||
let signable = SignableTransaction {
|
||||
keys,
|
||||
transcript,
|
||||
actual: match signable(plan, Some(tx_fee))? {
|
||||
Some(signable) => signable,
|
||||
None => return Ok((None, branch_outputs)),
|
||||
},
|
||||
};
|
||||
let eventuality = signable.actual.eventuality().unwrap();
|
||||
Ok((Some((signable, eventuality)), branch_outputs))
|
||||
}
|
||||
|
||||
async fn attempt_send(
|
||||
&self,
|
||||
transaction: SignableTransaction,
|
||||
) -> Result<Self::TransactionMachine, NetworkError> {
|
||||
match transaction.actual.clone().multisig(transaction.keys.clone(), transaction.transcript) {
|
||||
Ok(machine) => Ok(machine),
|
||||
Err(e) => panic!("failed to create a multisig machine for TX: {e}"),
|
||||
}
|
||||
}
|
||||
|
||||
async fn publish_transaction(&self, tx: &Self::Transaction) -> Result<(), NetworkError> {
|
||||
match self.rpc.publish_transaction(tx).await {
|
||||
Ok(_) => 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}", hex::encode(tx.hash())),
|
||||
}
|
||||
}
|
||||
|
||||
async fn get_transaction(&self, id: &[u8; 32]) -> Result<Transaction, NetworkError> {
|
||||
self.rpc.get_transaction(*id).await.map_err(|_| NetworkError::ConnectionError)
|
||||
}
|
||||
|
||||
fn confirm_completion(&self, eventuality: &Eventuality, tx: &Transaction) -> bool {
|
||||
eventuality.matches(tx)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_block_number(&self, id: &[u8; 32]) -> usize {
|
||||
self.rpc.get_block(*id).await.unwrap().number()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_fee(&self) -> Self::Fee {
|
||||
use monero_serai::wallet::FeePriority;
|
||||
|
||||
self.rpc.get_fee(self.rpc.get_protocol().await.unwrap(), FeePriority::Low).await.unwrap()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn mine_block(&self) {
|
||||
// https://github.com/serai-dex/serai/issues/198
|
||||
sleep(std::time::Duration::from_millis(100)).await;
|
||||
|
||||
#[derive(serde::Deserialize, Debug)]
|
||||
struct EmptyResponse {}
|
||||
let _: EmptyResponse = self
|
||||
.rpc
|
||||
.rpc_call(
|
||||
"json_rpc",
|
||||
Some(serde_json::json!({
|
||||
"method": "generateblocks",
|
||||
"params": {
|
||||
"wallet_address": Self::test_address().to_string(),
|
||||
"amount_of_blocks": 1
|
||||
},
|
||||
})),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn test_send(&self, address: Self::Address) -> Block {
|
||||
use zeroize::Zeroizing;
|
||||
use rand_core::OsRng;
|
||||
use monero_serai::wallet::FeePriority;
|
||||
|
||||
let new_block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
for _ in 0 .. 80 {
|
||||
self.mine_block().await;
|
||||
}
|
||||
|
||||
let outputs = Self::test_scanner()
|
||||
.scan(&self.rpc, &self.rpc.get_block_by_number(new_block).await.unwrap())
|
||||
.await
|
||||
.unwrap()
|
||||
.swap_remove(0)
|
||||
.ignore_timelock();
|
||||
|
||||
let amount = outputs[0].commitment().amount;
|
||||
// The dust should always be sufficient for the fee
|
||||
let fee = Monero::DUST;
|
||||
|
||||
let protocol = self.rpc.get_protocol().await.unwrap();
|
||||
|
||||
let decoys = Decoys::select(
|
||||
&mut OsRng,
|
||||
&self.rpc,
|
||||
protocol.ring_len(),
|
||||
self.rpc.get_height().await.unwrap() - 1,
|
||||
&outputs,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let inputs = outputs.into_iter().zip(decoys.into_iter()).collect::<Vec<_>>();
|
||||
|
||||
let tx = MSignableTransaction::new(
|
||||
protocol,
|
||||
None,
|
||||
inputs,
|
||||
vec![(address.into(), amount - fee)],
|
||||
Some(Change::fingerprintable(Self::test_address().into())),
|
||||
vec![],
|
||||
self.rpc.get_fee(protocol, FeePriority::Low).await.unwrap(),
|
||||
)
|
||||
.unwrap()
|
||||
.sign(&mut OsRng, &Zeroizing::new(Scalar::ONE.0))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
self.rpc.publish_transaction(&tx).await.unwrap();
|
||||
for _ in 0 .. 10 {
|
||||
self.mine_block().await;
|
||||
}
|
||||
self.get_block(block).await.unwrap()
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user