Ethereum Integration (#557)

* Clean up Ethereum

* Consistent contract address for deployed contracts

* Flesh out Router a bit

* Add a Deployer for DoS-less deployment

* Implement Router-finding

* Use CREATE2 helper present in ethers

* Move from CREATE2 to CREATE

Bit more streamlined for our use case.

* Document ethereum-serai

* Tidy tests a bit

* Test updateSeraiKey

* Use encodePacked for updateSeraiKey

* Take in the block hash to read state during

* Add a Sandbox contract to the Ethereum integration

* Add retrieval of transfers from Ethereum

* Add inInstruction function to the Router

* Augment our handling of InInstructions events with a check the transfer event also exists

* Have the Deployer error upon failed deployments

* Add --via-ir

* Make get_transaction test-only

We only used it to get transactions to confirm the resolution of Eventualities.
Eventualities need to be modularized. By introducing the dedicated
confirm_completion function, we remove the need for a non-test get_transaction
AND begin this modularization (by no longer explicitly grabbing a transaction
to check with).

* Modularize Eventuality

Almost fully-deprecates the Transaction trait for Completion. Replaces
Transaction ID with Claim.

* Modularize the Scheduler behind a trait

* Add an extremely basic account Scheduler

* Add nonce uses, key rotation to the account scheduler

* Only report the account Scheduler empty after transferring keys

Also ban payments to the branch/change/forward addresses.

* Make fns reliant on state test-only

* Start of an Ethereum integration for the processor

* Add a session to the Router to prevent updateSeraiKey replaying

This would only happen if an old key was rotated to again, which would require
n-of-n collusion (already ridiculous and a valid fault attributable event). It
just clarifies the formal arguments.

* Add a RouterCommand + SignMachine for producing it to coins/ethereum

* Ethereum which compiles

* Have branch/change/forward return an option

Also defines a UtxoNetwork extension trait for MAX_INPUTS.

* Make external_address exclusively a test fn

* Move the "account" scheduler to "smart contract"

* Remove ABI artifact

* Move refund/forward Plan creation into the Processor

We create forward Plans in the scan path, and need to know their exact fees in
the scan path. This requires adding a somewhat wonky shim_forward_plan method
so we can obtain a Plan equivalent to the actual forward Plan for fee reasons,
yet don't expect it to be the actual forward Plan (which may be distinct if
the Plan pulls from the global state, such as with a nonce).

Also properly types a Scheduler addendum such that the SC scheduler isn't
cramming the nonce to use into the N::Output type.

* Flesh out the Ethereum integration more

* Two commits ago, into the **Scheduler, not Processor

* Remove misc TODOs in SC Scheduler

* Add constructor to RouterCommandMachine

* RouterCommand read, pairing with the prior added write

* Further add serialization methods

* Have the Router's key included with the InInstruction

This does not use the key at the time of the event. This uses the key at the
end of the block for the event. Its much simpler than getting the full event
streams for each, checking when they interlace.

This does not read the state. Every block, this makes a request for every
single key update and simply chooses the last one. This allows pruning state,
only keeping the event tree. Ideally, we'd also introduce a cache to reduce the
cost of the filter (small in events yielded, long in blocks searched).

Since Serai doesn't have any forwarding TXs, nor Branches, nor change, all of
our Plans should solely have payments out, and there's no expectation of a Plan
being made under one key broken by it being received by another key.

* Add read/write to InInstruction

* Abstract the ABI for Call/OutInstruction in ethereum-serai

* Fill out signable_transaction for Ethereum

* Move ethereum-serai to alloy

Resolves #331.

* Use the opaque sol macro instead of generated files

* Move the processor over to the now-alloy-based ethereum-serai

* Use the ecrecover provided by alloy

