Merge branch 'develop' into crypto-tweaks

This commit is contained in:
Luke Parker
2023-03-16 16:43:04 -04:00
committed by GitHub
173 changed files with 29638 additions and 3517 deletions

View File

@@ -0,0 +1,318 @@
use std::{io, collections::HashMap};
use async_trait::async_trait;
#[rustfmt::skip]
use bitcoin::{
hashes::Hash, schnorr::TweakedPublicKey, OutPoint, Transaction, Block, Network, Address
};
#[cfg(test)]
use bitcoin::{
secp256k1::{SECP256K1, SecretKey, Message},
PrivateKey, PublicKey, EcdsaSighashType,
blockdata::script::Builder,
PackedLockTime, Sequence, Script, Witness, TxIn, TxOut,
};
use transcript::RecommendedTranscript;
use k256::{
ProjectivePoint, Scalar,
elliptic_curve::sec1::{ToEncodedPoint, Tag},
};
use frost::{curve::Secp256k1, ThresholdKeys};
use bitcoin_serai::{
crypto::{x_only, make_even},
wallet::{SpendableOutput, TransactionMachine, SignableTransaction as BSignableTransaction},
rpc::Rpc,
};
use crate::coin::{CoinError, Block as BlockTrait, OutputType, Output as OutputTrait, Coin};
impl BlockTrait for Block {
type Id = [u8; 32];
fn id(&self) -> Self::Id {
self.block_hash().as_hash().into_inner()
}
}
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
pub struct Fee(u64);
#[derive(Clone, Debug)]
pub struct Output(SpendableOutput, OutputType);
impl OutputTrait for Output {
type Id = [u8; 36];
fn kind(&self) -> OutputType {
self.1
}
fn id(&self) -> Self::Id {
self.0.id()
}
fn amount(&self) -> u64 {
self.0.output.value
}
fn serialize(&self) -> Vec<u8> {
let mut res = self.0.serialize();
self.1.write(&mut res).unwrap();
res
}
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
Ok(Output(SpendableOutput::read(reader)?, OutputType::read(reader)?))
}
}
#[derive(Debug)]
pub struct SignableTransaction {
keys: ThresholdKeys<Secp256k1>,
transcript: RecommendedTranscript,
actual: BSignableTransaction,
}
fn next_key(mut key: ProjectivePoint, i: usize) -> (ProjectivePoint, Scalar) {
let mut offset = Scalar::ZERO;
for _ in 0 .. i {
key += ProjectivePoint::GENERATOR;
offset += Scalar::ONE;
let even_offset;
(key, even_offset) = make_even(key);
offset += Scalar::from(even_offset);
}
(key, offset)
}
fn branch(key: ProjectivePoint) -> (ProjectivePoint, Scalar) {
next_key(key, 1)
}
fn change(key: ProjectivePoint) -> (ProjectivePoint, Scalar) {
next_key(key, 2)
}
#[derive(Clone, Debug)]
pub struct Bitcoin {
pub(crate) rpc: Rpc,
}
impl Bitcoin {
pub async fn new(url: String) -> Bitcoin {
Bitcoin { rpc: Rpc::new(url) }
}
#[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!([self.rpc.get_block_hash(1).await.unwrap()]))
.await
.unwrap()
}
}
}
#[async_trait]
impl Coin for Bitcoin {
type Curve = Secp256k1;
type Fee = Fee;
type Transaction = Transaction;
type Block = Block;
type Output = Output;
type SignableTransaction = SignableTransaction;
type TransactionMachine = TransactionMachine;
type Address = Address;
const ID: &'static [u8] = b"Bitcoin";
const CONFIRMATIONS: usize = 3;
// TODO: Get hard numbers and tune
const MAX_INPUTS: usize = 128;
const MAX_OUTPUTS: usize = 16;
fn tweak_keys(&self, key: &mut ThresholdKeys<Self::Curve>) {
let (_, offset) = make_even(key.group_key());
*key = key.offset(Scalar::from(offset));
}
fn address(&self, key: ProjectivePoint) -> Self::Address {
debug_assert!(key.to_encoded_point(true).tag() == Tag::CompressedEvenY, "YKey is odd");
Address::p2tr_tweaked(
TweakedPublicKey::dangerous_assume_tweaked(x_only(&key)),
Network::Regtest,
)
}
fn branch_address(&self, key: ProjectivePoint) -> Self::Address {
self.address(branch(key).0)
}
async fn get_latest_block_number(&self) -> Result<usize, CoinError> {
Ok(self.rpc.get_latest_block_number().await.map_err(|_| CoinError::ConnectionError)?)
}
async fn get_block(&self, number: usize) -> Result<Self::Block, CoinError> {
let block_hash =
self.rpc.get_block_hash(number).await.map_err(|_| CoinError::ConnectionError)?;
self.rpc.get_block(&block_hash).await.map_err(|_| CoinError::ConnectionError)
}
async fn get_outputs(
&self,
block: &Self::Block,
key: ProjectivePoint,
) -> Result<Vec<Self::Output>, CoinError> {
let external = (key, Scalar::ZERO);
let branch = branch(key);
let change = change(key);
let entry =
|pair: (_, _), kind| (self.address(pair.0).script_pubkey().to_bytes(), (pair.1, kind));
let scripts = HashMap::from([
entry(external, OutputType::External),
entry(branch, OutputType::Branch),
entry(change, OutputType::Change),
]);
let mut outputs = Vec::new();
// Skip the coinbase transaction which is burdened by maturity
for tx in &block.txdata[1 ..] {
for (vout, output) in tx.output.iter().enumerate() {
if let Some(info) = scripts.get(&output.script_pubkey.to_bytes()) {
outputs.push(Output(
SpendableOutput {
offset: info.0,
output: output.clone(),
outpoint: OutPoint { txid: tx.txid(), vout: u32::try_from(vout).unwrap() },
},
info.1,
));
}
}
}
Ok(outputs)
}
async fn prepare_send(
&self,
keys: ThresholdKeys<Secp256k1>,
transcript: RecommendedTranscript,
_: usize,
mut inputs: Vec<Output>,
payments: &[(Address, u64)],
change_key: Option<ProjectivePoint>,
fee: Fee,
) -> Result<Self::SignableTransaction, CoinError> {
Ok(SignableTransaction {
keys,
transcript,
actual: BSignableTransaction::new(
inputs.drain(..).map(|input| input.0).collect(),
payments,
change_key.map(|change_key| self.address(change(change_key).0)),
fee.0,
)
.ok_or(CoinError::NotEnoughFunds)?,
})
}
async fn attempt_send(
&self,
transaction: Self::SignableTransaction,
) -> Result<Self::TransactionMachine, CoinError> {
transaction
.actual
.clone()
.multisig(transaction.keys.clone(), transaction.transcript.clone())
.await
.map_err(|_| CoinError::ConnectionError)
}
async fn publish_transaction(&self, tx: &Self::Transaction) -> Result<Vec<u8>, CoinError> {
Ok(self.rpc.send_raw_transaction(tx).await.unwrap().to_vec())
}
#[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,
Address::p2sh(&Script::new(), Network::Regtest).unwrap().to_string()
]),
)
.await
.unwrap();
}
#[cfg(test)]
async fn test_send(&self, address: Self::Address) {
let secret_key = SecretKey::new(&mut rand_core::OsRng);
let private_key = PrivateKey::new(secret_key, Network::Regtest);
let public_key = PublicKey::from_private_key(SECP256K1, &private_key);
let main_addr = Address::p2pkh(&public_key, Network::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;
}
// TODO: Consider grabbing bdk as a dev dependency
let tx = self.get_block(new_block).await.unwrap().txdata.swap_remove(0);
let mut tx = Transaction {
version: 2,
lock_time: PackedLockTime::ZERO,
input: vec![TxIn {
previous_output: OutPoint { txid: tx.txid(), vout: 0 },
script_sig: Script::default(),
sequence: Sequence(u32::MAX),
witness: Witness::default(),
}],
output: vec![TxOut {
value: tx.output[0].value - 10000,
script_pubkey: address.script_pubkey(),
}],
};
let mut der = SECP256K1
.sign_ecdsa_low_r(
&Message::from(
tx.signature_hash(0, &main_addr.script_pubkey(), EcdsaSighashType::All.to_u32())
.as_hash(),
),
&private_key.inner,
)
.serialize_der()
.to_vec();
der.push(1);
tx.input[0].script_sig = Builder::new().push_slice(&der).push_key(&public_key).into_script();
self.rpc.send_raw_transaction(&tx).await.unwrap();
for _ in 0 .. Self::CONFIRMATIONS {
self.mine_block().await;
}
}
}

