Merge pull request #65 from comit-network/eventloop

Drive swarm in separate async task
This commit is contained in:
rishflab 2020-12-10 14:38:20 +11:00 committed by GitHub
commit 205b879967
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 730 additions and 310 deletions

View File

@ -37,6 +37,7 @@ use xmr_btc::{
};
mod amounts;
pub mod event_loop;
mod execution;
mod message0;
mod message1;
@ -117,7 +118,7 @@ pub async fn swap(
OutEvent::ConnectionEstablished(bob) => {
info!("Connection established with: {}", bob);
}
OutEvent::Request(amounts::OutEvent::Btc { btc, channel }) => {
OutEvent::Request(amounts::OutEvent { btc, channel }) => {
let amounts = calculate_amounts(btc);
last_amounts = Some(amounts);
swarm.send_amounts(channel, amounts);
@ -144,9 +145,6 @@ pub async fn swap(
punish_address,
);
info!("Commencing handshake");
swarm.set_state0(state.clone());
state0 = Some(state)
}
OutEvent::Message0(msg) => {
@ -386,6 +384,20 @@ pub struct Behaviour {
}
impl Behaviour {
pub fn new(state: State0) -> Self {
let identity = Keypair::generate_ed25519();
Self {
pt: PeerTracker::default(),
amounts: Amounts::default(),
message0: Message0::new(state),
message1: Message1::default(),
message2: Message2::default(),
message3: Message3::default(),
identity,
}
}
pub fn identity(&self) -> Keypair {
self.identity.clone()
}
@ -401,13 +413,6 @@ impl Behaviour {
info!("Sent amounts response");
}
// TODO(Franck) remove
/// Message0 gets sent within the network layer using this state0.
pub fn set_state0(&mut self, state: State0) {
debug!("Set state 0");
let _ = self.message0.set_state(state);
}
/// Send Message1 to Bob in response to receiving his Message1.
pub fn send_message1(
&mut self,
@ -429,22 +434,6 @@ impl Behaviour {
}
}
impl Default for Behaviour {
fn default() -> Self {
let identity = Keypair::generate_ed25519();
Self {
pt: PeerTracker::default(),
amounts: Amounts::default(),
message0: Message0::default(),
message1: Message1::default(),
message2: Message2::default(),
message3: Message3::default(),
identity,
}
}
}
fn calculate_amounts(btc: ::bitcoin::Amount) -> SwapAmounts {
// TODO (Franck): This should instead verify that the received amounts matches
// the command line arguments This value corresponds to 100 XMR per BTC

View File

@ -13,14 +13,15 @@ use std::{
};
use tracing::{debug, error};
use crate::network::request_response::{AliceToBob, AmountsProtocol, BobToAlice, Codec, TIMEOUT};
use crate::{
alice::amounts,
network::request_response::{AliceToBob, AmountsProtocol, BobToAlice, Codec, TIMEOUT},
};
#[derive(Debug)]
pub enum OutEvent {
Btc {
btc: ::bitcoin::Amount,
channel: ResponseChannel<AliceToBob>,
},
pub struct OutEvent {
pub btc: ::bitcoin::Amount,
pub channel: ResponseChannel<AliceToBob>,
}
/// A `NetworkBehaviour` that represents getting the amounts of an XMR/BTC swap.
@ -82,7 +83,7 @@ impl NetworkBehaviourEventProcess<RequestResponseEvent<BobToAlice, AliceToBob>>
} => {
if let BobToAlice::AmountsFromBtc(btc) = request {
debug!("Received amounts request");
self.events.push_back(OutEvent::Btc { btc, channel })
self.events.push_back(amounts::OutEvent { btc, channel })
}
}
RequestResponseEvent::Message {

View File

@ -0,0 +1,225 @@
use crate::{
alice::{Behaviour, OutEvent},
network::{request_response::AliceToBob, transport::SwapTransport, TokioExecutor},
SwapAmounts,
};
use anyhow::{Context, Result};
use futures::FutureExt;
use libp2p::{
core::Multiaddr, futures::StreamExt, request_response::ResponseChannel, PeerId, Swarm,
};
use tokio::sync::mpsc::{Receiver, Sender};
use xmr_btc::{alice, bob};
pub struct Channels<T> {
sender: Sender<T>,
receiver: Receiver<T>,
}
impl<T> Channels<T> {
pub fn new() -> Channels<T> {
let (sender, receiver) = tokio::sync::mpsc::channel(100);
Channels { sender, receiver }
}
}
impl<T> Default for Channels<T> {
fn default() -> Self {
Self::new()
}
}
pub struct EventLoopHandle {
pub msg0: Receiver<bob::Message0>,
pub msg1: Receiver<(bob::Message1, ResponseChannel<AliceToBob>)>,
pub msg2: Receiver<(bob::Message2, ResponseChannel<AliceToBob>)>,
pub msg3: Receiver<bob::Message3>,
pub request: Receiver<crate::alice::amounts::OutEvent>,
pub conn_established: Receiver<PeerId>,
pub send_amounts: Sender<(ResponseChannel<AliceToBob>, SwapAmounts)>,
pub send_msg1: Sender<(ResponseChannel<AliceToBob>, alice::Message1)>,
pub send_msg2: Sender<(ResponseChannel<AliceToBob>, alice::Message2)>,
}
impl EventLoopHandle {
pub async fn recv_conn_established(&mut self) -> Result<PeerId> {
self.conn_established
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive connection established from Bob"))
}
pub async fn recv_message0(&mut self) -> Result<bob::Message0> {
self.msg0
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive message 0 from Bob"))
}
pub async fn recv_message1(&mut self) -> Result<(bob::Message1, ResponseChannel<AliceToBob>)> {
self.msg1
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive message 1 from Bob"))
}
pub async fn recv_message2(&mut self) -> Result<(bob::Message2, ResponseChannel<AliceToBob>)> {
self.msg2
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed o receive message 2 from Bob"))
}
pub async fn recv_message3(&mut self) -> Result<bob::Message3> {
self.msg3.recv().await.ok_or_else(|| {
anyhow::Error::msg("Failed to receive Bitcoin encrypted signature from Bob")
})
}
pub async fn recv_request(&mut self) -> Result<crate::alice::amounts::OutEvent> {
self.request
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive amounts request from Bob"))
}
pub async fn send_amounts(
&mut self,
channel: ResponseChannel<AliceToBob>,
amounts: SwapAmounts,
) -> Result<()> {
let _ = self.send_amounts.send((channel, amounts)).await?;
Ok(())
}
pub async fn send_message1(
&mut self,
channel: ResponseChannel<AliceToBob>,
msg: alice::Message1,
) -> Result<()> {
let _ = self.send_msg1.send((channel, msg)).await?;
Ok(())
}
pub async fn send_message2(
&mut self,
channel: ResponseChannel<AliceToBob>,
msg: alice::Message2,
) -> Result<()> {
let _ = self.send_msg2.send((channel, msg)).await?;
Ok(())
}
}
pub struct EventLoop {
pub swarm: libp2p::Swarm<Behaviour>,
pub msg0: Sender<bob::Message0>,
pub msg1: Sender<(bob::Message1, ResponseChannel<AliceToBob>)>,
pub msg2: Sender<(bob::Message2, ResponseChannel<AliceToBob>)>,
pub msg3: Sender<bob::Message3>,
pub request: Sender<crate::alice::amounts::OutEvent>,
pub conn_established: Sender<PeerId>,
pub send_amounts: Receiver<(ResponseChannel<AliceToBob>, SwapAmounts)>,
pub send_msg1: Receiver<(ResponseChannel<AliceToBob>, alice::Message1)>,
pub send_msg2: Receiver<(ResponseChannel<AliceToBob>, alice::Message2)>,
}
impl EventLoop {
pub fn new(
transport: SwapTransport,
behaviour: Behaviour,
listen: Multiaddr,
) -> Result<(Self, EventLoopHandle)> {
let local_peer_id = behaviour.peer_id();
let mut swarm = libp2p::swarm::SwarmBuilder::new(transport, behaviour, local_peer_id)
.executor(Box::new(TokioExecutor {
handle: tokio::runtime::Handle::current(),
}))
.build();
Swarm::listen_on(&mut swarm, listen.clone())
.with_context(|| format!("Address is not supported: {:#}", listen))?;
let msg0 = Channels::new();
let msg1 = Channels::new();
let msg2 = Channels::new();
let msg3 = Channels::new();
let request = Channels::new();
let conn_established = Channels::new();
let send_amounts = Channels::new();
let send_msg1 = Channels::new();
let send_msg2 = Channels::new();
let driver = EventLoop {
swarm,
msg0: msg0.sender,
msg1: msg1.sender,
msg2: msg2.sender,
msg3: msg3.sender,
request: request.sender,
conn_established: conn_established.sender,
send_amounts: send_amounts.receiver,
send_msg1: send_msg1.receiver,
send_msg2: send_msg2.receiver,
};
let handle = EventLoopHandle {
msg0: msg0.receiver,
msg1: msg1.receiver,
msg2: msg2.receiver,
msg3: msg3.receiver,
request: request.receiver,
conn_established: conn_established.receiver,
send_amounts: send_amounts.sender,
send_msg1: send_msg1.sender,
send_msg2: send_msg2.sender,
};
Ok((driver, handle))
}
pub async fn run(&mut self) {
loop {
tokio::select! {
swarm_event = self.swarm.next().fuse() => {
match swarm_event {
OutEvent::ConnectionEstablished(alice) => {
let _ = self.conn_established.send(alice).await;
}
OutEvent::Message0(msg) => {
let _ = self.msg0.send(msg).await;
}
OutEvent::Message1 { msg, channel } => {
let _ = self.msg1.send((msg, channel)).await;
}
OutEvent::Message2 { msg, channel } => {
let _ = self.msg2.send((msg, channel)).await;
}
OutEvent::Message3(msg) => {
let _ = self.msg3.send(msg).await;
}
OutEvent::Request(event) => {
let _ = self.request.send(event).await;
}
}
},
amounts = self.send_amounts.next().fuse() => {
if let Some((channel, amounts)) = amounts {
self.swarm.send_amounts(channel, amounts);
}
},
msg1 = self.send_msg1.next().fuse() => {
if let Some((channel, msg)) = msg1 {
self.swarm.send_message1(channel, msg);
}
},
msg2 = self.send_msg2.next().fuse() => {
if let Some((channel, msg)) = msg2 {
self.swarm.send_message2(channel, msg);
}
},
}
}
}
}

View File

@ -1,8 +1,6 @@
use crate::{
alice::{amounts, OutEvent, Swarm},
bitcoin, monero,
network::request_response::AliceToBob,
SwapAmounts, PUNISH_TIMELOCK, REFUND_TIMELOCK,
alice::event_loop::EventLoopHandle, bitcoin, monero, network::request_response::AliceToBob,
SwapAmounts,
};
use anyhow::{bail, Context, Result};
use ecdsa_fun::{adaptor::Adaptor, nonce::Deterministic};
@ -11,13 +9,14 @@ use futures::{
pin_mut,
};
use libp2p::request_response::ResponseChannel;
use sha2::Sha256;
use std::{sync::Arc, time::Duration};
use tokio::time::timeout;
use tracing::trace;
use tracing::{info, trace};
use xmr_btc::{
alice,
alice::{State0, State3},
alice::State3,
bitcoin::{
poll_until_block_height_is_gte, BlockHeight, BroadcastSignedTransaction,
EncryptedSignature, GetRawTransaction, TransactionBlockHeight, TxCancel, TxLock, TxRefund,
@ -29,91 +28,53 @@ use xmr_btc::{
};
pub async fn negotiate(
state0: xmr_btc::alice::State0,
amounts: SwapAmounts,
a: bitcoin::SecretKey,
s_a: cross_curve_dleq::Scalar,
v_a: monero::PrivateViewKey,
swarm: &mut Swarm,
bitcoin_wallet: Arc<bitcoin::Wallet>,
// a: bitcoin::SecretKey,
// s_a: cross_curve_dleq::Scalar,
// v_a: monero::PrivateViewKey,
swarm_handle: &mut EventLoopHandle,
// bitcoin_wallet: Arc<bitcoin::Wallet>,
config: Config,
) -> Result<(ResponseChannel<AliceToBob>, State3)> {
trace!("Starting negotiate");
let event = timeout(config.bob_time_to_act, swarm.next())
.await
.context("Failed to receive dial connection from Bob")?;
match event {
OutEvent::ConnectionEstablished(_bob_peer_id) => {}
other => bail!("Unexpected event received: {:?}", other),
}
let event = timeout(config.bob_time_to_act, swarm.next())
// todo: we can move this out, we dont need to timeout here
let _peer_id = timeout(config.bob_time_to_act, swarm_handle.recv_conn_established())
.await
.context("Failed to receive amounts from Bob")?;
let (btc, channel) = match event {
OutEvent::Request(amounts::OutEvent::Btc { btc, channel }) => (btc, channel),
other => bail!("Unexpected event received: {:?}", other),
};
.context("Failed to receive dial connection from Bob")??;
if btc != amounts.btc {
let event = timeout(config.bob_time_to_act, swarm_handle.recv_request())
.await
.context("Failed to receive amounts from Bob")??;
if event.btc != amounts.btc {
bail!(
"Bob proposed a different amount; got {}, expected: {}",
btc,
event.btc,
amounts.btc
);
}
// TODO: get an ack from libp2p2
swarm.send_amounts(channel, amounts);
let redeem_address = bitcoin_wallet.as_ref().new_address().await?;
let punish_address = redeem_address.clone();
swarm_handle.send_amounts(event.channel, amounts).await?;
let state0 = State0::new(
a,
s_a,
v_a,
amounts.btc,
amounts.xmr,
REFUND_TIMELOCK,
PUNISH_TIMELOCK,
redeem_address,
punish_address,
);
let bob_message0 = timeout(config.bob_time_to_act, swarm_handle.recv_message0()).await??;
// TODO(Franck): Understand why this is needed.
swarm.set_state0(state0.clone());
let state1 = state0.receive(bob_message0)?;
let event = timeout(config.bob_time_to_act, swarm.next())
.await
.context("Failed to receive message 0 from Bob")?;
let message0 = match event {
OutEvent::Message0(msg) => msg,
other => bail!("Unexpected event received: {:?}", other),
};
let (bob_message1, channel) =
timeout(config.bob_time_to_act, swarm_handle.recv_message1()).await??;
let state1 = state0.receive(message0)?;
let state2 = state1.receive(bob_message1);
let event = timeout(config.bob_time_to_act, swarm.next())
.await
.context("Failed to receive message 1 from Bob")?;
let (msg, channel) = match event {
OutEvent::Message1 { msg, channel } => (msg, channel),
other => bail!("Unexpected event: {:?}", other),
};
swarm_handle
.send_message1(channel, state2.next_message())
.await?;
let state2 = state1.receive(msg);
let (bob_message2, channel) =
timeout(config.bob_time_to_act, swarm_handle.recv_message2()).await??;
let message1 = state2.next_message();
swarm.send_message1(channel, message1);
let event = timeout(config.bob_time_to_act, swarm.next())
.await
.context("Failed to receive message 2 from Bob")?;
let (msg, channel) = match event {
OutEvent::Message2 { msg, channel } => (msg, channel),
other => bail!("Unexpected event: {:?}", other),
};
let state3 = state2.receive(msg)?;
let state3 = state2.receive(bob_message2)?;
Ok((channel, state3))
}
@ -146,7 +107,7 @@ pub async fn lock_xmr<W>(
channel: ResponseChannel<AliceToBob>,
amounts: SwapAmounts,
state3: State3,
swarm: &mut Swarm,
swarm: &mut EventLoopHandle,
monero_wallet: Arc<W>,
) -> Result<()>
where
@ -165,28 +126,23 @@ where
// TODO(Franck): Wait for Monero to be confirmed once
swarm.send_message2(channel, alice::Message2 {
tx_lock_proof: transfer_proof,
});
swarm
.send_message2(channel, alice::Message2 {
tx_lock_proof: transfer_proof,
})
.await?;
Ok(())
}
pub async fn wait_for_bitcoin_encrypted_signature(
swarm: &mut Swarm,
swarm: &mut EventLoopHandle,
timeout_duration: Duration,
) -> Result<EncryptedSignature> {
let event = timeout(timeout_duration, swarm.next())
let msg3 = timeout(timeout_duration, swarm.recv_message3())
.await
.context("Failed to receive Bitcoin encrypted signature from Bob")?;
match event {
OutEvent::Message3(msg) => Ok(msg.tx_redeem_encsig),
other => bail!(
"Expected Bob's Bitcoin redeem encrypted signature, got: {:?}",
other
),
}
.context("Failed to receive Bitcoin encrypted signature from Bob")??;
Ok(msg3.tx_redeem_encsig)
}
pub fn build_bitcoin_redeem_transaction(
@ -227,6 +183,7 @@ pub async fn publish_bitcoin_redeem_transaction<W>(
where
W: BroadcastSignedTransaction + WaitForTransactionFinality,
{
info!("Attempting to publish bitcoin redeem txn");
let tx_id = bitcoin_wallet
.broadcast_signed_transaction(redeem_tx)
.await?;

View File

@ -1,4 +1,3 @@
use anyhow::{bail, Result};
use libp2p::{
request_response::{
handler::RequestProtocol, ProtocolSupport, RequestResponse, RequestResponseConfig,
@ -32,17 +31,24 @@ pub struct Message0 {
#[behaviour(ignore)]
events: VecDeque<OutEvent>,
#[behaviour(ignore)]
state: Option<State0>,
state: State0,
}
impl Message0 {
pub fn set_state(&mut self, state: State0) -> Result<()> {
if self.state.is_some() {
bail!("Trying to set state a second time");
}
self.state = Some(state);
pub fn new(state: State0) -> Self {
let timeout = Duration::from_secs(TIMEOUT);
let mut config = RequestResponseConfig::default();
config.set_request_timeout(timeout);
Ok(())
Self {
rr: RequestResponse::new(
Codec::default(),
vec![(Message0Protocol, ProtocolSupport::Full)],
config,
),
events: Default::default(),
state,
}
}
fn poll(
@ -58,24 +64,6 @@ impl Message0 {
}
}
impl Default for Message0 {
fn default() -> Self {
let timeout = Duration::from_secs(TIMEOUT);
let mut config = RequestResponseConfig::default();
config.set_request_timeout(timeout);
Self {
rr: RequestResponse::new(
Codec::default(),
vec![(Message0Protocol, ProtocolSupport::Full)],
config,
),
events: Default::default(),
state: None,
}
}
}
impl NetworkBehaviourEventProcess<RequestResponseEvent<BobToAlice, AliceToBob>> for Message0 {
fn inject_event(&mut self, event: RequestResponseEvent<BobToAlice, AliceToBob>) {
match event {
@ -88,13 +76,9 @@ impl NetworkBehaviourEventProcess<RequestResponseEvent<BobToAlice, AliceToBob>>
} => {
if let BobToAlice::Message0(msg) = request {
debug!("Received Message0");
let response = match &self.state {
None => panic!("No state, did you forget to set it?"),
Some(state) => {
// TODO: Get OsRng from somewhere?
AliceToBob::Message0(state.next_message(&mut OsRng))
}
};
let response = AliceToBob::Message0(self.state.next_message(&mut OsRng));
self.rr.send_response(channel, response);
debug!("Sent Message0");

View File

@ -2,17 +2,15 @@
//! Alice holds XMR and wishes receive BTC.
use crate::{
alice::{
event_loop::EventLoopHandle,
execution::{
build_bitcoin_punish_transaction, build_bitcoin_redeem_transaction,
extract_monero_private_key, lock_xmr, negotiate, publish_bitcoin_punish_transaction,
publish_bitcoin_redeem_transaction, publish_cancel_transaction,
wait_for_bitcoin_encrypted_signature, wait_for_bitcoin_refund, wait_for_locked_bitcoin,
},
Swarm,
},
bitcoin,
bitcoin::EncryptedSignature,
monero,
network::request_response::AliceToBob,
SwapAmounts,
};
@ -27,10 +25,9 @@ use rand::{CryptoRng, RngCore};
use std::{fmt, sync::Arc};
use tracing::info;
use xmr_btc::{
alice::State3,
alice::{State0, State3},
bitcoin::{TransactionBlockHeight, TxCancel, TxRefund, WatchForRawTransaction},
config::Config,
cross_curve_dleq,
monero::CreateWalletForOutput,
};
@ -44,9 +41,7 @@ impl<T> Rng for T where T: RngCore + CryptoRng + Send {}
pub enum AliceState {
Started {
amounts: SwapAmounts,
a: bitcoin::SecretKey,
s_a: cross_curve_dleq::Scalar,
v_a: monero::PrivateViewKey,
state0: State0,
},
Negotiated {
channel: ResponseChannel<AliceToBob>,
@ -109,11 +104,11 @@ impl fmt::Display for AliceState {
pub async fn swap(
state: AliceState,
swarm: Swarm,
swarm: EventLoopHandle,
bitcoin_wallet: Arc<crate::bitcoin::Wallet>,
monero_wallet: Arc<crate::monero::Wallet>,
config: Config,
) -> Result<(AliceState, Swarm)> {
) -> Result<(AliceState, EventLoopHandle)> {
run_until(
state,
is_complete,
@ -147,32 +142,18 @@ pub fn is_xmr_locked(state: &AliceState) -> bool {
pub async fn run_until(
state: AliceState,
is_target_state: fn(&AliceState) -> bool,
mut swarm: Swarm,
mut swarm: EventLoopHandle,
bitcoin_wallet: Arc<crate::bitcoin::Wallet>,
monero_wallet: Arc<crate::monero::Wallet>,
config: Config,
) -> Result<(AliceState, Swarm)> {
) -> Result<(AliceState, EventLoopHandle)> {
info!("Current state:{}", state);
if is_target_state(&state) {
Ok((state, swarm))
} else {
match state {
AliceState::Started {
amounts,
a,
s_a,
v_a,
} => {
let (channel, state3) = negotiate(
amounts,
a,
s_a,
v_a,
&mut swarm,
bitcoin_wallet.clone(),
config,
)
.await?;
AliceState::Started { amounts, state0 } => {
let (channel, state3) = negotiate(state0, amounts, &mut swarm, config).await?;
run_until(
AliceState::Negotiated {

View File

@ -16,6 +16,7 @@ use anyhow::Result;
use futures::{channel::mpsc, StreamExt};
use libp2p::Multiaddr;
use prettytable::{row, Table};
use rand::rngs::OsRng;
use std::{io, io::Write, process, sync::Arc};
use structopt::StructOpt;
use swap::{
@ -25,9 +26,10 @@ use swap::{
network::transport::{build, build_tor, SwapTransport},
recover::recover,
storage::Database,
Cmd, Rsp, SwapAmounts,
Cmd, Rsp, SwapAmounts, PUNISH_TIMELOCK, REFUND_TIMELOCK,
};
use tracing::info;
use xmr_btc::{alice::State0, cross_curve_dleq};
#[macro_use]
extern crate prettytable;
@ -50,7 +52,34 @@ async fn main() -> Result<()> {
} => {
info!("running swap node as Alice ...");
let behaviour = alice::Behaviour::default();
let bitcoin_wallet = bitcoin::Wallet::new("alice", bitcoind_url)
.await
.expect("failed to create bitcoin wallet");
let bitcoin_wallet = Arc::new(bitcoin_wallet);
let monero_wallet = Arc::new(monero::Wallet::new(monerod_url));
let rng = &mut OsRng;
let a = bitcoin::SecretKey::new_random(rng);
let s_a = cross_curve_dleq::Scalar::random(rng);
let v_a = xmr_btc::monero::PrivateViewKey::new_random(rng);
let redeem_address = bitcoin_wallet.as_ref().new_address().await?;
let punish_address = redeem_address.clone();
let state0 = State0::new(
a,
s_a,
v_a,
// todo: get from CLI args
bitcoin::Amount::from_sat(100),
// todo: get from CLI args
monero::Amount::from_piconero(1000000),
REFUND_TIMELOCK,
PUNISH_TIMELOCK,
redeem_address,
punish_address,
);
let behaviour = alice::Behaviour::new(state0);
let local_key_pair = behaviour.identity();
let (listen_addr, _ac, transport) = match tor_port {
@ -72,13 +101,6 @@ async fn main() -> Result<()> {
}
};
let bitcoin_wallet = bitcoin::Wallet::new("alice", bitcoind_url)
.await
.expect("failed to create bitcoin wallet");
let bitcoin_wallet = Arc::new(bitcoin_wallet);
let monero_wallet = Arc::new(monero::Wallet::new(monerod_url));
swap_as_alice(
bitcoin_wallet,
monero_wallet,

View File

@ -17,6 +17,7 @@ use tracing::{debug, info, warn};
use uuid::Uuid;
mod amounts;
pub mod event_loop;
mod execution;
mod message0;
mod message1;

237
swap/src/bob/event_loop.rs Normal file
View File

@ -0,0 +1,237 @@
use crate::{
bob::{Behaviour, OutEvent},
network::{transport::SwapTransport, TokioExecutor},
};
use anyhow::Result;
use futures::FutureExt;
use libp2p::{core::Multiaddr, PeerId};
use tokio::{
stream::StreamExt,
sync::mpsc::{Receiver, Sender},
};
use tracing::info;
use xmr_btc::{alice, bitcoin::EncryptedSignature, bob};
pub struct Channels<T> {
sender: Sender<T>,
receiver: Receiver<T>,
}
impl<T> Channels<T> {
pub fn new() -> Channels<T> {
let (sender, receiver) = tokio::sync::mpsc::channel(100);
Channels { sender, receiver }
}
}
impl<T> Default for Channels<T> {
fn default() -> Self {
Self::new()
}
}
pub struct EventLoopHandle {
pub msg0: Receiver<alice::Message0>,
pub msg1: Receiver<alice::Message1>,
pub msg2: Receiver<alice::Message2>,
pub request_amounts: Sender<(PeerId, ::bitcoin::Amount)>,
pub conn_established: Receiver<PeerId>,
pub dial_alice: Sender<Multiaddr>,
pub send_msg0: Sender<(PeerId, bob::Message0)>,
pub send_msg1: Sender<(PeerId, bob::Message1)>,
pub send_msg2: Sender<(PeerId, bob::Message2)>,
pub send_msg3: Sender<(PeerId, EncryptedSignature)>,
}
impl EventLoopHandle {
pub async fn recv_conn_established(&mut self) -> Result<PeerId> {
self.conn_established
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive connection established from Bob"))
}
pub async fn recv_message0(&mut self) -> Result<alice::Message0> {
self.msg0
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive message 0 from Bob"))
}
pub async fn recv_message1(&mut self) -> Result<alice::Message1> {
self.msg1
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed to receive message 1 from Bob"))
}
pub async fn recv_message2(&mut self) -> Result<alice::Message2> {
self.msg2
.recv()
.await
.ok_or_else(|| anyhow::Error::msg("Failed o receive message 2 from Bob"))
}
pub async fn dial_alice(&mut self, addr: Multiaddr) -> Result<()> {
info!("sending msg to ourselves to dial alice: {}", addr);
let _ = self.dial_alice.send(addr).await?;
Ok(())
}
pub async fn request_amounts(
&mut self,
peer_id: PeerId,
btc_amount: ::bitcoin::Amount,
) -> Result<()> {
let _ = self.request_amounts.send((peer_id, btc_amount)).await?;
Ok(())
}
pub async fn send_message0(&mut self, peer_id: PeerId, msg: bob::Message0) -> Result<()> {
let _ = self.send_msg0.send((peer_id, msg)).await?;
Ok(())
}
pub async fn send_message1(&mut self, peer_id: PeerId, msg: bob::Message1) -> Result<()> {
let _ = self.send_msg1.send((peer_id, msg)).await?;
Ok(())
}
pub async fn send_message2(&mut self, peer_id: PeerId, msg: bob::Message2) -> Result<()> {
let _ = self.send_msg2.send((peer_id, msg)).await?;
Ok(())
}
pub async fn send_message3(
&mut self,
peer_id: PeerId,
tx_redeem_encsig: EncryptedSignature,
) -> Result<()> {
let _ = self.send_msg3.send((peer_id, tx_redeem_encsig)).await?;
Ok(())
}
}
pub struct EventLoop {
pub swarm: libp2p::Swarm<Behaviour>,
pub msg0: Sender<alice::Message0>,
pub msg1: Sender<alice::Message1>,
pub msg2: Sender<alice::Message2>,
pub conn_established: Sender<PeerId>,
pub request_amounts: Receiver<(PeerId, ::bitcoin::Amount)>,
pub dial_alice: Receiver<Multiaddr>,
pub send_msg0: Receiver<(PeerId, bob::Message0)>,
pub send_msg1: Receiver<(PeerId, bob::Message1)>,
pub send_msg2: Receiver<(PeerId, bob::Message2)>,
pub send_msg3: Receiver<(PeerId, EncryptedSignature)>,
}
impl EventLoop {
pub fn new(transport: SwapTransport, behaviour: Behaviour) -> Result<(Self, EventLoopHandle)> {
let local_peer_id = behaviour.peer_id();
let swarm = libp2p::swarm::SwarmBuilder::new(transport, behaviour, local_peer_id)
.executor(Box::new(TokioExecutor {
handle: tokio::runtime::Handle::current(),
}))
.build();
let amounts = Channels::new();
let msg0 = Channels::new();
let msg1 = Channels::new();
let msg2 = Channels::new();
let conn_established = Channels::new();
let dial_alice = Channels::new();
let send_msg0 = Channels::new();
let send_msg1 = Channels::new();
let send_msg2 = Channels::new();
let send_msg3 = Channels::new();
let driver = EventLoop {
swarm,
request_amounts: amounts.receiver,
msg0: msg0.sender,
msg1: msg1.sender,
msg2: msg2.sender,
conn_established: conn_established.sender,
dial_alice: dial_alice.receiver,
send_msg0: send_msg0.receiver,
send_msg1: send_msg1.receiver,
send_msg2: send_msg2.receiver,
send_msg3: send_msg3.receiver,
};
let handle = EventLoopHandle {
request_amounts: amounts.sender,
msg0: msg0.receiver,
msg1: msg1.receiver,
msg2: msg2.receiver,
conn_established: conn_established.receiver,
dial_alice: dial_alice.sender,
send_msg0: send_msg0.sender,
send_msg1: send_msg1.sender,
send_msg2: send_msg2.sender,
send_msg3: send_msg3.sender,
};
Ok((driver, handle))
}
pub async fn run(mut self) {
loop {
tokio::select! {
swarm_event = self.swarm.next().fuse() => {
match swarm_event {
OutEvent::ConnectionEstablished(alice) => {
let _ = self.conn_established.send(alice).await;
}
OutEvent::Amounts(_amounts) => info!("Amounts received from Alice"),
OutEvent::Message0(msg) => {
let _ = self.msg0.send(msg).await;
}
OutEvent::Message1(msg) => {
let _ = self.msg1.send(msg).await;
}
OutEvent::Message2(msg) => {
let _ = self.msg2.send(msg).await;
}
OutEvent::Message3 => info!("Alice acknowledged message 3 received"),
}
},
addr = self.dial_alice.next().fuse() => {
if let Some(addr) = addr {
info!("dialing alice: {}", addr);
libp2p::Swarm::dial_addr(&mut self.swarm, addr).expect("Could not dial alice");
}
},
amounts = self.request_amounts.next().fuse() => {
if let Some((peer_id, btc_amount)) = amounts {
self.swarm.request_amounts(peer_id, btc_amount.as_sat());
}
},
msg0 = self.send_msg0.next().fuse() => {
if let Some((peer_id, msg)) = msg0 {
self.swarm.send_message0(peer_id, msg);
}
}
msg1 = self.send_msg1.next().fuse() => {
if let Some((peer_id, msg)) = msg1 {
self.swarm.send_message1(peer_id, msg);
}
},
msg2 = self.send_msg2.next().fuse() => {
if let Some((peer_id, msg)) = msg2 {
self.swarm.send_message2(peer_id, msg);
}
},
msg3 = self.send_msg3.next().fuse() => {
if let Some((peer_id, tx_redeem_encsig)) = msg3 {
self.swarm.send_message3(peer_id, tx_redeem_encsig);
}
}
}
}
}
}

View File

@ -1,7 +1,4 @@
use crate::{
bob::{OutEvent, Swarm},
SwapAmounts,
};
use crate::{bob::event_loop::EventLoopHandle, SwapAmounts};
use anyhow::Result;
use libp2p::core::Multiaddr;
use rand::{CryptoRng, RngCore};
@ -11,7 +8,7 @@ use xmr_btc::bob::State2;
pub async fn negotiate<R>(
state0: xmr_btc::bob::State0,
amounts: SwapAmounts,
swarm: &mut Swarm,
swarm: &mut EventLoopHandle,
addr: Multiaddr,
mut rng: R,
bitcoin_wallet: Arc<crate::bitcoin::Wallet>,
@ -19,34 +16,28 @@ pub async fn negotiate<R>(
where
R: RngCore + CryptoRng + Send,
{
libp2p::Swarm::dial_addr(swarm, addr)?;
tracing::trace!("Starting negotiate");
swarm.dial_alice(addr).await?;
let alice = match swarm.next().await {
OutEvent::ConnectionEstablished(alice) => alice,
other => panic!("unexpected event: {:?}", other),
};
let alice = swarm.recv_conn_established().await?;
swarm.request_amounts(alice.clone(), amounts.btc.as_sat());
swarm.request_amounts(alice.clone(), amounts.btc).await?;
// todo: see if we can remove
let (_btc, _xmr) = match swarm.next().await {
OutEvent::Amounts(amounts) => (amounts.btc, amounts.xmr),
other => panic!("unexpected event: {:?}", other),
};
swarm
.send_message0(alice.clone(), state0.next_message(&mut rng))
.await?;
let msg0 = swarm.recv_message0().await?;
let state1 = state0.receive(bitcoin_wallet.as_ref(), msg0).await?;
swarm.send_message0(alice.clone(), state0.next_message(&mut rng));
let state1 = match swarm.next().await {
OutEvent::Message0(msg) => state0.receive(bitcoin_wallet.as_ref(), msg).await?,
other => panic!("unexpected event: {:?}", other),
};
swarm
.send_message1(alice.clone(), state1.next_message())
.await?;
let msg1 = swarm.recv_message1().await?;
let state2 = state1.receive(msg1)?;
swarm.send_message1(alice.clone(), state1.next_message());
let state2 = match swarm.next().await {
OutEvent::Message1(msg) => state1.receive(msg)?,
other => panic!("unexpected event: {:?}", other),
};
swarm.send_message2(alice.clone(), state2.next_message());
swarm
.send_message2(alice.clone(), state2.next_message())
.await?;
Ok(state2)
}

View File

@ -1,5 +1,5 @@
use crate::{
bob::{execution::negotiate, OutEvent, Swarm},
bob::{event_loop::EventLoopHandle, execution::negotiate},
storage::Database,
SwapAmounts,
};
@ -8,7 +8,7 @@ use async_recursion::async_recursion;
use libp2p::{core::Multiaddr, PeerId};
use rand::{CryptoRng, RngCore};
use std::{fmt, sync::Arc};
use tracing::{debug, info};
use tracing::info;
use uuid::Uuid;
use xmr_btc::bob::{self};
@ -53,7 +53,7 @@ impl fmt::Display for BobState {
pub async fn swap<R>(
state: BobState,
swarm: Swarm,
swarm: EventLoopHandle,
db: Database,
bitcoin_wallet: Arc<crate::bitcoin::Wallet>,
monero_wallet: Arc<crate::monero::Wallet>,
@ -100,7 +100,7 @@ pub fn is_xmr_locked(state: &BobState) -> bool {
pub async fn run_until<R>(
state: BobState,
is_target_state: fn(&BobState) -> bool,
mut swarm: Swarm,
mut swarm: EventLoopHandle,
db: Database,
bitcoin_wallet: Arc<crate::bitcoin::Wallet>,
monero_wallet: Arc<crate::monero::Wallet>,
@ -162,14 +162,11 @@ where
// Watch for Alice to Lock Xmr or for t1 to elapse
BobState::BtcLocked(state3, alice_peer_id) => {
// todo: watch until t1, not indefinetely
let state4 = match swarm.next().await {
OutEvent::Message2(msg) => {
state3
.watch_for_lock_xmr(monero_wallet.as_ref(), msg)
.await?
}
other => panic!("unexpected event: {:?}", other),
};
let msg2 = swarm.recv_message2().await?;
let state4 = state3
.watch_for_lock_xmr(monero_wallet.as_ref(), msg2)
.await?;
run_until(
BobState::XmrLocked(state4, alice_peer_id),
is_target_state,
@ -190,18 +187,9 @@ where
// What if Alice fails to receive this? Should we always resend?
// todo: If we cannot dial Alice we should go to EncSigSent. Maybe dialing
// should happen in this arm?
swarm.send_message3(alice_peer_id.clone(), tx_redeem_encsig);
// Sadly we have to poll the swarm to get make sure the message is sent?
// FIXME: Having to wait for Alice's response here is a big problem, because
// we're stuck if she doesn't send her response back. I believe this is
// currently necessary, so we may have to rework this and/or how we use libp2p
match swarm.next().await {
OutEvent::Message3 => {
debug!("Got Message3 empty response");
}
other => panic!("unexpected event: {:?}", other),
};
swarm
.send_message3(alice_peer_id.clone(), tx_redeem_encsig)
.await?;
run_until(
BobState::EncSigSent(state, alice_peer_id),
@ -283,22 +271,10 @@ where
// }
Ok(BobState::BtcRefunded)
}
BobState::BtcRefunded => {
info!("btc refunded");
Ok(BobState::BtcRefunded)
}
BobState::Punished => {
info!("punished");
Ok(BobState::Punished)
}
BobState::SafelyAborted => {
info!("safely aborted");
Ok(BobState::SafelyAborted)
}
BobState::XmrRedeemed => {
info!("xmr redeemed");
Ok(BobState::XmrRedeemed)
}
BobState::BtcRefunded => Ok(BobState::BtcRefunded),
BobState::Punished => Ok(BobState::Punished),
BobState::SafelyAborted => Ok(BobState::SafelyAborted),
BobState::XmrRedeemed => Ok(BobState::XmrRedeemed),
}
}
}

View File

@ -14,8 +14,8 @@ pub mod state;
pub mod storage;
pub mod tor;
pub const REFUND_TIMELOCK: u32 = 10; // Relative timelock, this is number of blocks. TODO: What should it be?
pub const PUNISH_TIMELOCK: u32 = 10; // FIXME: What should this be?
pub const REFUND_TIMELOCK: u32 = 50; // Relative timelock, this is number of blocks. TODO: What should it be?
pub const PUNISH_TIMELOCK: u32 = 50; // FIXME: What should this be?
pub type Never = std::convert::Infallible;

View File

@ -10,16 +10,17 @@ use swap::{
};
use tempfile::tempdir;
use testcontainers::clients::Cli;
use tracing_subscriber::util::SubscriberInitExt as _;
use uuid::Uuid;
use xmr_btc::{bitcoin, config::Config, cross_curve_dleq};
use xmr_btc::{alice::State0, bitcoin, config::Config, cross_curve_dleq};
/// Run the following tests with RUST_MIN_STACK=10000000
#[tokio::test]
async fn happy_path() {
use tracing_subscriber::util::SubscriberInitExt as _;
let _guard = tracing_subscriber::fmt()
.with_env_filter("trace,hyper=warn")
.with_env_filter("swap=trace,xmr_btc=trace,monero_harness=info")
.with_ansi(false)
.set_default();
let cli = Cli::default();
@ -44,7 +45,14 @@ async fn happy_path() {
.parse()
.expect("failed to parse Alice's address");
let (alice_state, alice_swarm, alice_btc_wallet, alice_xmr_wallet, alice_peer_id) = init_alice(
let (
alice_state,
mut alice_swarm_driver,
alice_swarm_handle,
alice_btc_wallet,
alice_xmr_wallet,
alice_peer_id,
) = init_alice(
&bitcoind,
&monero,
btc_to_swap,
@ -55,29 +63,32 @@ async fn happy_path() {
)
.await;
let (bob_state, bob_swarm, bob_btc_wallet, bob_xmr_wallet, bob_db) = init_bob(
alice_multiaddr,
alice_peer_id,
&bitcoind,
&monero,
btc_to_swap,
btc_bob,
xmr_to_swap,
xmr_bob,
)
.await;
let (bob_state, bob_swarm_driver, bob_swarm_handle, bob_btc_wallet, bob_xmr_wallet, bob_db) =
init_bob(
alice_multiaddr,
alice_peer_id,
&bitcoind,
&monero,
btc_to_swap,
btc_bob,
xmr_to_swap,
xmr_bob,
)
.await;
let alice_swap = alice::swap::swap(
let alice_swap_fut = alice::swap::swap(
alice_state,
alice_swarm,
alice_swarm_handle,
alice_btc_wallet.clone(),
alice_xmr_wallet.clone(),
Config::regtest(),
);
let bob_swap = bob::swap::swap(
let _alice_swarm_fut = tokio::spawn(async move { alice_swarm_driver.run().await });
let bob_swap_fut = bob::swap::swap(
bob_state,
bob_swarm,
bob_swarm_handle,
bob_db,
bob_btc_wallet.clone(),
bob_xmr_wallet.clone(),
@ -85,7 +96,9 @@ async fn happy_path() {
Uuid::new_v4(),
);
try_join(alice_swap, bob_swap).await.unwrap();
let _bob_swarm_fut = tokio::spawn(async move { bob_swarm_driver.run().await });
try_join(alice_swap_fut, bob_swap_fut).await.unwrap();
let btc_alice_final = alice_btc_wallet.as_ref().balance().await.unwrap();
let btc_bob_final = bob_btc_wallet.as_ref().balance().await.unwrap();
@ -109,8 +122,10 @@ async fn happy_path() {
/// the encsig and fail to refund or redeem. Alice punishes.
#[tokio::test]
async fn alice_punishes_if_bob_never_acts_after_fund() {
use tracing_subscriber::util::SubscriberInitExt as _;
let _guard = tracing_subscriber::fmt()
.with_env_filter("trace,hyper=warn")
.with_env_filter("swap=info,xmr_btc=info")
.with_ansi(false)
.set_default();
let cli = Cli::default();
@ -134,7 +149,14 @@ async fn alice_punishes_if_bob_never_acts_after_fund() {
.parse()
.expect("failed to parse Alice's address");
let (alice_state, alice_swarm, alice_btc_wallet, alice_xmr_wallet, alice_peer_id) = init_alice(
let (
alice_state,
mut alice_swarm,
alice_swarm_handle,
alice_btc_wallet,
alice_xmr_wallet,
alice_peer_id,
) = init_alice(
&bitcoind,
&monero,
btc_to_swap,
@ -145,22 +167,23 @@ async fn alice_punishes_if_bob_never_acts_after_fund() {
)
.await;
let (bob_state, bob_swarm, bob_btc_wallet, bob_xmr_wallet, bob_db) = init_bob(
alice_multiaddr,
alice_peer_id,
&bitcoind,
&monero,
btc_to_swap,
bob_btc_starting_balance,
xmr_to_swap,
bob_xmr_starting_balance,
)
.await;
let (bob_state, bob_swarm_driver, bob_swarm_handle, bob_btc_wallet, bob_xmr_wallet, bob_db) =
init_bob(
alice_multiaddr,
alice_peer_id,
&bitcoind,
&monero,
btc_to_swap,
bob_btc_starting_balance,
xmr_to_swap,
bob_xmr_starting_balance,
)
.await;
let bob_xmr_locked_fut = bob::swap::run_until(
bob_state,
bob::swap::is_xmr_locked,
bob_swarm,
bob_swarm_handle,
bob_db,
bob_btc_wallet.clone(),
bob_xmr_wallet.clone(),
@ -168,14 +191,18 @@ async fn alice_punishes_if_bob_never_acts_after_fund() {
Uuid::new_v4(),
);
let _bob_swarm_fut = tokio::spawn(async move { bob_swarm_driver.run().await });
let alice_fut = alice::swap::swap(
alice_state,
alice_swarm,
alice_swarm_handle,
alice_btc_wallet.clone(),
alice_xmr_wallet.clone(),
Config::regtest(),
);
let _alice_swarm_fut = tokio::spawn(async move { alice_swarm.run().await });
// Wait until alice has locked xmr and bob h as locked btc
let ((alice_state, _), _bob_state) = try_join(alice_fut, bob_xmr_locked_fut).await.unwrap();
@ -192,10 +219,11 @@ async fn init_alice(
_btc_starting_balance: bitcoin::Amount,
xmr_to_swap: xmr_btc::monero::Amount,
xmr_starting_balance: xmr_btc::monero::Amount,
alice_multiaddr: Multiaddr,
listen: Multiaddr,
) -> (
AliceState,
alice::Swarm,
alice::event_loop::EventLoop,
alice::event_loop::EventLoopHandle,
Arc<swap::bitcoin::Wallet>,
Arc<swap::monero::Wallet>,
PeerId,
@ -220,27 +248,45 @@ async fn init_alice(
xmr: xmr_to_swap,
};
let alice_behaviour = alice::Behaviour::default();
let alice_peer_id = alice_behaviour.peer_id();
let alice_transport = build(alice_behaviour.identity()).unwrap();
let rng = &mut OsRng;
let alice_state = {
let (alice_state, alice_behaviour) = {
let rng = &mut OsRng;
let a = bitcoin::SecretKey::new_random(rng);
let s_a = cross_curve_dleq::Scalar::random(rng);
let v_a = xmr_btc::monero::PrivateViewKey::new_random(rng);
AliceState::Started {
amounts,
let redeem_address = alice_btc_wallet.as_ref().new_address().await.unwrap();
let punish_address = redeem_address.clone();
let state0 = State0::new(
a,
s_a,
v_a,
}
amounts.btc,
amounts.xmr,
REFUND_TIMELOCK,
PUNISH_TIMELOCK,
redeem_address,
punish_address,
);
// let msg0 = AliceToBob::Message0(self.state.next_message(&mut OsRng));
(
AliceState::Started {
amounts,
state0: state0.clone(),
},
alice::Behaviour::new(state0),
)
};
let alice_swarm = alice::new_swarm(alice_multiaddr, alice_transport, alice_behaviour).unwrap();
let alice_peer_id = alice_behaviour.peer_id();
let alice_transport = build(alice_behaviour.identity()).unwrap();
let (swarm_driver, handle) =
alice::event_loop::EventLoop::new(alice_transport, alice_behaviour, listen).unwrap();
(
alice_state,
alice_swarm,
swarm_driver,
handle,
alice_btc_wallet,
alice_xmr_wallet,
alice_peer_id,
@ -259,7 +305,8 @@ async fn init_bob(
xmr_stating_balance: xmr_btc::monero::Amount,
) -> (
BobState,
bob::Swarm,
bob::event_loop::EventLoop,
bob::event_loop::EventLoopHandle,
Arc<swap::bitcoin::Wallet>,
Arc<swap::monero::Wallet>,
Database,
@ -309,7 +356,16 @@ async fn init_bob(
peer_id: alice_peer_id,
addr: alice_multiaddr,
};
let bob_swarm = bob::new_swarm(bob_transport, bob_behaviour).unwrap();
(bob_state, bob_swarm, bob_btc_wallet, bob_xmr_wallet, bob_db)
let (swarm_driver, swarm_handle) =
bob::event_loop::EventLoop::new(bob_transport, bob_behaviour).unwrap();
(
bob_state,
swarm_driver,
swarm_handle,
bob_btc_wallet,
bob_xmr_wallet,
bob_db,
)
}