* Have the SC use nonce for rotation, not session (an independent nonce which wasn't synchronized)

* Always use the latest keys for SC scheduled plans

* get_eventuality_completions for Ethereum

* Finish fleshing out the processor Ethereum integration as needed for serai-processor tests

This doesn't not support any actual deployments, not even the ones simulated by
serai-processor-docker-tests.

* Add alloy-simple-request-transport to the GH workflows

* cargo update

* Clarify a few comments and make one check more robust

* Use a string for 27.0 in .github

* Remove optional from no-longer-optional dependencies in processor

* Add alloy to git deny exception

* Fix no longer optional specification in processor's binaries feature

* Use a version of foundry from 2024

* Correct fetching Bitcoin TXs in the processor docker tests

* Update rustls to resolve RUSTSEC warnings

* Use the monthly nightly foundry, not the deleted daily nightly
This commit is contained in:
Luke Parker
2024-04-21 06:02:12 -04:00
committed by GitHub
parent 43083dfd49
commit 0f0db14f05
58 changed files with 5031 additions and 1385 deletions

View File

@@ -1,6 +1,37 @@
use alloy_sol_types::sol;
#[rustfmt::skip]
#[allow(warnings)]
#[allow(needless_pass_by_value)]
#[allow(clippy::all)]
pub(crate) mod schnorr;
#[allow(clippy::ignored_unit_patterns)]
#[allow(clippy::redundant_closure_for_method_calls)]
mod erc20_container {
use super::*;
sol!("contracts/IERC20.sol");
}
pub use erc20_container::IERC20 as erc20;
#[rustfmt::skip]
#[allow(warnings)]
#[allow(needless_pass_by_value)]
#[allow(clippy::all)]
pub(crate) mod router;
#[allow(clippy::ignored_unit_patterns)]
#[allow(clippy::redundant_closure_for_method_calls)]
mod deployer_container {
use super::*;
sol!("contracts/Deployer.sol");
}
pub use deployer_container::Deployer as deployer;
#[rustfmt::skip]
#[allow(warnings)]
#[allow(needless_pass_by_value)]
#[allow(clippy::all)]
#[allow(clippy::ignored_unit_patterns)]
#[allow(clippy::redundant_closure_for_method_calls)]
mod router_container {
use super::*;
sol!(Router, "artifacts/Router.abi");
}
pub use router_container::Router as router;

View File

@@ -1,91 +1,185 @@
use sha3::{Digest, Keccak256};
use group::ff::PrimeField;
use k256::{
elliptic_curve::{
bigint::ArrayEncoding, ops::Reduce, point::AffineCoordinates, sec1::ToEncodedPoint,
},
ProjectivePoint, Scalar, U256,
elliptic_curve::{ops::Reduce, point::AffineCoordinates, sec1::ToEncodedPoint},
ProjectivePoint, Scalar, U256 as KU256,
};
#[cfg(test)]
use k256::{elliptic_curve::point::DecompressPoint, AffinePoint};
use frost::{
algorithm::{Hram, SchnorrSignature},
curve::Secp256k1,
curve::{Ciphersuite, Secp256k1},
};
use alloy_core::primitives::{Parity, Signature as AlloySignature};
use alloy_consensus::{SignableTransaction, Signed, TxLegacy};
use crate::abi::router::{Signature as AbiSignature};
pub(crate) fn keccak256(data: &[u8]) -> [u8; 32] {
Keccak256::digest(data).into()
alloy_core::primitives::keccak256(data).into()
}
pub(crate) fn address(point: &ProjectivePoint) -> [u8; 20] {
pub(crate) fn hash_to_scalar(data: &[u8]) -> Scalar {
<Scalar as Reduce<KU256>>::reduce_bytes(&keccak256(data).into())
}
pub fn address(point: &ProjectivePoint) -> [u8; 20] {
let encoded_point = point.to_encoded_point(false);
// Last 20 bytes of the hash of the concatenated x and y coordinates
// We obtain the concatenated x and y coordinates via the uncompressed encoding of the point
keccak256(&encoded_point.as_ref()[1 .. 65])[12 ..].try_into().unwrap()
}
pub(crate) fn deterministically_sign(tx: &TxLegacy) -> Signed<TxLegacy> {
assert!(
tx.chain_id.is_none(),
"chain ID was Some when deterministically signing a TX (causing a non-deterministic signer)"
);
let sig_hash = tx.signature_hash().0;
let mut r = hash_to_scalar(&[sig_hash.as_slice(), b"r"].concat());
let mut s = hash_to_scalar(&[sig_hash.as_slice(), b"s"].concat());
loop {
let r_bytes: [u8; 32] = r.to_repr().into();
let s_bytes: [u8; 32] = s.to_repr().into();
let v = Parity::NonEip155(false);
let signature =
AlloySignature::from_scalars_and_parity(r_bytes.into(), s_bytes.into(), v).unwrap();
let tx = tx.clone().into_signed(signature);
if tx.recover_signer().is_ok() {
return tx;
}
// Re-hash until valid
r = hash_to_scalar(r_bytes.as_ref());
s = hash_to_scalar(s_bytes.as_ref());
}
}
/// The public key for a Schnorr-signing account.
#[allow(non_snake_case)]
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
pub struct PublicKey {
pub A: ProjectivePoint,
pub px: Scalar,
pub parity: u8,
pub(crate) A: ProjectivePoint,
pub(crate) px: Scalar,
}
impl PublicKey {
/// Construct a new `PublicKey`.
///
/// This will return None if the provided point isn't eligible to be a public key (due to
/// bounds such as parity).
#[allow(non_snake_case)]
pub fn new(A: ProjectivePoint) -> Option<PublicKey> {
let affine = A.to_affine();
let parity = u8::from(bool::from(affine.y_is_odd())) + 27;
if parity != 27 {
// Only allow even keys to save a word within Ethereum
let is_odd = bool::from(affine.y_is_odd());
if is_odd {
None?;
}
let x_coord = affine.x();
let x_coord_scalar = <Scalar as Reduce<U256>>::reduce_bytes(&x_coord);
let x_coord_scalar = <Scalar as Reduce<KU256>>::reduce_bytes(&x_coord);
// Return None if a reduction would occur
// Reductions would be incredibly unlikely and shouldn't be an issue, yet it's one less
// headache/concern to have
// This does ban a trivial amoount of public keys
if x_coord_scalar.to_repr() != x_coord {
None?;
}
Some(PublicKey { A, px: x_coord_scalar, parity })
Some(PublicKey { A, px: x_coord_scalar })
}
pub fn point(&self) -> ProjectivePoint {
self.A
}
pub(crate) fn eth_repr(&self) -> [u8; 32] {
self.px.to_repr().into()
}
#[cfg(test)]
pub(crate) fn from_eth_repr(repr: [u8; 32]) -> Option<Self> {
#[allow(non_snake_case)]
let A = Option::<AffinePoint>::from(AffinePoint::decompress(&repr.into(), 0.into()))?.into();
Option::from(Scalar::from_repr(repr.into())).map(|px| PublicKey { A, px })
}
}
/// The HRAm to use for the Schnorr contract.
#[derive(Clone, Default)]
pub struct EthereumHram {}
impl Hram<Secp256k1> for EthereumHram {
#[allow(non_snake_case)]
fn hram(R: &ProjectivePoint, A: &ProjectivePoint, m: &[u8]) -> Scalar {
let a_encoded_point = A.to_encoded_point(true);
let mut a_encoded = a_encoded_point.as_ref().to_owned();
a_encoded[0] += 25; // Ethereum uses 27/28 for point parity
assert!((a_encoded[0] == 27) || (a_encoded[0] == 28));
let x_coord = A.to_affine().x();
let mut data = address(R).to_vec();
data.append(&mut a_encoded);
data.extend(x_coord.as_slice());
data.extend(m);
Scalar::reduce(U256::from_be_slice(&keccak256(&data)))
<Scalar as Reduce<KU256>>::reduce_bytes(&keccak256(&data).into())
}
}
/// A signature for the Schnorr contract.
#[derive(Clone, Copy, PartialEq, Eq, Debug)]
pub struct Signature {
pub(crate) c: Scalar,
pub(crate) s: Scalar,
}
impl Signature {
pub fn verify(&self, public_key: &PublicKey, message: &[u8]) -> bool {
#[allow(non_snake_case)]
let R = (Secp256k1::generator() * self.s) - (public_key.A * self.c);
EthereumHram::hram(&R, &public_key.A, message) == self.c
}
/// Construct a new `Signature`.
///
/// This will return None if the signature is invalid.
pub fn new(
public_key: &PublicKey,
chain_id: U256,
m: &[u8],
message: &[u8],
signature: SchnorrSignature<Secp256k1>,
) -> Option<Signature> {
let c = EthereumHram::hram(
&signature.R,
&public_key.A,
&[chain_id.to_be_byte_array().as_slice(), &keccak256(m)].concat(),
);
let c = EthereumHram::hram(&signature.R, &public_key.A, message);
if !signature.verify(public_key.A, c) {
None?;
}
Some(Signature { c, s: signature.s })
let res = Signature { c, s: signature.s };
assert!(res.verify(public_key, message));
Some(res)
}
pub fn c(&self) -> Scalar {
self.c
}
pub fn s(&self) -> Scalar {
self.s
}
pub fn to_bytes(&self) -> [u8; 64] {
let mut res = [0; 64];
res[.. 32].copy_from_slice(self.c.to_repr().as_ref());
res[32 ..].copy_from_slice(self.s.to_repr().as_ref());
res
}
pub fn from_bytes(bytes: [u8; 64]) -> std::io::Result<Self> {
let mut reader = bytes.as_slice();
let c = Secp256k1::read_F(&mut reader)?;
let s = Secp256k1::read_F(&mut reader)?;
Ok(Signature { c, s })
}
}
impl From<&Signature> for AbiSignature {
fn from(sig: &Signature) -> AbiSignature {
let c: [u8; 32] = sig.c.to_repr().into();
let s: [u8; 32] = sig.s.to_repr().into();
AbiSignature { c: c.into(), s: s.into() }
}
}

View File

@@ -0,0 +1,119 @@
use std::sync::Arc;
use alloy_core::primitives::{hex::FromHex, Address, B256, U256, Bytes, TxKind};
use alloy_consensus::{Signed, TxLegacy};
use alloy_sol_types::{SolCall, SolEvent};
use alloy_rpc_types::{BlockNumberOrTag, Filter};
use alloy_simple_request_transport::SimpleRequest;
use alloy_provider::{Provider, RootProvider};
use crate::{
Error,
crypto::{self, keccak256, PublicKey},
router::Router,
};
pub use crate::abi::deployer as abi;
/// The Deployer contract for the Router contract.
///
/// This Deployer has a deterministic address, letting it be immediately identified on any
/// compatible chain. It then supports retrieving the Router contract's address (which isn't
/// deterministic) using a single log query.
#[derive(Clone, Debug)]
pub struct Deployer;
impl Deployer {
/// Obtain the transaction to deploy this contract, already signed.
///
/// The account this transaction is sent from (which is populated in `from`) must be sufficiently
/// funded for this transaction to be submitted. This account has no known private key to anyone,
/// so ETH sent can be neither misappropriated nor returned.
pub fn deployment_tx() -> Signed<TxLegacy> {
let bytecode = include_str!("../artifacts/Deployer.bin");
let bytecode =
Bytes::from_hex(bytecode).expect("compiled-in Deployer bytecode wasn't valid hex");
let tx = TxLegacy {
chain_id: None,
nonce: 0,
gas_price: 100_000_000_000u128,
// TODO: Use a more accurate gas limit
gas_limit: 1_000_000u128,
to: TxKind::Create,
value: U256::ZERO,
input: bytecode,
};
crypto::deterministically_sign(&tx)
}
/// Obtain the deterministic address for this contract.
pub fn address() -> [u8; 20] {
let deployer_deployer =
Self::deployment_tx().recover_signer().expect("deployment_tx didn't have a valid signature");
**Address::create(&deployer_deployer, 0)
}
/// Construct a new view of the `Deployer`.
pub async fn new(provider: Arc<RootProvider<SimpleRequest>>) -> Result<Option<Self>, Error> {
let address = Self::address();
#[cfg(not(test))]
let required_block = BlockNumberOrTag::Finalized;
#[cfg(test)]
let required_block = BlockNumberOrTag::Latest;
let code = provider
.get_code_at(address.into(), required_block.into())
.await
.map_err(|_| Error::ConnectionError)?;
// Contract has yet to be deployed
if code.is_empty() {
return Ok(None);
}
Ok(Some(Self))
}
/// Yield the `ContractCall` necessary to deploy the Router.
pub fn deploy_router(&self, key: &PublicKey) -> TxLegacy {
TxLegacy {
to: TxKind::Call(Self::address().into()),
input: abi::deployCall::new((Router::init_code(key).into(),)).abi_encode().into(),
gas_limit: 1_000_000,
..Default::default()
}
}
/// Find the first Router deployed with the specified key as its first key.
///
/// This is the Router Serai will use, and is the only way to construct a `Router`.
pub async fn find_router(
&self,
provider: Arc<RootProvider<SimpleRequest>>,
key: &PublicKey,
) -> Result<Option<Router>, Error> {
let init_code = Router::init_code(key);
let init_code_hash = keccak256(&init_code);
#[cfg(not(test))]
let to_block = BlockNumberOrTag::Finalized;
#[cfg(test)]
let to_block = BlockNumberOrTag::Latest;
// Find the first log using this init code (where the init code is binding to the key)
let filter =
Filter::new().from_block(0).to_block(to_block).address(Address::from(Self::address()));
let filter = filter.event_signature(abi::Deployment::SIGNATURE_HASH);
let filter = filter.topic1(B256::from(init_code_hash));
let logs = provider.get_logs(&filter).await.map_err(|_| Error::ConnectionError)?;
let Some(first_log) = logs.first() else { return Ok(None) };
let router = first_log
.log_decode::<abi::Deployment>()
.map_err(|_| Error::ConnectionError)?
.inner
.data
.created;
Ok(Some(Router::new(provider, router)))
}
}

118
coins/ethereum/src/erc20.rs Normal file
View File

@@ -0,0 +1,118 @@
use std::{sync::Arc, collections::HashSet};
use alloy_core::primitives::{Address, B256, U256};
use alloy_sol_types::{SolInterface, SolEvent};
use alloy_rpc_types::{BlockNumberOrTag, Filter};
use alloy_simple_request_transport::SimpleRequest;
use alloy_provider::{Provider, RootProvider};
use crate::Error;
pub use crate::abi::erc20 as abi;
use abi::{IERC20Calls, Transfer, transferCall, transferFromCall};
#[derive(Clone, Debug)]
pub struct TopLevelErc20Transfer {
pub id: [u8; 32],
pub from: [u8; 20],
pub amount: U256,
pub data: Vec<u8>,
}
/// A view for an ERC20 contract.
#[derive(Clone, Debug)]
pub struct ERC20(Arc<RootProvider<SimpleRequest>>, Address);
impl ERC20 {
/// Construct a new view of the specified ERC20 contract.
///
/// This checks a contract is deployed at that address yet does not check the contract is
/// actually an ERC20.
pub async fn new(
provider: Arc<RootProvider<SimpleRequest>>,
address: [u8; 20],
) -> Result<Option<Self>, Error> {
let code = provider
.get_code_at(address.into(), BlockNumberOrTag::Finalized.into())
.await
.map_err(|_| Error::ConnectionError)?;
// Contract has yet to be deployed
if code.is_empty() {
return Ok(None);
}
Ok(Some(Self(provider.clone(), Address::from(&address))))
}
pub async fn top_level_transfers(
&self,
block: u64,
to: [u8; 20],
) -> Result<Vec<TopLevelErc20Transfer>, Error> {
let filter = Filter::new().from_block(block).to_block(block).address(self.1);
let filter = filter.event_signature(Transfer::SIGNATURE_HASH);
let mut to_topic = [0; 32];
to_topic[12 ..].copy_from_slice(&to);
let filter = filter.topic2(B256::from(to_topic));
let logs = self.0.get_logs(&filter).await.map_err(|_| Error::ConnectionError)?;
let mut handled = HashSet::new();
let mut top_level_transfers = vec![];
for log in logs {
// Double check the address which emitted this log
if log.address() != self.1 {
Err(Error::ConnectionError)?;
}
let tx_id = log.transaction_hash.ok_or(Error::ConnectionError)?;
let tx = self.0.get_transaction_by_hash(tx_id).await.map_err(|_| Error::ConnectionError)?;
// If this is a top-level call...
if tx.to == Some(self.1) {
// And we recognize the call...
// Don't validate the encoding as this can't be re-encoded to an identical bytestring due
// to the InInstruction appended
if let Ok(call) = IERC20Calls::abi_decode(&tx.input, false) {
// Extract the top-level call's from/to/value
let (from, call_to, value) = match call {
IERC20Calls::transfer(transferCall { to: call_to, value }) => (tx.from, call_to, value),
IERC20Calls::transferFrom(transferFromCall { from, to: call_to, value }) => {
(from, call_to, value)
}
// Treat any other function selectors as unrecognized
_ => continue,
};
let log = log.log_decode::<Transfer>().map_err(|_| Error::ConnectionError)?.inner.data;
// Ensure the top-level transfer is equivalent, and this presumably isn't a log for an
// internal transfer
if (log.from != from) || (call_to != to) || (value != log.value) {
continue;
}
// Now that the top-level transfer is confirmed to be equivalent to the log, ensure it's
// the only log we handle
if handled.contains(&tx_id) {
continue;
}
handled.insert(tx_id);
// Read the data appended after
let encoded = call.abi_encode();
let data = tx.input.as_ref()[encoded.len() ..].to_vec();
// Push the transfer
top_level_transfers.push(TopLevelErc20Transfer {
// Since we'll only handle one log for this TX, set the ID to the TX ID
id: *tx_id,
from: *log.from.0,
amount: log.value,
data,
});
}
}
}
Ok(top_level_transfers)
}
}

View File

@@ -1,16 +1,30 @@
use thiserror::Error;
pub use alloy_core;
pub use alloy_consensus;
pub use alloy_rpc_types;
pub use alloy_simple_request_transport;
pub use alloy_rpc_client;
pub use alloy_provider;
pub mod crypto;
pub(crate) mod abi;
pub mod schnorr;
pub mod erc20;
pub mod deployer;
pub mod router;
pub mod machine;
#[cfg(test)]
mod tests;
#[derive(Error, Debug)]
#[derive(Clone, Copy, PartialEq, Eq, Debug, Error)]
pub enum Error {
#[error("failed to verify Schnorr signature")]
InvalidSignature,
#[error("couldn't make call/send TX")]
ConnectionError,
}

View File

@@ -0,0 +1,414 @@
use std::{
io::{self, Read},
collections::HashMap,
};
use rand_core::{RngCore, CryptoRng};
use transcript::{Transcript, RecommendedTranscript};
use group::GroupEncoding;
use frost::{
curve::{Ciphersuite, Secp256k1},
Participant, ThresholdKeys, FrostError,
algorithm::Schnorr,
sign::*,
};
use alloy_core::primitives::U256;
use crate::{
crypto::{PublicKey, EthereumHram, Signature},
router::{
abi::{Call as AbiCall, OutInstruction as AbiOutInstruction},
Router,
},
};
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct Call {
pub to: [u8; 20],
pub value: U256,
pub data: Vec<u8>,
}
impl Call {
pub fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let mut to = [0; 20];
reader.read_exact(&mut to)?;
let value = {
let mut value_bytes = [0; 32];
reader.read_exact(&mut value_bytes)?;
U256::from_le_slice(&value_bytes)
};
let mut data_len = {
let mut data_len = [0; 4];
reader.read_exact(&mut data_len)?;
usize::try_from(u32::from_le_bytes(data_len)).expect("u32 couldn't fit within a usize")
};
// A valid DoS would be to claim a 4 GB data is present for only 4 bytes
// We read this in 1 KB chunks to only read data actually present (with a max DoS of 1 KB)
let mut data = vec![];
while data_len > 0 {
let chunk_len = data_len.min(1024);
let mut chunk = vec![0; chunk_len];
reader.read_exact(&mut chunk)?;
data.extend(&chunk);
data_len -= chunk_len;
}
Ok(Call { to, value, data })
}
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
writer.write_all(&self.to)?;
writer.write_all(&self.value.as_le_bytes())?;
let data_len = u32::try_from(self.data.len())
.map_err(|_| io::Error::other("call data length exceeded 2**32"))?;
writer.write_all(&data_len.to_le_bytes())?;
writer.write_all(&self.data)
}
}
impl From<Call> for AbiCall {
fn from(call: Call) -> AbiCall {
AbiCall { to: call.to.into(), value: call.value, data: call.data.into() }
}
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub enum OutInstructionTarget {
Direct([u8; 20]),
Calls(Vec<Call>),
}
impl OutInstructionTarget {
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let mut kind = [0xff];
reader.read_exact(&mut kind)?;
match kind[0] {
0 => {
let mut addr = [0; 20];
reader.read_exact(&mut addr)?;
Ok(OutInstructionTarget::Direct(addr))
}
1 => {
let mut calls_len = [0; 4];
reader.read_exact(&mut calls_len)?;
let calls_len = u32::from_le_bytes(calls_len);
let mut calls = vec![];
for _ in 0 .. calls_len {
calls.push(Call::read(reader)?);
}
Ok(OutInstructionTarget::Calls(calls))
}
_ => Err(io::Error::other("unrecognized OutInstructionTarget"))?,
}
}
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
match self {
OutInstructionTarget::Direct(addr) => {
writer.write_all(&[0])?;
writer.write_all(addr)?;
}
OutInstructionTarget::Calls(calls) => {
writer.write_all(&[1])?;
let call_len = u32::try_from(calls.len())
.map_err(|_| io::Error::other("amount of calls exceeded 2**32"))?;
writer.write_all(&call_len.to_le_bytes())?;
for call in calls {
call.write(writer)?;
}
}
}
Ok(())
}
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct OutInstruction {
pub target: OutInstructionTarget,
pub value: U256,
}
impl OutInstruction {
fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let target = OutInstructionTarget::read(reader)?;
let value = {
let mut value_bytes = [0; 32];
reader.read_exact(&mut value_bytes)?;
U256::from_le_slice(&value_bytes)
};
Ok(OutInstruction { target, value })
}
fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
self.target.write(writer)?;
writer.write_all(&self.value.as_le_bytes())
}
}
impl From<OutInstruction> for AbiOutInstruction {
fn from(instruction: OutInstruction) -> AbiOutInstruction {
match instruction.target {
OutInstructionTarget::Direct(addr) => {
AbiOutInstruction { to: addr.into(), calls: vec![], value: instruction.value }
}
OutInstructionTarget::Calls(calls) => AbiOutInstruction {
to: [0; 20].into(),
calls: calls.into_iter().map(Into::into).collect(),
value: instruction.value,
},
}
}
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub enum RouterCommand {
UpdateSeraiKey { chain_id: U256, nonce: U256, key: PublicKey },
Execute { chain_id: U256, nonce: U256, outs: Vec<OutInstruction> },
}
impl RouterCommand {
pub fn msg(&self) -> Vec<u8> {
match self {
RouterCommand::UpdateSeraiKey { chain_id, nonce, key } => {
Router::update_serai_key_message(*chain_id, *nonce, key)
}
RouterCommand::Execute { chain_id, nonce, outs } => Router::execute_message(
*chain_id,
*nonce,
outs.iter().map(|out| out.clone().into()).collect(),
),
}
}
pub fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let mut kind = [0xff];
reader.read_exact(&mut kind)?;
match kind[0] {
0 => {
let mut chain_id = [0; 32];
reader.read_exact(&mut chain_id)?;
let mut nonce = [0; 32];
reader.read_exact(&mut nonce)?;
let key = PublicKey::new(Secp256k1::read_G(reader)?)
.ok_or(io::Error::other("key for RouterCommand doesn't have an eth representation"))?;
Ok(RouterCommand::UpdateSeraiKey {
chain_id: U256::from_le_slice(&chain_id),
nonce: U256::from_le_slice(&nonce),
key,
})
}
1 => {
let mut chain_id = [0; 32];
reader.read_exact(&mut chain_id)?;
let chain_id = U256::from_le_slice(&chain_id);
let mut nonce = [0; 32];
reader.read_exact(&mut nonce)?;
let nonce = U256::from_le_slice(&nonce);
let mut outs_len = [0; 4];
reader.read_exact(&mut outs_len)?;
let outs_len = u32::from_le_bytes(outs_len);
let mut outs = vec![];
for _ in 0 .. outs_len {
outs.push(OutInstruction::read(reader)?);
}
Ok(RouterCommand::Execute { chain_id, nonce, outs })
}
_ => Err(io::Error::other("reading unknown type of RouterCommand"))?,
}
}
pub fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
match self {
RouterCommand::UpdateSeraiKey { chain_id, nonce, key } => {
writer.write_all(&[0])?;
writer.write_all(&chain_id.as_le_bytes())?;
writer.write_all(&nonce.as_le_bytes())?;
writer.write_all(&key.A.to_bytes())
}
RouterCommand::Execute { chain_id, nonce, outs } => {
writer.write_all(&[1])?;
writer.write_all(&chain_id.as_le_bytes())?;
writer.write_all(&nonce.as_le_bytes())?;
writer.write_all(&u32::try_from(outs.len()).unwrap().to_le_bytes())?;
for out in outs {
out.write(writer)?;
}
Ok(())
}
}
}
pub fn serialize(&self) -> Vec<u8> {
let mut res = vec![];
self.write(&mut res).unwrap();
res
}
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct SignedRouterCommand {
command: RouterCommand,
signature: Signature,
}
impl SignedRouterCommand {
pub fn new(key: &PublicKey, command: RouterCommand, signature: &[u8; 64]) -> Option<Self> {
let c = Secp256k1::read_F(&mut &signature[.. 32]).ok()?;
let s = Secp256k1::read_F(&mut &signature[32 ..]).ok()?;
let signature = Signature { c, s };
if !signature.verify(key, &command.msg()) {
None?
}
Some(SignedRouterCommand { command, signature })
}
pub fn command(&self) -> &RouterCommand {
&self.command
}
pub fn signature(&self) -> &Signature {
&self.signature
}
pub fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let command = RouterCommand::read(reader)?;
let mut sig = [0; 64];
reader.read_exact(&mut sig)?;
let signature = Signature::from_bytes(sig)?;
Ok(SignedRouterCommand { command, signature })
}
pub fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
self.command.write(writer)?;
writer.write_all(&self.signature.to_bytes())
}
}
pub struct RouterCommandMachine {
key: PublicKey,
command: RouterCommand,
machine: AlgorithmMachine<Secp256k1, Schnorr<Secp256k1, RecommendedTranscript, EthereumHram>>,
}
impl RouterCommandMachine {
pub fn new(keys: ThresholdKeys<Secp256k1>, command: RouterCommand) -> Option<Self> {
// The Schnorr algorithm should be fine without this, even when using the IETF variant
// If this is better and more comprehensive, we should do it, even if not necessary
let mut transcript = RecommendedTranscript::new(b"ethereum-serai RouterCommandMachine v0.1");
let key = keys.group_key();
transcript.append_message(b"key", key.to_bytes());
transcript.append_message(b"command", command.serialize());
Some(Self {
key: PublicKey::new(key)?,
command,
machine: AlgorithmMachine::new(Schnorr::new(transcript), keys),
})
}
}
impl PreprocessMachine for RouterCommandMachine {
type Preprocess = Preprocess<Secp256k1, ()>;
type Signature = SignedRouterCommand;
type SignMachine = RouterCommandSignMachine;
fn preprocess<R: RngCore + CryptoRng>(
self,
rng: &mut R,
) -> (Self::SignMachine, Self::Preprocess) {
let (machine, preprocess) = self.machine.preprocess(rng);
(RouterCommandSignMachine { key: self.key, command: self.command, machine }, preprocess)
}
}
pub struct RouterCommandSignMachine {
key: PublicKey,
command: RouterCommand,
machine: AlgorithmSignMachine<Secp256k1, Schnorr<Secp256k1, RecommendedTranscript, EthereumHram>>,
}
impl SignMachine<SignedRouterCommand> for RouterCommandSignMachine {
type Params = ();
type Keys = ThresholdKeys<Secp256k1>;
type Preprocess = Preprocess<Secp256k1, ()>;
type SignatureShare = SignatureShare<Secp256k1>;
type SignatureMachine = RouterCommandSignatureMachine;
fn cache(self) -> CachedPreprocess {
unimplemented!(
"RouterCommand machines don't support caching their preprocesses due to {}",
"being already bound to a specific command"
);
}
fn from_cache(
(): (),
_: ThresholdKeys<Secp256k1>,
_: CachedPreprocess,
) -> (Self, Self::Preprocess) {
unimplemented!(
"RouterCommand machines don't support caching their preprocesses due to {}",
"being already bound to a specific command"
);
}
fn read_preprocess<R: Read>(&self, reader: &mut R) -> io::Result<Self::Preprocess> {
self.machine.read_preprocess(reader)
}
fn sign(
self,
commitments: HashMap<Participant, Self::Preprocess>,
msg: &[u8],
) -> Result<(RouterCommandSignatureMachine, Self::SignatureShare), FrostError> {
if !msg.is_empty() {
panic!("message was passed to a RouterCommand machine when it generates its own");
}
let (machine, share) = self.machine.sign(commitments, &self.command.msg())?;
Ok((RouterCommandSignatureMachine { key: self.key, command: self.command, machine }, share))
}
}
pub struct RouterCommandSignatureMachine {
key: PublicKey,
command: RouterCommand,
machine:
AlgorithmSignatureMachine<Secp256k1, Schnorr<Secp256k1, RecommendedTranscript, EthereumHram>>,
}
impl SignatureMachine<SignedRouterCommand> for RouterCommandSignatureMachine {
type SignatureShare = SignatureShare<Secp256k1>;
fn read_share<R: Read>(&self, reader: &mut R) -> io::Result<Self::SignatureShare> {
self.machine.read_share(reader)
}
fn complete(
self,
shares: HashMap<Participant, Self::SignatureShare>,
) -> Result<SignedRouterCommand, FrostError> {
let sig = self.machine.complete(shares)?;
let signature = Signature::new(&self.key, &self.command.msg(), sig)
.expect("machine produced an invalid signature");
Ok(SignedRouterCommand { command: self.command, signature })
}
}

