mirror of
https://github.com/serai-dex/serai.git
synced 2025-12-08 20:29:23 +00:00
Split processor into bitcoin-processor, ethereum-processor, monero-processor
This commit is contained in:
@@ -1,942 +0,0 @@
|
||||
use std::{sync::OnceLock, 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,
|
||||
key::{Parity, XOnlyPublicKey},
|
||||
consensus::{Encodable, Decodable},
|
||||
script::Instruction,
|
||||
Transaction, Block, ScriptBuf,
|
||||
opcodes::all::{OP_SHA256, OP_EQUALVERIFY},
|
||||
},
|
||||
wallet::{
|
||||
tweak_keys, p2tr_script_buf, 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,
|
||||
absolute::LockTime,
|
||||
Amount as BAmount, Sequence, Script, Witness, OutPoint,
|
||||
transaction::Version,
|
||||
blockdata::transaction::{TxIn, TxOut},
|
||||
};
|
||||
|
||||
use serai_client::{
|
||||
primitives::{MAX_DATA_LEN, Coin, NetworkId, Amount, Balance},
|
||||
networks::bitcoin::Address,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
networks::{
|
||||
NetworkError, Block as BlockTrait, OutputType, Output as OutputTrait,
|
||||
Transaction as TransactionTrait, SignableTransaction as SignableTransactionTrait,
|
||||
Eventuality as EventualityTrait, EventualitiesTracker, Network, UtxoNetwork,
|
||||
},
|
||||
Payment,
|
||||
multisigs::scheduler::utxo::Scheduler,
|
||||
};
|
||||
|
||||
#[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,
|
||||
presumed_origin: Option<Address>,
|
||||
output: ReceivedOutput,
|
||||
data: Vec<u8>,
|
||||
}
|
||||
|
||||
impl OutputTrait<Bitcoin> 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 tx_id(&self) -> [u8; 32] {
|
||||
let mut hash = *self.output.outpoint().txid.as_raw_hash().as_byte_array();
|
||||
hash.reverse();
|
||||
hash
|
||||
}
|
||||
|
||||
fn key(&self) -> ProjectivePoint {
|
||||
let script = &self.output.output().script_pubkey;
|
||||
assert!(script.is_p2tr());
|
||||
let Instruction::PushBytes(key) = script.instructions_minimal().last().unwrap().unwrap() else {
|
||||
panic!("last item in v1 Taproot script wasn't bytes")
|
||||
};
|
||||
let key = XOnlyPublicKey::from_slice(key.as_ref())
|
||||
.expect("last item in v1 Taproot script wasn't x-only public key");
|
||||
Secp256k1::read_G(&mut key.public_key(Parity::Even).serialize().as_slice()).unwrap() -
|
||||
(ProjectivePoint::GENERATOR * self.output.offset())
|
||||
}
|
||||
|
||||
fn presumed_origin(&self) -> Option<Address> {
|
||||
self.presumed_origin.clone()
|
||||
}
|
||||
|
||||
fn balance(&self) -> Balance {
|
||||
Balance { coin: Coin::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)?;
|
||||
let presumed_origin: Option<Vec<u8>> = self.presumed_origin.clone().map(Into::into);
|
||||
writer.write_all(&presumed_origin.encode())?;
|
||||
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>(mut reader: &mut R) -> io::Result<Self> {
|
||||
Ok(Output {
|
||||
kind: OutputType::read(reader)?,
|
||||
presumed_origin: {
|
||||
let mut io_reader = scale::IoReader(reader);
|
||||
let res = Option::<Vec<u8>>::decode(&mut io_reader)
|
||||
.unwrap()
|
||||
.map(|address| Address::try_from(address).unwrap());
|
||||
reader = io_reader.0;
|
||||
res
|
||||
},
|
||||
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.compute_txid().as_raw_hash().as_byte_array();
|
||||
hash.reverse();
|
||||
hash
|
||||
}
|
||||
|
||||
#[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
|
||||
.to_sat();
|
||||
}
|
||||
for output in &self.output {
|
||||
value -= output.value.to_sat();
|
||||
}
|
||||
value
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct Eventuality([u8; 32]);
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Default, Debug)]
|
||||
pub struct EmptyClaim;
|
||||
impl AsRef<[u8]> for EmptyClaim {
|
||||
fn as_ref(&self) -> &[u8] {
|
||||
&[]
|
||||
}
|
||||
}
|
||||
impl AsMut<[u8]> for EmptyClaim {
|
||||
fn as_mut(&mut self) -> &mut [u8] {
|
||||
&mut []
|
||||
}
|
||||
}
|
||||
|
||||
impl EventualityTrait for Eventuality {
|
||||
type Claim = EmptyClaim;
|
||||
type Completion = Transaction;
|
||||
|
||||
fn lookup(&self) -> Vec<u8> {
|
||||
self.0.to_vec()
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
let mut id = [0; 32];
|
||||
reader
|
||||
.read_exact(&mut id)
|
||||
.map_err(|_| io::Error::other("couldn't decode ID in eventuality"))?;
|
||||
Ok(Eventuality(id))
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
self.0.to_vec()
|
||||
}
|
||||
|
||||
fn claim(_: &Transaction) -> EmptyClaim {
|
||||
EmptyClaim
|
||||
}
|
||||
fn serialize_completion(completion: &Transaction) -> Vec<u8> {
|
||||
let mut buf = vec![];
|
||||
completion.consensus_encode(&mut buf).unwrap();
|
||||
buf
|
||||
}
|
||||
fn read_completion<R: io::Read>(reader: &mut R) -> io::Result<Transaction> {
|
||||
Transaction::consensus_decode(&mut io::BufReader::with_capacity(0, reader))
|
||||
.map_err(|e| io::Error::other(format!("{e}")))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct SignableTransaction {
|
||||
actual: BSignableTransaction,
|
||||
}
|
||||
impl PartialEq for SignableTransaction {
|
||||
fn eq(&self, other: &SignableTransaction) -> bool {
|
||||
self.actual == other.actual
|
||||
}
|
||||
}
|
||||
impl Eq for SignableTransaction {}
|
||||
impl SignableTransactionTrait for SignableTransaction {
|
||||
fn fee(&self) -> u64 {
|
||||
self.actual.fee()
|
||||
}
|
||||
}
|
||||
|
||||
#[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
|
||||
let mut timestamps = vec![u64::from(self.header.time)];
|
||||
let mut parent = self.parent();
|
||||
// BIP-0113 uses a median of the prior 11 blocks
|
||||
while timestamps.len() < 11 {
|
||||
let mut parent_block;
|
||||
while {
|
||||
parent_block = rpc.rpc.get_block(&parent).await;
|
||||
parent_block.is_err()
|
||||
} {
|
||||
log::error!("couldn't get parent block when trying to get block time: {parent_block:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
}
|
||||
let parent_block = parent_block.unwrap();
|
||||
timestamps.push(u64::from(parent_block.header.time));
|
||||
parent = parent_block.parent();
|
||||
|
||||
if parent == [0; 32] {
|
||||
break;
|
||||
}
|
||||
}
|
||||
timestamps.sort();
|
||||
timestamps[timestamps.len() / 2]
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
fn eq(&self, _: &Self) -> bool {
|
||||
true
|
||||
}
|
||||
}
|
||||
impl Eq for Bitcoin {}
|
||||
|
||||
impl Bitcoin {
|
||||
pub async fn new(url: String) -> Bitcoin {
|
||||
let mut res = Rpc::new(url.clone()).await;
|
||||
while let Err(e) = res {
|
||||
log::error!("couldn't connect to Bitcoin node: {e:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
res = Rpc::new(url.clone()).await;
|
||||
}
|
||||
Bitcoin { rpc: res.unwrap() }
|
||||
}
|
||||
|
||||
#[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()
|
||||
}
|
||||
}
|
||||
|
||||
// This function panics on a node which doesn't follow the Bitcoin protocol, which is deemed fine
|
||||
async fn median_fee(&self, block: &Block) -> Result<Fee, NetworkError> {
|
||||
let mut fees = vec![];
|
||||
if block.txdata.len() > 1 {
|
||||
for tx in &block.txdata[1 ..] {
|
||||
let mut in_value = 0;
|
||||
for input in &tx.input {
|
||||
let mut input_tx = input.previous_output.txid.to_raw_hash().to_byte_array();
|
||||
input_tx.reverse();
|
||||
in_value += self
|
||||
.rpc
|
||||
.get_transaction(&input_tx)
|
||||
.await
|
||||
.map_err(|_| NetworkError::ConnectionError)?
|
||||
.output[usize::try_from(input.previous_output.vout).unwrap()]
|
||||
.value
|
||||
.to_sat();
|
||||
}
|
||||
let out = tx.output.iter().map(|output| output.value.to_sat()).sum::<u64>();
|
||||
fees.push((in_value - out) / u64::try_from(tx.vsize()).unwrap());
|
||||
}
|
||||
}
|
||||
fees.sort();
|
||||
let fee = fees.get(fees.len() / 2).copied().unwrap_or(0);
|
||||
|
||||
// The DUST constant documentation notes a relay rule practically enforcing a
|
||||
// 1000 sat/kilo-vbyte minimum fee.
|
||||
Ok(Fee(fee.max(1)))
|
||||
}
|
||||
|
||||
async fn make_signable_transaction(
|
||||
&self,
|
||||
block_number: usize,
|
||||
inputs: &[Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Address>,
|
||||
calculating_fee: bool,
|
||||
) -> Result<Option<BSignableTransaction>, NetworkError> {
|
||||
for payment in payments {
|
||||
assert_eq!(payment.balance.coin, Coin::Bitcoin);
|
||||
}
|
||||
|
||||
// TODO2: Use an fee representative of several blocks, cached inside Self
|
||||
let block_for_fee = self.get_block(block_number).await?;
|
||||
let fee = self.median_fee(&block_for_fee).await?;
|
||||
|
||||
let payments = payments
|
||||
.iter()
|
||||
.map(|payment| {
|
||||
(
|
||||
payment.address.clone().into(),
|
||||
// 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
|
||||
if calculating_fee { Self::DUST } else { payment.balance.amount.0 },
|
||||
)
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
match BSignableTransaction::new(
|
||||
inputs.iter().map(|input| input.output.clone()).collect(),
|
||||
&payments,
|
||||
change.clone().map(Into::into),
|
||||
None,
|
||||
fee.0,
|
||||
) {
|
||||
Ok(signable) => Ok(Some(signable)),
|
||||
Err(TransactionError::NoInputs) => {
|
||||
panic!("trying to create a bitcoin transaction without inputs")
|
||||
}
|
||||
// No outputs left and the change isn't worth enough/not even enough funds to pay the fee
|
||||
Err(TransactionError::NoOutputs | TransactionError::NotEnoughFunds) => Ok(None),
|
||||
// amortize_fee removes payments which fall below the dust threshold
|
||||
Err(TransactionError::DustPayment) => panic!("dust payment despite removing dust"),
|
||||
Err(TransactionError::TooMuchData) => {
|
||||
panic!("too much data despite not specifying data")
|
||||
}
|
||||
Err(TransactionError::TooLowFee) => {
|
||||
panic!("created a transaction whose fee is below the minimum")
|
||||
}
|
||||
Err(TransactionError::TooLargeTransaction) => {
|
||||
panic!("created a too large transaction despite limiting inputs/outputs")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Expected script has to start with SHA256 PUSH MSG_HASH OP_EQUALVERIFY ..
|
||||
fn segwit_data_pattern(script: &ScriptBuf) -> Option<bool> {
|
||||
let mut ins = script.instructions();
|
||||
|
||||
// first item should be SHA256 code
|
||||
if ins.next()?.ok()?.opcode()? != OP_SHA256 {
|
||||
return Some(false);
|
||||
}
|
||||
|
||||
// next should be a data push
|
||||
ins.next()?.ok()?.push_bytes()?;
|
||||
|
||||
// next should be a equality check
|
||||
if ins.next()?.ok()?.opcode()? != OP_EQUALVERIFY {
|
||||
return Some(false);
|
||||
}
|
||||
|
||||
Some(true)
|
||||
}
|
||||
|
||||
fn extract_serai_data(tx: &Transaction) -> Vec<u8> {
|
||||
// check outputs
|
||||
let mut data = (|| {
|
||||
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![]
|
||||
})();
|
||||
|
||||
// check inputs
|
||||
if data.is_empty() {
|
||||
for input in &tx.input {
|
||||
let witness = input.witness.to_vec();
|
||||
// expected witness at least has to have 2 items, msg and the redeem script.
|
||||
if witness.len() >= 2 {
|
||||
let redeem_script = ScriptBuf::from_bytes(witness.last().unwrap().clone());
|
||||
if Self::segwit_data_pattern(&redeem_script) == Some(true) {
|
||||
data.clone_from(&witness[witness.len() - 2]); // len() - 1 is the redeem_script
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
data.truncate(MAX_DATA_LEN.try_into().unwrap());
|
||||
data
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
pub fn sign_btc_input_for_p2pkh(
|
||||
tx: &Transaction,
|
||||
input_index: usize,
|
||||
private_key: &PrivateKey,
|
||||
) -> ScriptBuf {
|
||||
use bitcoin_serai::bitcoin::{Network as BNetwork, Address as BAddress};
|
||||
|
||||
let public_key = PublicKey::from_private_key(SECP256K1, private_key);
|
||||
let main_addr = BAddress::p2pkh(public_key, BNetwork::Regtest);
|
||||
|
||||
let mut der = SECP256K1
|
||||
.sign_ecdsa_low_r(
|
||||
&Message::from_digest_slice(
|
||||
SighashCache::new(tx)
|
||||
.legacy_signature_hash(
|
||||
input_index,
|
||||
&main_addr.script_pubkey(),
|
||||
EcdsaSighashType::All.to_u32(),
|
||||
)
|
||||
.unwrap()
|
||||
.to_raw_hash()
|
||||
.as_ref(),
|
||||
)
|
||||
.unwrap(),
|
||||
&private_key.inner,
|
||||
)
|
||||
.serialize_der()
|
||||
.to_vec();
|
||||
der.push(1);
|
||||
|
||||
ScriptBuf::builder()
|
||||
.push_slice(PushBytesBuf::try_from(der).unwrap())
|
||||
.push_key(&public_key)
|
||||
.into_script()
|
||||
}
|
||||
}
|
||||
|
||||
// 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"),
|
||||
)
|
||||
.expect("couldn't create Serai-representable address for P2TR script")
|
||||
}
|
||||
|
||||
#[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
|
||||
// 684 weight units, 171 vbytes, round up to 200
|
||||
// 200 vbytes at 1 sat/weight (our current minimum fee, 4 sat/vbyte) = 800 sat fee for the
|
||||
// aggregation TX
|
||||
const COST_TO_AGGREGATE: u64 = 800;
|
||||
|
||||
const MAX_OUTPUTS: usize = MAX_OUTPUTS;
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn external_address(&self, key: ProjectivePoint) -> Address {
|
||||
address_from_key(key)
|
||||
}
|
||||
|
||||
fn branch_address(key: ProjectivePoint) -> Option<Address> {
|
||||
let (_, offsets, _) = scanner(key);
|
||||
Some(address_from_key(key + (ProjectivePoint::GENERATOR * offsets[&OutputType::Branch])))
|
||||
}
|
||||
|
||||
fn change_address(key: ProjectivePoint) -> Option<Address> {
|
||||
let (_, offsets, _) = scanner(key);
|
||||
Some(address_from_key(key + (ProjectivePoint::GENERATOR * offsets[&OutputType::Change])))
|
||||
}
|
||||
|
||||
fn forward_address(key: ProjectivePoint) -> Option<Address> {
|
||||
let (_, offsets, _) = scanner(key);
|
||||
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()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn check_eventuality_by_claim(
|
||||
&self,
|
||||
eventuality: &Self::Eventuality,
|
||||
_: &EmptyClaim,
|
||||
) -> bool {
|
||||
self.rpc.get_transaction(&eventuality.0).await.is_ok()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_transaction_by_eventuality(&self, _: usize, id: &Eventuality) -> Transaction {
|
||||
self.rpc.get_transaction(&id.0).await.unwrap()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn mine_block(&self) {
|
||||
use bitcoin_serai::bitcoin::{Network as BNetwork, Address as BAddress};
|
||||
|
||||
self
|
||||
.rpc
|
||||
.rpc_call::<Vec<String>>(
|
||||
"generatetoaddress",
|
||||
serde_json::json!([1, BAddress::p2sh(Script::new(), BNetwork::Regtest).unwrap()]),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn test_send(&self, address: Address) -> Block {
|
||||
use bitcoin_serai::bitcoin::{Network as BNetwork, Address as BAddress};
|
||||
|
||||
let secret_key = SecretKey::new(&mut rand_core::OsRng);
|
||||
let private_key = PrivateKey::new(secret_key, BNetwork::Regtest);
|
||||
let public_key = PublicKey::from_private_key(SECP256K1, &private_key);
|
||||
let main_addr = BAddress::p2pkh(public_key, BNetwork::Regtest);
|
||||
|
||||
let new_block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
self
|
||||
.rpc
|
||||
.rpc_call::<Vec<String>>("generatetoaddress", serde_json::json!([100, main_addr]))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let tx = self.get_block(new_block).await.unwrap().txdata.swap_remove(0);
|
||||
let mut tx = Transaction {
|
||||
version: Version(2),
|
||||
lock_time: LockTime::ZERO,
|
||||
input: vec![TxIn {
|
||||
previous_output: OutPoint { txid: tx.compute_txid(), vout: 0 },
|
||||
script_sig: Script::new().into(),
|
||||
sequence: Sequence(u32::MAX),
|
||||
witness: Witness::default(),
|
||||
}],
|
||||
output: vec![TxOut {
|
||||
value: tx.output[0].value - BAmount::from_sat(10000),
|
||||
script_pubkey: address.clone().into(),
|
||||
}],
|
||||
};
|
||||
tx.input[0].script_sig = Self::sign_btc_input_for_p2pkh(&tx, 0, &private_key);
|
||||
|
||||
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()
|
||||
}
|
||||
}
|
||||
|
||||
impl UtxoNetwork for Bitcoin {
|
||||
const MAX_INPUTS: usize = MAX_INPUTS;
|
||||
}
|
||||
@@ -1,936 +0,0 @@
|
||||
use core::{fmt, time::Duration};
|
||||
use std::{
|
||||
sync::Arc,
|
||||
collections::{HashSet, HashMap},
|
||||
io,
|
||||
};
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use ciphersuite::{group::GroupEncoding, Ciphersuite, Secp256k1};
|
||||
use frost::ThresholdKeys;
|
||||
|
||||
use ethereum_serai::{
|
||||
alloy::{
|
||||
primitives::U256,
|
||||
rpc_types::{BlockTransactionsKind, BlockNumberOrTag, Transaction},
|
||||
simple_request_transport::SimpleRequest,
|
||||
rpc_client::ClientBuilder,
|
||||
provider::{Provider, RootProvider},
|
||||
},
|
||||
crypto::{PublicKey, Signature},
|
||||
erc20::Erc20,
|
||||
deployer::Deployer,
|
||||
router::{Router, Coin as EthereumCoin, InInstruction as EthereumInInstruction},
|
||||
machine::*,
|
||||
};
|
||||
#[cfg(test)]
|
||||
use ethereum_serai::alloy::primitives::B256;
|
||||
|
||||
use tokio::{
|
||||
time::sleep,
|
||||
sync::{RwLock, RwLockReadGuard},
|
||||
};
|
||||
#[cfg(not(test))]
|
||||
use tokio::{
|
||||
io::{AsyncReadExt, AsyncWriteExt},
|
||||
net::TcpStream,
|
||||
};
|
||||
|
||||
use serai_client::{
|
||||
primitives::{Coin, Amount, Balance, NetworkId},
|
||||
validator_sets::primitives::Session,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
Db, Payment,
|
||||
networks::{
|
||||
OutputType, Output, Transaction as TransactionTrait, SignableTransaction, Block,
|
||||
Eventuality as EventualityTrait, EventualitiesTracker, NetworkError, Network,
|
||||
},
|
||||
key_gen::NetworkKeyDb,
|
||||
multisigs::scheduler::{
|
||||
Scheduler as SchedulerTrait,
|
||||
smart_contract::{Addendum, Scheduler},
|
||||
},
|
||||
};
|
||||
|
||||
#[cfg(not(test))]
|
||||
const DAI: [u8; 20] =
|
||||
match const_hex::const_decode_to_array(b"0x6B175474E89094C44Da98b954EedeAC495271d0F") {
|
||||
Ok(res) => res,
|
||||
Err(_) => panic!("invalid non-test DAI hex address"),
|
||||
};
|
||||
#[cfg(test)] // TODO
|
||||
const DAI: [u8; 20] =
|
||||
match const_hex::const_decode_to_array(b"0000000000000000000000000000000000000000") {
|
||||
Ok(res) => res,
|
||||
Err(_) => panic!("invalid test DAI hex address"),
|
||||
};
|
||||
|
||||
fn coin_to_serai_coin(coin: &EthereumCoin) -> Option<Coin> {
|
||||
match coin {
|
||||
EthereumCoin::Ether => Some(Coin::Ether),
|
||||
EthereumCoin::Erc20(token) => {
|
||||
if *token == DAI {
|
||||
return Some(Coin::Dai);
|
||||
}
|
||||
None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn amount_to_serai_amount(coin: Coin, amount: U256) -> Amount {
|
||||
assert_eq!(coin.network(), NetworkId::Ethereum);
|
||||
assert_eq!(coin.decimals(), 8);
|
||||
// Remove 10 decimals so we go from 18 decimals to 8 decimals
|
||||
let divisor = U256::from(10_000_000_000u64);
|
||||
// This is valid up to 184b, which is assumed for the coins allowed
|
||||
Amount(u64::try_from(amount / divisor).unwrap())
|
||||
}
|
||||
|
||||
fn balance_to_ethereum_amount(balance: Balance) -> U256 {
|
||||
assert_eq!(balance.coin.network(), NetworkId::Ethereum);
|
||||
assert_eq!(balance.coin.decimals(), 8);
|
||||
// Restore 10 decimals so we go from 8 decimals to 18 decimals
|
||||
let factor = U256::from(10_000_000_000u64);
|
||||
U256::from(balance.amount.0) * factor
|
||||
}
|
||||
|
||||
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
|
||||
pub struct Address(pub [u8; 20]);
|
||||
impl TryFrom<Vec<u8>> for Address {
|
||||
type Error = ();
|
||||
fn try_from(bytes: Vec<u8>) -> Result<Address, ()> {
|
||||
if bytes.len() != 20 {
|
||||
Err(())?;
|
||||
}
|
||||
let mut res = [0; 20];
|
||||
res.copy_from_slice(&bytes);
|
||||
Ok(Address(res))
|
||||
}
|
||||
}
|
||||
impl TryInto<Vec<u8>> for Address {
|
||||
type Error = ();
|
||||
fn try_into(self) -> Result<Vec<u8>, ()> {
|
||||
Ok(self.0.to_vec())
|
||||
}
|
||||
}
|
||||
|
||||
impl fmt::Display for Address {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
ethereum_serai::alloy::primitives::Address::from(self.0).fmt(f)
|
||||
}
|
||||
}
|
||||
|
||||
impl SignableTransaction for RouterCommand {
|
||||
fn fee(&self) -> u64 {
|
||||
// Return a fee of 0 as we'll handle amortization on our end
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<D: Db> TransactionTrait<Ethereum<D>> for Transaction {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> Self::Id {
|
||||
self.hash.0
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, _network: &Ethereum<D>) -> u64 {
|
||||
// Return a fee of 0 as we'll handle amortization on our end
|
||||
0
|
||||
}
|
||||
}
|
||||
|
||||
// We use 32-block Epochs to represent blocks.
|
||||
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
|
||||
pub struct Epoch {
|
||||
// The hash of the block which ended the prior Epoch.
|
||||
prior_end_hash: [u8; 32],
|
||||
// The first block number within this Epoch.
|
||||
start: u64,
|
||||
// The hash of the last block within this Epoch.
|
||||
end_hash: [u8; 32],
|
||||
// The monotonic time for this Epoch.
|
||||
time: u64,
|
||||
}
|
||||
|
||||
impl Epoch {
|
||||
fn end(&self) -> u64 {
|
||||
self.start + 31
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<D: Db> Block<Ethereum<D>> for Epoch {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> [u8; 32] {
|
||||
self.end_hash
|
||||
}
|
||||
fn parent(&self) -> [u8; 32] {
|
||||
self.prior_end_hash
|
||||
}
|
||||
async fn time(&self, _: &Ethereum<D>) -> u64 {
|
||||
self.time
|
||||
}
|
||||
}
|
||||
|
||||
impl<D: Db> Output<Ethereum<D>> for EthereumInInstruction {
|
||||
type Id = [u8; 32];
|
||||
|
||||
fn kind(&self) -> OutputType {
|
||||
OutputType::External
|
||||
}
|
||||
|
||||
fn id(&self) -> Self::Id {
|
||||
let mut id = [0; 40];
|
||||
id[.. 32].copy_from_slice(&self.id.0);
|
||||
id[32 ..].copy_from_slice(&self.id.1.to_le_bytes());
|
||||
*ethereum_serai::alloy::primitives::keccak256(id)
|
||||
}
|
||||
fn tx_id(&self) -> [u8; 32] {
|
||||
self.id.0
|
||||
}
|
||||
fn key(&self) -> <Secp256k1 as Ciphersuite>::G {
|
||||
self.key_at_end_of_block
|
||||
}
|
||||
|
||||
fn presumed_origin(&self) -> Option<Address> {
|
||||
Some(Address(self.from))
|
||||
}
|
||||
|
||||
fn balance(&self) -> Balance {
|
||||
let coin = coin_to_serai_coin(&self.coin).unwrap_or_else(|| {
|
||||
panic!(
|
||||
"requesting coin for an EthereumInInstruction with a coin {}",
|
||||
"we don't handle. this never should have been yielded"
|
||||
)
|
||||
});
|
||||
Balance { coin, amount: amount_to_serai_amount(coin, self.amount) }
|
||||
}
|
||||
fn data(&self) -> &[u8] {
|
||||
&self.data
|
||||
}
|
||||
|
||||
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
|
||||
EthereumInInstruction::write(self, writer)
|
||||
}
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
EthereumInInstruction::read(reader)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct Claim {
|
||||
signature: [u8; 64],
|
||||
}
|
||||
impl AsRef<[u8]> for Claim {
|
||||
fn as_ref(&self) -> &[u8] {
|
||||
&self.signature
|
||||
}
|
||||
}
|
||||
impl AsMut<[u8]> for Claim {
|
||||
fn as_mut(&mut self) -> &mut [u8] {
|
||||
&mut self.signature
|
||||
}
|
||||
}
|
||||
impl Default for Claim {
|
||||
fn default() -> Self {
|
||||
Self { signature: [0; 64] }
|
||||
}
|
||||
}
|
||||
impl From<&Signature> for Claim {
|
||||
fn from(sig: &Signature) -> Self {
|
||||
Self { signature: sig.to_bytes() }
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct Eventuality(PublicKey, RouterCommand);
|
||||
impl EventualityTrait for Eventuality {
|
||||
type Claim = Claim;
|
||||
type Completion = SignedRouterCommand;
|
||||
|
||||
fn lookup(&self) -> Vec<u8> {
|
||||
match self.1 {
|
||||
RouterCommand::UpdateSeraiKey { nonce, .. } | RouterCommand::Execute { nonce, .. } => {
|
||||
nonce.as_le_bytes().to_vec()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
let point = Secp256k1::read_G(reader)?;
|
||||
let command = RouterCommand::read(reader)?;
|
||||
Ok(Eventuality(
|
||||
PublicKey::new(point).ok_or(io::Error::other("unusable key within Eventuality"))?,
|
||||
command,
|
||||
))
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
let mut res = vec![];
|
||||
res.extend(self.0.point().to_bytes().as_slice());
|
||||
self.1.write(&mut res).unwrap();
|
||||
res
|
||||
}
|
||||
|
||||
fn claim(completion: &Self::Completion) -> Self::Claim {
|
||||
Claim::from(completion.signature())
|
||||
}
|
||||
fn serialize_completion(completion: &Self::Completion) -> Vec<u8> {
|
||||
let mut res = vec![];
|
||||
completion.write(&mut res).unwrap();
|
||||
res
|
||||
}
|
||||
fn read_completion<R: io::Read>(reader: &mut R) -> io::Result<Self::Completion> {
|
||||
SignedRouterCommand::read(reader)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct Ethereum<D: Db> {
|
||||
// This DB is solely used to access the first key generated, as needed to determine the Router's
|
||||
// address. Accordingly, all methods present are consistent to a Serai chain with a finalized
|
||||
// first key (regardless of local state), and this is safe.
|
||||
db: D,
|
||||
#[cfg_attr(test, allow(unused))]
|
||||
relayer_url: String,
|
||||
provider: Arc<RootProvider<SimpleRequest>>,
|
||||
deployer: Deployer,
|
||||
router: Arc<RwLock<Option<Router>>>,
|
||||
}
|
||||
impl<D: Db> PartialEq for Ethereum<D> {
|
||||
fn eq(&self, _other: &Ethereum<D>) -> bool {
|
||||
true
|
||||
}
|
||||
}
|
||||
impl<D: Db> fmt::Debug for Ethereum<D> {
|
||||
fn fmt(&self, fmt: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
fmt
|
||||
.debug_struct("Ethereum")
|
||||
.field("deployer", &self.deployer)
|
||||
.field("router", &self.router)
|
||||
.finish_non_exhaustive()
|
||||
}
|
||||
}
|
||||
impl<D: Db> Ethereum<D> {
|
||||
pub async fn new(db: D, daemon_url: String, relayer_url: String) -> Self {
|
||||
let provider = Arc::new(RootProvider::new(
|
||||
ClientBuilder::default().transport(SimpleRequest::new(daemon_url), true),
|
||||
));
|
||||
|
||||
let mut deployer = Deployer::new(provider.clone()).await;
|
||||
while !matches!(deployer, Ok(Some(_))) {
|
||||
log::error!("Deployer wasn't deployed yet or networking error");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
deployer = Deployer::new(provider.clone()).await;
|
||||
}
|
||||
let deployer = deployer.unwrap().unwrap();
|
||||
|
||||
dbg!(&relayer_url);
|
||||
dbg!(relayer_url.len());
|
||||
Ethereum { db, relayer_url, provider, deployer, router: Arc::new(RwLock::new(None)) }
|
||||
}
|
||||
|
||||
// Obtain a reference to the Router, sleeping until it's deployed if it hasn't already been.
|
||||
// This is guaranteed to return Some.
|
||||
pub async fn router(&self) -> RwLockReadGuard<'_, Option<Router>> {
|
||||
// If we've already instantiated the Router, return a read reference
|
||||
{
|
||||
let router = self.router.read().await;
|
||||
if router.is_some() {
|
||||
return router;
|
||||
}
|
||||
}
|
||||
|
||||
// Instantiate it
|
||||
let mut router = self.router.write().await;
|
||||
// If another attempt beat us to it, return
|
||||
if router.is_some() {
|
||||
drop(router);
|
||||
return self.router.read().await;
|
||||
}
|
||||
|
||||
// Get the first key from the DB
|
||||
let first_key =
|
||||
NetworkKeyDb::get(&self.db, Session(0)).expect("getting outputs before confirming a key");
|
||||
let key = Secp256k1::read_G(&mut first_key.as_slice()).unwrap();
|
||||
let public_key = PublicKey::new(key).unwrap();
|
||||
|
||||
// Find the router
|
||||
let mut found = self.deployer.find_router(self.provider.clone(), &public_key).await;
|
||||
while !matches!(found, Ok(Some(_))) {
|
||||
log::error!("Router wasn't deployed yet or networking error");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
found = self.deployer.find_router(self.provider.clone(), &public_key).await;
|
||||
}
|
||||
|
||||
// Set it
|
||||
*router = Some(found.unwrap().unwrap());
|
||||
|
||||
// Downgrade to a read lock
|
||||
// Explicitly doesn't use `downgrade` so that another pending write txn can realize it's no
|
||||
// longer necessary
|
||||
drop(router);
|
||||
self.router.read().await
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl<D: Db> Network for Ethereum<D> {
|
||||
type Curve = Secp256k1;
|
||||
|
||||
type Transaction = Transaction;
|
||||
type Block = Epoch;
|
||||
|
||||
type Output = EthereumInInstruction;
|
||||
type SignableTransaction = RouterCommand;
|
||||
type Eventuality = Eventuality;
|
||||
type TransactionMachine = RouterCommandMachine;
|
||||
|
||||
type Scheduler = Scheduler<Self>;
|
||||
|
||||
type Address = Address;
|
||||
|
||||
const NETWORK: NetworkId = NetworkId::Ethereum;
|
||||
const ID: &'static str = "Ethereum";
|
||||
const ESTIMATED_BLOCK_TIME_IN_SECONDS: usize = 32 * 12;
|
||||
const CONFIRMATIONS: usize = 1;
|
||||
|
||||
const DUST: u64 = 0; // TODO
|
||||
|
||||
const COST_TO_AGGREGATE: u64 = 0;
|
||||
|
||||
// TODO: usize::max, with a merkle tree in the router
|
||||
const MAX_OUTPUTS: usize = 256;
|
||||
|
||||
fn tweak_keys(keys: &mut ThresholdKeys<Self::Curve>) {
|
||||
while PublicKey::new(keys.group_key()).is_none() {
|
||||
*keys = keys.offset(<Secp256k1 as Ciphersuite>::F::ONE);
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn external_address(&self, _key: <Secp256k1 as Ciphersuite>::G) -> Address {
|
||||
Address(self.router().await.as_ref().unwrap().address())
|
||||
}
|
||||
|
||||
fn branch_address(_key: <Secp256k1 as Ciphersuite>::G) -> Option<Address> {
|
||||
None
|
||||
}
|
||||
|
||||
fn change_address(_key: <Secp256k1 as Ciphersuite>::G) -> Option<Address> {
|
||||
None
|
||||
}
|
||||
|
||||
fn forward_address(_key: <Secp256k1 as Ciphersuite>::G) -> Option<Address> {
|
||||
None
|
||||
}
|
||||
|
||||
async fn get_latest_block_number(&self) -> Result<usize, NetworkError> {
|
||||
let actual_number = self
|
||||
.provider
|
||||
.get_block(BlockNumberOrTag::Finalized.into(), BlockTransactionsKind::Hashes)
|
||||
.await
|
||||
.map_err(|_| NetworkError::ConnectionError)?
|
||||
.ok_or(NetworkError::ConnectionError)?
|
||||
.header
|
||||
.number;
|
||||
// Error if there hasn't been a full epoch yet
|
||||
if actual_number < 32 {
|
||||
Err(NetworkError::ConnectionError)?
|
||||
}
|
||||
// If this is 33, the division will return 1, yet 1 is the epoch in progress
|
||||
let latest_full_epoch = (actual_number / 32).saturating_sub(1);
|
||||
Ok(latest_full_epoch.try_into().unwrap())
|
||||
}
|
||||
|
||||
async fn get_block(&self, number: usize) -> Result<Self::Block, NetworkError> {
|
||||
let latest_finalized = self.get_latest_block_number().await?;
|
||||
if number > latest_finalized {
|
||||
Err(NetworkError::ConnectionError)?
|
||||
}
|
||||
|
||||
let start = number * 32;
|
||||
let prior_end_hash = if start == 0 {
|
||||
[0; 32]
|
||||
} else {
|
||||
self
|
||||
.provider
|
||||
.get_block(u64::try_from(start - 1).unwrap().into(), BlockTransactionsKind::Hashes)
|
||||
.await
|
||||
.ok()
|
||||
.flatten()
|
||||
.ok_or(NetworkError::ConnectionError)?
|
||||
.header
|
||||
.hash
|
||||
.into()
|
||||
};
|
||||
|
||||
let end_header = self
|
||||
.provider
|
||||
.get_block(u64::try_from(start + 31).unwrap().into(), BlockTransactionsKind::Hashes)
|
||||
.await
|
||||
.ok()
|
||||
.flatten()
|
||||
.ok_or(NetworkError::ConnectionError)?
|
||||
.header;
|
||||
|
||||
let end_hash = end_header.hash.into();
|
||||
let time = end_header.timestamp;
|
||||
|
||||
Ok(Epoch { prior_end_hash, start: start.try_into().unwrap(), end_hash, time })
|
||||
}
|
||||
|
||||
async fn get_outputs(
|
||||
&self,
|
||||
block: &Self::Block,
|
||||
_: <Secp256k1 as Ciphersuite>::G,
|
||||
) -> Vec<Self::Output> {
|
||||
let router = self.router().await;
|
||||
let router = router.as_ref().unwrap();
|
||||
// Grab the key at the end of the epoch
|
||||
let key_at_end_of_block = loop {
|
||||
match router.key_at_end_of_block(block.start + 31).await {
|
||||
Ok(Some(key)) => break key,
|
||||
Ok(None) => return vec![],
|
||||
Err(e) => {
|
||||
log::error!("couldn't connect to router for the key at the end of the block: {e:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
let mut all_events = vec![];
|
||||
let mut top_level_txids = HashSet::new();
|
||||
for erc20_addr in [DAI] {
|
||||
let erc20 = Erc20::new(self.provider.clone(), erc20_addr);
|
||||
|
||||
for block in block.start .. (block.start + 32) {
|
||||
let transfers = loop {
|
||||
match erc20.top_level_transfers(block, router.address()).await {
|
||||
Ok(transfers) => break transfers,
|
||||
Err(e) => {
|
||||
log::error!("couldn't connect to Ethereum node for the top-level transfers: {e:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
for transfer in transfers {
|
||||
top_level_txids.insert(transfer.id);
|
||||
all_events.push(EthereumInInstruction {
|
||||
id: (transfer.id, 0),
|
||||
from: transfer.from,
|
||||
coin: EthereumCoin::Erc20(erc20_addr),
|
||||
amount: transfer.amount,
|
||||
data: transfer.data,
|
||||
key_at_end_of_block,
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for block in block.start .. (block.start + 32) {
|
||||
let mut events = router.in_instructions(block, &HashSet::from([DAI])).await;
|
||||
while let Err(e) = events {
|
||||
log::error!("couldn't connect to Ethereum node for the Router's events: {e:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
events = router.in_instructions(block, &HashSet::from([DAI])).await;
|
||||
}
|
||||
let mut events = events.unwrap();
|
||||
for event in &mut events {
|
||||
// A transaction should either be a top-level transfer or a Router InInstruction
|
||||
if top_level_txids.contains(&event.id.0) {
|
||||
panic!("top-level transfer had {} and router had {:?}", hex::encode(event.id.0), event);
|
||||
}
|
||||
// Overwrite the key at end of block to key at end of epoch
|
||||
event.key_at_end_of_block = key_at_end_of_block;
|
||||
}
|
||||
all_events.extend(events);
|
||||
}
|
||||
|
||||
for event in &all_events {
|
||||
assert!(
|
||||
coin_to_serai_coin(&event.coin).is_some(),
|
||||
"router yielded events for unrecognized coins"
|
||||
);
|
||||
}
|
||||
all_events
|
||||
}
|
||||
|
||||
async fn get_eventuality_completions(
|
||||
&self,
|
||||
eventualities: &mut EventualitiesTracker<Self::Eventuality>,
|
||||
block: &Self::Block,
|
||||
) -> HashMap<
|
||||
[u8; 32],
|
||||
(
|
||||
usize,
|
||||
<Self::Transaction as TransactionTrait<Self>>::Id,
|
||||
<Self::Eventuality as EventualityTrait>::Completion,
|
||||
),
|
||||
> {
|
||||
let mut res = HashMap::new();
|
||||
if eventualities.map.is_empty() {
|
||||
return res;
|
||||
}
|
||||
|
||||
let router = self.router().await;
|
||||
let router = router.as_ref().unwrap();
|
||||
|
||||
let past_scanned_epoch = loop {
|
||||
match self.get_block(eventualities.block_number).await {
|
||||
Ok(block) => break block,
|
||||
Err(e) => log::error!("couldn't get the last scanned block in the tracker: {}", e),
|
||||
}
|
||||
sleep(Duration::from_secs(10)).await;
|
||||
};
|
||||
assert_eq!(
|
||||
past_scanned_epoch.start / 32,
|
||||
u64::try_from(eventualities.block_number).unwrap(),
|
||||
"assumption of tracker block number's relation to epoch start is incorrect"
|
||||
);
|
||||
|
||||
// Iterate from after the epoch number in the tracker to the end of this epoch
|
||||
for block_num in (past_scanned_epoch.end() + 1) ..= block.end() {
|
||||
let executed = loop {
|
||||
match router.executed_commands(block_num).await {
|
||||
Ok(executed) => break executed,
|
||||
Err(e) => log::error!("couldn't get the executed commands in block {block_num}: {e}"),
|
||||
}
|
||||
sleep(Duration::from_secs(10)).await;
|
||||
};
|
||||
|
||||
for executed in executed {
|
||||
let lookup = executed.nonce.to_le_bytes().to_vec();
|
||||
if let Some((plan_id, eventuality)) = eventualities.map.get(&lookup) {
|
||||
if let Some(command) =
|
||||
SignedRouterCommand::new(&eventuality.0, eventuality.1.clone(), &executed.signature)
|
||||
{
|
||||
res.insert(*plan_id, (block_num.try_into().unwrap(), executed.tx_id, command));
|
||||
eventualities.map.remove(&lookup);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
eventualities.block_number = (block.start / 32).try_into().unwrap();
|
||||
|
||||
res
|
||||
}
|
||||
|
||||
async fn needed_fee(
|
||||
&self,
|
||||
_block_number: usize,
|
||||
inputs: &[Self::Output],
|
||||
_payments: &[Payment<Self>],
|
||||
_change: &Option<Self::Address>,
|
||||
) -> Result<Option<u64>, NetworkError> {
|
||||
assert_eq!(inputs.len(), 0);
|
||||
// Claim no fee is needed so we can perform amortization ourselves
|
||||
Ok(Some(0))
|
||||
}
|
||||
|
||||
async fn signable_transaction(
|
||||
&self,
|
||||
_block_number: usize,
|
||||
_plan_id: &[u8; 32],
|
||||
key: <Self::Curve as Ciphersuite>::G,
|
||||
inputs: &[Self::Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Self::Address>,
|
||||
scheduler_addendum: &<Self::Scheduler as SchedulerTrait<Self>>::Addendum,
|
||||
) -> Result<Option<(Self::SignableTransaction, Self::Eventuality)>, NetworkError> {
|
||||
assert_eq!(inputs.len(), 0);
|
||||
assert!(change.is_none());
|
||||
let chain_id = self.provider.get_chain_id().await.map_err(|_| NetworkError::ConnectionError)?;
|
||||
|
||||
// TODO: Perform fee amortization (in scheduler?
|
||||
// TODO: Make this function internal and have needed_fee properly return None as expected?
|
||||
// TODO: signable_transaction is written as cannot return None if needed_fee returns Some
|
||||
// TODO: Why can this return None at all if it isn't allowed to return None?
|
||||
|
||||
let command = match scheduler_addendum {
|
||||
Addendum::Nonce(nonce) => RouterCommand::Execute {
|
||||
chain_id: U256::try_from(chain_id).unwrap(),
|
||||
nonce: U256::try_from(*nonce).unwrap(),
|
||||
outs: payments
|
||||
.iter()
|
||||
.filter_map(|payment| {
|
||||
Some(OutInstruction {
|
||||
target: if let Some(data) = payment.data.as_ref() {
|
||||
// This introspects the Call serialization format, expecting the first 20 bytes to
|
||||
// be the address
|
||||
// This avoids wasting the 20-bytes allocated within address
|
||||
let full_data = [payment.address.0.as_slice(), data].concat();
|
||||
let mut reader = full_data.as_slice();
|
||||
|
||||
let mut calls = vec![];
|
||||
while !reader.is_empty() {
|
||||
calls.push(Call::read(&mut reader).ok()?)
|
||||
}
|
||||
// The above must have executed at least once since reader contains the address
|
||||
assert_eq!(calls[0].to, payment.address.0);
|
||||
|
||||
OutInstructionTarget::Calls(calls)
|
||||
} else {
|
||||
OutInstructionTarget::Direct(payment.address.0)
|
||||
},
|
||||
value: {
|
||||
assert_eq!(payment.balance.coin, Coin::Ether); // TODO
|
||||
balance_to_ethereum_amount(payment.balance)
|
||||
},
|
||||
})
|
||||
})
|
||||
.collect(),
|
||||
},
|
||||
Addendum::RotateTo { nonce, new_key } => {
|
||||
assert!(payments.is_empty());
|
||||
RouterCommand::UpdateSeraiKey {
|
||||
chain_id: U256::try_from(chain_id).unwrap(),
|
||||
nonce: U256::try_from(*nonce).unwrap(),
|
||||
key: PublicKey::new(*new_key).expect("new key wasn't a valid ETH public key"),
|
||||
}
|
||||
}
|
||||
};
|
||||
Ok(Some((
|
||||
command.clone(),
|
||||
Eventuality(PublicKey::new(key).expect("key wasn't a valid ETH public key"), command),
|
||||
)))
|
||||
}
|
||||
|
||||
async fn attempt_sign(
|
||||
&self,
|
||||
keys: ThresholdKeys<Self::Curve>,
|
||||
transaction: Self::SignableTransaction,
|
||||
) -> Result<Self::TransactionMachine, NetworkError> {
|
||||
Ok(
|
||||
RouterCommandMachine::new(keys, transaction)
|
||||
.expect("keys weren't usable to sign router commands"),
|
||||
)
|
||||
}
|
||||
|
||||
async fn publish_completion(
|
||||
&self,
|
||||
completion: &<Self::Eventuality as EventualityTrait>::Completion,
|
||||
) -> Result<(), NetworkError> {
|
||||
// Publish this to the dedicated TX server for a solver to actually publish
|
||||
#[cfg(not(test))]
|
||||
{
|
||||
let mut msg = vec![];
|
||||
match completion.command() {
|
||||
RouterCommand::UpdateSeraiKey { nonce, .. } | RouterCommand::Execute { nonce, .. } => {
|
||||
msg.extend(&u32::try_from(nonce).unwrap().to_le_bytes());
|
||||
}
|
||||
}
|
||||
completion.write(&mut msg).unwrap();
|
||||
|
||||
let Ok(mut socket) = TcpStream::connect(&self.relayer_url).await else {
|
||||
log::warn!("couldn't connect to the relayer server");
|
||||
Err(NetworkError::ConnectionError)?
|
||||
};
|
||||
let Ok(()) = socket.write_all(&u32::try_from(msg.len()).unwrap().to_le_bytes()).await else {
|
||||
log::warn!("couldn't send the message's len to the relayer server");
|
||||
Err(NetworkError::ConnectionError)?
|
||||
};
|
||||
let Ok(()) = socket.write_all(&msg).await else {
|
||||
log::warn!("couldn't write the message to the relayer server");
|
||||
Err(NetworkError::ConnectionError)?
|
||||
};
|
||||
if socket.read_u8().await.ok() != Some(1) {
|
||||
log::warn!("didn't get the ack from the relayer server");
|
||||
Err(NetworkError::ConnectionError)?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// Publish this using a dummy account we fund with magic RPC commands
|
||||
#[cfg(test)]
|
||||
{
|
||||
let router = self.router().await;
|
||||
let router = router.as_ref().unwrap();
|
||||
|
||||
let mut tx = match completion.command() {
|
||||
RouterCommand::UpdateSeraiKey { key, .. } => {
|
||||
router.update_serai_key(key, completion.signature())
|
||||
}
|
||||
RouterCommand::Execute { outs, .. } => router.execute(
|
||||
&outs.iter().cloned().map(Into::into).collect::<Vec<_>>(),
|
||||
completion.signature(),
|
||||
),
|
||||
};
|
||||
tx.gas_limit = 1_000_000u64.into();
|
||||
tx.gas_price = 1_000_000_000u64.into();
|
||||
let tx = ethereum_serai::crypto::deterministically_sign(&tx);
|
||||
|
||||
if self.provider.get_transaction_by_hash(*tx.hash()).await.unwrap().is_none() {
|
||||
self
|
||||
.provider
|
||||
.raw_request::<_, ()>(
|
||||
"anvil_setBalance".into(),
|
||||
[
|
||||
tx.recover_signer().unwrap().to_string(),
|
||||
(U256::from(tx.tx().gas_limit) * U256::from(tx.tx().gas_price)).to_string(),
|
||||
],
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let (tx, sig, _) = tx.into_parts();
|
||||
let mut bytes = vec![];
|
||||
tx.encode_with_signature_fields(&sig, &mut bytes);
|
||||
let pending_tx = self.provider.send_raw_transaction(&bytes).await.unwrap();
|
||||
self.mine_block().await;
|
||||
assert!(pending_tx.get_receipt().await.unwrap().status());
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
async fn confirm_completion(
|
||||
&self,
|
||||
eventuality: &Self::Eventuality,
|
||||
claim: &<Self::Eventuality as EventualityTrait>::Claim,
|
||||
) -> Result<Option<<Self::Eventuality as EventualityTrait>::Completion>, NetworkError> {
|
||||
Ok(SignedRouterCommand::new(&eventuality.0, eventuality.1.clone(), &claim.signature))
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_block_number(&self, id: &<Self::Block as Block<Self>>::Id) -> usize {
|
||||
self
|
||||
.provider
|
||||
.get_block(B256::from(*id).into(), BlockTransactionsKind::Hashes)
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap()
|
||||
.header
|
||||
.number
|
||||
.try_into()
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn check_eventuality_by_claim(
|
||||
&self,
|
||||
eventuality: &Self::Eventuality,
|
||||
claim: &<Self::Eventuality as EventualityTrait>::Claim,
|
||||
) -> bool {
|
||||
SignedRouterCommand::new(&eventuality.0, eventuality.1.clone(), &claim.signature).is_some()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_transaction_by_eventuality(
|
||||
&self,
|
||||
block: usize,
|
||||
eventuality: &Self::Eventuality,
|
||||
) -> Self::Transaction {
|
||||
// We mine 96 blocks to ensure the 32 blocks relevant are finalized
|
||||
// Back-check the prior two epochs in response to this
|
||||
// TODO: Review why this is sub(3) and not sub(2)
|
||||
for block in block.saturating_sub(3) ..= block {
|
||||
match eventuality.1 {
|
||||
RouterCommand::UpdateSeraiKey { nonce, .. } | RouterCommand::Execute { nonce, .. } => {
|
||||
let router = self.router().await;
|
||||
let router = router.as_ref().unwrap();
|
||||
|
||||
let block = u64::try_from(block).unwrap();
|
||||
let filter = router
|
||||
.key_updated_filter()
|
||||
.from_block(block * 32)
|
||||
.to_block(((block + 1) * 32) - 1)
|
||||
.topic1(nonce);
|
||||
let logs = self.provider.get_logs(&filter).await.unwrap();
|
||||
if let Some(log) = logs.first() {
|
||||
return self
|
||||
.provider
|
||||
.get_transaction_by_hash(log.clone().transaction_hash.unwrap())
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
};
|
||||
|
||||
let filter = router
|
||||
.executed_filter()
|
||||
.from_block(block * 32)
|
||||
.to_block(((block + 1) * 32) - 1)
|
||||
.topic1(nonce);
|
||||
let logs = self.provider.get_logs(&filter).await.unwrap();
|
||||
if logs.is_empty() {
|
||||
continue;
|
||||
}
|
||||
return self
|
||||
.provider
|
||||
.get_transaction_by_hash(logs[0].transaction_hash.unwrap())
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
}
|
||||
}
|
||||
}
|
||||
panic!("couldn't find completion in any three of checked blocks");
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn mine_block(&self) {
|
||||
self.provider.raw_request::<_, ()>("anvil_mine".into(), [96]).await.unwrap();
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn test_send(&self, send_to: Self::Address) -> Self::Block {
|
||||
use rand_core::OsRng;
|
||||
use ciphersuite::group::ff::Field;
|
||||
use ethereum_serai::alloy::sol_types::SolCall;
|
||||
|
||||
let key = <Secp256k1 as Ciphersuite>::F::random(&mut OsRng);
|
||||
let address = ethereum_serai::crypto::address(&(Secp256k1::generator() * key));
|
||||
|
||||
// Set a 1.1 ETH balance
|
||||
self
|
||||
.provider
|
||||
.raw_request::<_, ()>(
|
||||
"anvil_setBalance".into(),
|
||||
[Address(address).to_string(), "1100000000000000000".into()],
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let value = U256::from_str_radix("1000000000000000000", 10).unwrap();
|
||||
let tx = ethereum_serai::alloy::consensus::TxLegacy {
|
||||
chain_id: None,
|
||||
nonce: 0,
|
||||
gas_price: 1_000_000_000u128,
|
||||
gas_limit: 200_000u128,
|
||||
to: ethereum_serai::alloy::primitives::TxKind::Call(send_to.0.into()),
|
||||
// 1 ETH
|
||||
value,
|
||||
input: ethereum_serai::router::abi::inInstructionCall::new((
|
||||
[0; 20].into(),
|
||||
value,
|
||||
vec![].into(),
|
||||
))
|
||||
.abi_encode()
|
||||
.into(),
|
||||
};
|
||||
|
||||
use ethereum_serai::alloy::{primitives::Signature, consensus::SignableTransaction};
|
||||
let sig = k256::ecdsa::SigningKey::from(k256::elliptic_curve::NonZeroScalar::new(key).unwrap())
|
||||
.sign_prehash_recoverable(tx.signature_hash().as_ref())
|
||||
.unwrap();
|
||||
|
||||
let mut bytes = vec![];
|
||||
tx.encode_with_signature_fields(&Signature::from(sig), &mut bytes);
|
||||
let pending_tx = self.provider.send_raw_transaction(&bytes).await.ok().unwrap();
|
||||
|
||||
// Mine an epoch containing this TX
|
||||
self.mine_block().await;
|
||||
assert!(pending_tx.get_receipt().await.unwrap().status());
|
||||
// Yield the freshly mined block
|
||||
self.get_block(self.get_latest_block_number().await.unwrap()).await.unwrap()
|
||||
}
|
||||
}
|
||||
@@ -1,658 +0,0 @@
|
||||
use core::{fmt::Debug, time::Duration};
|
||||
use std::{io, collections::HashMap};
|
||||
|
||||
use async_trait::async_trait;
|
||||
use thiserror::Error;
|
||||
|
||||
use frost::{
|
||||
dkg::evrf::EvrfCurve,
|
||||
curve::{Ciphersuite, Curve},
|
||||
ThresholdKeys,
|
||||
sign::PreprocessMachine,
|
||||
};
|
||||
|
||||
use serai_client::primitives::{NetworkId, Balance};
|
||||
|
||||
use log::error;
|
||||
|
||||
use tokio::time::sleep;
|
||||
|
||||
#[cfg(feature = "bitcoin")]
|
||||
pub mod bitcoin;
|
||||
#[cfg(feature = "bitcoin")]
|
||||
pub use self::bitcoin::Bitcoin;
|
||||
|
||||
#[cfg(feature = "ethereum")]
|
||||
pub mod ethereum;
|
||||
#[cfg(feature = "ethereum")]
|
||||
pub use ethereum::Ethereum;
|
||||
|
||||
#[cfg(feature = "monero")]
|
||||
pub mod monero;
|
||||
#[cfg(feature = "monero")]
|
||||
pub use monero::Monero;
|
||||
|
||||
use crate::{Payment, Plan, multisigs::scheduler::Scheduler};
|
||||
|
||||
#[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,
|
||||
|
||||
// Forwarded output from the prior multisig
|
||||
Forwarded,
|
||||
}
|
||||
|
||||
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,
|
||||
OutputType::Forwarded => 3,
|
||||
}])
|
||||
}
|
||||
|
||||
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,
|
||||
3 => OutputType::Forwarded,
|
||||
_ => Err(io::Error::other("invalid OutputType"))?,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
pub trait Output<N: Network>: Send + Sync + Sized + Clone + PartialEq + Eq + Debug {
|
||||
type Id: 'static + Id;
|
||||
|
||||
fn kind(&self) -> OutputType;
|
||||
|
||||
fn id(&self) -> Self::Id;
|
||||
fn tx_id(&self) -> <N::Transaction as Transaction<N>>::Id; // TODO: Review use of
|
||||
fn key(&self) -> <N::Curve as Ciphersuite>::G;
|
||||
|
||||
fn presumed_origin(&self) -> Option<N::Address>;
|
||||
|
||||
fn balance(&self) -> Balance;
|
||||
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 + PartialEq + Debug {
|
||||
type Id: 'static + Id;
|
||||
fn id(&self) -> Self::Id;
|
||||
// TODO: Move to Balance
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, network: &N) -> u64;
|
||||
}
|
||||
|
||||
pub trait SignableTransaction: Send + Sync + Clone + Debug {
|
||||
// TODO: Move to Balance
|
||||
fn fee(&self) -> u64;
|
||||
}
|
||||
|
||||
pub trait Eventuality: Send + Sync + Clone + PartialEq + Debug {
|
||||
type Claim: Send + Sync + Clone + PartialEq + Default + AsRef<[u8]> + AsMut<[u8]> + Debug;
|
||||
type Completion: Send + Sync + Clone + PartialEq + Debug;
|
||||
|
||||
fn lookup(&self) -> Vec<u8>;
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self>;
|
||||
fn serialize(&self) -> Vec<u8>;
|
||||
|
||||
fn claim(completion: &Self::Completion) -> Self::Claim;
|
||||
|
||||
// TODO: Make a dedicated Completion trait
|
||||
fn serialize_completion(completion: &Self::Completion) -> Vec<u8>;
|
||||
fn read_completion<R: io::Read>(reader: &mut R) -> io::Result<Self::Completion>;
|
||||
}
|
||||
|
||||
#[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()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
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;
|
||||
/// The monotonic network time at this block.
|
||||
///
|
||||
/// This call is presumed to be expensive and should only be called sparingly.
|
||||
async fn time(&self, rpc: &N) -> u64;
|
||||
}
|
||||
|
||||
// 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
|
||||
fn drop_branches<N: Network>(
|
||||
key: <N::Curve as Ciphersuite>::G,
|
||||
payments: &[Payment<N>],
|
||||
) -> Vec<PostFeeBranch> {
|
||||
let mut branch_outputs = vec![];
|
||||
for payment in payments {
|
||||
if Some(&payment.address) == N::branch_address(key).as_ref() {
|
||||
branch_outputs.push(PostFeeBranch { expected: payment.balance.amount.0, actual: None });
|
||||
}
|
||||
}
|
||||
branch_outputs
|
||||
}
|
||||
|
||||
pub struct PreparedSend<N: Network> {
|
||||
/// None for the transaction if the SignableTransaction was dropped due to lack of value.
|
||||
pub tx: Option<(N::SignableTransaction, N::Eventuality)>,
|
||||
pub post_fee_branches: Vec<PostFeeBranch>,
|
||||
/// The updated operating costs after preparing this transaction.
|
||||
pub operating_costs: u64,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
#[rustfmt::skip]
|
||||
pub trait Network: 'static + Send + Sync + Clone + PartialEq + Debug {
|
||||
/// The elliptic curve used for this network.
|
||||
type Curve: Curve
|
||||
+ EvrfCurve<EmbeddedCurve: Ciphersuite<G: ec_divisors::DivisorCurve<FieldElement = <Self::Curve as Ciphersuite>::F>>>;
|
||||
|
||||
/// The type representing the transaction for this network.
|
||||
type Transaction: Transaction<Self>; // TODO: Review use of
|
||||
/// 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<Self>;
|
||||
/// The type containing all information on a planned transaction, waiting to be signed.
|
||||
type SignableTransaction: SignableTransaction;
|
||||
/// The type containing all information to check if a plan was completed.
|
||||
///
|
||||
/// This must be binding to both the outputs expected and the plan ID.
|
||||
type Eventuality: Eventuality;
|
||||
/// The FROST machine to sign a transaction.
|
||||
type TransactionMachine: PreprocessMachine<
|
||||
Signature = <Self::Eventuality as Eventuality>::Completion,
|
||||
>;
|
||||
|
||||
/// The scheduler for this network.
|
||||
type Scheduler: Scheduler<Self>;
|
||||
|
||||
/// 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 estimated amount of time a block will take.
|
||||
const ESTIMATED_BLOCK_TIME_IN_SECONDS: usize;
|
||||
/// The amount of confirmations required to consider a block 'final'.
|
||||
const CONFIRMATIONS: 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.
|
||||
///
|
||||
/// For any received output, there's the cost to spend the output. This value MUST exceed the
|
||||
/// cost to spend said output, and should by a notable margin (not just 2x, yet an order of
|
||||
/// magnitude).
|
||||
// TODO: Dust needs to be diversified per Coin
|
||||
const DUST: u64;
|
||||
|
||||
/// The cost to perform input aggregation with a 2-input 1-output TX.
|
||||
const COST_TO_AGGREGATE: 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.
|
||||
#[cfg(test)]
|
||||
async fn external_address(&self, key: <Self::Curve as Ciphersuite>::G) -> Self::Address;
|
||||
/// Address for the given group key to use for scheduled branches.
|
||||
fn branch_address(key: <Self::Curve as Ciphersuite>::G) -> Option<Self::Address>;
|
||||
/// Address for the given group key to use for change.
|
||||
fn change_address(key: <Self::Curve as Ciphersuite>::G) -> Option<Self::Address>;
|
||||
/// Address for forwarded outputs from prior multisigs.
|
||||
///
|
||||
/// forward_address must only return None if explicit forwarding isn't necessary.
|
||||
fn forward_address(key: <Self::Curve as Ciphersuite>::G) -> Option<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 latest block's number, retrying until success.
|
||||
async fn get_latest_block_number_with_retries(&self) -> usize {
|
||||
loop {
|
||||
match self.get_latest_block_number().await {
|
||||
Ok(number) => {
|
||||
return number;
|
||||
}
|
||||
Err(e) => {
|
||||
error!(
|
||||
"couldn't get the latest block number in the with retry get_latest_block_number: {e:?}",
|
||||
);
|
||||
sleep(Duration::from_secs(10)).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Get a block, retrying until success.
|
||||
async fn get_block_with_retries(&self, block_number: usize) -> Self::Block {
|
||||
loop {
|
||||
match self.get_block(block_number).await {
|
||||
Ok(block) => {
|
||||
return block;
|
||||
}
|
||||
Err(e) => {
|
||||
error!("couldn't get block {block_number} in the with retry get_block: {:?}", e);
|
||||
sleep(Duration::from_secs(10)).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the outputs within a block for a specific key.
|
||||
async fn get_outputs(
|
||||
&self,
|
||||
block: &Self::Block,
|
||||
key: <Self::Curve as Ciphersuite>::G,
|
||||
) -> Vec<Self::Output>;
|
||||
|
||||
/// Get the registered eventualities completed within this block, and any prior blocks which
|
||||
/// registered eventualities may have been completed in.
|
||||
///
|
||||
/// This may panic if not fed a block greater than the tracker's block number.
|
||||
///
|
||||
/// Plan ID -> (block number, TX ID, completion)
|
||||
// TODO: get_eventuality_completions_internal + provided get_eventuality_completions for common
|
||||
// code
|
||||
// TODO: Consider having this return the Transaction + the Completion?
|
||||
// Or Transaction with extract_completion?
|
||||
async fn get_eventuality_completions(
|
||||
&self,
|
||||
eventualities: &mut EventualitiesTracker<Self::Eventuality>,
|
||||
block: &Self::Block,
|
||||
) -> HashMap<
|
||||
[u8; 32],
|
||||
(
|
||||
usize,
|
||||
<Self::Transaction as Transaction<Self>>::Id,
|
||||
<Self::Eventuality as Eventuality>::Completion,
|
||||
),
|
||||
>;
|
||||
|
||||
/// Returns the needed fee to fulfill this Plan at this fee rate.
|
||||
///
|
||||
/// Returns None if this Plan isn't fulfillable (such as when the fee exceeds the input value).
|
||||
async fn needed_fee(
|
||||
&self,
|
||||
block_number: usize,
|
||||
inputs: &[Self::Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Self::Address>,
|
||||
) -> Result<Option<u64>, NetworkError>;
|
||||
|
||||
/// Create a SignableTransaction for the given Plan.
|
||||
///
|
||||
/// The expected flow is:
|
||||
/// 1) Call needed_fee
|
||||
/// 2) If the Plan is fulfillable, amortize the fee
|
||||
/// 3) Call signable_transaction *which MUST NOT return None if the above was done properly*
|
||||
///
|
||||
/// This takes a destructured Plan as some of these arguments are malleated from the original
|
||||
/// Plan.
|
||||
// TODO: Explicit AmortizedPlan?
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
async fn signable_transaction(
|
||||
&self,
|
||||
block_number: usize,
|
||||
plan_id: &[u8; 32],
|
||||
key: <Self::Curve as Ciphersuite>::G,
|
||||
inputs: &[Self::Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Self::Address>,
|
||||
scheduler_addendum: &<Self::Scheduler as Scheduler<Self>>::Addendum,
|
||||
) -> Result<Option<(Self::SignableTransaction, Self::Eventuality)>, NetworkError>;
|
||||
|
||||
/// Prepare a SignableTransaction for a transaction.
|
||||
///
|
||||
/// This must not persist anything as we will prepare Plans we never intend to execute.
|
||||
async fn prepare_send(
|
||||
&self,
|
||||
block_number: usize,
|
||||
plan: Plan<Self>,
|
||||
operating_costs: u64,
|
||||
) -> Result<PreparedSend<Self>, NetworkError> {
|
||||
// Sanity check this has at least one output planned
|
||||
assert!((!plan.payments.is_empty()) || plan.change.is_some());
|
||||
|
||||
let plan_id = plan.id();
|
||||
let Plan { key, inputs, mut payments, change, scheduler_addendum } = plan;
|
||||
let theoretical_change_amount = if change.is_some() {
|
||||
inputs.iter().map(|input| input.balance().amount.0).sum::<u64>() -
|
||||
payments.iter().map(|payment| payment.balance.amount.0).sum::<u64>()
|
||||
} else {
|
||||
0
|
||||
};
|
||||
|
||||
let Some(tx_fee) = self.needed_fee(block_number, &inputs, &payments, &change).await? else {
|
||||
// This Plan is not fulfillable
|
||||
// TODO: Have Plan explicitly distinguish payments and branches in two separate Vecs?
|
||||
return Ok(PreparedSend {
|
||||
tx: None,
|
||||
// Have all of its branches dropped
|
||||
post_fee_branches: drop_branches(key, &payments),
|
||||
// This plan expects a change output valued at sum(inputs) - sum(outputs)
|
||||
// Since we can no longer create this change output, it becomes an operating cost
|
||||
// TODO: Look at input restoration to reduce this operating cost
|
||||
operating_costs: operating_costs +
|
||||
if change.is_some() { theoretical_change_amount } else { 0 },
|
||||
});
|
||||
};
|
||||
|
||||
// Amortize the fee over the plan's payments
|
||||
let (post_fee_branches, mut operating_costs) = (|| {
|
||||
// If we're creating a change output, letting us recoup coins, amortize the operating costs
|
||||
// as well
|
||||
let total_fee = tx_fee + if change.is_some() { operating_costs } else { 0 };
|
||||
|
||||
let original_outputs = payments.iter().map(|payment| payment.balance.amount.0).sum::<u64>();
|
||||
// If this isn't enough for the total fee, drop and move on
|
||||
if original_outputs < total_fee {
|
||||
let mut remaining_operating_costs = operating_costs;
|
||||
if change.is_some() {
|
||||
// Operating costs increase by the TX fee
|
||||
remaining_operating_costs += tx_fee;
|
||||
// Yet decrease by the payments we managed to drop
|
||||
remaining_operating_costs = remaining_operating_costs.saturating_sub(original_outputs);
|
||||
}
|
||||
return (drop_branches(key, &payments), remaining_operating_costs);
|
||||
}
|
||||
|
||||
let initial_payment_amounts =
|
||||
payments.iter().map(|payment| payment.balance.amount.0).collect::<Vec<_>>();
|
||||
|
||||
// Amortize the transaction fee across outputs
|
||||
let mut remaining_fee = total_fee;
|
||||
// Run as many times as needed until we can successfully subtract this fee
|
||||
while remaining_fee != 0 {
|
||||
// This shouldn't be a / by 0 as these payments have enough value to cover the fee
|
||||
let this_iter_fee = remaining_fee / u64::try_from(payments.len()).unwrap();
|
||||
let mut overage = remaining_fee % u64::try_from(payments.len()).unwrap();
|
||||
for payment in &mut payments {
|
||||
let this_payment_fee = this_iter_fee + overage;
|
||||
// Only subtract the overage once
|
||||
overage = 0;
|
||||
|
||||
let subtractable = payment.balance.amount.0.min(this_payment_fee);
|
||||
remaining_fee -= subtractable;
|
||||
payment.balance.amount.0 -= subtractable;
|
||||
}
|
||||
}
|
||||
|
||||
// If any payment is now below the dust threshold, set its value to 0 so it'll be dropped
|
||||
for payment in &mut payments {
|
||||
if payment.balance.amount.0 < Self::DUST {
|
||||
payment.balance.amount.0 = 0;
|
||||
}
|
||||
}
|
||||
|
||||
// Note the branch outputs' new values
|
||||
let mut branch_outputs = vec![];
|
||||
for (initial_amount, payment) in initial_payment_amounts.into_iter().zip(&payments) {
|
||||
if Some(&payment.address) == Self::branch_address(key).as_ref() {
|
||||
branch_outputs.push(PostFeeBranch {
|
||||
expected: initial_amount,
|
||||
actual: if payment.balance.amount.0 == 0 {
|
||||
None
|
||||
} else {
|
||||
Some(payment.balance.amount.0)
|
||||
},
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
// Drop payments now worth 0
|
||||
payments = payments
|
||||
.drain(..)
|
||||
.filter(|payment| {
|
||||
if payment.balance.amount.0 != 0 {
|
||||
true
|
||||
} else {
|
||||
log::debug!("dropping dust payment from plan {}", hex::encode(plan_id));
|
||||
false
|
||||
}
|
||||
})
|
||||
.collect();
|
||||
|
||||
// Sanity check the fee was successfully amortized
|
||||
let new_outputs = payments.iter().map(|payment| payment.balance.amount.0).sum::<u64>();
|
||||
assert!((new_outputs + total_fee) <= original_outputs);
|
||||
|
||||
(
|
||||
branch_outputs,
|
||||
if change.is_none() {
|
||||
// If the change is None, this had no effect on the operating costs
|
||||
operating_costs
|
||||
} else {
|
||||
// Since the change is some, and we successfully amortized, the operating costs were
|
||||
// recouped
|
||||
0
|
||||
},
|
||||
)
|
||||
})();
|
||||
|
||||
let Some(tx) = self
|
||||
.signable_transaction(
|
||||
block_number,
|
||||
&plan_id,
|
||||
key,
|
||||
&inputs,
|
||||
&payments,
|
||||
&change,
|
||||
&scheduler_addendum,
|
||||
)
|
||||
.await?
|
||||
else {
|
||||
panic!(
|
||||
"{}. {}: {}, {}: {:?}, {}: {:?}, {}: {:?}, {}: {}, {}: {:?}",
|
||||
"signable_transaction returned None for a TX we prior successfully calculated the fee for",
|
||||
"id",
|
||||
hex::encode(plan_id),
|
||||
"inputs",
|
||||
inputs,
|
||||
"post-amortization payments",
|
||||
payments,
|
||||
"change",
|
||||
change,
|
||||
"successfully amoritized fee",
|
||||
tx_fee,
|
||||
"scheduler's addendum",
|
||||
scheduler_addendum,
|
||||
)
|
||||
};
|
||||
|
||||
if change.is_some() {
|
||||
let on_chain_expected_change =
|
||||
inputs.iter().map(|input| input.balance().amount.0).sum::<u64>() -
|
||||
payments.iter().map(|payment| payment.balance.amount.0).sum::<u64>() -
|
||||
tx_fee;
|
||||
// If the change value is less than the dust threshold, it becomes an operating cost
|
||||
// This may be slightly inaccurate as dropping payments may reduce the fee, raising the
|
||||
// change above dust
|
||||
// That's fine since it'd have to be in a very precarious state AND then it's over-eager in
|
||||
// tabulating costs
|
||||
if on_chain_expected_change < Self::DUST {
|
||||
operating_costs += theoretical_change_amount;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(PreparedSend { tx: Some(tx), post_fee_branches, operating_costs })
|
||||
}
|
||||
|
||||
/// Attempt to sign a SignableTransaction.
|
||||
async fn attempt_sign(
|
||||
&self,
|
||||
keys: ThresholdKeys<Self::Curve>,
|
||||
transaction: Self::SignableTransaction,
|
||||
) -> Result<Self::TransactionMachine, NetworkError>;
|
||||
|
||||
/// Publish a completion.
|
||||
async fn publish_completion(
|
||||
&self,
|
||||
completion: &<Self::Eventuality as Eventuality>::Completion,
|
||||
) -> Result<(), NetworkError>;
|
||||
|
||||
/// Confirm a plan was completed by the specified transaction, per our bounds.
|
||||
///
|
||||
/// Returns Err if there was an error with the confirmation methodology.
|
||||
/// Returns Ok(None) if this is not a valid completion.
|
||||
/// Returns Ok(Some(_)) with the completion if it's valid.
|
||||
async fn confirm_completion(
|
||||
&self,
|
||||
eventuality: &Self::Eventuality,
|
||||
claim: &<Self::Eventuality as Eventuality>::Claim,
|
||||
) -> Result<Option<<Self::Eventuality as Eventuality>::Completion>, NetworkError>;
|
||||
|
||||
/// Get a block's number by its ID.
|
||||
#[cfg(test)]
|
||||
async fn get_block_number(&self, id: &<Self::Block as Block<Self>>::Id) -> usize;
|
||||
|
||||
/// Check an Eventuality is fulfilled by a claim.
|
||||
#[cfg(test)]
|
||||
async fn check_eventuality_by_claim(
|
||||
&self,
|
||||
eventuality: &Self::Eventuality,
|
||||
claim: &<Self::Eventuality as Eventuality>::Claim,
|
||||
) -> bool;
|
||||
|
||||
/// Get a transaction by the Eventuality it completes.
|
||||
#[cfg(test)]
|
||||
async fn get_transaction_by_eventuality(
|
||||
&self,
|
||||
block: usize,
|
||||
eventuality: &Self::Eventuality,
|
||||
) -> Self::Transaction;
|
||||
|
||||
#[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;
|
||||
}
|
||||
|
||||
pub trait UtxoNetwork: Network {
|
||||
/// 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;
|
||||
}
|
||||
@@ -1,807 +0,0 @@
|
||||
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_simple_request_rpc::SimpleRequestRpc;
|
||||
use monero_wallet::{
|
||||
ringct::RctType,
|
||||
transaction::Transaction,
|
||||
block::Block,
|
||||
rpc::{FeeRate, RpcError, Rpc},
|
||||
address::{Network as MoneroNetwork, SubaddressIndex},
|
||||
ViewPair, GuaranteedViewPair, WalletOutput, OutputWithDecoys, GuaranteedScanner,
|
||||
send::{
|
||||
SendError, Change, SignableTransaction as MSignableTransaction, Eventuality, TransactionMachine,
|
||||
},
|
||||
};
|
||||
#[cfg(test)]
|
||||
use monero_wallet::Scanner;
|
||||
|
||||
use tokio::time::sleep;
|
||||
|
||||
pub use serai_client::{
|
||||
primitives::{MAX_DATA_LEN, Coin, NetworkId, Amount, Balance},
|
||||
networks::monero::Address,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
Payment, additional_key,
|
||||
networks::{
|
||||
NetworkError, Block as BlockTrait, OutputType, Output as OutputTrait,
|
||||
Transaction as TransactionTrait, SignableTransaction as SignableTransactionTrait,
|
||||
Eventuality as EventualityTrait, EventualitiesTracker, Network, UtxoNetwork,
|
||||
},
|
||||
multisigs::scheduler::utxo::Scheduler,
|
||||
};
|
||||
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
pub struct Output(WalletOutput);
|
||||
|
||||
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);
|
||||
const FORWARD_SUBADDRESS: Option<SubaddressIndex> = SubaddressIndex::new(3, 0);
|
||||
|
||||
impl OutputTrait<Monero> 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.subaddress() {
|
||||
EXTERNAL_SUBADDRESS => OutputType::External,
|
||||
BRANCH_SUBADDRESS => OutputType::Branch,
|
||||
CHANGE_SUBADDRESS => OutputType::Change,
|
||||
FORWARD_SUBADDRESS => OutputType::Forwarded,
|
||||
_ => panic!("unrecognized address was scanned for"),
|
||||
}
|
||||
}
|
||||
|
||||
fn id(&self) -> Self::Id {
|
||||
self.0.key().compress().to_bytes()
|
||||
}
|
||||
|
||||
fn tx_id(&self) -> [u8; 32] {
|
||||
self.0.transaction()
|
||||
}
|
||||
|
||||
fn key(&self) -> EdwardsPoint {
|
||||
EdwardsPoint(self.0.key() - (EdwardsPoint::generator().0 * self.0.key_offset()))
|
||||
}
|
||||
|
||||
fn presumed_origin(&self) -> Option<Address> {
|
||||
None
|
||||
}
|
||||
|
||||
fn balance(&self) -> Balance {
|
||||
Balance { coin: Coin::Monero, amount: Amount(self.0.commitment().amount) }
|
||||
}
|
||||
|
||||
fn data(&self) -> &[u8] {
|
||||
let Some(data) = self.0.arbitrary_data().first() else { return &[] };
|
||||
// If the data is too large, prune it
|
||||
// This should cause decoding the instruction to fail, and trigger a refund as appropriate
|
||||
if data.len() > usize::try_from(MAX_DATA_LEN).unwrap() {
|
||||
return &[];
|
||||
}
|
||||
data
|
||||
}
|
||||
|
||||
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
|
||||
self.0.write(writer)?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
Ok(Output(WalletOutput::read(reader)?))
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: Consider ([u8; 32], TransactionPruned)
|
||||
#[async_trait]
|
||||
impl TransactionTrait<Monero> for Transaction {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> Self::Id {
|
||||
self.hash()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn fee(&self, _: &Monero) -> u64 {
|
||||
match self {
|
||||
Transaction::V1 { .. } => panic!("v1 TX in test-only function"),
|
||||
Transaction::V2 { ref proofs, .. } => proofs.as_ref().unwrap().base.fee,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl EventualityTrait for Eventuality {
|
||||
type Claim = [u8; 32];
|
||||
type Completion = Transaction;
|
||||
|
||||
// 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()
|
||||
}
|
||||
|
||||
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
|
||||
Eventuality::read(reader)
|
||||
}
|
||||
fn serialize(&self) -> Vec<u8> {
|
||||
self.serialize()
|
||||
}
|
||||
|
||||
fn claim(tx: &Transaction) -> [u8; 32] {
|
||||
tx.id()
|
||||
}
|
||||
fn serialize_completion(completion: &Transaction) -> Vec<u8> {
|
||||
completion.serialize()
|
||||
}
|
||||
fn read_completion<R: io::Read>(reader: &mut R) -> io::Result<Transaction> {
|
||||
Transaction::read(reader)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct SignableTransaction(MSignableTransaction);
|
||||
impl SignableTransactionTrait for SignableTransaction {
|
||||
fn fee(&self) -> u64 {
|
||||
self.0.necessary_fee()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl BlockTrait<Monero> for Block {
|
||||
type Id = [u8; 32];
|
||||
fn id(&self) -> Self::Id {
|
||||
self.hash()
|
||||
}
|
||||
|
||||
fn parent(&self) -> Self::Id {
|
||||
self.header.previous
|
||||
}
|
||||
|
||||
async fn time(&self, rpc: &Monero) -> u64 {
|
||||
// Constant from Monero
|
||||
const BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW: usize = 60;
|
||||
|
||||
// If Monero doesn't have enough blocks to build a window, it doesn't define a network time
|
||||
if (self.number().unwrap() + 1) < BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW {
|
||||
// Use the block number as the time
|
||||
return u64::try_from(self.number().unwrap()).unwrap();
|
||||
}
|
||||
|
||||
let mut timestamps = vec![self.header.timestamp];
|
||||
let mut parent = self.parent();
|
||||
while timestamps.len() < BLOCKCHAIN_TIMESTAMP_CHECK_WINDOW {
|
||||
let mut parent_block;
|
||||
while {
|
||||
parent_block = rpc.rpc.get_block(parent).await;
|
||||
parent_block.is_err()
|
||||
} {
|
||||
log::error!("couldn't get parent block when trying to get block time: {parent_block:?}");
|
||||
sleep(Duration::from_secs(5)).await;
|
||||
}
|
||||
let parent_block = parent_block.unwrap();
|
||||
timestamps.push(parent_block.header.timestamp);
|
||||
parent = parent_block.parent();
|
||||
|
||||
if parent_block.number().unwrap() == 0 {
|
||||
break;
|
||||
}
|
||||
}
|
||||
timestamps.sort();
|
||||
|
||||
// Because 60 has two medians, Monero's epee picks the in-between value, calculated by the
|
||||
// following formula (from the "get_mid" function)
|
||||
let n = timestamps.len() / 2;
|
||||
let a = timestamps[n - 1];
|
||||
let b = timestamps[n];
|
||||
#[rustfmt::skip] // Enables Ctrl+F'ing for everything after the `= `
|
||||
let res = (a/2) + (b/2) + ((a - 2*(a/2)) + (b - 2*(b/2)))/2;
|
||||
// Technically, res may be 1 if all prior blocks had a timestamp by 0, which would break
|
||||
// monotonicity with our above definition of height as time
|
||||
// Monero also solely requires the block's time not be less than the median, it doesn't ensure
|
||||
// it advances the median forward
|
||||
// Ensure monotonicity despite both these issues by adding the block number to the median time
|
||||
res + u64::try_from(self.number().unwrap()).unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct Monero {
|
||||
rpc: SimpleRequestRpc,
|
||||
}
|
||||
// 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 {}
|
||||
|
||||
#[allow(clippy::needless_pass_by_value)] // Needed to satisfy API expectations
|
||||
fn map_rpc_err(err: RpcError) -> NetworkError {
|
||||
if let RpcError::InvalidNode(reason) = &err {
|
||||
log::error!("Monero RpcError::InvalidNode({reason})");
|
||||
} else {
|
||||
log::debug!("Monero RpcError {err:?}");
|
||||
}
|
||||
NetworkError::ConnectionError
|
||||
}
|
||||
|
||||
enum MakeSignableTransactionResult {
|
||||
Fee(u64),
|
||||
SignableTransaction(MSignableTransaction),
|
||||
}
|
||||
|
||||
impl Monero {
|
||||
pub async fn new(url: String) -> Monero {
|
||||
let mut res = SimpleRequestRpc::new(url.clone()).await;
|
||||
while let Err(e) = res {
|
||||
log::error!("couldn't connect to Monero node: {e:?}");
|
||||
tokio::time::sleep(Duration::from_secs(5)).await;
|
||||
res = SimpleRequestRpc::new(url.clone()).await;
|
||||
}
|
||||
Monero { rpc: res.unwrap() }
|
||||
}
|
||||
|
||||
fn view_pair(spend: EdwardsPoint) -> GuaranteedViewPair {
|
||||
GuaranteedViewPair::new(spend.0, Zeroizing::new(additional_key::<Monero>(0).0)).unwrap()
|
||||
}
|
||||
|
||||
fn address_internal(spend: EdwardsPoint, subaddress: Option<SubaddressIndex>) -> Address {
|
||||
Address::new(Self::view_pair(spend).address(MoneroNetwork::Mainnet, subaddress, None)).unwrap()
|
||||
}
|
||||
|
||||
fn scanner(spend: EdwardsPoint) -> GuaranteedScanner {
|
||||
let mut scanner = GuaranteedScanner::new(Self::view_pair(spend));
|
||||
debug_assert!(EXTERNAL_SUBADDRESS.is_none());
|
||||
scanner.register_subaddress(BRANCH_SUBADDRESS.unwrap());
|
||||
scanner.register_subaddress(CHANGE_SUBADDRESS.unwrap());
|
||||
scanner.register_subaddress(FORWARD_SUBADDRESS.unwrap());
|
||||
scanner
|
||||
}
|
||||
|
||||
async fn median_fee(&self, block: &Block) -> Result<FeeRate, NetworkError> {
|
||||
let mut fees = vec![];
|
||||
for tx_hash in &block.transactions {
|
||||
let tx =
|
||||
self.rpc.get_transaction(*tx_hash).await.map_err(|_| NetworkError::ConnectionError)?;
|
||||
// Only consider fees from RCT transactions, else the fee property read wouldn't be accurate
|
||||
let fee = match &tx {
|
||||
Transaction::V2 { proofs: Some(proofs), .. } => proofs.base.fee,
|
||||
_ => continue,
|
||||
};
|
||||
fees.push(fee / u64::try_from(tx.weight()).unwrap());
|
||||
}
|
||||
fees.sort();
|
||||
let fee = fees.get(fees.len() / 2).copied().unwrap_or(0);
|
||||
|
||||
// TODO: Set a sane minimum fee
|
||||
const MINIMUM_FEE: u64 = 1_500_000;
|
||||
Ok(FeeRate::new(fee.max(MINIMUM_FEE), 10000).unwrap())
|
||||
}
|
||||
|
||||
async fn make_signable_transaction(
|
||||
&self,
|
||||
block_number: usize,
|
||||
plan_id: &[u8; 32],
|
||||
inputs: &[Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Address>,
|
||||
calculating_fee: bool,
|
||||
) -> Result<Option<MakeSignableTransactionResult>, NetworkError> {
|
||||
for payment in payments {
|
||||
assert_eq!(payment.balance.coin, Coin::Monero);
|
||||
}
|
||||
|
||||
// TODO2: Use an fee representative of several blocks, cached inside Self
|
||||
let block_for_fee = self.get_block(block_number).await?;
|
||||
let fee_rate = self.median_fee(&block_for_fee).await?;
|
||||
|
||||
// Determine the RCT proofs to make based off the hard fork
|
||||
// TODO: Make a fn for this block which is duplicated with tests
|
||||
let rct_type = match block_for_fee.header.hardfork_version {
|
||||
14 => RctType::ClsagBulletproof,
|
||||
15 | 16 => RctType::ClsagBulletproofPlus,
|
||||
_ => panic!("Monero hard forked and the processor wasn't updated for it"),
|
||||
};
|
||||
|
||||
let mut transcript =
|
||||
RecommendedTranscript::new(b"Serai Processor Monero Transaction Transcript");
|
||||
transcript.append_message(b"plan", plan_id);
|
||||
|
||||
// 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 mut inputs_actual = Vec::with_capacity(inputs.len());
|
||||
for input in inputs {
|
||||
inputs_actual.push(
|
||||
OutputWithDecoys::fingerprintable_deterministic_new(
|
||||
&mut ChaCha20Rng::from_seed(transcript.rng_seed(b"decoys")),
|
||||
&self.rpc,
|
||||
// TODO: Have Decoys take RctType
|
||||
match rct_type {
|
||||
RctType::ClsagBulletproof => 11,
|
||||
RctType::ClsagBulletproofPlus => 16,
|
||||
_ => panic!("selecting decoys for an unsupported RctType"),
|
||||
},
|
||||
block_number + 1,
|
||||
input.0.clone(),
|
||||
)
|
||||
.await
|
||||
.map_err(map_rpc_err)?,
|
||||
);
|
||||
}
|
||||
|
||||
// Monero requires at least two outputs
|
||||
// If we only have one output planned, add a dummy payment
|
||||
let mut payments = payments.to_vec();
|
||||
let outputs = payments.len() + usize::from(u8::from(change.is_some()));
|
||||
if outputs == 0 {
|
||||
return Ok(None);
|
||||
} else if outputs == 1 {
|
||||
payments.push(Payment {
|
||||
address: Address::new(
|
||||
ViewPair::new(EdwardsPoint::generator().0, Zeroizing::new(Scalar::ONE.0))
|
||||
.unwrap()
|
||||
.legacy_address(MoneroNetwork::Mainnet),
|
||||
)
|
||||
.unwrap(),
|
||||
balance: Balance { coin: Coin::Monero, amount: Amount(0) },
|
||||
data: None,
|
||||
});
|
||||
}
|
||||
|
||||
let payments = payments
|
||||
.into_iter()
|
||||
.map(|payment| (payment.address.into(), payment.balance.amount.0))
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
match MSignableTransaction::new(
|
||||
rct_type,
|
||||
// Use the plan ID as the outgoing view key
|
||||
Zeroizing::new(*plan_id),
|
||||
inputs_actual,
|
||||
payments,
|
||||
Change::fingerprintable(change.as_ref().map(|change| change.clone().into())),
|
||||
vec![],
|
||||
fee_rate,
|
||||
) {
|
||||
Ok(signable) => Ok(Some({
|
||||
if calculating_fee {
|
||||
MakeSignableTransactionResult::Fee(signable.necessary_fee())
|
||||
} else {
|
||||
MakeSignableTransactionResult::SignableTransaction(signable)
|
||||
}
|
||||
})),
|
||||
Err(e) => match e {
|
||||
SendError::UnsupportedRctType => {
|
||||
panic!("trying to use an RctType unsupported by monero-wallet")
|
||||
}
|
||||
SendError::NoInputs |
|
||||
SendError::InvalidDecoyQuantity |
|
||||
SendError::NoOutputs |
|
||||
SendError::TooManyOutputs |
|
||||
SendError::NoChange |
|
||||
SendError::TooMuchArbitraryData |
|
||||
SendError::TooLargeTransaction |
|
||||
SendError::WrongPrivateKey => {
|
||||
panic!("created an invalid Monero transaction: {e}");
|
||||
}
|
||||
SendError::MultiplePaymentIds => {
|
||||
panic!("multiple payment IDs despite not supporting integrated addresses");
|
||||
}
|
||||
SendError::NotEnoughFunds { inputs, outputs, necessary_fee } => {
|
||||
log::debug!(
|
||||
"Monero NotEnoughFunds. inputs: {:?}, outputs: {:?}, necessary_fee: {necessary_fee:?}",
|
||||
inputs,
|
||||
outputs
|
||||
);
|
||||
match necessary_fee {
|
||||
Some(necessary_fee) => {
|
||||
// If we're solely calculating the fee, return the fee this TX will cost
|
||||
if calculating_fee {
|
||||
Ok(Some(MakeSignableTransactionResult::Fee(necessary_fee)))
|
||||
} else {
|
||||
// If we're actually trying to make the TX, return None
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
// We didn't have enough funds to even cover the outputs
|
||||
None => {
|
||||
// Ensure we're not misinterpreting this
|
||||
assert!(outputs > inputs);
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
}
|
||||
SendError::MaliciousSerialization | SendError::ClsagError(_) | SendError::FrostError(_) => {
|
||||
panic!("supposedly unreachable (at this time) Monero error: {e}");
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn test_view_pair() -> ViewPair {
|
||||
ViewPair::new(*EdwardsPoint::generator(), Zeroizing::new(Scalar::ONE.0)).unwrap()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn test_scanner() -> Scanner {
|
||||
Scanner::new(Self::test_view_pair())
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
fn test_address() -> Address {
|
||||
Address::new(Self::test_view_pair().legacy_address(MoneroNetwork::Mainnet)).unwrap()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl Network for Monero {
|
||||
type Curve = Ed25519;
|
||||
|
||||
type Transaction = Transaction;
|
||||
type Block = Block;
|
||||
|
||||
type Output = Output;
|
||||
type SignableTransaction = SignableTransaction;
|
||||
type Eventuality = Eventuality;
|
||||
type TransactionMachine = TransactionMachine;
|
||||
|
||||
type Scheduler = Scheduler<Monero>;
|
||||
|
||||
type Address = Address;
|
||||
|
||||
const NETWORK: NetworkId = NetworkId::Monero;
|
||||
const ID: &'static str = "Monero";
|
||||
const ESTIMATED_BLOCK_TIME_IN_SECONDS: usize = 120;
|
||||
const CONFIRMATIONS: usize = 10;
|
||||
|
||||
const MAX_OUTPUTS: usize = 16;
|
||||
|
||||
// 0.01 XMR
|
||||
const DUST: u64 = 10000000000;
|
||||
|
||||
// TODO
|
||||
const COST_TO_AGGREGATE: u64 = 0;
|
||||
|
||||
// Monero doesn't require/benefit from tweaking
|
||||
fn tweak_keys(_: &mut ThresholdKeys<Self::Curve>) {}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn external_address(&self, key: EdwardsPoint) -> Address {
|
||||
Self::address_internal(key, EXTERNAL_SUBADDRESS)
|
||||
}
|
||||
|
||||
fn branch_address(key: EdwardsPoint) -> Option<Address> {
|
||||
Some(Self::address_internal(key, BRANCH_SUBADDRESS))
|
||||
}
|
||||
|
||||
fn change_address(key: EdwardsPoint) -> Option<Address> {
|
||||
Some(Self::address_internal(key, CHANGE_SUBADDRESS))
|
||||
}
|
||||
|
||||
fn forward_address(key: EdwardsPoint) -> Option<Address> {
|
||||
Some(Self::address_internal(key, FORWARD_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(map_rpc_err)? - 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(map_rpc_err)?)
|
||||
.await
|
||||
.map_err(map_rpc_err)?,
|
||||
)
|
||||
}
|
||||
|
||||
async fn get_outputs(&self, block: &Block, key: EdwardsPoint) -> Vec<Output> {
|
||||
let outputs = loop {
|
||||
match self
|
||||
.rpc
|
||||
.get_scannable_block(block.clone())
|
||||
.await
|
||||
.map_err(|e| format!("{e:?}"))
|
||||
.and_then(|block| Self::scanner(key).scan(block).map_err(|e| format!("{e:?}")))
|
||||
{
|
||||
Ok(outputs) => break outputs,
|
||||
Err(e) => {
|
||||
log::error!("couldn't scan block {}: {e:?}", hex::encode(block.id()));
|
||||
sleep(Duration::from_secs(60)).await;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Miner transactions are required to explicitly state their timelock, so this does exclude
|
||||
// those (which have an extended timelock we don't want to deal with)
|
||||
let raw_outputs = outputs.not_additionally_locked();
|
||||
let mut outputs = Vec::with_capacity(raw_outputs.len());
|
||||
for output in raw_outputs {
|
||||
// This should be pointless as we shouldn't be able to scan for any other subaddress
|
||||
// This just helps ensures nothing invalid makes it through
|
||||
assert!([EXTERNAL_SUBADDRESS, BRANCH_SUBADDRESS, CHANGE_SUBADDRESS, FORWARD_SUBADDRESS]
|
||||
.contains(&output.subaddress()));
|
||||
|
||||
outputs.push(Output(output));
|
||||
}
|
||||
|
||||
outputs
|
||||
}
|
||||
|
||||
async fn get_eventuality_completions(
|
||||
&self,
|
||||
eventualities: &mut EventualitiesTracker<Eventuality>,
|
||||
block: &Block,
|
||||
) -> HashMap<[u8; 32], (usize, [u8; 32], Transaction)> {
|
||||
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], (usize, [u8; 32], Transaction)>,
|
||||
) {
|
||||
for hash in &block.transactions {
|
||||
let tx = {
|
||||
let mut tx;
|
||||
while {
|
||||
tx = network.rpc.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.clone().into()) {
|
||||
res.insert(
|
||||
eventualities.map.remove(&tx.prefix().extra).unwrap().0,
|
||||
(block.number().unwrap(), tx.id(), tx),
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
eventualities.block_number += 1;
|
||||
assert_eq!(eventualities.block_number, block.number().unwrap());
|
||||
}
|
||||
|
||||
for block_num in (eventualities.block_number + 1) .. block.number().unwrap() {
|
||||
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().unwrap());
|
||||
|
||||
res
|
||||
}
|
||||
|
||||
async fn needed_fee(
|
||||
&self,
|
||||
block_number: usize,
|
||||
inputs: &[Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Address>,
|
||||
) -> Result<Option<u64>, NetworkError> {
|
||||
let res = self
|
||||
.make_signable_transaction(block_number, &[0; 32], inputs, payments, change, true)
|
||||
.await?;
|
||||
let Some(res) = res else { return Ok(None) };
|
||||
let MakeSignableTransactionResult::Fee(fee) = res else {
|
||||
panic!("told make_signable_transaction calculating_fee and got transaction")
|
||||
};
|
||||
Ok(Some(fee))
|
||||
}
|
||||
|
||||
async fn signable_transaction(
|
||||
&self,
|
||||
block_number: usize,
|
||||
plan_id: &[u8; 32],
|
||||
_key: EdwardsPoint,
|
||||
inputs: &[Output],
|
||||
payments: &[Payment<Self>],
|
||||
change: &Option<Address>,
|
||||
(): &(),
|
||||
) -> Result<Option<(Self::SignableTransaction, Self::Eventuality)>, NetworkError> {
|
||||
let res = self
|
||||
.make_signable_transaction(block_number, plan_id, inputs, payments, change, false)
|
||||
.await?;
|
||||
let Some(res) = res else { return Ok(None) };
|
||||
let MakeSignableTransactionResult::SignableTransaction(signable) = res else {
|
||||
panic!("told make_signable_transaction not calculating_fee and got fee")
|
||||
};
|
||||
|
||||
let signable = SignableTransaction(signable);
|
||||
let eventuality = signable.0.clone().into();
|
||||
Ok(Some((signable, eventuality)))
|
||||
}
|
||||
|
||||
async fn attempt_sign(
|
||||
&self,
|
||||
keys: ThresholdKeys<Self::Curve>,
|
||||
transaction: SignableTransaction,
|
||||
) -> Result<Self::TransactionMachine, NetworkError> {
|
||||
match transaction.0.clone().multisig(keys) {
|
||||
Ok(machine) => Ok(machine),
|
||||
Err(e) => panic!("failed to create a multisig machine for TX: {e}"),
|
||||
}
|
||||
}
|
||||
|
||||
async fn publish_completion(&self, tx: &Transaction) -> Result<(), NetworkError> {
|
||||
match self.rpc.publish_transaction(tx).await {
|
||||
Ok(()) => Ok(()),
|
||||
Err(RpcError::ConnectionError(e)) => {
|
||||
log::debug!("Monero ConnectionError: {e}");
|
||||
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 confirm_completion(
|
||||
&self,
|
||||
eventuality: &Eventuality,
|
||||
id: &[u8; 32],
|
||||
) -> Result<Option<Transaction>, NetworkError> {
|
||||
let tx = self.rpc.get_transaction(*id).await.map_err(map_rpc_err)?;
|
||||
if eventuality.matches(&tx.clone().into()) {
|
||||
Ok(Some(tx))
|
||||
} else {
|
||||
Ok(None)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_block_number(&self, id: &[u8; 32]) -> usize {
|
||||
self.rpc.get_block(*id).await.unwrap().number().unwrap()
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn check_eventuality_by_claim(
|
||||
&self,
|
||||
eventuality: &Self::Eventuality,
|
||||
claim: &[u8; 32],
|
||||
) -> bool {
|
||||
return eventuality.matches(&self.rpc.get_pruned_transaction(*claim).await.unwrap());
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn get_transaction_by_eventuality(
|
||||
&self,
|
||||
block: usize,
|
||||
eventuality: &Eventuality,
|
||||
) -> Transaction {
|
||||
let block = self.rpc.get_block_by_number(block).await.unwrap();
|
||||
for tx in &block.transactions {
|
||||
let tx = self.rpc.get_transaction(*tx).await.unwrap();
|
||||
if eventuality.matches(&tx.clone().into()) {
|
||||
return tx;
|
||||
}
|
||||
}
|
||||
panic!("block didn't have a transaction for this eventuality")
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn mine_block(&self) {
|
||||
// https://github.com/serai-dex/serai/issues/198
|
||||
sleep(std::time::Duration::from_millis(100)).await;
|
||||
self.rpc.generate_blocks(&Self::test_address().into(), 1).await.unwrap();
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
async fn test_send(&self, address: Address) -> Block {
|
||||
use zeroize::Zeroizing;
|
||||
use rand_core::{RngCore, OsRng};
|
||||
use monero_wallet::rpc::FeePriority;
|
||||
|
||||
let new_block = self.get_latest_block_number().await.unwrap() + 1;
|
||||
for _ in 0 .. 80 {
|
||||
self.mine_block().await;
|
||||
}
|
||||
|
||||
let new_block = self.rpc.get_block_by_number(new_block).await.unwrap();
|
||||
let mut outputs = Self::test_scanner()
|
||||
.scan(self.rpc.get_scannable_block(new_block.clone()).await.unwrap())
|
||||
.unwrap()
|
||||
.ignore_additional_timelock();
|
||||
let output = outputs.swap_remove(0);
|
||||
|
||||
let amount = output.commitment().amount;
|
||||
// The dust should always be sufficient for the fee
|
||||
let fee = Monero::DUST;
|
||||
|
||||
let rct_type = match new_block.header.hardfork_version {
|
||||
14 => RctType::ClsagBulletproof,
|
||||
15 | 16 => RctType::ClsagBulletproofPlus,
|
||||
_ => panic!("Monero hard forked and the processor wasn't updated for it"),
|
||||
};
|
||||
|
||||
let output = OutputWithDecoys::fingerprintable_deterministic_new(
|
||||
&mut OsRng,
|
||||
&self.rpc,
|
||||
match rct_type {
|
||||
RctType::ClsagBulletproof => 11,
|
||||
RctType::ClsagBulletproofPlus => 16,
|
||||
_ => panic!("selecting decoys for an unsupported RctType"),
|
||||
},
|
||||
self.rpc.get_height().await.unwrap(),
|
||||
output,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let mut outgoing_view_key = Zeroizing::new([0; 32]);
|
||||
OsRng.fill_bytes(outgoing_view_key.as_mut());
|
||||
let tx = MSignableTransaction::new(
|
||||
rct_type,
|
||||
outgoing_view_key,
|
||||
vec![output],
|
||||
vec![(address.into(), amount - fee)],
|
||||
Change::fingerprintable(Some(Self::test_address().into())),
|
||||
vec![],
|
||||
self.rpc.get_fee_rate(FeePriority::Unimportant).await.unwrap(),
|
||||
)
|
||||
.unwrap()
|
||||
.sign(&mut OsRng, &Zeroizing::new(Scalar::ONE.0))
|
||||
.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()
|
||||
}
|
||||
}
|
||||
|
||||
impl UtxoNetwork for Monero {
|
||||
// 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;
|
||||
}
|
||||
Reference in New Issue
Block a user