View File

@@ -1,4 +1,4 @@
use std::marker::Send;
use std::io;
use async_trait::async_trait;
use thiserror::Error;
@@ -10,23 +10,63 @@ use frost::{
sign::PreprocessMachine,
};
pub mod bitcoin;
pub use self::bitcoin::Bitcoin;
pub mod monero;
pub use self::monero::Monero;
#[derive(Clone, Error, Debug)]
#[derive(Clone, Copy, Error, Debug)]
pub enum CoinError {
#[error("failed to connect to coin daemon")]
ConnectionError,
#[error("not enough funds")]
NotEnoughFunds,
}
pub trait Block: Sized + Clone {
type Id: Clone + Copy + AsRef<[u8]>;
fn id(&self) -> Self::Id;
}
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
pub enum OutputType {
External,
Branch,
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: Sized + Clone {
type Id: AsRef<[u8]>;
type Id: Clone + Copy + AsRef<[u8]>;
fn kind(&self) -> OutputType;
fn id(&self) -> Self::Id;
fn amount(&self) -> u64;
fn serialize(&self) -> Vec<u8>;
fn deserialize<R: std::io::Read>(reader: &mut R) -> std::io::Result<Self>;
fn read<R: std::io::Read>(reader: &mut R) -> std::io::Result<Self>;
}
#[async_trait]
@@ -35,7 +75,7 @@ pub trait Coin {
type Fee: Copy;
type Transaction;
type Block;
type Block: Block;
type Output: Output;
type SignableTransaction;
@@ -48,8 +88,13 @@ pub trait Coin {
const MAX_INPUTS: usize;
const MAX_OUTPUTS: usize; // TODO: Decide if this includes change or not
fn tweak_keys(&self, key: &mut ThresholdKeys<Self::Curve>);
/// Address for the given group key to receive external coins to.
// Doesn't have to take self, enables some level of caching which is pleasant
fn address(&self, key: <Self::Curve as Ciphersuite>::G) -> Self::Address;
/// Address for the given group key to use for scheduled branches.
fn branch_address(&self, key: <Self::Curve as Ciphersuite>::G) -> Self::Address;
async fn get_latest_block_number(&self) -> Result<usize, CoinError>;
async fn get_block(&self, number: usize) -> Result<Self::Block, CoinError>;
@@ -59,9 +104,7 @@ pub trait Coin {
key: <Self::Curve as Ciphersuite>::G,
) -> Result<Vec<Self::Output>, CoinError>;
// TODO: Remove
async fn is_confirmed(&self, tx: &[u8]) -> Result<bool, CoinError>;
#[allow(clippy::too_many_arguments)]
async fn prepare_send(
&self,
keys: ThresholdKeys<Self::Curve>,
@@ -69,6 +112,7 @@ pub trait Coin {
block_number: usize,
inputs: Vec<Self::Output>,
payments: &[(Self::Address, u64)],
change: Option<<Self::Curve as Ciphersuite>::G>,
fee: Self::Fee,
) -> Result<Self::SignableTransaction, CoinError>;
@@ -77,10 +121,7 @@ pub trait Coin {
transaction: Self::SignableTransaction,
) -> Result<Self::TransactionMachine, CoinError>;
async fn publish_transaction(
&self,
tx: &Self::Transaction,
) -> Result<(Vec<u8>, Vec<<Self::Output as Output>::Id>), CoinError>;
async fn publish_transaction(&self, tx: &Self::Transaction) -> Result<Vec<u8>, CoinError>;
#[cfg(test)]
async fn get_fee(&self) -> Self::Fee;

View File

@@ -10,20 +10,29 @@ use frost::{curve::Ed25519, ThresholdKeys};
use monero_serai::{
transaction::Transaction,
block::Block,
block::Block as MBlock,
rpc::Rpc,
wallet::{
ViewPair, Scanner,
address::{Network, MoneroAddress},
Fee, SpendableOutput, SignableTransaction as MSignableTransaction, TransactionMachine,
address::{Network, SubaddressIndex, AddressSpec, MoneroAddress},
Fee, SpendableOutput, Change, SignableTransaction as MSignableTransaction, TransactionMachine,
},
};
use crate::{
additional_key,
coin::{CoinError, Output as OutputTrait, Coin},
coin::{CoinError, Block as BlockTrait, OutputType, Output as OutputTrait, Coin},
};
#[derive(Clone, Debug)]
pub struct Block([u8; 32], MBlock);
impl BlockTrait for Block {
type Id = [u8; 32];
fn id(&self) -> Self::Id {
self.0
}
}
#[derive(Clone, Debug)]
pub struct Output(SpendableOutput);
impl From<SpendableOutput> for Output {
@@ -32,12 +41,25 @@ impl From<SpendableOutput> for Output {
}
}
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 the Monero library offers a variant which allows senders to ensure their TXs have unique
// output keys, Serai can still be targeted using the classic 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()
}
@@ -50,8 +72,8 @@ impl OutputTrait for Output {
self.0.serialize()
}
fn deserialize<R: std::io::Read>(reader: &mut R) -> std::io::Result<Self> {
SpendableOutput::deserialize(reader).map(Output)
fn read<R: std::io::Read>(reader: &mut R) -> std::io::Result<Self> {
SpendableOutput::read(reader).map(Output)
}
}
@@ -75,23 +97,43 @@ impl Monero {
Monero { rpc: Rpc::new(url).unwrap(), view: Zeroizing::new(additional_key::<Monero>(0).0) }
}
fn scanner(&self, spend: dfg::EdwardsPoint) -> Scanner {
Scanner::from_view(ViewPair::new(spend.0, self.view.clone()), Network::Mainnet, None)
fn view_pair(&self, spend: dfg::EdwardsPoint) -> ViewPair {
ViewPair::new(spend.0, self.view.clone())
}
#[cfg(test)]
fn empty_scanner() -> Scanner {
use group::Group;
Scanner::from_view(
ViewPair::new(*dfg::EdwardsPoint::generator(), Zeroizing::new(Scalar::one())),
fn address_internal(
&self,
spend: dfg::EdwardsPoint,
subaddress: Option<SubaddressIndex>,
) -> MoneroAddress {
self.view_pair(spend).address(
Network::Mainnet,
Some(std::collections::HashSet::new()),
AddressSpec::Featured { subaddress, payment_id: None, guaranteed: true },
)
}
fn scanner(&self, spend: dfg::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 empty_address() -> MoneroAddress {
Self::empty_scanner().address()
fn test_view_pair() -> ViewPair {
use group::Group;
ViewPair::new(*dfg::EdwardsPoint::generator(), Zeroizing::new(Scalar::one()))
}
#[cfg(test)]
fn test_scanner() -> Scanner {
Scanner::from_view(Self::test_view_pair(), Some(std::collections::HashSet::new()))
}
#[cfg(test)]
fn test_address() -> MoneroAddress {
Self::test_view_pair().address(Network::Mainnet, AddressSpec::Standard)
}
}
@@ -120,8 +162,15 @@ impl Coin for Monero {
const MAX_INPUTS: usize = 128;
const MAX_OUTPUTS: usize = 16;
// Monero doesn't require/benefit from tweaking
fn tweak_keys(&self, _: &mut ThresholdKeys<Self::Curve>) {}
fn address(&self, key: dfg::EdwardsPoint) -> Self::Address {
self.scanner(key).address()
self.address_internal(key, EXTERNAL_SUBADDRESS)
}
fn branch_address(&self, key: dfg::EdwardsPoint) -> Self::Address {
self.address_internal(key, BRANCH_SUBADDRESS)
}
async fn get_latest_block_number(&self) -> Result<usize, CoinError> {
@@ -130,7 +179,9 @@ impl Coin for Monero {
}
async fn get_block(&self, number: usize) -> Result<Self::Block, CoinError> {
self.rpc.get_block(number).await.map_err(|_| CoinError::ConnectionError)
let hash = self.rpc.get_block_hash(number).await.map_err(|_| CoinError::ConnectionError)?;
let block = self.rpc.get_block(hash).await.map_err(|_| CoinError::ConnectionError)?;
Ok(Block(hash, block))
}
async fn get_outputs(
@@ -138,27 +189,33 @@ impl Coin for Monero {
block: &Self::Block,
key: dfg::EdwardsPoint,
) -> Result<Vec<Self::Output>, CoinError> {
Ok(
self
.scanner(key)
.scan(&self.rpc, block)
.await
.map_err(|_| CoinError::ConnectionError)?
.iter()
.flat_map(|outputs| outputs.not_locked())
.map(Output::from)
.collect(),
)
}
async fn is_confirmed(&self, tx: &[u8]) -> Result<bool, CoinError> {
let tx_block_number = self
.rpc
.get_transaction_block_number(tx)
let mut transactions = self
.scanner(key)
.scan(&self.rpc, &block.1)
.await
.map_err(|_| CoinError::ConnectionError)?
.unwrap_or(usize::MAX);
Ok((self.get_latest_block_number().await?.saturating_sub(tx_block_number) + 1) >= 10)
.iter()
.map(|outputs| outputs.not_locked())
.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 transaction in transactions.iter_mut() {
*transaction = transaction
.drain(..)
.filter(|output| {
[EXTERNAL_SUBADDRESS, BRANCH_SUBADDRESS, CHANGE_SUBADDRESS]
.contains(&output.output.metadata.subaddress)
})
.collect();
}
Ok(
transactions
.drain(..)
.flat_map(|mut outputs| outputs.drain(..).map(Output::from).collect::<Vec<_>>())
.collect(),
)
}
async fn prepare_send(
@@ -168,9 +225,9 @@ impl Coin for Monero {
block_number: usize,
mut inputs: Vec<Output>,
payments: &[(MoneroAddress, u64)],
change: Option<dfg::EdwardsPoint>,
fee: Fee,
) -> Result<SignableTransaction, CoinError> {
let spend = keys.group_key();
Ok(SignableTransaction {
keys,
transcript,
@@ -179,7 +236,8 @@ impl Coin for Monero {
self.rpc.get_protocol().await.unwrap(), // TODO: Make this deterministic
inputs.drain(..).map(|input| input.0).collect(),
payments.to_vec(),
Some(self.address(spend)),
change
.map(|change| Change::fingerprintable(self.address_internal(change, CHANGE_SUBADDRESS))),
vec![],
fee,
)
@@ -204,12 +262,9 @@ impl Coin for Monero {
.map_err(|_| CoinError::ConnectionError)
}
async fn publish_transaction(
&self,
tx: &Self::Transaction,
) -> Result<(Vec<u8>, Vec<<Self::Output as OutputTrait>::Id>), CoinError> {
async fn publish_transaction(&self, tx: &Self::Transaction) -> Result<Vec<u8>, CoinError> {
self.rpc.publish_transaction(tx).await.map_err(|_| CoinError::ConnectionError)?;
Ok((tx.hash().to_vec(), tx.prefix.outputs.iter().map(|output| output.key.to_bytes()).collect()))
Ok(tx.hash().to_vec())
}
#[cfg(test)]
@@ -228,7 +283,7 @@ impl Coin for Monero {
Some(serde_json::json!({
"method": "generateblocks",
"params": {
"wallet_address": Self::empty_address().to_string(),
"wallet_address": Self::test_address().to_string(),
"amount_of_blocks": 10
},
})),
@@ -249,8 +304,8 @@ impl Coin for Monero {
self.mine_block().await;
}
let outputs = Self::empty_scanner()
.scan(&self.rpc, &self.rpc.get_block(new_block).await.unwrap())
let outputs = Self::test_scanner()
.scan(&self.rpc, &self.rpc.get_block_by_number(new_block).await.unwrap())
.await
.unwrap()
.swap_remove(0)
@@ -262,7 +317,7 @@ impl Coin for Monero {
self.rpc.get_protocol().await.unwrap(),
outputs,
vec![(address, amount - fee)],
Some(Self::empty_address()),
Some(Change::new(&Self::test_view_pair(), true)),
vec![],
self.rpc.get_fee().await.unwrap(),
)

View File

@@ -0,0 +1,12 @@
use crate::{
coin::{Coin, Bitcoin},
tests::test_send,
};
#[tokio::test]
async fn bitcoin() {
let bitcoin = Bitcoin::new("http://serai:seraidex@127.0.0.1:18443".to_string()).await;
bitcoin.fresh_chain().await;
let fee = bitcoin.get_fee().await;
test_send(bitcoin, fee).await;
}

View File

@@ -1,120 +1,5 @@
use std::{
sync::{Arc, RwLock},
collections::HashMap,
};
mod send;
pub(crate) use send::test_send;
use async_trait::async_trait;
use rand_core::OsRng;
use frost::Participant;
use crate::{
NetworkError, Network,
coin::{Coin, Monero},
wallet::{WalletKeys, MemCoinDb, Wallet},
};
#[derive(Clone)]
struct LocalNetwork {
i: Participant,
size: u16,
round: usize,
#[allow(clippy::type_complexity)]
rounds: Arc<RwLock<Vec<HashMap<Participant, Vec<u8>>>>>,
}
impl LocalNetwork {
fn new(size: u16) -> Vec<LocalNetwork> {
let rounds = Arc::new(RwLock::new(vec![]));
let mut res = vec![];
for i in 1 ..= size {
res.push(LocalNetwork {
i: Participant::new(i).unwrap(),
size,
round: 0,
rounds: rounds.clone(),
});
}
res
}
}
#[async_trait]
impl Network for LocalNetwork {
async fn round(&mut self, data: Vec<u8>) -> Result<HashMap<Participant, Vec<u8>>, NetworkError> {
{
let mut rounds = self.rounds.write().unwrap();
if rounds.len() == self.round {
rounds.push(HashMap::new());
}
rounds[self.round].insert(self.i, data);
}
while {
let read = self.rounds.try_read().unwrap();
read[self.round].len() != usize::from(self.size)
} {
tokio::task::yield_now().await;
}
let mut res = self.rounds.try_read().unwrap()[self.round].clone();
res.remove(&self.i);
self.round += 1;
Ok(res)
}
}
async fn test_send<C: Coin + Clone>(coin: C, fee: C::Fee) {
// Mine blocks so there's a confirmed block
coin.mine_block().await;
let latest = coin.get_latest_block_number().await.unwrap();
let mut keys = frost::tests::key_gen::<_, C::Curve>(&mut OsRng);
let threshold = keys[&Participant::new(1).unwrap()].params().t();
let mut networks = LocalNetwork::new(threshold);
let mut wallets = vec![];
for i in 1 ..= threshold {
let mut wallet = Wallet::new(MemCoinDb::new(), coin.clone());
wallet.acknowledge_block(0, latest);
wallet.add_keys(&WalletKeys::new(keys.remove(&Participant::new(i).unwrap()).unwrap(), 0));
wallets.push(wallet);
}
// Get the chain to a length where blocks have sufficient confirmations
while (latest + (C::CONFIRMATIONS - 1)) > coin.get_latest_block_number().await.unwrap() {
coin.mine_block().await;
}
for wallet in wallets.iter_mut() {
// Poll to activate the keys
wallet.poll().await.unwrap();
}
coin.test_send(wallets[0].address()).await;
let mut futures = vec![];
for (network, wallet) in networks.iter_mut().zip(wallets.iter_mut()) {
wallet.poll().await.unwrap();
let latest = coin.get_latest_block_number().await.unwrap();
wallet.acknowledge_block(1, latest - (C::CONFIRMATIONS - 1));
let signable = wallet
.prepare_sends(1, vec![(wallet.address(), 10000000000)], fee)
.await
.unwrap()
.1
.swap_remove(0);
futures.push(wallet.attempt_send(network, signable));
}
println!("{:?}", hex::encode(futures::future::join_all(futures).await.swap_remove(0).unwrap().0));
}
#[tokio::test]
async fn monero() {
let monero = Monero::new("http://127.0.0.1:18081".to_string()).await;
let fee = monero.get_fee().await;
test_send(monero, fee).await;
}
mod bitcoin;
mod monero;

View File

@@ -0,0 +1,11 @@
use crate::{
coin::{Coin, Monero},
tests::test_send,
};
#[tokio::test]
async fn monero() {
let monero = Monero::new("http://127.0.0.1:18081".to_string()).await;
let fee = monero.get_fee().await;
test_send(monero, fee).await;
}

106
processor/src/tests/send.rs Normal file
View File

@@ -0,0 +1,106 @@
use std::{
sync::{Arc, RwLock},
collections::HashMap,
};
use async_trait::async_trait;
use rand_core::OsRng;
use crate::{
NetworkError, Network,
coin::Coin,
wallet::{WalletKeys, MemCoinDb, Wallet},
};
#[derive(Clone)]
struct LocalNetwork {
i: u16,
size: u16,
round: usize,
#[allow(clippy::type_complexity)]
rounds: Arc<RwLock<Vec<HashMap<u16, Vec<u8>>>>>,
}
impl LocalNetwork {
fn new(size: u16) -> Vec<LocalNetwork> {
let rounds = Arc::new(RwLock::new(vec![]));
let mut res = vec![];
for i in 1 ..= size {
res.push(LocalNetwork { i, size, round: 0, rounds: rounds.clone() });
}
res
}
}
#[async_trait]
impl Network for LocalNetwork {
async fn round(&mut self, data: Vec<u8>) -> Result<HashMap<u16, Vec<u8>>, NetworkError> {
{
let mut rounds = self.rounds.write().unwrap();
if rounds.len() == self.round {
rounds.push(HashMap::new());
}
rounds[self.round].insert(self.i, data);
}
while {
let read = self.rounds.try_read().unwrap();
read[self.round].len() != usize::from(self.size)
} {
tokio::task::yield_now().await;
}
let mut res = self.rounds.try_read().unwrap()[self.round].clone();
res.remove(&self.i);
self.round += 1;
Ok(res)
}
}
pub async fn test_send<C: Coin + Clone>(coin: C, fee: C::Fee) {
// Mine blocks so there's a confirmed block
coin.mine_block().await;
let latest = coin.get_latest_block_number().await.unwrap();
let mut keys = frost::tests::key_gen::<_, C::Curve>(&mut OsRng);
let threshold = keys[&1].params().t();
let mut networks = LocalNetwork::new(threshold);
let mut wallets = vec![];
for i in 1 ..= threshold {
let mut wallet = Wallet::new(MemCoinDb::new(), coin.clone());
wallet.acknowledge_block(0, latest);
wallet.add_keys(&WalletKeys::new(keys.remove(&i).unwrap(), 0));
wallets.push(wallet);
}
// Get the chain to a length where blocks have sufficient confirmations
while (latest + (C::CONFIRMATIONS - 1)) > coin.get_latest_block_number().await.unwrap() {
coin.mine_block().await;
}
for wallet in wallets.iter_mut() {
// Poll to activate the keys
wallet.poll().await.unwrap();
}
coin.test_send(wallets[0].address()).await;
let mut futures = vec![];
for (network, wallet) in networks.iter_mut().zip(wallets.iter_mut()) {
wallet.poll().await.unwrap();
let latest = coin.get_latest_block_number().await.unwrap();
wallet.acknowledge_block(1, latest - (C::CONFIRMATIONS - 1));
let signable = wallet
.prepare_sends(1, vec![(wallet.address(), 100000000)], fee)
.await
.unwrap()
.1
.swap_remove(0);
futures.push(wallet.attempt_send(network, signable));
}
println!("{:?}", hex::encode(futures::future::join_all(futures).await.swap_remove(0).unwrap()));
}

View File

@@ -225,18 +225,16 @@ impl<D: CoinDb, C: Coin> Wallet<D, C> {
}
pub fn add_keys(&mut self, keys: &WalletKeys<C::Curve>) {
self.pending.push((self.acknowledged_block(keys.creation_block), keys.bind(C::ID)));
let creation_block = keys.creation_block;
let mut keys = keys.bind(C::ID);
self.coin.tweak_keys(&mut keys);
self.pending.push((self.acknowledged_block(creation_block), keys));
}
pub fn address(&self) -> C::Address {
self.coin.address(self.keys[self.keys.len() - 1].0.group_key())
}
// TODO: Remove
pub async fn is_confirmed(&mut self, tx: &[u8]) -> Result<bool, CoinError> {
self.coin.is_confirmed(tx).await
}
pub async fn poll(&mut self) -> Result<(), CoinError> {
if self.coin.get_latest_block_number().await? < (C::CONFIRMATIONS - 1) {
return Ok(());
@@ -267,8 +265,7 @@ impl<D: CoinDb, C: Coin> Wallet<D, C> {
.coin
.get_outputs(&block, keys.group_key())
.await?
.iter()
.cloned()
.drain(..)
.filter(|output| self.db.add_output(output)),
);
}
@@ -287,7 +284,7 @@ impl<D: CoinDb, C: Coin> Wallet<D, C> {
pub async fn prepare_sends(
&mut self,
canonical: usize,
payments: Vec<(C::Address, u64)>,
mut payments: Vec<(C::Address, u64)>,
fee: C::Fee,
) -> Result<(Vec<(C::Address, u64)>, Vec<C::SignableTransaction>), CoinError> {
if payments.is_empty() {
@@ -301,7 +298,6 @@ impl<D: CoinDb, C: Coin> Wallet<D, C> {
// As each payment re-appears, let mut payments = schedule[payment] where the only input is
// the source payment
// let (mut payments, schedule) = schedule(payments);
let mut payments = payments;
let mut txs = vec![];
for (keys, outputs) in self.keys.iter_mut() {
@@ -325,7 +321,15 @@ impl<D: CoinDb, C: Coin> Wallet<D, C> {
let tx = self
.coin
.prepare_send(keys.clone(), transcript, acknowledged_block, inputs, &outputs, fee)
.prepare_send(
keys.clone(),
transcript,
acknowledged_block,
inputs,
&outputs,
Some(keys.group_key()),
fee,
)
.await?;
// self.db.save_tx(tx) // TODO
txs.push(tx);
@@ -339,7 +343,7 @@ impl<D: CoinDb, C: Coin> Wallet<D, C> {
&mut self,
network: &mut N,
prepared: C::SignableTransaction,
) -> Result<(Vec<u8>, Vec<<C::Output as Output>::Id>), SignError> {
) -> Result<Vec<u8>, SignError> {
let attempt = self.coin.attempt_send(prepared).await.map_err(SignError::CoinError)?;
let (attempt, commitments) = attempt.preprocess(&mut OsRng);