View File

@@ -1,30 +1,426 @@
pub use crate::abi::router::*;
use std::{sync::Arc, io, collections::HashSet};
/*
use crate::crypto::{ProcessedSignature, PublicKey};
use ethers::{contract::ContractFactory, prelude::*, solc::artifacts::contract::ContractBytecode};
use eyre::Result;
use std::{convert::From, fs::File, sync::Arc};
use k256::{
elliptic_curve::{group::GroupEncoding, sec1},
ProjectivePoint,
};
pub async fn router_update_public_key<M: Middleware + 'static>(
contract: &Router<M>,
public_key: &PublicKey,
signature: &ProcessedSignature,
) -> std::result::Result<Option<TransactionReceipt>, eyre::ErrReport> {
let tx = contract.update_public_key(public_key.px.to_bytes().into(), signature.into());
let pending_tx = tx.send().await?;
let receipt = pending_tx.await?;
Ok(receipt)
use alloy_core::primitives::{hex::FromHex, Address, U256, Bytes, TxKind};
#[cfg(test)]
use alloy_core::primitives::B256;
use alloy_consensus::TxLegacy;
use alloy_sol_types::{SolValue, SolConstructor, SolCall, SolEvent};
use alloy_rpc_types::Filter;
#[cfg(test)]
use alloy_rpc_types::{BlockId, TransactionRequest, TransactionInput};
use alloy_simple_request_transport::SimpleRequest;
use alloy_provider::{Provider, RootProvider};
pub use crate::{
Error,
crypto::{PublicKey, Signature},
abi::{erc20::Transfer, router as abi},
};
use abi::{SeraiKeyUpdated, InInstruction as InInstructionEvent, Executed as ExecutedEvent};
#[derive(Clone, PartialEq, Eq, Debug)]
pub enum Coin {
Ether,
Erc20([u8; 20]),
}
pub async fn router_execute<M: Middleware + 'static>(
contract: &Router<M>,
txs: Vec<Rtransaction>,
signature: &ProcessedSignature,
) -> std::result::Result<Option<TransactionReceipt>, eyre::ErrReport> {
let tx = contract.execute(txs, signature.into()).send();
let pending_tx = tx.send().await?;
let receipt = pending_tx.await?;
Ok(receipt)
impl Coin {
pub fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let mut kind = [0xff];
reader.read_exact(&mut kind)?;
Ok(match kind[0] {
0 => Coin::Ether,
1 => {
let mut address = [0; 20];
reader.read_exact(&mut address)?;
Coin::Erc20(address)
}
_ => Err(io::Error::other("unrecognized Coin type"))?,
})
}
pub fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
match self {
Coin::Ether => writer.write_all(&[0]),
Coin::Erc20(token) => {
writer.write_all(&[1])?;
writer.write_all(token)
}
}
}
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct InInstruction {
pub id: ([u8; 32], u64),
pub from: [u8; 20],
pub coin: Coin,
pub amount: U256,
pub data: Vec<u8>,
pub key_at_end_of_block: ProjectivePoint,
}
impl InInstruction {
pub fn read<R: io::Read>(reader: &mut R) -> io::Result<Self> {
let id = {
let mut id_hash = [0; 32];
reader.read_exact(&mut id_hash)?;
let mut id_pos = [0; 8];
reader.read_exact(&mut id_pos)?;
let id_pos = u64::from_le_bytes(id_pos);
(id_hash, id_pos)
};
let mut from = [0; 20];
reader.read_exact(&mut from)?;
let coin = Coin::read(reader)?;
let mut amount = [0; 32];
reader.read_exact(&mut amount)?;
let amount = U256::from_le_slice(&amount);
let mut data_len = [0; 4];
reader.read_exact(&mut data_len)?;
let data_len = usize::try_from(u32::from_le_bytes(data_len))
.map_err(|_| io::Error::other("InInstruction data exceeded 2**32 in length"))?;
let mut data = vec![0; data_len];
reader.read_exact(&mut data)?;
let mut key_at_end_of_block = <ProjectivePoint as GroupEncoding>::Repr::default();
reader.read_exact(&mut key_at_end_of_block)?;
let key_at_end_of_block = Option::from(ProjectivePoint::from_bytes(&key_at_end_of_block))
.ok_or(io::Error::other("InInstruction had key at end of block which wasn't valid"))?;
Ok(InInstruction { id, from, coin, amount, data, key_at_end_of_block })
}
pub fn write<W: io::Write>(&self, writer: &mut W) -> io::Result<()> {
writer.write_all(&self.id.0)?;
writer.write_all(&self.id.1.to_le_bytes())?;
writer.write_all(&self.from)?;
self.coin.write(writer)?;
writer.write_all(&self.amount.as_le_bytes())?;
writer.write_all(
&u32::try_from(self.data.len())
.map_err(|_| {
io::Error::other("InInstruction being written had data exceeding 2**32 in length")
})?
.to_le_bytes(),
)?;
writer.write_all(&self.data)?;
writer.write_all(&self.key_at_end_of_block.to_bytes())
}
}
#[derive(Clone, PartialEq, Eq, Debug)]
pub struct Executed {
pub tx_id: [u8; 32],
pub nonce: u64,
pub signature: [u8; 64],
}
/// The contract Serai uses to manage its state.
#[derive(Clone, Debug)]
pub struct Router(Arc<RootProvider<SimpleRequest>>, Address);
impl Router {
pub(crate) fn code() -> Vec<u8> {
let bytecode = include_str!("../artifacts/Router.bin");
Bytes::from_hex(bytecode).expect("compiled-in Router bytecode wasn't valid hex").to_vec()
}
pub(crate) fn init_code(key: &PublicKey) -> Vec<u8> {
let mut bytecode = Self::code();
// Append the constructor arguments
bytecode.extend((abi::constructorCall { _seraiKey: key.eth_repr().into() }).abi_encode());
bytecode
}
// This isn't pub in order to force users to use `Deployer::find_router`.
pub(crate) fn new(provider: Arc<RootProvider<SimpleRequest>>, address: Address) -> Self {
Self(provider, address)
}
pub fn address(&self) -> [u8; 20] {
**self.1
}
/// Get the key for Serai at the specified block.
#[cfg(test)]
pub async fn serai_key(&self, at: [u8; 32]) -> Result<PublicKey, Error> {
let call = TransactionRequest::default()
.to(Some(self.1))
.input(TransactionInput::new(abi::seraiKeyCall::new(()).abi_encode().into()));
let bytes = self
.0
.call(&call, Some(BlockId::Hash(B256::from(at).into())))
.await
.map_err(|_| Error::ConnectionError)?;
let res =
abi::seraiKeyCall::abi_decode_returns(&bytes, true).map_err(|_| Error::ConnectionError)?;
PublicKey::from_eth_repr(res._0.0).ok_or(Error::ConnectionError)
}
/// Get the message to be signed in order to update the key for Serai.
pub(crate) fn update_serai_key_message(chain_id: U256, nonce: U256, key: &PublicKey) -> Vec<u8> {
let mut buffer = b"updateSeraiKey".to_vec();
buffer.extend(&chain_id.to_be_bytes::<32>());
buffer.extend(&nonce.to_be_bytes::<32>());
buffer.extend(&key.eth_repr());
buffer
}
/// Update the key representing Serai.
pub fn update_serai_key(&self, public_key: &PublicKey, sig: &Signature) -> TxLegacy {
// TODO: Set a more accurate gas
TxLegacy {
to: TxKind::Call(self.1),
input: abi::updateSeraiKeyCall::new((public_key.eth_repr().into(), sig.into()))
.abi_encode()
.into(),
gas_limit: 100_000,
..Default::default()
}
}
/// Get the current nonce for the published batches.
#[cfg(test)]
pub async fn nonce(&self, at: [u8; 32]) -> Result<U256, Error> {
let call = TransactionRequest::default()
.to(Some(self.1))
.input(TransactionInput::new(abi::nonceCall::new(()).abi_encode().into()));
let bytes = self
.0
.call(&call, Some(BlockId::Hash(B256::from(at).into())))
.await
.map_err(|_| Error::ConnectionError)?;
let res =
abi::nonceCall::abi_decode_returns(&bytes, true).map_err(|_| Error::ConnectionError)?;
Ok(res._0)
}
/// Get the message to be signed in order to update the key for Serai.
pub(crate) fn execute_message(
chain_id: U256,
nonce: U256,
outs: Vec<abi::OutInstruction>,
) -> Vec<u8> {
("execute".to_string(), chain_id, nonce, outs).abi_encode_params()
}
/// Execute a batch of `OutInstruction`s.
pub fn execute(&self, outs: &[abi::OutInstruction], sig: &Signature) -> TxLegacy {
TxLegacy {
to: TxKind::Call(self.1),
input: abi::executeCall::new((outs.to_vec(), sig.into())).abi_encode().into(),
// TODO
gas_limit: 100_000 + ((200_000 + 10_000) * u128::try_from(outs.len()).unwrap()),
..Default::default()
}
}
pub async fn in_instructions(
&self,
block: u64,
allowed_tokens: &HashSet<[u8; 20]>,
) -> Result<Vec<InInstruction>, Error> {
let key_at_end_of_block = {
let filter = Filter::new().from_block(0).to_block(block).address(self.1);
let filter = filter.event_signature(SeraiKeyUpdated::SIGNATURE_HASH);
let all_keys = self.0.get_logs(&filter).await.map_err(|_| Error::ConnectionError)?;
let last_key_x_coordinate_log = all_keys.last().ok_or(Error::ConnectionError)?;
let last_key_x_coordinate = last_key_x_coordinate_log
.log_decode::<SeraiKeyUpdated>()
.map_err(|_| Error::ConnectionError)?
.inner
.data
.key;
let mut compressed_point = <ProjectivePoint as GroupEncoding>::Repr::default();
compressed_point[0] = u8::from(sec1::Tag::CompressedEvenY);
compressed_point[1 ..].copy_from_slice(last_key_x_coordinate.as_slice());
ProjectivePoint::from_bytes(&compressed_point).expect("router's last key wasn't a valid key")
};
let filter = Filter::new().from_block(block).to_block(block).address(self.1);
let filter = filter.event_signature(InInstructionEvent::SIGNATURE_HASH);
let logs = self.0.get_logs(&filter).await.map_err(|_| Error::ConnectionError)?;
let mut transfer_check = HashSet::new();
let mut in_instructions = vec![];
for log in logs {
// Double check the address which emitted this log
if log.address() != self.1 {
Err(Error::ConnectionError)?;
}
let id = (
log.block_hash.ok_or(Error::ConnectionError)?.into(),
log.log_index.ok_or(Error::ConnectionError)?,
);
let tx_hash = log.transaction_hash.ok_or(Error::ConnectionError)?;
let tx = self.0.get_transaction_by_hash(tx_hash).await.map_err(|_| Error::ConnectionError)?;
let log =
log.log_decode::<InInstructionEvent>().map_err(|_| Error::ConnectionError)?.inner.data;
let coin = if log.coin.0 == [0; 20] {
Coin::Ether
} else {
let token = *log.coin.0;
if !allowed_tokens.contains(&token) {
continue;
}
// If this also counts as a top-level transfer via the token, drop it
//
// Necessary in order to handle a potential edge case with some theoretical token
// implementations
//
// This will either let it be handled by the top-level transfer hook or will drop it
// entirely on the side of caution
if tx.to == Some(token.into()) {
continue;
}
// Get all logs for this TX
let receipt = self
.0
.get_transaction_receipt(tx_hash)
.await
.map_err(|_| Error::ConnectionError)?
.ok_or(Error::ConnectionError)?;
let tx_logs = receipt.inner.logs();
// Find a matching transfer log
let mut found_transfer = false;
for tx_log in tx_logs {
let log_index = tx_log.log_index.ok_or(Error::ConnectionError)?;
// Ensure we didn't already use this transfer to check a distinct InInstruction event
if transfer_check.contains(&log_index) {
continue;
}
// Check if this log is from the token we expected to be transferred
if tx_log.address().0 != token {
continue;
}
// Check if this is a transfer log
// https://github.com/alloy-rs/core/issues/589
if tx_log.topics()[0] != Transfer::SIGNATURE_HASH {
continue;
}
let Ok(transfer) = Transfer::decode_log(&tx_log.inner.clone(), true) else { continue };
// Check if this is a transfer to us for the expected amount
if (transfer.to == self.1) && (transfer.value == log.amount) {
transfer_check.insert(log_index);
found_transfer = true;
break;
}
}
if !found_transfer {
// This shouldn't be a ConnectionError
// This is an exploit, a non-conforming ERC20, or an invalid connection
// This should halt the process which is sufficient, yet this is sub-optimal
// TODO
Err(Error::ConnectionError)?;
}
Coin::Erc20(token)
};
in_instructions.push(InInstruction {
id,
from: *log.from.0,
coin,
amount: log.amount,
data: log.instruction.as_ref().to_vec(),
key_at_end_of_block,
});
}
Ok(in_instructions)
}
pub async fn executed_commands(&self, block: u64) -> Result<Vec<Executed>, Error> {
let mut res = vec![];
{
let filter = Filter::new().from_block(block).to_block(block).address(self.1);
let filter = filter.event_signature(SeraiKeyUpdated::SIGNATURE_HASH);
let logs = self.0.get_logs(&filter).await.map_err(|_| Error::ConnectionError)?;
for log in logs {
// Double check the address which emitted this log
if log.address() != self.1 {
Err(Error::ConnectionError)?;
}
let tx_id = log.transaction_hash.ok_or(Error::ConnectionError)?.into();
let log =
log.log_decode::<SeraiKeyUpdated>().map_err(|_| Error::ConnectionError)?.inner.data;
let mut signature = [0; 64];
signature[.. 32].copy_from_slice(log.signature.c.as_ref());
signature[32 ..].copy_from_slice(log.signature.s.as_ref());
res.push(Executed {
tx_id,
nonce: log.nonce.try_into().map_err(|_| Error::ConnectionError)?,
signature,
});
}
}
{
let filter = Filter::new().from_block(block).to_block(block).address(self.1);
let filter = filter.event_signature(ExecutedEvent::SIGNATURE_HASH);
let logs = self.0.get_logs(&filter).await.map_err(|_| Error::ConnectionError)?;
for log in logs {
// Double check the address which emitted this log
if log.address() != self.1 {
Err(Error::ConnectionError)?;
}
let tx_id = log.transaction_hash.ok_or(Error::ConnectionError)?.into();
let log = log.log_decode::<ExecutedEvent>().map_err(|_| Error::ConnectionError)?.inner.data;
let mut signature = [0; 64];
signature[.. 32].copy_from_slice(log.signature.c.as_ref());
signature[32 ..].copy_from_slice(log.signature.s.as_ref());
res.push(Executed {
tx_id,
nonce: log.nonce.try_into().map_err(|_| Error::ConnectionError)?,
signature,
});
}
}
Ok(res)
}
#[cfg(feature = "tests")]
pub fn key_updated_filter(&self) -> Filter {
Filter::new().address(self.1).event_signature(SeraiKeyUpdated::SIGNATURE_HASH)
}
#[cfg(feature = "tests")]
pub fn executed_filter(&self) -> Filter {
Filter::new().address(self.1).event_signature(ExecutedEvent::SIGNATURE_HASH)
}
}
*/

