everywhere: support unix sockets

This patch adds support for listening on and connecting to unix sockets.

This requires having wrapper types for various tokio specific network
abstractions while also supporting things like serialization and
deserialization.

Unfortionately i was unable to find a published crate fulfilling these
requirements.

For this reason I've published a crate myself. Called `tokio-unix-tcp`,
it serves as a drop in replacement for Tokio's TCP and Unix network
types.

I plan to maintain this library outside the scope of this project as
well, in general the code should be simple and stable enough however
to not require maintainance going forward.

As i said this crate aims to support the requirement mentioned above.
In addition to this it also strives to be more correct about handling
the different types of unix sockets, which the libraries i reviewed
were weak at. A list of these crates can be found in the crate README
under "Related work".

---

The changes to netapp can be summarized as the following:

- `std::net::SocketAddr` has been replaced by
  `tokio_unix_tcp::NamedSocketAddr` in most places. This enum encapsulates
  a IP address and port as well as a path in its variants and describes
  a concrete socket address netapp can bind or connect to.

- In some places `tokio_unix_tcp::SocketAddr` is used instead of
  `tokio_unix_tcp::NamedSocketAddr` as mentioned above. This is due to
  the way unix sockets work:

  The remote peer of a client from the perspective of a server is not
  a concrete path but `unnamed`. They just share a file descriptor
  for the actual communication channel. The local address of the server
  is the actual file system path the server is listening on.

  In some cases netapp might be configured to connect to another peer
  using a unix socket and to not send a reachable IP address and port
  or unix socket path using the `HelloMessage`.

  As per the above (the client's remote address will be `unnamed`),
  we have no way of connecting back to that peer. This will currently
  cause the connection to be aborted by the server.

- Listening on Unix sockets requires some additional handling like
  removing a previous file at the bind path and setting a correct
  mode (defaulting to `0o222` currently). This is handled by
  `tokio_unix_tcp`.

---

I've tested these changes by including them in garage and running basic
administration commands against a node and by running the unit tests here.

Basalt peering is currently lacking a proper cost calculation for unix
sockets - I'm sadly not familiar with this code.
This commit is contained in:
networkException 2023-10-20 02:38:41 +02:00
parent e6976dea5a
commit 47a470b281
Signed by: networkException
GPG key ID: E3877443AE684391
9 changed files with 308 additions and 152 deletions

View file

@ -9,10 +9,12 @@ use serde::{Deserialize, Serialize};
use structopt::StructOpt; use structopt::StructOpt;
use sodiumoxide::crypto::auth; use sodiumoxide::crypto::auth;
use sodiumoxide::crypto::sign::ed25519; use sodiumoxide::crypto::sign::{ed25519, PublicKey};
use tokio::sync::watch; use tokio::sync::watch;
use tokio_unix_tcp::NamedSocketAddr;
use netapp::endpoint::*; use netapp::endpoint::*;
use netapp::message::*; use netapp::message::*;
use netapp::peering::basalt::*; use netapp::peering::basalt::*;
@ -96,9 +98,13 @@ async fn main() {
let netapp = NetApp::new(0u64, netid, privkey); let netapp = NetApp::new(0u64, netid, privkey);
let mut bootstrap_peers = vec![]; let mut bootstrap_peers: Vec<(PublicKey, NamedSocketAddr)> = vec![];
for peer in opt.bootstrap_peers.iter() { for peer in opt.bootstrap_peers.iter() {
bootstrap_peers.push(parse_peer_addr(peer).expect("Invalid peer address")); bootstrap_peers.push(
parse_peer_addr(peer)
.map(|(node_id, socket_bind_addr)| (node_id, socket_bind_addr.into()))
.expect("Invalid peer address"),
);
} }
let basalt_params = BasaltParams { let basalt_params = BasaltParams {

View file

@ -1,5 +1,5 @@
use std::io::Write; use std::io::Write;
use std::net::SocketAddr; use std::net;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
@ -9,10 +9,13 @@ use futures::{stream, StreamExt};
use log::*; use log::*;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
use structopt::StructOpt; use structopt::StructOpt;
use tokio::sync::watch; use tokio::sync::watch;
use tokio_unix_tcp::NamedSocketAddr;
use sodiumoxide::crypto::auth; use sodiumoxide::crypto::auth;
use sodiumoxide::crypto::sign::ed25519; use sodiumoxide::crypto::sign::{ed25519, PublicKey};
use netapp::endpoint::*; use netapp::endpoint::*;
use netapp::message::*; use netapp::message::*;
@ -75,21 +78,25 @@ async fn main() {
info!("Node public key: {}", hex::encode(privkey.public_key())); info!("Node public key: {}", hex::encode(privkey.public_key()));
let public_addr = opt.public_addr.map(|x| x.parse().unwrap()); let public_addr = opt.public_addr.map(|x| x.parse().unwrap());
let listen_addr: SocketAddr = opt.listen_addr.parse().unwrap(); let listen_addr: net::SocketAddr = opt.listen_addr.parse().unwrap();
info!("Node public address: {:?}", public_addr); info!("Node public address: {:?}", public_addr);
info!("Node listen address: {}", listen_addr); info!("Node listen address: {}", listen_addr);
let netapp = NetApp::new(0u64, netid.clone(), privkey.clone()); let netapp = NetApp::new(0u64, netid.clone(), privkey.clone());
let mut bootstrap_peers = vec![]; let mut bootstrap_peers: Vec<(PublicKey, NamedSocketAddr)> = vec![];
for peer in opt.bootstrap_peers.iter() { for peer in opt.bootstrap_peers.iter() {
bootstrap_peers.push(parse_peer_addr(peer).expect("Invalid peer address")); bootstrap_peers.push(
parse_peer_addr(peer)
.map(|(node_id, socket_bind_addr)| (node_id, socket_bind_addr.into()))
.expect("Invalid peer address"),
);
} }
let peering = FullMeshPeeringStrategy::new( let peering = FullMeshPeeringStrategy::new(
netapp.clone(), netapp.clone(),
bootstrap_peers, bootstrap_peers,
public_addr.map(|a| SocketAddr::new(a, listen_addr.port())), public_addr.map(|a| net::SocketAddr::new(a, listen_addr.port()).into()),
); );
info!("Add more peers to this mesh by running: fullmesh -n {} -l 127.0.0.1:$((1000 + $RANDOM)) -b {}@{}", info!("Add more peers to this mesh by running: fullmesh -n {} -l 127.0.0.1:$((1000 + $RANDOM)) -b {}@{}",
@ -108,7 +115,7 @@ async fn main() {
tokio::join!( tokio::join!(
example.exchange_loop(watch_cancel.clone()), example.exchange_loop(watch_cancel.clone()),
netapp.listen(listen_addr, public_addr, watch_cancel.clone()), netapp.listen(listen_addr.into(), public_addr, watch_cancel.clone()),
peering.run(watch_cancel), peering.run(watch_cancel),
); );
} }

View file

@ -1,5 +1,4 @@
use std::collections::HashMap; use std::collections::HashMap;
use std::net::SocketAddr;
use std::pin::Pin; use std::pin::Pin;
use std::sync::atomic::{self, AtomicU32}; use std::sync::atomic::{self, AtomicU32};
use std::sync::{Arc, Mutex}; use std::sync::{Arc, Mutex};
@ -13,11 +12,13 @@ use log::{debug, error, trace};
use futures::io::AsyncReadExt; use futures::io::AsyncReadExt;
use futures::Stream; use futures::Stream;
use kuska_handshake::async_std::{handshake_client, BoxStream}; use kuska_handshake::async_std::{handshake_client, BoxStream};
use tokio::net::TcpStream;
use tokio::select; use tokio::select;
use tokio::sync::{mpsc, oneshot, watch}; use tokio::sync::{mpsc, oneshot, watch};
use tokio_util::compat::*; use tokio_util::compat::*;
use tokio_unix_tcp::SocketAddr;
use tokio_unix_tcp::Stream as SocketStream;
#[cfg(feature = "telemetry")] #[cfg(feature = "telemetry")]
use opentelemetry::{ use opentelemetry::{
trace::{FutureExt, Span, SpanKind, TraceContextExt, Tracer}, trace::{FutureExt, Span, SpanKind, TraceContextExt, Tracer},
@ -47,7 +48,7 @@ pub(crate) struct ClientConn {
impl ClientConn { impl ClientConn {
pub(crate) async fn init( pub(crate) async fn init(
netapp: Arc<NetApp>, netapp: Arc<NetApp>,
socket: TcpStream, socket: SocketStream,
peer_id: NodeID, peer_id: NodeID,
) -> Result<(), Error> { ) -> Result<(), Error> {
let remote_addr = socket.peer_addr()?; let remote_addr = socket.peer_addr()?;

View file

@ -1,5 +1,6 @@
use std::collections::HashMap; use std::collections::HashMap;
use std::net::{IpAddr, SocketAddr}; use std::net::{self, IpAddr};
use std::path::PathBuf;
use std::sync::{Arc, RwLock}; use std::sync::{Arc, RwLock};
use log::{debug, error, info, trace, warn}; use log::{debug, error, info, trace, warn};
@ -13,10 +14,11 @@ use sodiumoxide::crypto::sign::ed25519;
use futures::stream::futures_unordered::FuturesUnordered; use futures::stream::futures_unordered::FuturesUnordered;
use futures::stream::StreamExt; use futures::stream::StreamExt;
use tokio::net::{TcpListener, TcpStream};
use tokio::select; use tokio::select;
use tokio::sync::{mpsc, watch}; use tokio::sync::{mpsc, watch};
use tokio_unix_tcp::{Listener, NamedSocketAddr, SocketAddr, Stream, UnixSocketAddr};
use crate::client::*; use crate::client::*;
use crate::endpoint::*; use crate::endpoint::*;
use crate::error::*; use crate::error::*;
@ -39,16 +41,19 @@ pub(crate) type VersionTag = [u8; 16];
pub(crate) const NETAPP_VERSION_TAG: u64 = 0x6e65746170700005; // netapp 0x0005 pub(crate) const NETAPP_VERSION_TAG: u64 = 0x6e65746170700005; // netapp 0x0005
#[derive(Serialize, Deserialize, Debug)] #[derive(Serialize, Deserialize, Debug)]
pub(crate) struct HelloMessage { pub(crate) enum HelloMessage {
pub server_addr: Option<IpAddr>, Tcp {
pub server_port: u16, server_addr: Option<IpAddr>,
server_port: u16,
},
Unix(PathBuf),
} }
impl Message for HelloMessage { impl Message for HelloMessage {
type Response = (); type Response = ();
} }
type OnConnectHandler = Box<dyn Fn(NodeID, SocketAddr, bool) + Send + Sync>; type OnConnectHandler = Box<dyn Fn(NodeID, NamedSocketAddr, bool) + Send + Sync>;
type OnDisconnectHandler = Box<dyn Fn(NodeID, bool) + Send + Sync>; type OnDisconnectHandler = Box<dyn Fn(NodeID, bool) + Send + Sync>;
/// NetApp is the main class that handles incoming and outgoing connections. /// NetApp is the main class that handles incoming and outgoing connections.
@ -81,9 +86,12 @@ pub struct NetApp {
on_disconnected_handler: ArcSwapOption<OnDisconnectHandler>, on_disconnected_handler: ArcSwapOption<OnDisconnectHandler>,
} }
struct ListenParams { enum ListenParams {
listen_addr: SocketAddr, Tcp {
public_addr: Option<IpAddr>, listen_addr: net::SocketAddr,
public_addr: Option<IpAddr>,
},
Unix(PathBuf),
} }
impl NetApp { impl NetApp {
@ -129,7 +137,7 @@ impl NetApp {
/// as the peering strategy will need to set this itself. /// as the peering strategy will need to set this itself.
pub fn on_connected<F>(&self, handler: F) pub fn on_connected<F>(&self, handler: F)
where where
F: Fn(NodeID, SocketAddr, bool) + Sized + Send + Sync + 'static, F: Fn(NodeID, NamedSocketAddr, bool) + Sized + Send + Sync + 'static,
{ {
self.on_connected_handler self.on_connected_handler
.store(Some(Arc::new(Box::new(handler)))); .store(Some(Arc::new(Box::new(handler))));
@ -179,14 +187,18 @@ impl NetApp {
/// If this is not called, the NetApp instance remains a passive client. /// If this is not called, the NetApp instance remains a passive client.
pub async fn listen( pub async fn listen(
self: Arc<Self>, self: Arc<Self>,
listen_addr: SocketAddr, listen_addr: NamedSocketAddr,
public_addr: Option<IpAddr>, public_addr: Option<IpAddr>,
mut must_exit: watch::Receiver<bool>, mut must_exit: watch::Receiver<bool>,
) { ) {
let listen_params = ListenParams { let listen_params = match &listen_addr {
listen_addr, NamedSocketAddr::Inet(addr) => ListenParams::Tcp {
public_addr, listen_addr: *addr,
public_addr,
},
NamedSocketAddr::Unix(path) => ListenParams::Unix(path.to_path_buf()),
}; };
if self if self
.listen_params .listen_params
.swap(Some(Arc::new(listen_params))) .swap(Some(Arc::new(listen_params)))
@ -195,7 +207,9 @@ impl NetApp {
error!("Trying to listen on NetApp but we're already listening!"); error!("Trying to listen on NetApp but we're already listening!");
} }
let listener = TcpListener::bind(listen_addr).await.unwrap(); let listener = Listener::bind_and_prepare_unix(&listen_addr, true, None)
.await
.unwrap();
info!("Listening on {}", listen_addr); info!("Listening on {}", listen_addr);
let (conn_in, mut conn_out) = mpsc::unbounded_channel(); let (conn_in, mut conn_out) = mpsc::unbounded_channel();
@ -276,18 +290,22 @@ impl NetApp {
self.on_disconnected_handler.store(None); self.on_disconnected_handler.store(None);
} }
/// Attempt to connect to a peer, given by its ip:port and its public key. /// Attempt to connect to a peer, given by its ip:port / unix socket path
/// The public key will be checked during the secret handshake process. /// and its public key. The public key will be checked during the secret
/// This function returns once the connection has been established and a /// handshake process. This function returns once the connection has been
/// successfull handshake was made. At this point we can send messages to /// established and a successful handshake was made. At this point we can
/// the other node with `Netapp::request` /// send messages to the other node with `Netapp::request`
pub async fn try_connect(self: Arc<Self>, ip: SocketAddr, id: NodeID) -> Result<(), Error> { pub async fn try_connect(
self: Arc<Self>,
addr: NamedSocketAddr,
id: NodeID,
) -> Result<(), Error> {
// Don't connect to ourself, we don't care // Don't connect to ourself, we don't care
// but pretend we did // but pretend we did
if id == self.id { if id == self.id {
tokio::spawn(async move { tokio::spawn(async move {
if let Some(h) = self.on_connected_handler.load().as_ref() { if let Some(h) = self.on_connected_handler.load().as_ref() {
h(id, ip, false); h(id, addr, false);
} }
}); });
return Ok(()); return Ok(());
@ -298,8 +316,8 @@ impl NetApp {
return Ok(()); return Ok(());
} }
let socket = TcpStream::connect(ip).await?; let socket = Stream::connect(&addr).await?;
info!("Connected to {}, negotiating handshake...", ip); info!("Connected to {}, negotiating handshake...", addr);
ClientConn::init(self, socket, id).await?; ClientConn::init(self, socket, id).await?;
Ok(()) Ok(())
} }
@ -392,23 +410,29 @@ impl NetApp {
} }
if let Some(h) = self.on_connected_handler.load().as_ref() { if let Some(h) = self.on_connected_handler.load().as_ref() {
h(conn.peer_id, conn.remote_addr, false); h(
conn.peer_id,
conn.remote_addr.clone().to_named_socket_addr().unwrap(),
false,
);
} }
if let Some(lp) = self.listen_params.load_full() { if let Some(lp) = self.listen_params.load_full() {
let server_addr = lp.public_addr; let hello_message: HelloMessage = match lp.as_ref() {
let server_port = lp.listen_addr.port(); ListenParams::Tcp {
listen_addr,
public_addr,
} => HelloMessage::Tcp {
server_addr: *public_addr,
server_port: listen_addr.port(),
},
ListenParams::Unix(listen_path) => HelloMessage::Unix(listen_path.clone()),
};
let hello_endpoint = self.hello_endpoint.load_full().unwrap(); let hello_endpoint = self.hello_endpoint.load_full().unwrap();
tokio::spawn(async move { tokio::spawn(async move {
hello_endpoint hello_endpoint
.call( .call(&conn.peer_id, hello_message, PRIO_NORMAL)
&conn.peer_id,
HelloMessage {
server_addr,
server_port,
},
PRIO_NORMAL,
)
.await .await
.map(|_| ()) .map(|_| ())
.log_err("Sending hello message"); .log_err("Sending hello message");
@ -443,9 +467,64 @@ impl EndpointHandler<HelloMessage> for NetApp {
debug!("Hello from {:?}: {:?}", hex::encode(&from[..8]), msg); debug!("Hello from {:?}: {:?}", hex::encode(&from[..8]), msg);
if let Some(h) = self.on_connected_handler.load().as_ref() { if let Some(h) = self.on_connected_handler.load().as_ref() {
if let Some(c) = self.server_conns.read().unwrap().get(&from) { if let Some(c) = self.server_conns.read().unwrap().get(&from) {
let remote_ip = msg.server_addr.unwrap_or_else(|| c.remote_addr.ip()); let remote_addr = match (msg, c.remote_addr.clone()) {
let remote_addr = SocketAddr::new(remote_ip, msg.server_port); (
h(from, remote_addr, true); HelloMessage::Tcp {
server_addr,
server_port,
},
SocketAddr::Inet(remote_addr),
) => {
let remote_ip = server_addr.unwrap_or_else(|| remote_addr.ip());
Some(NamedSocketAddr::Inet(net::SocketAddr::new(
remote_ip,
*server_port,
)))
}
(
HelloMessage::Tcp {
server_addr: Some(server_addr),
server_port,
},
SocketAddr::Unix(_),
) => Some(NamedSocketAddr::Inet(net::SocketAddr::new(
*server_addr,
*server_port,
))),
(
HelloMessage::Tcp {
server_addr: None,
server_port: _,
},
SocketAddr::Unix(UnixSocketAddr::Pathname(pathname)),
) => Some(NamedSocketAddr::Unix(pathname)),
(
HelloMessage::Tcp {
server_addr: None,
server_port: _,
},
SocketAddr::Unix(UnixSocketAddr::AbstractOrUnnamed),
) => {
error!("Remote client connected to us via a unix socket and is listening on TCP without a public address specified.");
error!("Their peer address is an unnamed unix socket, as such we have no way of connecting back to them. Disconnecting.");
None
}
(HelloMessage::Unix(server_path), _) => {
Some(NamedSocketAddr::Unix(server_path.clone()))
}
};
if let Some(remote_addr) = remote_addr {
h(from, remote_addr, true);
} else {
self.disconnect(&from);
}
} }
} }
} }

View file

@ -1,5 +1,5 @@
use std::collections::HashSet; use std::collections::HashSet;
use std::net::SocketAddr; use std::net;
use std::sync::{Arc, RwLock}; use std::sync::{Arc, RwLock};
use std::time::Duration; use std::time::Duration;
@ -13,6 +13,8 @@ use sodiumoxide::crypto::hash;
use tokio::sync::watch; use tokio::sync::watch;
use tokio_unix_tcp::NamedSocketAddr;
use crate::endpoint::*; use crate::endpoint::*;
use crate::message::*; use crate::message::*;
use crate::netapp::*; use crate::netapp::*;
@ -40,10 +42,10 @@ impl Message for PushMessage {
type Seed = [u8; 32]; type Seed = [u8; 32];
#[derive(Hash, Clone, Copy, Debug, PartialOrd, PartialEq, Eq, Serialize, Deserialize)] #[derive(Hash, Clone, Debug, PartialOrd, PartialEq, Eq, Serialize, Deserialize)]
struct Peer { struct Peer {
id: NodeID, id: NodeID,
addr: SocketAddr, addr: NamedSocketAddr,
} }
type Cost = [u8; 40]; type Cost = [u8; 40];
@ -57,24 +59,28 @@ impl Peer {
let mut cost = [0u8; 40]; let mut cost = [0u8; 40];
match self.addr { match self.addr {
SocketAddr::V4(v4addr) => { NamedSocketAddr::Inet(addr) => match addr {
let v4ip = v4addr.ip().octets(); net::SocketAddr::V4(v4addr) => {
let v4ip = v4addr.ip().octets();
for i in 0..4 { for i in 0..4 {
let mut h = hasher; let mut h = hasher;
h.update(&v4ip[..i + 1]); h.update(&v4ip[..i + 1]);
cost[i * 8..(i + 1) * 8].copy_from_slice(&h.finalize()[..8]); cost[i * 8..(i + 1) * 8].copy_from_slice(&h.finalize()[..8]);
}
} }
} net::SocketAddr::V6(v6addr) => {
SocketAddr::V6(v6addr) => { let v6ip = v6addr.ip().octets();
let v6ip = v6addr.ip().octets();
for i in 0..4 { for i in 0..4 {
let mut h = hasher; let mut h = hasher;
h.update(&v6ip[..i + 2]); h.update(&v6ip[..i + 2]);
cost[i * 8..(i + 1) * 8].copy_from_slice(&h.finalize()[..8]); cost[i * 8..(i + 1) * 8].copy_from_slice(&h.finalize()[..8]);
}
} }
} },
// FIXME: What should the cost calculation be here?
NamedSocketAddr::Unix(_) => {}
} }
{ {
@ -94,7 +100,10 @@ struct BasaltSlot {
impl BasaltSlot { impl BasaltSlot {
fn cost(&self) -> Cost { fn cost(&self) -> Cost {
self.peer.map(|p| p.cost(&self.seed)).unwrap_or(MAX_COST) self.peer
.clone()
.map(|p| p.cost(&self.seed))
.unwrap_or(MAX_COST)
} }
} }
@ -117,7 +126,7 @@ impl BasaltView {
fn current_peers(&self) -> HashSet<Peer> { fn current_peers(&self) -> HashSet<Peer> {
self.slots self.slots
.iter() .iter()
.filter_map(|s| s.peer) .filter_map(|s| s.peer.clone())
.collect::<HashSet<_>>() .collect::<HashSet<_>>()
} }
fn current_peers_vec(&self) -> Vec<Peer> { fn current_peers_vec(&self) -> Vec<Peer> {
@ -139,7 +148,7 @@ impl BasaltView {
let mut rng = thread_rng(); let mut rng = thread_rng();
for _i in 0..count { for _i in 0..count {
let idx = rng.gen_range(0..possibles.len()); let idx = rng.gen_range(0..possibles.len());
ret.push(self.slots[possibles[idx]].peer.unwrap()); ret.push(self.slots[possibles[idx]].peer.clone().unwrap());
} }
ret ret
} }
@ -158,7 +167,7 @@ impl BasaltView {
peer.addr, peer.addr,
hex::encode(peer_cost) hex::encode(peer_cost)
); );
self.slots[i].peer = Some(*peer); self.slots[i].peer = Some(peer.clone());
slot_cost = peer_cost; slot_cost = peer_cost;
} }
} }
@ -172,7 +181,7 @@ impl BasaltView {
fn disconnected(&mut self, id: NodeID) { fn disconnected(&mut self, id: NodeID) {
let mut cleared_slots = vec![]; let mut cleared_slots = vec![];
for i in 0..self.slots.len() { for i in 0..self.slots.len() {
if let Some(p) = self.slots[i].peer { if let Some(p) = self.slots[i].peer.clone() {
if p.id == id { if p.id == id {
self.slots[i].peer = None; self.slots[i].peer = None;
cleared_slots.push(i); cleared_slots.push(i);
@ -204,7 +213,7 @@ impl BasaltView {
let peer_cost = peer.cost(&self.slots[i].seed); let peer_cost = peer.cost(&self.slots[i].seed);
if peer_cost < min_cost { if peer_cost < min_cost {
min_cost = peer_cost; min_cost = peer_cost;
min_peer = Some(*peer); min_peer = Some(peer.clone());
} }
} }
if let Some(p) = min_peer { if let Some(p) = min_peer {
@ -251,14 +260,14 @@ pub struct Basalt {
impl Basalt { impl Basalt {
pub fn new( pub fn new(
netapp: Arc<NetApp>, netapp: Arc<NetApp>,
bootstrap_list: Vec<(NodeID, SocketAddr)>, bootstrap_list: Vec<(NodeID, NamedSocketAddr)>,
param: BasaltParams, param: BasaltParams,
) -> Arc<Self> { ) -> Arc<Self> {
let bootstrap_peers = bootstrap_list let bootstrap_peers = bootstrap_list
.iter() .iter()
.map(|(id, addr)| Peer { .map(|(id, addr)| Peer {
id: *id, id: *id,
addr: *addr, addr: addr.clone(),
}) })
.collect::<Vec<_>>(); .collect::<Vec<_>>();
@ -280,9 +289,11 @@ impl Basalt {
basalt.push_endpoint.set_handler(basalt.clone()); basalt.push_endpoint.set_handler(basalt.clone());
let basalt2 = basalt.clone(); let basalt2 = basalt.clone();
netapp.on_connected(move |id: NodeID, addr: SocketAddr, is_incoming: bool| { netapp.on_connected(
basalt2.on_connected(id, addr, is_incoming); move |id: NodeID, addr: NamedSocketAddr, is_incoming: bool| {
}); basalt2.on_connected(id, addr, is_incoming);
},
);
let basalt2 = basalt.clone(); let basalt2 = basalt.clone();
netapp.on_disconnected(move |id: NodeID, is_incoming: bool| { netapp.on_disconnected(move |id: NodeID, is_incoming: bool| {
@ -307,7 +318,7 @@ impl Basalt {
pub async fn run(self: Arc<Self>, must_exit: watch::Receiver<bool>) { pub async fn run(self: Arc<Self>, must_exit: watch::Receiver<bool>) {
for peer in self.bootstrap_peers.iter() { for peer in self.bootstrap_peers.iter() {
tokio::spawn(self.clone().try_connect(*peer)); tokio::spawn(self.clone().try_connect(peer.clone()));
} }
tokio::join!( tokio::join!(
@ -386,7 +397,7 @@ impl Basalt {
let mut to_retry_maybe = self.bootstrap_peers.clone(); let mut to_retry_maybe = self.bootstrap_peers.clone();
for (peer, _) in self.backlog.read().unwrap().iter() { for (peer, _) in self.backlog.read().unwrap().iter() {
if !self.bootstrap_peers.contains(peer) { if !self.bootstrap_peers.contains(peer) {
to_retry_maybe.push(*peer); to_retry_maybe.push(peer.clone());
} }
} }
self.handle_peer_list(&to_retry_maybe[..]); self.handle_peer_list(&to_retry_maybe[..]);
@ -397,7 +408,7 @@ impl Basalt {
let to_connect = self.view.read().unwrap().should_try_list(peers); let to_connect = self.view.read().unwrap().should_try_list(peers);
for peer in to_connect.iter() { for peer in to_connect.iter() {
tokio::spawn(self.clone().try_connect(*peer)); tokio::spawn(self.clone().try_connect(peer.clone()));
} }
} }
@ -406,16 +417,20 @@ impl Basalt {
let view = self.view.read().unwrap(); let view = self.view.read().unwrap();
let mut attempts = self.current_attempts.write().unwrap(); let mut attempts = self.current_attempts.write().unwrap();
if view.slots.iter().any(|x| x.peer == Some(peer)) { if view.slots.iter().any(|x| x.peer == Some(peer.clone())) {
return; return;
} }
if attempts.contains(&peer) { if attempts.contains(&peer) {
return; return;
} }
attempts.insert(peer); attempts.insert(peer.clone());
} }
let res = self.netapp.clone().try_connect(peer.addr, peer.id).await; let res = self
.netapp
.clone()
.try_connect(peer.addr.clone(), peer.id)
.await;
trace!("Connection attempt to {}: {:?}", peer.addr, res); trace!("Connection attempt to {}: {:?}", peer.addr, res);
self.current_attempts.write().unwrap().remove(&peer); self.current_attempts.write().unwrap().remove(&peer);
@ -425,7 +440,7 @@ impl Basalt {
} }
} }
fn on_connected(self: &Arc<Self>, id: NodeID, addr: SocketAddr, is_incoming: bool) { fn on_connected(self: &Arc<Self>, id: NodeID, addr: NamedSocketAddr, is_incoming: bool) {
if is_incoming { if is_incoming {
self.handle_peer_list(&[Peer { id, addr }][..]); self.handle_peer_list(&[Peer { id, addr }][..]);
} else { } else {
@ -434,7 +449,7 @@ impl Basalt {
let mut backlog = self.backlog.write().unwrap(); let mut backlog = self.backlog.write().unwrap();
if backlog.get(&peer).is_none() { if backlog.get(&peer).is_none() {
backlog.put(peer, ()); backlog.put(peer.clone(), ());
} }
drop(backlog); drop(backlog);

View file

@ -1,5 +1,4 @@
use std::collections::{HashMap, VecDeque}; use std::collections::{HashMap, VecDeque};
use std::net::SocketAddr;
use std::sync::atomic::{self, AtomicU64}; use std::sync::atomic::{self, AtomicU64};
use std::sync::{Arc, RwLock}; use std::sync::{Arc, RwLock};
use std::time::{Duration, Instant}; use std::time::{Duration, Instant};
@ -12,6 +11,8 @@ use serde::{Deserialize, Serialize};
use tokio::select; use tokio::select;
use tokio::sync::watch; use tokio::sync::watch;
use tokio_unix_tcp::NamedSocketAddr;
use sodiumoxide::crypto::hash; use sodiumoxide::crypto::hash;
use crate::endpoint::*; use crate::endpoint::*;
@ -43,7 +44,7 @@ impl Message for PingMessage {
#[derive(Serialize, Deserialize)] #[derive(Serialize, Deserialize)]
struct PeerListMessage { struct PeerListMessage {
pub list: Vec<(NodeID, SocketAddr)>, pub list: Vec<(NodeID, NamedSocketAddr)>,
} }
impl Message for PeerListMessage { impl Message for PeerListMessage {
@ -57,9 +58,9 @@ struct PeerInfoInternal {
// addr is the currently connected address, // addr is the currently connected address,
// or the last address we were connected to, // or the last address we were connected to,
// or an arbitrary address some other peer gave us // or an arbitrary address some other peer gave us
addr: SocketAddr, addr: NamedSocketAddr,
// all_addrs contains all of the addresses everyone gave us // all_addrs contains all of the addresses everyone gave us
all_addrs: Vec<SocketAddr>, all_addrs: Vec<NamedSocketAddr>,
state: PeerConnState, state: PeerConnState,
last_send_ping: Option<Instant>, last_send_ping: Option<Instant>,
@ -69,9 +70,9 @@ struct PeerInfoInternal {
} }
impl PeerInfoInternal { impl PeerInfoInternal {
fn new(addr: SocketAddr, state: PeerConnState) -> Self { fn new(addr: NamedSocketAddr, state: PeerConnState) -> Self {
Self { Self {
addr, addr: addr.clone(),
all_addrs: vec![addr], all_addrs: vec![addr],
state, state,
last_send_ping: None, last_send_ping: None,
@ -83,12 +84,12 @@ impl PeerInfoInternal {
} }
/// Information that the full mesh peering strategy can return about the peers it knows of /// Information that the full mesh peering strategy can return about the peers it knows of
#[derive(Copy, Clone, Debug)] #[derive(Clone, Debug)]
pub struct PeerInfo { pub struct PeerInfo {
/// The node's identifier (its public key) /// The node's identifier (its public key)
pub id: NodeID, pub id: NodeID,
/// The node's network address /// The node's network address
pub addr: SocketAddr, pub addr: NamedSocketAddr,
/// The current status of our connection to this node /// The current status of our connection to this node
pub state: PeerConnState, pub state: PeerConnState,
/// The last time at which the node was seen /// The last time at which the node was seen
@ -153,11 +154,11 @@ impl KnownHosts {
fn update_hash(&mut self) { fn update_hash(&mut self) {
self.hash = Self::calculate_hash(&self.list); self.hash = Self::calculate_hash(&self.list);
} }
fn map_into_vec(input: &HashMap<NodeID, PeerInfoInternal>) -> Vec<(NodeID, SocketAddr)> { fn map_into_vec(input: &HashMap<NodeID, PeerInfoInternal>) -> Vec<(NodeID, NamedSocketAddr)> {
let mut list = Vec::with_capacity(input.len()); let mut list = Vec::with_capacity(input.len());
for (id, peer) in input.iter() { for (id, peer) in input.iter() {
if peer.state == PeerConnState::Connected || peer.state == PeerConnState::Ourself { if peer.state == PeerConnState::Connected || peer.state == PeerConnState::Ourself {
list.push((*id, peer.addr)); list.push((*id, peer.addr.clone()));
} }
} }
list list
@ -196,8 +197,8 @@ impl FullMeshPeeringStrategy {
/// to all of the nodes specified in the bootstrap list. /// to all of the nodes specified in the bootstrap list.
pub fn new( pub fn new(
netapp: Arc<NetApp>, netapp: Arc<NetApp>,
bootstrap_list: Vec<(NodeID, SocketAddr)>, bootstrap_list: Vec<(NodeID, NamedSocketAddr)>,
our_addr: Option<SocketAddr>, our_addr: Option<NamedSocketAddr>,
) -> Arc<Self> { ) -> Arc<Self> {
let mut known_hosts = KnownHosts::new(); let mut known_hosts = KnownHosts::new();
for (id, addr) in bootstrap_list { for (id, addr) in bootstrap_list {
@ -232,10 +233,12 @@ impl FullMeshPeeringStrategy {
strat.peer_list_endpoint.set_handler(strat.clone()); strat.peer_list_endpoint.set_handler(strat.clone());
let strat2 = strat.clone(); let strat2 = strat.clone();
netapp.on_connected(move |id: NodeID, addr: SocketAddr, is_incoming: bool| { netapp.on_connected(
let strat2 = strat2.clone(); move |id: NodeID, addr: NamedSocketAddr, is_incoming: bool| {
strat2.on_connected(id, addr, is_incoming); let strat2 = strat2.clone();
}); strat2.on_connected(id, addr, is_incoming);
},
);
let strat2 = strat.clone(); let strat2 = strat.clone();
netapp.on_disconnected(move |id: NodeID, is_incoming: bool| { netapp.on_disconnected(move |id: NodeID, is_incoming: bool| {
@ -318,7 +321,11 @@ impl FullMeshPeeringStrategy {
.filter(|x| **x != h.addr) .filter(|x| **x != h.addr)
.cloned() .cloned()
.collect::<Vec<_>>(); .collect::<Vec<_>>();
tokio::spawn(self.clone().try_connect(id, h.addr, alternate_addrs)); tokio::spawn(self.clone().try_connect(
id,
h.addr.clone(),
alternate_addrs,
));
} }
} }
} }
@ -351,7 +358,7 @@ impl FullMeshPeeringStrategy {
if !pings.is_empty() { if !pings.is_empty() {
pub_peer_list.push(PeerInfo { pub_peer_list.push(PeerInfo {
id: *id, id: *id,
addr: info.addr, addr: info.addr.clone(),
state: info.state, state: info.state,
last_seen: info.last_seen, last_seen: info.last_seen,
avg_ping: Some( avg_ping: Some(
@ -366,7 +373,7 @@ impl FullMeshPeeringStrategy {
} else { } else {
pub_peer_list.push(PeerInfo { pub_peer_list.push(PeerInfo {
id: *id, id: *id,
addr: info.addr, addr: info.addr.clone(),
state: info.state, state: info.state,
last_seen: info.last_seen, last_seen: info.last_seen,
avg_ping: None, avg_ping: None,
@ -458,18 +465,20 @@ impl FullMeshPeeringStrategy {
} }
} }
fn handle_peer_list(&self, list: &[(NodeID, SocketAddr)]) { fn handle_peer_list(&self, list: &[(NodeID, NamedSocketAddr)]) {
let mut known_hosts = self.known_hosts.write().unwrap(); let mut known_hosts = self.known_hosts.write().unwrap();
let mut changed = false; let mut changed = false;
for (id, addr) in list.iter() { for (id, addr) in list.iter() {
if let Some(kh) = known_hosts.list.get_mut(id) { if let Some(kh) = known_hosts.list.get_mut(id) {
if !kh.all_addrs.contains(addr) { if !kh.all_addrs.contains(addr) {
kh.all_addrs.push(*addr); kh.all_addrs.push(addr.clone());
changed = true; changed = true;
} }
} else { } else {
known_hosts.list.insert(*id, self.new_peer(id, *addr)); known_hosts
.list
.insert(*id, self.new_peer(id, addr.clone()));
changed = true; changed = true;
} }
} }
@ -483,16 +492,16 @@ impl FullMeshPeeringStrategy {
async fn try_connect( async fn try_connect(
self: Arc<Self>, self: Arc<Self>,
id: NodeID, id: NodeID,
default_addr: SocketAddr, default_addr: NamedSocketAddr,
alternate_addrs: Vec<SocketAddr>, alternate_addrs: Vec<NamedSocketAddr>,
) { ) {
let conn_addr = { let conn_addr = {
let mut ret = None; let mut ret = None;
for addr in [default_addr].iter().chain(alternate_addrs.iter()) { for addr in [default_addr].iter().chain(alternate_addrs.iter()) {
debug!("Trying address {} for peer {}", addr, hex::encode(&id[..8])); debug!("Trying address {} for peer {}", addr, hex::encode(&id[..8]));
match self.netapp.clone().try_connect(*addr, id).await { match self.netapp.clone().try_connect(addr.clone(), id).await {
Ok(()) => { Ok(()) => {
ret = Some(*addr); ret = Some(addr.clone());
break; break;
} }
Err(e) => { Err(e) => {
@ -533,7 +542,7 @@ impl FullMeshPeeringStrategy {
} }
} }
fn on_connected(self: Arc<Self>, id: NodeID, addr: SocketAddr, is_incoming: bool) { fn on_connected(self: Arc<Self>, id: NodeID, addr: NamedSocketAddr, is_incoming: bool) {
let mut known_hosts = self.known_hosts.write().unwrap(); let mut known_hosts = self.known_hosts.write().unwrap();
if is_incoming { if is_incoming {
if let Some(host) = known_hosts.list.get_mut(&id) { if let Some(host) = known_hosts.list.get_mut(&id) {
@ -551,7 +560,7 @@ impl FullMeshPeeringStrategy {
); );
if let Some(host) = known_hosts.list.get_mut(&id) { if let Some(host) = known_hosts.list.get_mut(&id) {
host.state = PeerConnState::Connected; host.state = PeerConnState::Connected;
host.addr = addr; host.addr = addr.clone();
if !host.all_addrs.contains(&addr) { if !host.all_addrs.contains(&addr) {
host.all_addrs.push(addr); host.all_addrs.push(addr);
} }
@ -577,7 +586,7 @@ impl FullMeshPeeringStrategy {
} }
} }
fn new_peer(&self, id: &NodeID, addr: SocketAddr) -> PeerInfoInternal { fn new_peer(&self, id: &NodeID, addr: NamedSocketAddr) -> PeerInfoInternal {
let state = if *id == self.netapp.id { let state = if *id == self.netapp.id {
PeerConnState::Ourself PeerConnState::Ourself
} else { } else {

View file

@ -1,5 +1,4 @@
use std::collections::HashMap; use std::collections::HashMap;
use std::net::SocketAddr;
use std::sync::{Arc, Mutex}; use std::sync::{Arc, Mutex};
use arc_swap::ArcSwapOption; use arc_swap::ArcSwapOption;
@ -8,11 +7,12 @@ use log::*;
use futures::io::{AsyncReadExt, AsyncWriteExt}; use futures::io::{AsyncReadExt, AsyncWriteExt};
use kuska_handshake::async_std::{handshake_server, BoxStream}; use kuska_handshake::async_std::{handshake_server, BoxStream};
use tokio::net::TcpStream;
use tokio::select; use tokio::select;
use tokio::sync::{mpsc, watch}; use tokio::sync::{mpsc, watch};
use tokio_util::compat::*; use tokio_util::compat::*;
use tokio_unix_tcp::{SocketAddr, Stream};
#[cfg(feature = "telemetry")] #[cfg(feature = "telemetry")]
use opentelemetry::{ use opentelemetry::{
trace::{FutureExt, Span, SpanKind, TraceContextExt, TraceId, Tracer}, trace::{FutureExt, Span, SpanKind, TraceContextExt, TraceId, Tracer},
@ -61,7 +61,7 @@ pub(crate) struct ServerConn {
impl ServerConn { impl ServerConn {
pub(crate) async fn run( pub(crate) async fn run(
netapp: Arc<NetApp>, netapp: Arc<NetApp>,
socket: TcpStream, socket: Stream,
must_exit: watch::Receiver<bool>, must_exit: watch::Receiver<bool>,
) -> Result<(), Error> { ) -> Result<(), Error> {
let remote_addr = socket.peer_addr()?; let remote_addr = socket.peer_addr()?;

View file

@ -1,10 +1,11 @@
use std::net::SocketAddr;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use tokio::select; use tokio::select;
use tokio::sync::watch; use tokio::sync::watch;
use tokio_unix_tcp::NamedSocketAddr;
use sodiumoxide::crypto::auth; use sodiumoxide::crypto::auth;
use sodiumoxide::crypto::sign::ed25519; use sodiumoxide::crypto::sign::ed25519;
@ -37,14 +38,20 @@ async fn run_test_inner() {
let (pk2, sk2) = ed25519::gen_keypair(); let (pk2, sk2) = ed25519::gen_keypair();
let (pk3, sk3) = ed25519::gen_keypair(); let (pk3, sk3) = ed25519::gen_keypair();
let addr1: SocketAddr = "127.0.0.1:19991".parse().unwrap(); let addr1: NamedSocketAddr = "127.0.0.1:19991".parse().unwrap();
let addr2: SocketAddr = "127.0.0.1:19992".parse().unwrap(); let addr2: NamedSocketAddr = "127.0.0.1:19992".parse().unwrap();
let addr3: SocketAddr = "127.0.0.1:19993".parse().unwrap(); let addr3: NamedSocketAddr = "127.0.0.1:19993".parse().unwrap();
let (stop_tx, stop_rx) = watch::channel(false); let (stop_tx, stop_rx) = watch::channel(false);
let (thread1, _netapp1, peering1) = let (thread1, _netapp1, peering1) = run_netapp(
run_netapp(netid.clone(), pk1, sk1, addr1, vec![], stop_rx.clone()); netid.clone(),
pk1,
sk1,
addr1.clone(),
vec![],
stop_rx.clone(),
);
tokio::time::sleep(Duration::from_secs(2)).await; tokio::time::sleep(Duration::from_secs(2)).await;
// Connect second node and check it peers with everyone // Connect second node and check it peers with everyone
@ -52,8 +59,8 @@ async fn run_test_inner() {
netid.clone(), netid.clone(),
pk2, pk2,
sk2, sk2,
addr2, addr2.clone(),
vec![(pk1, addr1)], vec![(pk1, addr1.into())],
stop_rx.clone(), stop_rx.clone(),
); );
tokio::time::sleep(Duration::from_secs(5)).await; tokio::time::sleep(Duration::from_secs(5)).await;
@ -67,8 +74,14 @@ async fn run_test_inner() {
assert_eq!(pl2.len(), 2); assert_eq!(pl2.len(), 2);
// Connect third ndoe and check it peers with everyone // Connect third ndoe and check it peers with everyone
let (thread3, _netapp3, peering3) = let (thread3, _netapp3, peering3) = run_netapp(
run_netapp(netid, pk3, sk3, addr3, vec![(pk2, addr2)], stop_rx.clone()); netid,
pk3,
sk3,
addr3,
vec![(pk2, addr2.into())],
stop_rx.clone(),
);
tokio::time::sleep(Duration::from_secs(5)).await; tokio::time::sleep(Duration::from_secs(5)).await;
let pl1 = peering1.get_peer_list(); let pl1 = peering1.get_peer_list();
@ -94,8 +107,8 @@ fn run_netapp(
netid: auth::Key, netid: auth::Key,
_pk: NodeID, _pk: NodeID,
sk: ed25519::SecretKey, sk: ed25519::SecretKey,
listen_addr: SocketAddr, listen_addr: NamedSocketAddr,
bootstrap_peers: Vec<(NodeID, SocketAddr)>, bootstrap_peers: Vec<(NodeID, NamedSocketAddr)>,
must_exit: watch::Receiver<bool>, must_exit: watch::Receiver<bool>,
) -> ( ) -> (
tokio::task::JoinHandle<()>, tokio::task::JoinHandle<()>,

View file

@ -1,10 +1,14 @@
use std::net::SocketAddr; use std::net;
use std::path::PathBuf;
use std::str::FromStr;
use log::info; use log::info;
use serde::Serialize; use serde::Serialize;
use tokio::sync::watch; use tokio::sync::watch;
use tokio_unix_tcp::{NamedSocketAddr, UnixSocketAddr};
use crate::netapp::*; use crate::netapp::*;
/// Utility function: encodes any serializable value in MessagePack binary format /// Utility function: encodes any serializable value in MessagePack binary format
@ -56,24 +60,37 @@ pub fn watch_ctrl_c() -> watch::Receiver<bool> {
} }
/// Parse a peer's address including public key, written in the format: /// Parse a peer's address including public key, written in the format:
/// `<public key hex>@<ip>:<port>` /// `<public key hex>@<ip>:<port>` or
pub fn parse_peer_addr(peer: &str) -> Option<(NodeID, SocketAddr)> { /// `<public key hex>@<path>` for unix domain sockets
let delim = peer.find('@')?; pub fn parse_peer_addr(peer: &str) -> Option<(NodeID, NamedSocketAddr)> {
let (key, ip) = peer.split_at(delim);
let pubkey = NodeID::from_slice(&hex::decode(key).ok()?)?;
let ip = ip[1..].parse::<SocketAddr>().ok()?;
Some((pubkey, ip))
}
/// Parse and resolve a peer's address including public key, written in the format:
/// `<public key hex>@<ip or hostname>:<port>`
pub fn parse_and_resolve_peer_addr(peer: &str) -> Option<(NodeID, Vec<SocketAddr>)> {
use std::net::ToSocketAddrs;
let delim = peer.find('@')?; let delim = peer.find('@')?;
let (key, host) = peer.split_at(delim); let (key, host) = peer.split_at(delim);
let pubkey = NodeID::from_slice(&hex::decode(key).ok()?)?; let pubkey = NodeID::from_slice(&hex::decode(key).ok()?)?;
let hosts = host[1..].to_socket_addrs().ok()?.collect::<Vec<_>>(); let host = NamedSocketAddr::from_str(&host[1..]).ok()?;
Some((pubkey, host))
}
/// Parse and resolve a peer's address including public key, written in the format:
/// `<public key hex>@<ip or hostname>:<port>` or
/// `<public key hex>@<path>` for unix domain sockets
pub fn parse_and_resolve_peer_addr(peer: &str) -> Option<(NodeID, Vec<NamedSocketAddr>)> {
let delim = peer.find('@')?;
let (key, host) = peer.split_at(delim);
let pubkey = NodeID::from_slice(&hex::decode(key).ok()?)?;
let host = &host[1..];
let hosts = if UnixSocketAddr::is_pathname(host) {
vec![PathBuf::from_str(host).unwrap().into()]
} else {
use std::net::ToSocketAddrs;
host.parse::<net::SocketAddr>()
.ok()?
.to_socket_addrs()
.ok()?
.map(NamedSocketAddr::Inet)
.collect::<Vec<_>>()
};
if hosts.is_empty() { if hosts.is_empty() {
return None; return None;
} }
@ -81,14 +98,23 @@ pub fn parse_and_resolve_peer_addr(peer: &str) -> Option<(NodeID, Vec<SocketAddr
} }
/// async version of parse_and_resolve_peer_addr /// async version of parse_and_resolve_peer_addr
pub async fn parse_and_resolve_peer_addr_async(peer: &str) -> Option<(NodeID, Vec<SocketAddr>)> { pub async fn parse_and_resolve_peer_addr_async(
peer: &str,
) -> Option<(NodeID, Vec<NamedSocketAddr>)> {
let delim = peer.find('@')?; let delim = peer.find('@')?;
let (key, host) = peer.split_at(delim); let (key, host) = peer.split_at(delim);
let pubkey = NodeID::from_slice(&hex::decode(key).ok()?)?; let pubkey = NodeID::from_slice(&hex::decode(key).ok()?)?;
let hosts = tokio::net::lookup_host(&host[1..]) let host = &host[1..];
.await let hosts = if UnixSocketAddr::is_pathname(host) {
.ok()? vec![PathBuf::from_str(host).unwrap().into()]
.collect::<Vec<_>>(); } else {
tokio::net::lookup_host(host)
.await
.ok()?
.map(NamedSocketAddr::Inet)
.collect::<Vec<_>>()
};
if hosts.is_empty() { if hosts.is_empty() {
return None; return None;
} }