View File

@@ -1,34 +0,0 @@
use eyre::{eyre, Result};
use group::ff::PrimeField;
use ethers_providers::{Provider, Http};
use crate::{
Error,
crypto::{keccak256, PublicKey, Signature},
};
pub use crate::abi::schnorr::*;
pub async fn call_verify(
contract: &Schnorr<Provider<Http>>,
public_key: &PublicKey,
message: &[u8],
signature: &Signature,
) -> Result<()> {
if contract
.verify(
public_key.parity,
public_key.px.to_repr().into(),
keccak256(message),
signature.c.to_repr().into(),
signature.s.to_repr().into(),
)
.call()
.await?
{
Ok(())
} else {
Err(eyre!(Error::InvalidSignature))
}
}

View File

@@ -0,0 +1,13 @@
use alloy_sol_types::sol;
#[rustfmt::skip]
#[allow(warnings)]
#[allow(needless_pass_by_value)]
#[allow(clippy::all)]
#[allow(clippy::ignored_unit_patterns)]
#[allow(clippy::redundant_closure_for_method_calls)]
mod schnorr_container {
use super::*;
sol!("src/tests/contracts/Schnorr.sol");
}
pub(crate) use schnorr_container::TestSchnorr as schnorr;

View File

@@ -0,0 +1,51 @@
// SPDX-License-Identifier: AGPLv3
pragma solidity ^0.8.0;
contract TestERC20 {
event Transfer(address indexed from, address indexed to, uint256 value);
event Approval(address indexed owner, address indexed spender, uint256 value);
function name() public pure returns (string memory) {
return "Test ERC20";
}
function symbol() public pure returns (string memory) {
return "TEST";
}
function decimals() public pure returns (uint8) {
return 18;
}
function totalSupply() public pure returns (uint256) {
return 1_000_000 * 10e18;
}
mapping(address => uint256) balances;
mapping(address => mapping(address => uint256)) allowances;
constructor() {
balances[msg.sender] = totalSupply();
}
function balanceOf(address owner) public view returns (uint256) {
return balances[owner];
}
function transfer(address to, uint256 value) public returns (bool) {
balances[msg.sender] -= value;
balances[to] += value;
return true;
}
function transferFrom(address from, address to, uint256 value) public returns (bool) {
allowances[from][msg.sender] -= value;
balances[from] -= value;
balances[to] += value;
return true;
}
function approve(address spender, uint256 value) public returns (bool) {
allowances[msg.sender][spender] = value;
return true;
}
function allowance(address owner, address spender) public view returns (uint256) {
return allowances[owner][spender];
}
}

View File

@@ -0,0 +1,15 @@
// SPDX-License-Identifier: AGPLv3
pragma solidity ^0.8.0;
import "../../../contracts/Schnorr.sol";
contract TestSchnorr {
function verify(
bytes32 px,
bytes calldata message,
bytes32 c,
bytes32 s
) external pure returns (bool) {
return Schnorr.verify(px, message, c, s);
}
}

View File

@@ -1,49 +1,33 @@
use rand_core::OsRng;
use sha2::Sha256;
use sha3::{Digest, Keccak256};
use group::Group;
use group::ff::{Field, PrimeField};
use k256::{
ecdsa::{hazmat::SignPrimitive, signature::DigestVerifier, SigningKey, VerifyingKey},
elliptic_curve::{bigint::ArrayEncoding, ops::Reduce, point::DecompressPoint},
U256, Scalar, AffinePoint, ProjectivePoint,
ecdsa::{
self, hazmat::SignPrimitive, signature::hazmat::PrehashVerifier, SigningKey, VerifyingKey,
},
Scalar, ProjectivePoint,
};
use frost::{
curve::Secp256k1,
curve::{Ciphersuite, Secp256k1},
algorithm::{Hram, IetfSchnorr},
tests::{algorithm_machines, sign},
};
use crate::{crypto::*, tests::key_gen};
pub fn hash_to_scalar(data: &[u8]) -> Scalar {
Scalar::reduce(U256::from_be_slice(&keccak256(data)))
}
pub(crate) fn ecrecover(message: Scalar, v: u8, r: Scalar, s: Scalar) -> Option<[u8; 20]> {
if r.is_zero().into() || s.is_zero().into() || !((v == 27) || (v == 28)) {
return None;
}
#[allow(non_snake_case)]
let R = AffinePoint::decompress(&r.to_bytes(), (v - 27).into());
#[allow(non_snake_case)]
if let Some(R) = Option::<AffinePoint>::from(R) {
#[allow(non_snake_case)]
let R = ProjectivePoint::from(R);
let r = r.invert().unwrap();
let u1 = ProjectivePoint::GENERATOR * (-message * r);
let u2 = R * (s * r);
let key: ProjectivePoint = u1 + u2;
if !bool::from(key.is_identity()) {
return Some(address(&key));
}
}
None
// The ecrecover opcode, yet with parity replacing v
pub(crate) fn ecrecover(message: Scalar, odd_y: bool, r: Scalar, s: Scalar) -> Option<[u8; 20]> {
let sig = ecdsa::Signature::from_scalars(r, s).ok()?;
let message: [u8; 32] = message.to_repr().into();
alloy_core::primitives::Signature::from_signature_and_parity(
sig,
alloy_core::primitives::Parity::Parity(odd_y),
)
.ok()?
.recover_address_from_prehash(&alloy_core::primitives::B256::from(message))
.ok()
.map(Into::into)
}
#[test]
@@ -55,20 +39,23 @@ fn test_ecrecover() {
const MESSAGE: &[u8] = b"Hello, World!";
let (sig, recovery_id) = private
.as_nonzero_scalar()
.try_sign_prehashed_rfc6979::<Sha256>(&Keccak256::digest(MESSAGE), b"")
.try_sign_prehashed(
<Secp256k1 as Ciphersuite>::F::random(&mut OsRng),
&keccak256(MESSAGE).into(),
)
.unwrap();
// Sanity check the signature verifies
#[allow(clippy::unit_cmp)] // Intended to assert this wasn't changed to Result<bool>
{
assert_eq!(public.verify_digest(Keccak256::new_with_prefix(MESSAGE), &sig).unwrap(), ());
assert_eq!(public.verify_prehash(&keccak256(MESSAGE), &sig).unwrap(), ());
}
// Perform the ecrecover
assert_eq!(
ecrecover(
hash_to_scalar(MESSAGE),
u8::from(recovery_id.unwrap().is_y_odd()) + 27,
u8::from(recovery_id.unwrap().is_y_odd()) == 1,
*sig.r(),
*sig.s()
)
@@ -93,18 +80,13 @@ fn test_signing() {
pub fn preprocess_signature_for_ecrecover(
R: ProjectivePoint,
public_key: &PublicKey,
chain_id: U256,
m: &[u8],
s: Scalar,
) -> (u8, Scalar, Scalar) {
let c = EthereumHram::hram(
&R,
&public_key.A,
&[chain_id.to_be_byte_array().as_slice(), &keccak256(m)].concat(),
);
) -> (Scalar, Scalar) {
let c = EthereumHram::hram(&R, &public_key.A, m);
let sa = -(s * public_key.px);
let ca = -(c * public_key.px);
(public_key.parity, sa, ca)
(sa, ca)
}
#[test]
@@ -112,21 +94,12 @@ fn test_ecrecover_hack() {
let (keys, public_key) = key_gen();
const MESSAGE: &[u8] = b"Hello, World!";
let hashed_message = keccak256(MESSAGE);
let chain_id = U256::ONE;
let full_message = &[chain_id.to_be_byte_array().as_slice(), &hashed_message].concat();
let algo = IetfSchnorr::<Secp256k1, EthereumHram>::ietf();
let sig = sign(
&mut OsRng,
&algo,
keys.clone(),
algorithm_machines(&mut OsRng, &algo, &keys),
full_message,
);
let sig =
sign(&mut OsRng, &algo, keys.clone(), algorithm_machines(&mut OsRng, &algo, &keys), MESSAGE);
let (parity, sa, ca) =
preprocess_signature_for_ecrecover(sig.R, &public_key, chain_id, MESSAGE, sig.s);
let q = ecrecover(sa, parity, public_key.px, ca).unwrap();
let (sa, ca) = preprocess_signature_for_ecrecover(sig.R, &public_key, MESSAGE, sig.s);
let q = ecrecover(sa, false, public_key.px, ca).unwrap();
assert_eq!(q, address(&sig.R));
}

View File

@@ -1,21 +1,25 @@
use std::{sync::Arc, time::Duration, fs::File, collections::HashMap};
use std::{sync::Arc, collections::HashMap};
use rand_core::OsRng;
use group::ff::PrimeField;
use k256::{Scalar, ProjectivePoint};
use frost::{curve::Secp256k1, Participant, ThresholdKeys, tests::key_gen as frost_key_gen};
use ethers_core::{
types::{H160, Signature as EthersSignature},
abi::Abi,
use alloy_core::{
primitives::{Address, U256, Bytes, TxKind},
hex::FromHex,
};
use ethers_contract::ContractFactory;
use ethers_providers::{Middleware, Provider, Http};
use alloy_consensus::{SignableTransaction, TxLegacy};
use crate::crypto::PublicKey;
use alloy_rpc_types::TransactionReceipt;
use alloy_simple_request_transport::SimpleRequest;
use alloy_provider::{Provider, RootProvider};
use crate::crypto::{address, deterministically_sign, PublicKey};
mod crypto;
mod abi;
mod schnorr;
mod router;
@@ -36,57 +40,88 @@ pub fn key_gen() -> (HashMap<Participant, ThresholdKeys<Secp256k1>>, PublicKey)
(keys, public_key)
}
// TODO: Replace with a contract deployment from an unknown account, so the environment solely has
// to fund the deployer, not create/pass a wallet
// TODO: Deterministic deployments across chains
// TODO: Use a proper error here
pub async fn send(
provider: &RootProvider<SimpleRequest>,
wallet: &k256::ecdsa::SigningKey,
mut tx: TxLegacy,
) -> Option<TransactionReceipt> {
let verifying_key = *wallet.verifying_key().as_affine();
let address = Address::from(address(&verifying_key.into()));
// https://github.com/alloy-rs/alloy/issues/539
// let chain_id = provider.get_chain_id().await.unwrap();
// tx.chain_id = Some(chain_id);
tx.chain_id = None;
tx.nonce = provider.get_transaction_count(address, None).await.unwrap();
// 100 gwei
tx.gas_price = 100_000_000_000u128;
let sig = wallet.sign_prehash_recoverable(tx.signature_hash().as_ref()).unwrap();
assert_eq!(address, tx.clone().into_signed(sig.into()).recover_signer().unwrap());
assert!(
provider.get_balance(address, None).await.unwrap() >
((U256::from(tx.gas_price) * U256::from(tx.gas_limit)) + tx.value)
);
let mut bytes = vec![];
tx.encode_with_signature_fields(&sig.into(), &mut bytes);
let pending_tx = provider.send_raw_transaction(&bytes).await.ok()?;
pending_tx.get_receipt().await.ok()
}
pub async fn fund_account(
provider: &RootProvider<SimpleRequest>,
wallet: &k256::ecdsa::SigningKey,
to_fund: Address,
value: U256,
) -> Option<()> {
let funding_tx =
TxLegacy { to: TxKind::Call(to_fund), gas_limit: 21_000, value, ..Default::default() };
assert!(send(provider, wallet, funding_tx).await.unwrap().status());
Some(())
}
// TODO: Use a proper error here
pub async fn deploy_contract(
chain_id: u32,
client: Arc<Provider<Http>>,
client: Arc<RootProvider<SimpleRequest>>,
wallet: &k256::ecdsa::SigningKey,
name: &str,
) -> eyre::Result<H160> {
let abi: Abi =
serde_json::from_reader(File::open(format!("./artifacts/{name}.abi")).unwrap()).unwrap();
) -> Option<Address> {
let hex_bin_buf = std::fs::read_to_string(format!("./artifacts/{name}.bin")).unwrap();
let hex_bin =
if let Some(stripped) = hex_bin_buf.strip_prefix("0x") { stripped } else { &hex_bin_buf };
let bin = hex::decode(hex_bin).unwrap();
let factory = ContractFactory::new(abi, bin.into(), client.clone());
let bin = Bytes::from_hex(hex_bin).unwrap();
let mut deployment_tx = factory.deploy(())?.tx;
deployment_tx.set_chain_id(chain_id);
deployment_tx.set_gas(1_000_000);
let (max_fee_per_gas, max_priority_fee_per_gas) = client.estimate_eip1559_fees(None).await?;
deployment_tx.as_eip1559_mut().unwrap().max_fee_per_gas = Some(max_fee_per_gas);
deployment_tx.as_eip1559_mut().unwrap().max_priority_fee_per_gas = Some(max_priority_fee_per_gas);
let deployment_tx = TxLegacy {
chain_id: None,
nonce: 0,
// 100 gwei
gas_price: 100_000_000_000u128,
gas_limit: 1_000_000,
to: TxKind::Create,
value: U256::ZERO,
input: bin,
};
let sig_hash = deployment_tx.sighash();
let (sig, rid) = wallet.sign_prehash_recoverable(sig_hash.as_ref()).unwrap();
let deployment_tx = deterministically_sign(&deployment_tx);
// EIP-155 v
let mut v = u64::from(rid.to_byte());
assert!((v == 0) || (v == 1));
v += u64::from((chain_id * 2) + 35);
// Fund the deployer address
fund_account(
&client,
wallet,
deployment_tx.recover_signer().unwrap(),
U256::from(deployment_tx.tx().gas_limit) * U256::from(deployment_tx.tx().gas_price),
)
.await?;
let r = sig.r().to_repr();
let r_ref: &[u8] = r.as_ref();
let s = sig.s().to_repr();
let s_ref: &[u8] = s.as_ref();
let deployment_tx =
deployment_tx.rlp_signed(&EthersSignature { r: r_ref.into(), s: s_ref.into(), v });
let (deployment_tx, sig, _) = deployment_tx.into_parts();
let mut bytes = vec![];
deployment_tx.encode_with_signature_fields(&sig, &mut bytes);
let pending_tx = client.send_raw_transaction(&bytes).await.ok()?;
let receipt = pending_tx.get_receipt().await.ok()?;
assert!(receipt.status());
let pending_tx = client.send_raw_transaction(deployment_tx).await?;
let mut receipt;
while {
receipt = client.get_transaction_receipt(pending_tx.tx_hash()).await?;
receipt.is_none()
} {
tokio::time::sleep(Duration::from_secs(6)).await;
}
let receipt = receipt.unwrap();
assert!(receipt.status == Some(1.into()));
Ok(receipt.contract_address.unwrap())
Some(receipt.contract_address.unwrap())
}

View File

@@ -2,7 +2,8 @@ use std::{convert::TryFrom, sync::Arc, collections::HashMap};
use rand_core::OsRng;
use group::ff::PrimeField;
use group::Group;
use k256::ProjectivePoint;
use frost::{
curve::Secp256k1,
Participant, ThresholdKeys,
@@ -10,100 +11,173 @@ use frost::{
tests::{algorithm_machines, sign},
};
use ethers_core::{
types::{H160, U256, Bytes},
abi::AbiEncode,
utils::{Anvil, AnvilInstance},
};
use ethers_providers::{Middleware, Provider, Http};
use alloy_core::primitives::{Address, U256};
use alloy_simple_request_transport::SimpleRequest;
use alloy_rpc_client::ClientBuilder;
use alloy_provider::{Provider, RootProvider};
use alloy_node_bindings::{Anvil, AnvilInstance};
use crate::{
crypto::{keccak256, PublicKey, EthereumHram, Signature},
router::{self, *},
tests::{key_gen, deploy_contract},
crypto::*,
deployer::Deployer,
router::{Router, abi as router},
tests::{key_gen, send, fund_account},
};
async fn setup_test() -> (
u32,
AnvilInstance,
Router<Provider<Http>>,
Arc<RootProvider<SimpleRequest>>,
u64,
Router,
HashMap<Participant, ThresholdKeys<Secp256k1>>,
PublicKey,
) {
let anvil = Anvil::new().spawn();
let provider = Provider::<Http>::try_from(anvil.endpoint()).unwrap();
let chain_id = provider.get_chainid().await.unwrap().as_u32();
let provider = RootProvider::new(
ClientBuilder::default().transport(SimpleRequest::new(anvil.endpoint()), true),
);
let chain_id = provider.get_chain_id().await.unwrap();
let wallet = anvil.keys()[0].clone().into();
let client = Arc::new(provider);
let contract_address =
deploy_contract(chain_id, client.clone(), &wallet, "Router").await.unwrap();
let contract = Router::new(contract_address, client.clone());
// Make sure the Deployer constructor returns None, as it doesn't exist yet
assert!(Deployer::new(client.clone()).await.unwrap().is_none());
// Deploy the Deployer
let tx = Deployer::deployment_tx();
fund_account(
&client,
&wallet,
tx.recover_signer().unwrap(),
U256::from(tx.tx().gas_limit) * U256::from(tx.tx().gas_price),
)
.await
.unwrap();
let (tx, sig, _) = tx.into_parts();
let mut bytes = vec![];
tx.encode_with_signature_fields(&sig, &mut bytes);
let pending_tx = client.send_raw_transaction(&bytes).await.unwrap();
let receipt = pending_tx.get_receipt().await.unwrap();
assert!(receipt.status());
let deployer =
Deployer::new(client.clone()).await.expect("network error").expect("deployer wasn't deployed");
let (keys, public_key) = key_gen();
// Set the key to the threshold keys
let tx = contract.init_serai_key(public_key.px.to_repr().into()).gas(100_000);
let pending_tx = tx.send().await.unwrap();
let receipt = pending_tx.await.unwrap().unwrap();
assert!(receipt.status == Some(1.into()));
// Verify the Router constructor returns None, as it doesn't exist yet
assert!(deployer.find_router(client.clone(), &public_key).await.unwrap().is_none());
(chain_id, anvil, contract, keys, public_key)
// Deploy the router
let receipt = send(&client, &anvil.keys()[0].clone().into(), deployer.deploy_router(&public_key))
.await
.unwrap();
assert!(receipt.status());
let contract = deployer.find_router(client.clone(), &public_key).await.unwrap().unwrap();
(anvil, client, chain_id, contract, keys, public_key)
}
async fn latest_block_hash(client: &RootProvider<SimpleRequest>) -> [u8; 32] {
client
.get_block(client.get_block_number().await.unwrap().into(), false)
.await
.unwrap()
.unwrap()
.header
.hash
.unwrap()
.0
}
#[tokio::test]
async fn test_deploy_contract() {
setup_test().await;
let (_anvil, client, _, router, _, public_key) = setup_test().await;
let block_hash = latest_block_hash(&client).await;
assert_eq!(router.serai_key(block_hash).await.unwrap(), public_key);
assert_eq!(router.nonce(block_hash).await.unwrap(), U256::try_from(1u64).unwrap());
// TODO: Check it emitted SeraiKeyUpdated(public_key) at its genesis
}
pub fn hash_and_sign(
keys: &HashMap<Participant, ThresholdKeys<Secp256k1>>,
public_key: &PublicKey,
chain_id: U256,
message: &[u8],
) -> Signature {
let hashed_message = keccak256(message);
let mut chain_id_bytes = [0; 32];
chain_id.to_big_endian(&mut chain_id_bytes);
let full_message = &[chain_id_bytes.as_slice(), &hashed_message].concat();
let algo = IetfSchnorr::<Secp256k1, EthereumHram>::ietf();
let sig = sign(
&mut OsRng,
&algo,
keys.clone(),
algorithm_machines(&mut OsRng, &algo, keys),
full_message,
);
let sig =
sign(&mut OsRng, &algo, keys.clone(), algorithm_machines(&mut OsRng, &algo, keys), message);
Signature::new(public_key, k256::U256::from_words(chain_id.0), message, sig).unwrap()
Signature::new(public_key, message, sig).unwrap()
}
#[tokio::test]
async fn test_router_update_serai_key() {
let (anvil, client, chain_id, contract, keys, public_key) = setup_test().await;
let next_key = loop {
let point = ProjectivePoint::random(&mut OsRng);
let Some(next_key) = PublicKey::new(point) else { continue };
break next_key;
};
let message = Router::update_serai_key_message(
U256::try_from(chain_id).unwrap(),
U256::try_from(1u64).unwrap(),
&next_key,
);
let sig = hash_and_sign(&keys, &public_key, &message);
let first_block_hash = latest_block_hash(&client).await;
assert_eq!(contract.serai_key(first_block_hash).await.unwrap(), public_key);
let receipt =
send(&client, &anvil.keys()[0].clone().into(), contract.update_serai_key(&next_key, &sig))
.await
.unwrap();
assert!(receipt.status());
let second_block_hash = latest_block_hash(&client).await;
assert_eq!(contract.serai_key(second_block_hash).await.unwrap(), next_key);
// Check this does still offer the historical state
assert_eq!(contract.serai_key(first_block_hash).await.unwrap(), public_key);
// TODO: Check logs
println!("gas used: {:?}", receipt.gas_used);
// println!("logs: {:?}", receipt.logs);
}
#[tokio::test]
async fn test_router_execute() {
let (chain_id, _anvil, contract, keys, public_key) = setup_test().await;
let (anvil, client, chain_id, contract, keys, public_key) = setup_test().await;
let to = H160([0u8; 20]);
let value = U256([0u64; 4]);
let data = Bytes::from([0]);
let tx = OutInstruction { to, value, data: data.clone() };
let to = Address::from([0; 20]);
let value = U256::ZERO;
let tx = router::OutInstruction { to, value, calls: vec![] };
let txs = vec![tx];
let nonce_call = contract.nonce();
let nonce = nonce_call.call().await.unwrap();
let first_block_hash = latest_block_hash(&client).await;
let nonce = contract.nonce(first_block_hash).await.unwrap();
assert_eq!(nonce, U256::try_from(1u64).unwrap());
let encoded =
("execute".to_string(), nonce, vec![router::OutInstruction { to, value, data }]).encode();
let sig = hash_and_sign(&keys, &public_key, chain_id.into(), &encoded);
let message = Router::execute_message(U256::try_from(chain_id).unwrap(), nonce, txs.clone());
let sig = hash_and_sign(&keys, &public_key, &message);
let tx = contract
.execute(vec![tx], router::Signature { c: sig.c.to_repr().into(), s: sig.s.to_repr().into() })
.gas(300_000);
let pending_tx = tx.send().await.unwrap();
let receipt = dbg!(pending_tx.await.unwrap().unwrap());
assert!(receipt.status == Some(1.into()));
let receipt =
send(&client, &anvil.keys()[0].clone().into(), contract.execute(&txs, &sig)).await.unwrap();
assert!(receipt.status());
println!("gas used: {:?}", receipt.cumulative_gas_used);
println!("logs: {:?}", receipt.logs);
let second_block_hash = latest_block_hash(&client).await;
assert_eq!(contract.nonce(second_block_hash).await.unwrap(), U256::try_from(2u64).unwrap());
// Check this does still offer the historical state
assert_eq!(contract.nonce(first_block_hash).await.unwrap(), U256::try_from(1u64).unwrap());
// TODO: Check logs
println!("gas used: {:?}", receipt.gas_used);
// println!("logs: {:?}", receipt.logs);
}

View File

@@ -1,11 +1,9 @@
use std::{convert::TryFrom, sync::Arc};
use std::sync::Arc;
use rand_core::OsRng;
use ::k256::{elliptic_curve::bigint::ArrayEncoding, U256, Scalar};
use ethers_core::utils::{keccak256, Anvil, AnvilInstance};
use ethers_providers::{Middleware, Provider, Http};
use group::ff::PrimeField;
use k256::Scalar;
use frost::{
curve::Secp256k1,
@@ -13,24 +11,34 @@ use frost::{
tests::{algorithm_machines, sign},
};
use alloy_core::primitives::Address;
use alloy_sol_types::SolCall;
use alloy_rpc_types::{TransactionInput, TransactionRequest};
use alloy_simple_request_transport::SimpleRequest;
use alloy_rpc_client::ClientBuilder;
use alloy_provider::{Provider, RootProvider};
use alloy_node_bindings::{Anvil, AnvilInstance};
use crate::{
Error,
crypto::*,
schnorr::*,
tests::{key_gen, deploy_contract},
tests::{key_gen, deploy_contract, abi::schnorr as abi},
};
async fn setup_test() -> (u32, AnvilInstance, Schnorr<Provider<Http>>) {
async fn setup_test() -> (AnvilInstance, Arc<RootProvider<SimpleRequest>>, Address) {
let anvil = Anvil::new().spawn();
let provider = Provider::<Http>::try_from(anvil.endpoint()).unwrap();
let chain_id = provider.get_chainid().await.unwrap().as_u32();
let provider = RootProvider::new(
ClientBuilder::default().transport(SimpleRequest::new(anvil.endpoint()), true),
);
let wallet = anvil.keys()[0].clone().into();
let client = Arc::new(provider);
let contract_address =
deploy_contract(chain_id, client.clone(), &wallet, "Schnorr").await.unwrap();
let contract = Schnorr::new(contract_address, client.clone());
(chain_id, anvil, contract)
let address = deploy_contract(client.clone(), &wallet, "TestSchnorr").await.unwrap();
(anvil, client, address)
}
#[tokio::test]
@@ -38,30 +46,48 @@ async fn test_deploy_contract() {
setup_test().await;
}
pub async fn call_verify(
provider: &RootProvider<SimpleRequest>,
contract: Address,
public_key: &PublicKey,
message: &[u8],
signature: &Signature,
) -> Result<(), Error> {
let px: [u8; 32] = public_key.px.to_repr().into();
let c_bytes: [u8; 32] = signature.c.to_repr().into();
let s_bytes: [u8; 32] = signature.s.to_repr().into();
let call = TransactionRequest::default().to(Some(contract)).input(TransactionInput::new(
abi::verifyCall::new((px.into(), message.to_vec().into(), c_bytes.into(), s_bytes.into()))
.abi_encode()
.into(),
));
let bytes = provider.call(&call, None).await.map_err(|_| Error::ConnectionError)?;
let res =
abi::verifyCall::abi_decode_returns(&bytes, true).map_err(|_| Error::ConnectionError)?;
if res._0 {
Ok(())
} else {
Err(Error::InvalidSignature)
}
}
#[tokio::test]
async fn test_ecrecover_hack() {
let (chain_id, _anvil, contract) = setup_test().await;
let chain_id = U256::from(chain_id);
let (_anvil, client, contract) = setup_test().await;
let (keys, public_key) = key_gen();
const MESSAGE: &[u8] = b"Hello, World!";
let hashed_message = keccak256(MESSAGE);
let full_message = &[chain_id.to_be_byte_array().as_slice(), &hashed_message].concat();
let algo = IetfSchnorr::<Secp256k1, EthereumHram>::ietf();
let sig = sign(
&mut OsRng,
&algo,
keys.clone(),
algorithm_machines(&mut OsRng, &algo, &keys),
full_message,
);
let sig = Signature::new(&public_key, chain_id, MESSAGE, sig).unwrap();
let sig =
sign(&mut OsRng, &algo, keys.clone(), algorithm_machines(&mut OsRng, &algo, &keys), MESSAGE);
let sig = Signature::new(&public_key, MESSAGE, sig).unwrap();
call_verify(&contract, &public_key, MESSAGE, &sig).await.unwrap();
call_verify(&client, contract, &public_key, MESSAGE, &sig).await.unwrap();
// Test an invalid signature fails
let mut sig = sig;
sig.s += Scalar::ONE;
assert!(call_verify(&contract, &public_key, MESSAGE, &sig).await.is_err());
assert!(call_verify(&client, contract, &public_key, MESSAGE, &sig).await.is_err());
}