From dcb6ebd5af361db77d171b4fab6d146b13655e09 Mon Sep 17 00:00:00 2001 From: micolous Date: Thu, 27 Apr 2023 13:18:44 +1000 Subject: [PATCH] caBLE tunnel server (#291) * backend: add connection TTL and traffic limits * backend: return HTTP errors when peers are not available * common: add request / response copy functions * common: add Display implementation for CablePath * common: make CablePath struct more consistent * frontend: check routing_id * frontend: simplify plumbing * log things better * remove many unwraps * add more logging to library * cable-tunnel-server: increase TTL and add sample exchange * backend: implement flags * Migrated backend to dev version of hyper, and implement TLS. Works with Chrome and Safari. * Move some TLS bits out, add debugging * Migrate frontend to new version of hyper, and implement some basic routing. Move some boilerplate into common. Implement HTTPS frontend. * frontend: break loops, and remove excess state * shuffle more state * add docs * wip: self_tx mode, probably will remove * refactor out a bunch of futures stuff to propagate error states better * document tunnel server better, improve logging and error handling * Make http dependency optional * Define all cable-tunnel-server dependencies in the workspace, and share those with other components. * fix building cable docs * a bunch of wordsmithing, using tracing spans * document a bunch of things, instrument the frontend, make debug handler disableable * add some router tests * clippy * fmt --- Cargo.toml | 11 + cable-tunnel-server/README.md | 113 ++++ cable-tunnel-server/backend/Cargo.toml | 29 + cable-tunnel-server/backend/README.md | 108 ++++ cable-tunnel-server/backend/src/main.rs | 424 +++++++++++++ cable-tunnel-server/common/Cargo.toml | 25 + cable-tunnel-server/common/README.md | 12 + cable-tunnel-server/common/src/favicon.ico | Bin 0 -> 2238 bytes cable-tunnel-server/common/src/index.html | 10 + cable-tunnel-server/common/src/lib.rs | 679 +++++++++++++++++++++ cable-tunnel-server/common/src/tls.rs | 154 +++++ cable-tunnel-server/frontend/Cargo.toml | 26 + cable-tunnel-server/frontend/src/main.rs | 304 +++++++++ 13 files changed, 1895 insertions(+) create mode 100644 cable-tunnel-server/README.md create mode 100644 cable-tunnel-server/backend/Cargo.toml create mode 100644 cable-tunnel-server/backend/README.md create mode 100644 cable-tunnel-server/backend/src/main.rs create mode 100644 cable-tunnel-server/common/Cargo.toml create mode 100644 cable-tunnel-server/common/README.md create mode 100644 cable-tunnel-server/common/src/favicon.ico create mode 100644 cable-tunnel-server/common/src/index.html create mode 100644 cable-tunnel-server/common/src/lib.rs create mode 100644 cable-tunnel-server/common/src/tls.rs create mode 100644 cable-tunnel-server/frontend/Cargo.toml create mode 100644 cable-tunnel-server/frontend/src/main.rs diff --git a/Cargo.toml b/Cargo.toml index 8bfa5e28..74d86018 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -10,6 +10,10 @@ members = [ "webauthn-rs", # Authenticator interactions "webauthn-authenticator-rs", + # caBLE tunnel server + "cable-tunnel-server/backend", + "cable-tunnel-server/common", + "cable-tunnel-server/frontend", "fido-key-manager", # Authenticator CLI, "authenticator-cli", @@ -33,6 +37,7 @@ exclude = [ [workspace.dependencies] base64urlsafedata = { path = "./base64urlsafedata" } +cable-tunnel-server-common = { path = "./cable-tunnel-server/common" } webauthn-authenticator-rs = { path = "./webauthn-authenticator-rs" } webauthn-rs = { path = "./webauthn-rs" } webauthn-rs-core = { path = "./webauthn-rs-core" } @@ -44,6 +49,10 @@ clap = { version = "^3.2", features = ["derive", "env"] } compact_jwt = "0.2.3" futures = "^0.3.25" hex = "0.4.3" +http = "^0.2.9" +http-body = "=1.0.0-rc.2" +http-body-util = "=0.1.0-rc.2" +hyper = { version = "=1.0.0-rc.3", default-features = false, features = ["http1"] } nom = "7.1" openssl = "^0.10.41" rand = "0.8" @@ -53,8 +62,10 @@ serde_json = "^1.0.79" tide = "0.16" thiserror = "^1.0.37" tokio = { version = "1.22.0", features = ["sync", "test-util", "macros", "rt-multi-thread", "time"] } +tokio-native-tls = "^0.3.1" tokio-tungstenite = { version = "^0.18.0", features = ["native-tls"] } tracing = "^0.1.35" tracing-subscriber = { version = "0.3", features = ["env-filter", "std", "fmt"] } +tungstenite = { version = "^0.18.0", default-features = false, features = ["handshake"] } url = "2" uuid = "^1.1.2" diff --git a/cable-tunnel-server/README.md b/cable-tunnel-server/README.md new file mode 100644 index 00000000..20572d61 --- /dev/null +++ b/cable-tunnel-server/README.md @@ -0,0 +1,113 @@ +# webauthn-rs caBLE tunnel server + +**Important:** it is only necessary for an *authenticator vendor* to run a caBLE +tunnel service for their devices. Initiators (such as browsers and client +applications) connect to a tunnel service of the *authenticator's* choosing. + +**Warning:** this is still a work in progress, and not yet fully implemented. + +However, you can run a single-task tunnel service with the `backend` alone: +[see `./backend/README.md` for instructions][0]. + +[0]: ./backend/README.md + +## Background + +To facilitate two-way communication between an initiator (browser) and +authenticator (mobile phone), caBLE uses a WebSocket tunnel server. There are +tunnel servers run by Apple (`cable.auth.com`) and Google (`cable.ua5v.com`), +and a facility to procedurally generate new tunnel server domain names +([run `webauthn-authenticator-rs`' `cable_domain` example][1]). + +[1]: ../webauthn-authenticator-rs/examples/cable_tunnel.rs + +As far as the tunnel server is concerned, what happens is: + +1. The authenticator and initator choose a 16 byte tunnel ID. + +2. The authenticator connects to a tunnel server of its choosing, using HTTPS. + +3. The authenticator makes a WebSocket request to `/cable/new/${TUNNEL_ID}`[^new]. + +4. The tunnel server responds with a WebSocket handshake, and includes a 3 byte + routing ID in the HTTP response headers to indicate which task is serving + the request. + +5. The authenticator transmits the tunnel server ID and routing ID to the + initiator using an encrypted Bluetooth Low Energy advertisement. + +6. The initiator decrypts the advertisement, and connects to the tunnel server + using HTTPS. + +7. The initiator makes a WebSocket request to + `/cable/connect/${ROUTING_ID}/${TUNNEL_ID}`. + +8. The tunnel server responds with a WebSocket handshake. + +9. The tunnel server relays binary WebSocket messages between the authenticator + and initiator. + +The initiator starts a Noise channel with the authenticator for further +communication such that the tunnel server cannot read their communications, and +then does registration or authentication using the FIDO 2 protocol. + +Aside from implementing some basic request filtering, message limits and session +limits, the tunnel server implementations are very simple. The tunnel server +itself does not need to concern itself with the minutae of the Noise protocol - +it only needs to pass binary messages across the tunnel verbatim. + +[^new]: + This [follows Google's caBLE URL convention][2]. The URL used to establish a + new channel [is not part of the FIDO 2.2 specification][3]. + +[2]: https://source.chromium.org/chromium/chromium/src/+/main:device/fido/cable/v2_handshake.cc?q=symbol%3A%5Cbdevice%3A%3Acablev2%3A%3Atunnelserver%3A%3AGetNewTunnelURL%5Cb%20case%3Ayes +[3]: https://fidoalliance.org/specs/fido-v2.2-rd-20230321/fido-client-to-authenticator-protocol-v2.2-rd-20230321.html#ref-for-client-platform①⓪ + +## Design + +`webauthn-rs`' caBLE tunnel server consists of three parts: + +* [backend][]: serving binary which passes messages between the authenticator + and initiator on a known tunnel ID. + +* [frontend][]: serving binary which routes requests to a `backend` task based + on the routing ID (for `connect` / initiator requests), or some other load + balancing algorithm (for `new` / authenticator requests). + +* [common][]: contains all the shared web server, TLS and caBLE components for + the `backend` and `frontend` binaries. + +[backend]: ./backend/ +[frontend]: ./frontend/ +[common]: ./common/ + +### Backend + +**Source:** [`./backend/`][backend] + +It should be possible to run the `backend` without a `frontend` – in this case +the routing ID will be ignored, and all tunnels exist inside of a single serving +task. + +### Frontend + +**Warning:** The `frontend` is not yet fully implemented, and does not yet do +everything described here. This would be necessary to operate a +high-availability caBLE tunnel service. + +**Source:** [`./frontend/`][frontend] + +The `frontend` needs to do some basic request processing (for routing) before +handing off the connection to a `backend`: + +* For connecting to existing tunnels, the `frontend` needs to connect to + arbitrary `backend` tasks *in any location*. + +* For establishing new tunnels, the `frontend` should prefer to route to "local" + `backend` tasks, taking into account backend availability and load balancing. + +This will probably need some distributed lock service to allocate the routing +IDs. + +While it would be possible to route based on the tunnel ID *alone*, this would +make tunnel create / fetch operations (in the `backend`) global. diff --git a/cable-tunnel-server/backend/Cargo.toml b/cable-tunnel-server/backend/Cargo.toml new file mode 100644 index 00000000..7a5c03a5 --- /dev/null +++ b/cable-tunnel-server/backend/Cargo.toml @@ -0,0 +1,29 @@ +[package] +name = "cable-tunnel-server-backend" +version = "0.1.0" +authors = ["Michael Farrell "] +categories = ["authentication"] +description = "webauthn-rs caBLE tunnel server backend" +edition = "2021" +keywords = ["cable", "hybrid", "fido", "webauthn"] +license = "MPL-2.0" +readme = "README.md" +repository = "https://github.com/kanidm/webauthn-rs/" +rust-version = "1.66.0" + +[dependencies] +cable-tunnel-server-common.workspace = true + +clap.workspace = true +futures.workspace = true +hex.workspace = true +http-body.workspace = true +http-body-util.workspace = true +hyper = { workspace = true, features = ["server"] } +thiserror.workspace = true +tokio.workspace = true +tokio-native-tls.workspace = true +tokio-tungstenite.workspace = true +tracing.workspace = true +tracing-subscriber.workspace = true +tungstenite.workspace = true diff --git a/cable-tunnel-server/backend/README.md b/cable-tunnel-server/backend/README.md new file mode 100644 index 00000000..c3628e86 --- /dev/null +++ b/cable-tunnel-server/backend/README.md @@ -0,0 +1,108 @@ +# webauthn-rs cable-tunnel-server-backend + +This binary provides a caBLE tunnel server, which is intended for +*non-production use only*. + +The `backend` can run in two configurations: + +* a single-task configuration, directly serving requests with no frontend. + + In this configuration, caBLE [Routing IDs][background] are ignored, and it is + presumed all incoming requests can be served out of a single running task. + +* a multi-task configuration, with many frontend tasks. + + In this configuration, the backend presumes it has frontend tasks in front of + it to [handle caBLE Routing IDs][background]. However, the frontend is not yet + fully implemented. + +The `backend` is stateless, and is not capable of communicating with other +tasks on its own. Each tunnel exists within one (*and only one*) `backend` task, +and `backend` tasks never process caBLE [Routing IDs][background]. + +[background]: ../README.md#background + +## Building + +You can build the `backend` using Cargo: + +```sh +cargo build +``` + +This will output a binary to `./target/debug/cable-tunnel-server-backend`. + +You can also run the server via Cargo: + +```sh +cargo run -- --help +``` + +## Configuring the server + +The server is configured with command-line flags, which can be seen by running +the server with `--help`. + +To run the server at http://127.0.0.1:8080 (for testing with +`webauthn-authenticator-rs` built with the `cable-override-tunnel` feature): + +```sh +./cable-tunnel-server-backend \ + --bind-address 127.0.0.1:8080 \ + --insecure-http-server +``` + +To run the server with HTTPS and strict `Origin` header checks: + +```sh +./cable-tunnel-server-backend \ + --bind-address 192.0.2.1:443 \ + --tls-public-key /etc/ssl/certs/cable.example.com.pem \ + --tls-private-key /etc/ssl/certs/cable.example.com.key \ + --origin cable.example.com +``` + +> **Important:** caBLE has an algorithm to deriving tunnel server domain names – +> you cannot host the service on an arbitrary domain name of your choosing. +> +> Run [`webauthn-authenticator-rs`' `cable_domain` example][cable_domain] to +> derive hostnames at the command line. + +[cable_domain]: ../../webauthn-authenticator-rs/examples/cable_domain.rs + +## Logging + +By default, the server runs at log level `info`. This can be changed with the +`RUST_LOG` environment variable, using the +[log levels available in the `tracing` crate][log-levels]. + +The server logs the following at each level, plus all the messages in the levels +above it: + +* `error`: TLS handshake errors, TCP connection errors, incorrect or unknown + HTTP requests + +* `warn`: warnings about using unencrypted HTTP + +* `info`: (default) start-up messages, HTTP connection lifetime, HTTP request + logs, WebSocket tunnel lifetime + +* `debug`: n/a + +* `trace`: adds complete incoming HTTP requests, WebSocket tunnel messages + +[log-levels]: https://docs.rs/tracing/*/tracing/struct.Level.html + +## Monitoring + +The server exports some basic metrics at `/debug`: + +* `server_state.strong_count`: the number of strong references to + `Arc` + +* `peer_map`: a `HashMap` of all pending tunnels - those where the authenticator + has connected but the initiator has not yet connected. + + * `peer_map.capacity`: the capacity of the pending tunnels `HashMap` + + * `peer_map.len`: the number of pending tunnels diff --git a/cable-tunnel-server/backend/src/main.rs b/cable-tunnel-server/backend/src/main.rs new file mode 100644 index 00000000..9f3cc922 --- /dev/null +++ b/cable-tunnel-server/backend/src/main.rs @@ -0,0 +1,424 @@ +use std::{ + borrow::Cow, collections::HashMap, convert::Infallible, error::Error as StdError, + net::SocketAddr, sync::Arc, time::Duration, +}; + +use clap::{ArgAction, Parser, ValueHint}; + +use futures::{SinkExt, StreamExt}; +use http_body_util::Full; +use hyper::{ + body::{Bytes, Incoming}, + header::CONTENT_TYPE, + http::HeaderValue, + upgrade::Upgraded, + Request, Response, StatusCode, +}; +use tokio::{ + select, + sync::{ + mpsc::{channel, error::TrySendError, Receiver, Sender}, + RwLock, + }, +}; +use tokio_tungstenite::WebSocketStream; +use tungstenite::{ + error::CapacityError, + protocol::{frame::coding::CloseCode, CloseFrame, Message, Role, WebSocketConfig}, +}; + +use cable_tunnel_server_common::*; + +#[macro_use] +extern crate tracing; + +type Rx = Receiver; +type Tx = Sender; +type PeerMap = RwLock>; +const CHANNEL_BUFFER_SIZE: usize = 6; + +struct ServerState { + peer_map: PeerMap, + max_messages: u8, + max_length: usize, + origin: Option, + routing_id: RoutingId, + tunnel_ttl: Duration, + debug_handler: bool, +} + +#[derive(Debug, Parser)] +#[clap(about = "caBLE tunnel server backend")] +pub struct Flags { + /// Bind address and port for the server. + #[clap(long, default_value = "127.0.0.1:8080", value_name = "ADDR")] + bind_address: SocketAddr, + + /// The routing ID to report on new tunnel requests. This is a 3 byte, + /// base-16 encoded value (eg: `123456`). + /// + /// Note: all routing IDs will be accepted by the server for connect + /// requests, and are never checked against this value. + #[clap(long, default_value = "000000", value_parser = parse_hex::, value_name = "ID")] + routing_id: RoutingId, + + /// If set, the required Origin for requests sent to the WebSocket server. + /// + /// When not set, the tunnel server allows requests from any Origin, which + /// could allow non-caBLE use of the server. + #[clap(long, value_hint = ValueHint::Hostname)] + origin: Option, + + /// Maximum amount of time a tunnel may be open for, in seconds. The timer + /// starts when the authenticator first connects, even if there is no peer. + #[clap(long, default_value = "120", value_parser = parse_duration_secs, value_name = "SECONDS")] + tunnel_ttl: Duration, + + /// Maximum number of messages that may be sent to a tunnel by each peer in + /// a session. Once this limit has been reached, the tunnel will be closed. + #[clap(long, default_value = "16", value_name = "MESSAGES")] + max_messages: u8, + + /// Maximum message length which may be sent to a tunnel by a peer. If a + /// peer sends a longer message, the connection and tunnels will be closed. + #[clap(long, default_value = "16384", value_name = "BYTES")] + max_length: usize, + + #[clap(flatten)] + protocol: ServerTransportProtocol, + + /// Disables the `/debug` handler on the HTTP server. + #[clap(long = "no-debug-handler", action = ArgAction::SetFalse)] + debug_handler: bool, +} + +impl From<&Flags> for ServerState { + fn from(f: &Flags) -> Self { + Self { + peer_map: RwLock::new(HashMap::new()), + max_messages: f.max_messages, + max_length: f.max_length, + origin: f.origin.to_owned(), + routing_id: f.routing_id, + tunnel_ttl: f.tunnel_ttl, + debug_handler: f.debug_handler, + } + } +} + +/// A tunnel which is pending connection from an initiator. +struct Tunnel { + /// A channel to allow the authenticator to receive messages from the + /// initiator. + authenticator_rx: Rx, + /// A channel to allow the initiator to send messages to the authenticator. + initiator_tx: Tx, +} + +impl Tunnel { + pub fn new(authenticator_rx: Rx, initiator_tx: Tx) -> Self { + Self { + authenticator_rx, + initiator_tx, + } + } +} + +const PEER_DISCONNECTED_FRAME: CloseFrame = CloseFrame { + code: CloseCode::Normal, + reason: Cow::Borrowed("remote peer cleanly disconnected"), +}; + +#[derive(thiserror::Error, Debug)] +enum CableError { + #[error("remote peer sent erroneous frame")] + RemotePeerErrorFrame, + #[error("remote peer abnormally disconnected")] + RemotePeerAbnormallyDisconnected, + #[error("client sent a message which was too long")] + MessageTooLong, + #[error("client sent too many messages")] + TooManyMessages, + #[error("client sent unsupported message type")] + UnsupportedMessageType, + #[error("tunnel TTL exceeded")] + TtlExceeded, + #[error("WebSocket error: {0}")] + WebSocketError(tungstenite::Error), +} + +impl From for CableError { + fn from(e: tungstenite::Error) -> Self { + if matches!( + e, + tungstenite::Error::Capacity(CapacityError::MessageTooLong { .. }) + ) { + Self::MessageTooLong + } else { + Self::WebSocketError(e) + } + } +} + +impl CableError { + fn close_reason(&self) -> Option { + use CableError::*; + let code = match self { + RemotePeerErrorFrame => CloseCode::Policy, + RemotePeerAbnormallyDisconnected => CloseCode::Away, + MessageTooLong => CloseCode::Size, + TooManyMessages => CloseCode::Policy, + UnsupportedMessageType => CloseCode::Unsupported, + TtlExceeded => CloseCode::Policy, + // Don't expose other error types + _ => return None, + }; + + Some(CloseFrame { + code, + reason: self.to_string().into(), + }) + } + + /// Create a message to notify the remote peer about a local error. + fn peer_message(&self) -> Option { + use CableError::*; + let f = match self { + // Remote peer is already gone, don't notify. + RemotePeerErrorFrame | RemotePeerAbnormallyDisconnected => return None, + + // Other errors should notify peer + TtlExceeded => TtlExceeded.close_reason(), + TooManyMessages => TooManyMessages.close_reason(), + WebSocketError(tungstenite::Error::ConnectionClosed) => Some(PEER_DISCONNECTED_FRAME), + WebSocketError(_) => RemotePeerAbnormallyDisconnected.close_reason(), + _ => RemotePeerErrorFrame.close_reason(), + }; + + Some(Message::Close(f)) + } +} + +#[instrument(level = "info", skip_all, err, fields(addr = addr.to_string()))] +async fn handle_websocket( + state: Arc, + mut ws_stream: WebSocketStream, + tx: Tx, + mut rx: Rx, + addr: SocketAddr, +) -> Result<(), CableError> { + let mut message_count = 0u8; + + let r = tokio::time::timeout(state.tunnel_ttl, async { + loop { + select! { + r = rx.recv() => match r { + Some(msg) => { + // A message was received from the remote peer, send it onward. + match msg { + Message::Close(reason) => { + info!("remote peer closed connection: {reason:?}"); + ws_stream.close(reason).await?; + return Ok(()); + } + msg => { + ws_stream.send(msg).await?; + } + } + }, + None => { + // The peer disconnected + return Err(CableError::RemotePeerAbnormallyDisconnected); + } + }, + + r = ws_stream.next() => match r { + None => { + // Stream ended + info!("client disconnected"); + tx.try_send(Message::Close(Some(PEER_DISCONNECTED_FRAME.clone()))).ok(); + return Ok(()); + }, + Some(Err(e)) => { + // Websocket protocol error + error!("reading websocket: {e}"); + return Err(e.into()); + }, + Some(Ok(msg)) => { + // A message was received from the local peer, validate it and + // send it onward + if msg.is_close() { + info!("closing connection"); + tx.try_send(Message::Close(Some(PEER_DISCONNECTED_FRAME.clone()))).ok(); + ws_stream.close(None).await?; + return Ok(()); + } + + if msg.is_ping() || msg.is_pong() { + // Ignore PING/PONG messages, and don't count them towards + // quota. Tungstenite handles replies for us. + continue; + } + + let msg = if let Message::Binary(msg) = msg { + msg + } else { + // Drop connection on other message types. + return Err(CableError::UnsupportedMessageType); + }; + + // Count the message towards the quota + message_count += 1; + + if message_count > state.max_messages || message_count == u8::MAX { + return Err(CableError::TooManyMessages); + } + + trace!("message {message_count}: {}", hex::encode(&msg)); + match tx.try_send(Message::Binary(msg)) { + Err(TrySendError::Closed(_)) => + return Err(CableError::RemotePeerAbnormallyDisconnected), + Err(TrySendError::Full(_)) => + return Err(CableError::TooManyMessages), + Ok(_) => (), + } + } + } + } + } + }) + .await + // Convert Elapsed into TtlExceeded + .unwrap_or(Err(CableError::TtlExceeded)); + + if let Err(e) = &r { + // An error result indicates that no Close message has been sent + // already, and we may need to notify the peer. Sending messages or + // closing may fail at this stage, but we don't care. + if let Some(msg) = e.peer_message() { + tx.try_send(msg).ok(); + } + ws_stream.close(e.close_reason()).await.ok(); + } + + r +} + +#[instrument(level = "info", skip_all, fields( + req.method = req.method().to_string(), + req.path = req.uri().path(), +))] +async fn handle_request( + state: Arc, + addr: SocketAddr, + req: Request, +) -> Result>, Infallible> { + trace!("request payload: {req:?}"); + let mut req = req.map(|_| ()); + + let (mut res, mut path) = match Router::route(&req, state.origin.as_deref()) { + Router::Static(res) => return Ok(res), + Router::Websocket(res, path) => (res, path), + Router::Debug => { + return Ok(if state.debug_handler { + let peer_map_read = state.peer_map.read().await; + let debug = format!( + "server_state.strong_count = {}\npeer_map.capacity = {}\npeer_map.len = {}\n", + Arc::strong_count(&state), + peer_map_read.capacity(), + peer_map_read.len(), + ); + Response::builder() + .status(StatusCode::OK) + .header(CONTENT_TYPE, HeaderValue::from_static("text/plain")) + .body(Bytes::from(debug).into()) + .unwrap() + } else { + empty_response(StatusCode::NOT_FOUND) + }); + } + }; + + let (tx, rx) = match path.method { + CableMethod::New => { + // Add the routing ID to the response header. + path.routing_id.copy_from_slice(&state.routing_id); + path.insert_routing_id_header(res.headers_mut()); + + // Create both channels in the authenticator side, as the first one here + let (authenticator_tx, authenticator_rx) = channel(CHANNEL_BUFFER_SIZE); + let (initiator_tx, initiator_rx) = channel(CHANNEL_BUFFER_SIZE); + let tunnel = Tunnel::new(authenticator_rx, initiator_tx); + + // Put it in our peer_map, if we can... + { + let mut lock = state.peer_map.write().await; + if lock.contains_key(&path.tunnel_id) { + error!("tunnel already exists: {path}"); + return Ok(empty_response(StatusCode::CONFLICT)); + } + lock.insert(path.tunnel_id, tunnel); + } + + (authenticator_tx, initiator_rx) + } + + CableMethod::Connect => { + if let Some(c) = state.peer_map.write().await.remove(&path.tunnel_id) { + (c.initiator_tx, c.authenticator_rx) + } else { + error!("no peer available for tunnel: {path}"); + return Ok(empty_response(StatusCode::NOT_FOUND)); + } + } + }; + + tokio::task::spawn(async move { + let ss = state.clone(); + let config = Some(WebSocketConfig { + max_message_size: Some(ss.max_length), + max_frame_size: Some(ss.max_length), + ..Default::default() + }); + + match hyper::upgrade::on(&mut req).await { + Ok(upgraded) => { + let ws_stream = + WebSocketStream::from_raw_socket(upgraded, Role::Server, config).await; + handle_websocket(ss, ws_stream, tx, rx, addr).await.ok(); + } + Err(e) => { + error!("upgrade error: {e}"); + } + } + + if path.method == CableMethod::New { + // Remove any stale entry + state.peer_map.write().await.remove(&path.tunnel_id); + } + }); + + Ok(res) +} + +#[tokio::main] +async fn main() -> Result<(), Box> { + setup_logging(); + let flags = Flags::parse(); + let server_state = ServerState::from(&flags); + let tls_acceptor = flags.protocol.tls_acceptor()?; + info!( + "Starting webauthn-rs cable-tunnel-server-backend at {}", + flags.protocol.uri(&flags.bind_address)? + ); + + run_server( + flags.bind_address, + tls_acceptor, + server_state, + handle_request, + ) + .await?; + + Ok(()) +} diff --git a/cable-tunnel-server/common/Cargo.toml b/cable-tunnel-server/common/Cargo.toml new file mode 100644 index 00000000..71e756d2 --- /dev/null +++ b/cable-tunnel-server/common/Cargo.toml @@ -0,0 +1,25 @@ +[package] +name = "cable-tunnel-server-common" +version = "0.1.0" +authors = ["Michael Farrell "] +description = "Common components for webauthn-rs' caBLE tunnel server backend and frontend" +edition = "2021" +keywords = ["cable", "hybrid", "fido", "webauthn"] +license = "MPL-2.0" +readme = "README.md" +repository = "https://github.com/kanidm/webauthn-rs/" +rust-version = "1.66.0" + +[dependencies] +clap.workspace = true +hex.workspace = true +http-body.workspace = true +http-body-util.workspace = true +hyper = { workspace = true, features = ["server"] } +thiserror.workspace = true +tokio.workspace = true +tokio-native-tls.workspace = true +tokio-tungstenite.workspace = true +tracing.workspace = true +tracing-subscriber.workspace = true +tungstenite.workspace = true diff --git a/cable-tunnel-server/common/README.md b/cable-tunnel-server/common/README.md new file mode 100644 index 00000000..89f271b6 --- /dev/null +++ b/cable-tunnel-server/common/README.md @@ -0,0 +1,12 @@ +# cable-tunnel-server-common + +Common components for webauthn-rs' [caBLE tunnel server][] [`backend`][] and +[`frontend`][]. + +**Important:** this library is an internal implementation detail of webauthn-rs' +[caBLE tunnel server][], and has no guarantees of API stability whatsoever. It +is **not** intended for use outside of that context. + +[`backend`]: ../backend/ +[caBLE tunnel server]: ../README.md +[`frontend`]: ../frontend/ diff --git a/cable-tunnel-server/common/src/favicon.ico b/cable-tunnel-server/common/src/favicon.ico new file mode 100644 index 0000000000000000000000000000000000000000..2951078e6752259b51327fee40c03edd477acf2d GIT binary patch literal 2238 zcmdUvc~n$Y9LK+|y|Qe#vP@e{P(uMlMFEjbfl&rw1V>;TSrzcAxr>U1g(2=^vWSt6 zB5s(d;fBT_B%8p@z%c7Wpri$m9fa_xpZ-@80{)JMRJg=rw8- z=*-|G0}KOTK%o+5QuKOXW3JI?xEPNC*Vq8_j19pxF~EFNL%8XgG#2i9CXd4cvvF`T zwgS&=JgkkJfw3Wg;R-mKSi@u5M0id&0pG$HFbV_TaxyS+6Bb!bftQsze5|L#$94t; z6gzW&2<$8;ACL+5^t;$3PJpY7%(pz;N}8u#S|$;28PD9 z!AoJWFcol@V>qt>;3$)|t_02~3`n7=19~dp5_xb!uslm1l>i|J5h6rdar~?+)W2Eo_MhJfggbkEN zN)xc83Df-5G&W<$<}eyt0AD?=lr~HWxQ_|`4`3QV_xEc@_~uB!zk||=S<7^=4c5S8 z#a+w^QNcR60vw`-P3SLhU8%-_=w0yMpu*yfDlFYn1i!E{1Z+_vbbBdcMF&A56o@+{ z!hVq$;n8`B+;ONxPPCyisf{1+u2-{PRBS+6-=OHPg4oi_F7UP)sJod)P zAWF!^>7*oxj!LmVL4m`G1&BRXjCipWvByhkltG*`YN1xEq0wk4T4=Re=yW=;EK4bej;#V)UWtl|3RG5BqN=J2)z#Igsi~pV zqPDgcb#--k`0yd>>+8|b(16CqMl>}wp}DykEiEl*ZEZzcTN~Qj+wthpBXo3hptG~H zuTS*3>h!t(7elT6r(>$Wo&95M2RF>xhxcx0B-V(-W?XS;T|5<c{ZxbZ^3;WTUdUX HFBbm`+7M2j literal 0 HcmV?d00001 diff --git a/cable-tunnel-server/common/src/index.html b/cable-tunnel-server/common/src/index.html new file mode 100644 index 00000000..cea1dda3 --- /dev/null +++ b/cable-tunnel-server/common/src/index.html @@ -0,0 +1,10 @@ + + + + webauthn-rs cable-tunnel-server + + +

This is a caBLE tunnel server, but your client did not send a caBLE request.

+

webauthn-rs cable-tunnel-server

+ + diff --git a/cable-tunnel-server/common/src/lib.rs b/cable-tunnel-server/common/src/lib.rs new file mode 100644 index 00000000..090b5fa8 --- /dev/null +++ b/cable-tunnel-server/common/src/lib.rs @@ -0,0 +1,679 @@ +//! Common components for webauthn-rs' caBLE tunnel server. +//! +//! **Important**: this library is an internal implementation detail of +//! webauthn-rs' caBLE tunnel server, and has no guarantees of API stability +//! whatsoever. It is **not** intended for use outside of that context. +use std::{ + convert::Infallible, error::Error as StdError, fmt::Display, future::Future, mem::size_of, + net::SocketAddr, num::ParseIntError, sync::Arc, time::Duration, +}; + +use hex::{FromHex, FromHexError}; +use http_body::Body; +use http_body_util::{Empty, Full}; +use hyper::{ + body::{Bytes, Incoming}, + header::{CONTENT_TYPE, ORIGIN, SEC_WEBSOCKET_PROTOCOL}, + http::HeaderValue, + service::service_fn, + HeaderMap, Method, Request, Response, StatusCode, Uri, +}; +use tokio::net::TcpListener; +use tokio_native_tls::TlsAcceptor; +use tokio_tungstenite::MaybeTlsStream; +use tracing::Instrument; +use tracing_subscriber::{filter::LevelFilter, fmt::format::FmtSpan, EnvFilter}; +use tungstenite::handshake::server::create_response; + +#[macro_use] +extern crate tracing; + +mod tls; +pub use tls::*; + +pub type RoutingId = [u8; 3]; +pub type TunnelId = [u8; 16]; + +pub static CABLE_PROTOCOL: HeaderValue = HeaderValue::from_static("fido.cable"); +pub const CABLE_ROUTING_ID_HEADER: &str = "X-caBLE-Routing-ID"; + +pub const CABLE_NEW_PATH: &str = "/cable/new/"; +pub const CABLE_CONNECT_PATH: &str = "/cable/connect/"; + +pub const MAX_URL_LENGTH: usize = + CABLE_CONNECT_PATH.len() + ((size_of::() + size_of::()) * 2) + 2; + +const FAVICON: &[u8] = include_bytes!("favicon.ico"); +const INDEX: &str = include_str!("index.html"); + +/// Parses a Base-16 encoded string. +/// +/// This function is intended for use as a `clap` `value_parser`. +pub fn parse_hex(i: &str) -> Result +where + T: FromHex, +{ + FromHex::from_hex(i) +} + +/// Parses a duration as a number of seconds from a string. +/// +/// This function is intended for use as a `clap` `value_parser`. +pub fn parse_duration_secs(i: &str) -> Result { + i.parse::().map(Duration::from_secs) +} + +/// Path for caBLE WebSocket tunnel protocol. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct CablePath { + /// The method for the tunnel. + pub method: CableMethod, + /// The routing ID of the tunnel. + pub routing_id: RoutingId, + /// The tunnel ID of the tunnel. + pub tunnel_id: TunnelId, +} + +/// Method for caBLE WebSocket tunnel protocol. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum CableMethod { + /// Request from the authenticator to establish a new tunnel. This needs to + /// be allocated a routing ID. + New, + /// Request from the initiator to connect to an existing tunnel. + Connect, +} + +impl CablePath { + pub fn new(tunnel_id: TunnelId) -> Self { + Self { + method: CableMethod::New, + routing_id: [0; size_of::()], + tunnel_id, + } + } + + pub fn connect(routing_id: RoutingId, tunnel_id: TunnelId) -> Self { + Self { + method: CableMethod::Connect, + routing_id, + tunnel_id, + } + } + + /// Inserts the caBLE routing ID header into a HTTP response. + pub fn insert_routing_id_header(&self, headers: &mut HeaderMap) { + headers.insert( + CABLE_ROUTING_ID_HEADER, + HeaderValue::from_str(&hex::encode_upper(self.routing_id)).unwrap(), + ); + } +} + +impl Display for CablePath { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self.method { + CableMethod::New => { + write!(f, "{}{}", CABLE_NEW_PATH, hex::encode_upper(self.tunnel_id)) + } + CableMethod::Connect => write!( + f, + "{}{}/{}", + CABLE_CONNECT_PATH, + hex::encode_upper(self.routing_id), + hex::encode_upper(self.tunnel_id) + ), + } + } +} + +impl TryFrom<&str> for CablePath { + type Error = (); + fn try_from(path: &str) -> Result { + if path.len() > MAX_URL_LENGTH { + error!("path too long: {} > {MAX_URL_LENGTH} bytes", path.len()); + return Err(()); + } else if let Some(path) = path.strip_prefix(CABLE_NEW_PATH) { + let mut tunnel_id: TunnelId = [0; size_of::()]; + if hex::decode_to_slice(path, &mut tunnel_id).is_ok() { + return Ok(Self::new(tunnel_id)); + } + error!("invalid new path: {path}"); + } else if let Some(path) = path.strip_prefix(CABLE_CONNECT_PATH) { + let mut routing_id: RoutingId = [0; size_of::()]; + let mut tunnel_id: TunnelId = [0; size_of::()]; + + let mut splitter = path.split('/'); + + if splitter + .next() + .and_then(|c| hex::decode_to_slice(c, &mut routing_id).ok()) + .is_none() + { + error!("invalid routing_id in connect path: {path}"); + return Err(()); + } + + if splitter + .next() + .and_then(|c| hex::decode_to_slice(c, &mut tunnel_id).ok()) + .is_none() + { + error!("invalid tunnel_id in connect path: {path}"); + return Err(()); + } + + if splitter.next().is_some() { + error!("unexpected extra token in connect path: {path}"); + return Err(()); + } + + return Ok(Self::connect(routing_id, tunnel_id)); + } else { + error!("unknown path: {path}") + } + + Err(()) + } +} + +/// HTTP router for a caBLE WebSocket tunnel server. +#[derive(Debug)] +pub enum Router { + /// The web server should handle the request as a caBLE WebSocket + /// connection. + Websocket(Response>, CablePath), + /// The web server should return a static response. This may be an error + /// message. + Static(Response>), + + Debug, +} + +impl Router { + /// Routes an incoming HTTP request. + pub fn route(req: &Request<()>, origin: Option<&str>) -> Self { + if req.method() != Method::GET { + error!("method {} not allowed", req.method()); + let response = Response::builder() + .status(StatusCode::METHOD_NOT_ALLOWED) + .header("Allow", "GET") + .body(Default::default()) + .unwrap(); + return Self::Static(response); + } + + let path = match req.uri().path() { + "/" => { + return Self::Static( + Response::builder() + .status(StatusCode::OK) + .header(CONTENT_TYPE, "text/html") + .body(Bytes::from(INDEX).into()) + .unwrap(), + ) + } + "/favicon.ico" => { + return Self::Static( + Response::builder() + .status(StatusCode::OK) + .header(CONTENT_TYPE, "image/vnd.microsoft.icon") + .body(Bytes::from(FAVICON).into()) + .unwrap(), + ); + } + "/debug" => return Self::Debug, + path => match CablePath::try_from(path) { + Err(()) => { + return Self::Static(empty_response(StatusCode::NOT_FOUND)); + } + Ok(p) => p, + }, + }; + + let mut res = match create_response(req) { + Ok(r) => r, + Err(e) => { + error!("bad request for WebSocket: {e}"); + return Self::Static(empty_response(StatusCode::BAD_REQUEST)); + } + }; + + // At this point, we have something that looks like a WebSocket on the + // other side. We should check the parameters selected etc. + if !req + .headers() + .get(SEC_WEBSOCKET_PROTOCOL) + .map(|v| v == CABLE_PROTOCOL) + .unwrap_or_default() + { + error!("unsupported or missing WebSocket protocol"); + return Self::Static(empty_response(StatusCode::BAD_REQUEST)); + } + + // Check the origin header + if let Some(origin) = origin { + if !req + .headers() + .get(ORIGIN) + .and_then(|v| v.to_str().ok()) + .and_then(|v| v.parse::().ok()) + .map(|v| { + v.host() + .map(|o| o.eq_ignore_ascii_case(origin)) + .unwrap_or_default() + }) + .unwrap_or_default() + { + error!("incorrect or missing Origin header"); + return Self::Static(empty_response(StatusCode::FORBIDDEN)); + } + } + + // We have the correct protocol, include in the response + res.headers_mut() + .append(SEC_WEBSOCKET_PROTOCOL, CABLE_PROTOCOL.to_owned()); + let res = res.map(|_| Default::default()); + + Router::Websocket(res, path) + } + + #[cfg(test)] + pub(self) fn static_response(self) -> Option>> { + match self { + Self::Static(r) => Some(r), + _ => None, + } + } +} + +/// Make a [Response] with a given [StatusCode] and empty body. +pub fn empty_response, T: Default>(status: E) -> Response { + Response::builder() + .status(status) + .body(Default::default()) + .unwrap() +} + +/// Create a copy of an existing HTTP [Request], discarding the body. +pub fn copy_request_empty_body(r: &Request) -> Request> { + let mut o = Request::builder().method(r.method()).uri(r.uri()); + { + let headers = o.headers_mut().unwrap(); + headers.extend(r.headers().to_owned()); + } + + o.body(Default::default()).unwrap() +} + +/// Create a copy of an existing HTTP [Response], discarding the body. +pub fn copy_response_empty_body(r: &Response) -> Response> { + let mut o = Response::builder().status(r.status()); + { + let headers = o.headers_mut().unwrap(); + headers.extend(r.headers().to_owned()); + } + + o.body(Default::default()).unwrap() +} + +/// Run a HTTP server for the caBLE WebSocket tunnel. +pub async fn run_server( + bind_address: SocketAddr, + tls_acceptor: Option, + server_state: T, + mut request_handler: F, +) -> Result<(), Box> +where + F: FnMut(Arc, SocketAddr, Request) -> R + Copy + Send + Sync + 'static, + R: Future, Infallible>> + Send, + ResBody: Body + Send + 'static, + ::Error: Into>, + ::Data: Send, + T: Send + Sync + 'static, +{ + let server_state = Arc::new(server_state); + let tcp = TcpListener::bind(&bind_address).await?; + let tls_acceptor = tls_acceptor.map(Arc::new); + + loop { + let (stream, remote_addr) = match tcp.accept().await { + Ok(o) => o, + Err(e) => { + error!("tcp.accept: {e}"); + continue; + } + }; + let server_state = server_state.clone(); + let service = + service_fn(move |req| request_handler(server_state.clone(), remote_addr, req)); + let tls_acceptor = tls_acceptor.clone(); + + let span = info_span!("handle_connection", addr = remote_addr.to_string()); + tokio::task::spawn( + async move { + let stream = match tls_acceptor { + None => MaybeTlsStream::Plain(stream), + Some(tls_acceptor) => match tls_acceptor.accept(stream).await { + Ok(o) => MaybeTlsStream::NativeTls(o), + Err(e) => { + error!("tls_acceptor.accept: {e}"); + return; + } + }, + }; + + let conn = + hyper::server::conn::http1::Builder::new().serve_connection(stream, service); + let conn = conn.with_upgrades(); + + if let Err(e) = conn.await { + error!("connection error: {e}"); + } + } + .instrument(span), + ); + } +} + +/// Sets up logging for cable-tunnel-server binaries. +pub fn setup_logging() { + tracing_subscriber::fmt() + .with_env_filter( + EnvFilter::builder() + .with_default_directive(LevelFilter::INFO.into()) + .from_env_lossy(), + ) + .with_span_events(FmtSpan::CLOSE | FmtSpan::NEW) + .with_thread_ids(true) + // .with_file(true) + // .with_line_number(true) + .compact() + .init(); +} + +#[cfg(test)] +mod tests { + use http_body_util::BodyExt; + use tungstenite::client::IntoClientRequest; + + use super::*; + + #[test] + fn parse_urls() { + // Parse valid paths in upper case + assert_eq!( + CablePath::new(*b"hello, webauthn!"), + CablePath::try_from("/cable/new/68656C6C6F2C20776562617574686E21").unwrap() + ); + assert_eq!( + CablePath::connect(*b"abc", *b"hello, webauthn!"), + CablePath::try_from("/cable/connect/616263/68656C6C6F2C20776562617574686E21").unwrap() + ); + + // Converting to string should always return upper-case paths + assert_eq!( + "/cable/new/68656C6C6F2C20776562617574686E21", + CablePath::new(*b"hello, webauthn!").to_string(), + ); + assert_eq!( + "/cable/connect/616263/68656C6C6F2C20776562617574686E21", + CablePath::connect(*b"abc", *b"hello, webauthn!").to_string(), + ); + + // Parse valid paths in lower case + assert_eq!( + CablePath::new(*b"hello, webauthn!"), + CablePath::try_from("/cable/new/68656c6c6f2c20776562617574686e21").unwrap() + ); + assert_eq!( + CablePath::connect(*b"abc", *b"hello, webauthn!"), + CablePath::try_from("/cable/connect/616263/68656c6c6f2c20776562617574686e21").unwrap() + ); + + // Parsing lower-case paths should return strings in upper case. + assert_eq!( + "/cable/new/68656C6C6F2C20776562617574686E21", + CablePath::try_from("/cable/new/68656c6c6f2c20776562617574686e21") + .unwrap() + .to_string() + ); + assert_eq!( + "/cable/connect/616263/68656C6C6F2C20776562617574686E21", + CablePath::try_from("/cable/connect/616263/68656c6c6f2c20776562617574686e21") + .unwrap() + .to_string() + ); + + // Invalid paths + assert!(CablePath::try_from("/").is_err()); + + assert!(CablePath::try_from("/cable/new/").is_err()); + assert!(CablePath::try_from("/cable/new/not_hex_digits_here_but_32_chars").is_err()); + assert!(CablePath::try_from("/cable/new/C0FFEE").is_err()); + assert!(CablePath::try_from("/cable/new/C0FFEEC0FFEEC0FFEEC0FFEEC0FFEEC0FFEE").is_err()); + assert!(CablePath::try_from("/cable/new/68656C6C6F2C20776562617574686E21/").is_err()); + assert!(CablePath::try_from("/cable/new/../new/68656C6C6F2C20776562617574686E21").is_err()); + + assert!( + CablePath::try_from("/cable/connect/C0FFEE/not_hex_digits_here_but_32_chars").is_err() + ); + assert!(CablePath::try_from("/cable/connect/C0FFEE/COFFEE").is_err()); + assert!(CablePath::try_from("/cable/connect/C0/FFEE").is_err()); + assert!(CablePath::try_from("/cable/connect/C0/68656C6C6F2C20776562617574686E21").is_err()); + assert!( + CablePath::try_from("/cable/connect/C0F/68656C6C6F2C20776562617574686E21").is_err() + ); + assert!( + CablePath::try_from("/cable/connect/C0FFEECO/68656C6C6F2C20776562617574686E21") + .is_err() + ); + assert!( + CablePath::try_from("/cable/connect/C0FFEE/68656C6C6F2C20776562617574686E21/1234") + .is_err() + ); + assert!( + CablePath::try_from("/cable/connect/C0FFEE/68656C6C6F2C20776562617574686E21/").is_err() + ); + + // other nonsense + assert!(CablePath::try_from("cable/new/68656C6C6F2C20776562617574686E21").is_err()); + assert!(CablePath::try_from("../cable/new/68656C6C6F2C20776562617574686E21").is_err()); + assert!(CablePath::try_from("/../cable/new/68656C6C6F2C20776562617574686E21").is_err()); + assert!(CablePath::try_from("../../../etc/passwd").is_err()); + + // Should be rejected by length limits + assert!(CablePath::try_from(include_str!("lib.rs")).is_err()); + } + + async fn check_index_response(response: Response>) { + assert_eq!(StatusCode::OK, response.status()); + assert_eq!( + "text/html", + HeaderValue::to_str(response.headers().get(CONTENT_TYPE).unwrap()).unwrap() + ); + assert!(response.body().size_hint().exact().unwrap() > 16); + assert_eq!( + INDEX, + response + .into_body() + .frame() + .await + .unwrap() + .unwrap() + .into_data() + .unwrap() + ); + } + + async fn check_favicon_response(response: Response>) { + assert_eq!(StatusCode::OK, response.status()); + assert_eq!( + "image/vnd.microsoft.icon", + HeaderValue::to_str(response.headers().get(CONTENT_TYPE).unwrap()).unwrap() + ); + assert!(response.body().size_hint().exact().unwrap() > 16); + assert_eq!( + FAVICON, + response + .into_body() + .frame() + .await + .unwrap() + .unwrap() + .into_data() + .unwrap() + ); + } + + fn check_error_response(response: Response>, expected_status: StatusCode) { + assert_eq!(expected_status, response.status()); + assert_eq!(0, response.body().size_hint().exact().unwrap()); + } + + fn check_websocket_response(response: &Response>) -> bool { + assert_eq!(StatusCode::SWITCHING_PROTOCOLS, response.status()); + assert_eq!(0, response.body().size_hint().exact().unwrap()); + assert_eq!( + "fido.cable", + HeaderValue::to_str(response.headers().get(SEC_WEBSOCKET_PROTOCOL).unwrap()).unwrap() + ); + true + } + + const TEST_ORIGINS: [Option<&str>; 3] = + [None, Some("cable.example.com"), Some("cable.example.net")]; + + #[tokio::test] + async fn static_router() -> Result<(), Box> { + // Index handler, without origin + let request = Request::get("https://cable.example.com/").body(())?; + + for router_origin in TEST_ORIGINS { + check_index_response( + Router::route(&request, router_origin) + .static_response() + .ok_or("expected static response")?, + ) + .await; + } + + // Index handler, with origin + let request = Request::get("https://cable.example.com/") + .header(ORIGIN, "cable.example.com") + .body(())?; + + for router_origin in TEST_ORIGINS { + check_index_response( + Router::route(&request, router_origin) + .static_response() + .ok_or("expected static response")?, + ) + .await; + } + + // Favicon handler, without origin + let request = Request::get("https://cable.example.com/favicon.ico").body(())?; + + for router_origin in TEST_ORIGINS { + check_favicon_response( + Router::route(&request, router_origin) + .static_response() + .ok_or("expected static response")?, + ) + .await; + } + + // Favicon handler, with origin + let request = Request::get("https://cable.example.com/favicon.ico") + .header(ORIGIN, "cable.example.com") + .body(())?; + + for router_origin in TEST_ORIGINS { + check_favicon_response( + Router::route(&request, router_origin) + .static_response() + .ok_or("expected static response")?, + ) + .await; + } + Ok(()) + } + + #[test] + fn debug_router() -> Result<(), Box> { + // Debug handler, without origin + let request = Request::get("https://cable.example.com/debug").body(())?; + + for router_origin in TEST_ORIGINS { + assert!(matches!( + Router::route(&request, router_origin), + Router::Debug + )); + } + + // Debug handler, with origin + let request = Request::get("https://cable.example.com/debug") + .header(ORIGIN, "cable.example.com") + .body(())?; + + for router_origin in TEST_ORIGINS { + assert!(matches!( + Router::route(&request, router_origin), + Router::Debug + )); + } + + Ok(()) + } + + #[test] + fn websocket_router() -> Result<(), Box> { + // Make WebSocket request missing caBLE headers + let request = "wss://cable.example.com/cable/new/68656C6C6F2C20776562617574686E21" + .into_client_request()?; + check_error_response( + Router::route(&request, None) + .static_response() + .ok_or("expected static response")?, + StatusCode::BAD_REQUEST, + ); + + // With caBLE headers, but no Origin + let mut request = "wss://cable.example.com/cable/new/68656C6C6F2C20776562617574686E21" + .into_client_request()?; + request + .headers_mut() + .insert(SEC_WEBSOCKET_PROTOCOL, CABLE_PROTOCOL.to_owned()); + matches!( + Router::route(&request, None), + Router::Websocket(_, p) if p == CablePath::new(*b"hello, webauthn!") + ); + check_error_response( + Router::route(&request, Some("cable.example.com")) + .static_response() + .ok_or("expected static response")?, + StatusCode::FORBIDDEN, + ); + + // With caBLE headers and Origin + let mut request = "wss://cable.example.com/cable/new/68656C6C6F2C20776562617574686E21" + .into_client_request()?; + let headers = request.headers_mut(); + headers.insert(SEC_WEBSOCKET_PROTOCOL, CABLE_PROTOCOL.to_owned()); + headers.insert(ORIGIN, HeaderValue::from_static("cable.example.com")); + + for router_origin in [None, Some("cable.example.com")] { + matches!( + Router::route(&request, router_origin), + Router::Websocket(r, p) if p == CablePath::new(*b"hello, webauthn!") && check_websocket_response(&r) + ); + } + matches!( + Router::route(&request, Some("cable.example.net")), + Router::Websocket(r, p) if p == CablePath::new(*b"hello, webauthn!") && check_websocket_response(&r) + ); + + Ok(()) + } +} diff --git a/cable-tunnel-server/common/src/tls.rs b/cable-tunnel-server/common/src/tls.rs new file mode 100644 index 00000000..dc1f55d0 --- /dev/null +++ b/cable-tunnel-server/common/src/tls.rs @@ -0,0 +1,154 @@ +use std::{fs::read, net::SocketAddr, path::PathBuf}; + +use clap::{Args, ValueHint}; +use hyper::{http::uri::Builder, Uri}; +use tokio_native_tls::{ + native_tls::{self, Certificate, Identity, Protocol}, + TlsAcceptor, TlsConnector, +}; + +#[derive(thiserror::Error, Debug)] +pub enum TlsConfigError { + #[error( + "server was configured with a TLS public and/or private key, but HTTP mode is enabled" + )] + InsecureHttpWithKeys, + #[error("server was configured to use TLS, but public and/or private keys were not provided")] + TlsServerRequiresPublicAndPrivateKeys, + #[error("backend connection was configured with a trusted CA, but HTTP mode is enabled")] + InsecureHttpWithTrust, + #[error("backend connection was configured with a trust domain, but HTTP mode is enabled")] + InsecureHttpWithDomain, + #[error("problem reading key file: {0}")] + IoError(std::io::Error), + #[error("TLS error: {0}")] + TlsError(native_tls::Error), +} + +impl From for TlsConfigError { + fn from(e: std::io::Error) -> Self { + Self::IoError(e) + } +} + +impl From for TlsConfigError { + fn from(e: native_tls::Error) -> Self { + Self::TlsError(e) + } +} + +#[derive(Debug, Clone, Args)] +pub struct ServerTransportProtocol { + /// Runs an unencrypted HTTP server, rather than HTTPS. This is not suitable + /// for use with ordinary caBLE clients. + #[clap(long, conflicts_with_all(&["tls-public-key", "tls-private-key"]))] + insecure_http_server: bool, + + /// Path to the server's public key (certificate) in PEM format. + #[clap(long, value_name = "PEM", value_hint = ValueHint::FilePath)] + tls_public_key: Option, + + /// Path to the server's private key in PEM format. + #[clap(long, value_name = "PEM", value_hint = ValueHint::FilePath)] + tls_private_key: Option, +} + +impl ServerTransportProtocol { + /// Returns a [TlsAcceptor] for this [ServerTransportProtocol] + /// + /// Returns `None` if insecure HTTP connections should be used. + pub fn tls_acceptor(&self) -> Result, TlsConfigError> { + if self.insecure_http_server { + if self.tls_public_key.is_some() || self.tls_private_key.is_some() { + return Err(TlsConfigError::InsecureHttpWithKeys); + } + + warn!("Using unencrypted HTTP to serve requests. This is insecure, and won't work with ordinary caBLE clients!"); + return Ok(None); + } + + let (tls_public_key, tls_private_key) = match (&self.tls_public_key, &self.tls_private_key) + { + (Some(p), Some(q)) => (p, q), + _ => { + return Err(TlsConfigError::TlsServerRequiresPublicAndPrivateKeys); + } + }; + + let pem = read(tls_public_key)?; + let key = read(tls_private_key)?; + let identity = Identity::from_pkcs8(&pem, &key)?; + Ok(Some(TlsAcceptor::from( + native_tls::TlsAcceptor::builder(identity) + // We only support TLS 1.2 and later, because all caBLE clients support it. + // + // We don't care about breaking things like exceptionally ancient versions of Android. + .min_protocol_version(Some(Protocol::Tlsv12)) + .build()?, + ))) + } + + /// Returns a `http` or `https` URI for this [ServerTransportProtocol] for a + /// given [SocketAddr]. + pub fn uri(&self, addr: &SocketAddr) -> Result { + Builder::new() + .scheme(if self.insecure_http_server { + "http" + } else { + "https" + }) + .authority(addr.to_string()) + .path_and_query("/") + .build() + } +} + +#[derive(Debug, Clone, Args)] +pub struct BackendClientOptions { + /// Uses unencrypted HTTP to connect to backend tasks, rather than HTTPS. + #[clap(long, conflicts_with_all(&["trusted-ca", "domain"]))] + insecure_http_backend: bool, + + /// Public key of the root CA to trust when connecting to a backend task, + /// instead of using the system's built-in certificate trust store. + #[clap(long, value_name = "PEM", value_hint = ValueHint::FilePath)] + trusted_ca: Option, + + /// If set, use this domain name to validate the server-provided certificate + /// against, rather than the IP of the backend. + #[clap(long, value_name = "DOMAIN", value_hint = ValueHint::Hostname)] + pub domain: Option, +} + +impl BackendClientOptions { + /// Creates a [TlsConnector] for the [BackendClientOptions]. + /// + /// Returns `None` if insecure HTTP connections should be used. + pub fn tls_connector(&self) -> Result, TlsConfigError> { + if self.insecure_http_backend { + if self.trusted_ca.is_some() { + return Err(TlsConfigError::InsecureHttpWithTrust); + } + + if self.domain.is_some() { + return Err(TlsConfigError::InsecureHttpWithDomain); + } + + warn!("Using unencrypted HTTP to connect to backend tasks. This is insecure!"); + return Ok(None); + } + + let mut b = native_tls::TlsConnector::builder(); + b.min_protocol_version(Some(Protocol::Tlsv12)); + + if let Some(trusted_ca) = &self.trusted_ca { + let trusted_ca = read(trusted_ca)?; + let certificate = Certificate::from_pem(&trusted_ca)?; + + b.disable_built_in_roots(true) + .add_root_certificate(certificate); + } + + Ok(Some(TlsConnector::from(b.build()?))) + } +} diff --git a/cable-tunnel-server/frontend/Cargo.toml b/cable-tunnel-server/frontend/Cargo.toml new file mode 100644 index 00000000..4c2c8b25 --- /dev/null +++ b/cable-tunnel-server/frontend/Cargo.toml @@ -0,0 +1,26 @@ +[package] +name = "cable-tunnel-server-frontend" +version = "0.1.0" +authors = ["Michael Farrell "] +categories = ["authentication"] +description = "webauthn-rs caBLE tunnel server frontend" +edition = "2021" +keywords = ["cable", "hybrid", "fido", "webauthn"] +license = "MPL-2.0" +readme = "README.md" +repository = "https://github.com/kanidm/webauthn-rs/" +rust-version = "1.66.0" + +[dependencies] +cable-tunnel-server-common.workspace = true + +clap.workspace = true +hex.workspace = true +http-body-util.workspace = true +hyper = { workspace = true, features = ["client", "server"] } +tokio.workspace = true +tokio-native-tls.workspace = true +tokio-tungstenite.workspace = true +tracing.workspace = true +tracing-subscriber.workspace = true +tungstenite.workspace = true diff --git a/cable-tunnel-server/frontend/src/main.rs b/cable-tunnel-server/frontend/src/main.rs new file mode 100644 index 00000000..79464741 --- /dev/null +++ b/cable-tunnel-server/frontend/src/main.rs @@ -0,0 +1,304 @@ +use std::{ + collections::HashMap, convert::Infallible, error::Error as StdError, net::SocketAddr, sync::Arc, +}; + +use clap::{ArgAction, Parser, ValueHint}; + +use http_body_util::Full; +use hyper::{ + body::{Bytes, Incoming}, + header::CONTENT_TYPE, + http::HeaderValue, + Request, Response, StatusCode, +}; + +use cable_tunnel_server_common::*; +use tokio::{io::AsyncWriteExt, net::TcpStream, sync::RwLock}; +use tokio_native_tls::TlsConnector; +use tokio_tungstenite::MaybeTlsStream; + +#[macro_use] +extern crate tracing; + +const ROUTING_ID: RoutingId = [0xC0, 0xFF, 0xEE]; +const FORWARDED_HEADER: &str = "X-WebAuthnRS-Forwarded"; + +struct ServerState { + origin: Option, + tls_domain: Option, + backend_connector: Option, + + backends: RwLock>, + debug_handler: bool, +} + +#[derive(Debug, Parser)] +#[clap(about = "caBLE tunnel server backend")] +pub struct Flags { + /// Bind address and port for the server. + #[clap(long, default_value = "127.0.0.1:8080", value_name = "ADDR")] + bind_address: SocketAddr, + + // Address for the service backend. + #[clap(long, default_value = "127.0.0.1:8081", value_name = "ADDR")] + backend_address: SocketAddr, + + #[clap(flatten)] + backend_options: BackendClientOptions, + + /// If set, the required Origin for requests sent to the WebSocket server. + /// + /// When not set, the tunnel server allows requests from any Origin, which + /// could allow non-caBLE use of the server. + #[clap(long, value_hint = ValueHint::Hostname)] + origin: Option, + + #[clap(flatten)] + protocol: ServerTransportProtocol, + + /// Disables the `/debug` handler on the HTTP server. + #[clap(long = "no-debug-handler", action = ArgAction::SetFalse)] + debug_handler: bool, +} + +impl TryFrom<&Flags> for ServerState { + type Error = TlsConfigError; + + fn try_from(f: &Flags) -> Result { + Ok(Self { + origin: f.origin.to_owned(), + tls_domain: f.backend_options.domain.to_owned(), + backend_connector: f.backend_options.tls_connector()?, + backends: RwLock::new(HashMap::new()), + debug_handler: f.debug_handler, + }) + } +} + +#[instrument(level = "info", skip_all, fields( + req.method = req.method().to_string(), + req.path = req.uri().path(), + backend, +))] +async fn handle_request( + state: Arc, + _: SocketAddr, + req: Request, +) -> Result>, Infallible> { + trace!("request payload: {req:?}"); + + // Drop the request body from future manipulation, because we never use it. + let mut req = req.map(|_| ()); + + // Use our usual routing logic for static files, except ignore the crafted + // response for Websocket connections. + let mut path = match Router::route(&req, state.origin.as_deref()) { + Router::Static(res) => return Ok(res), + Router::Websocket(_, path) => path, + Router::Debug => { + return Ok(if state.debug_handler { + let backends_read = state.backends.read().await; + let backends_info: String = backends_read + .iter() + .map(|(k, v)| format!("backends[{}] = {v}\n", hex::encode_upper(k))) + .collect(); + let debug = format!( + "server_state.strong_count = {}\nbackends.capacity = {}\nbackends.len = {}\n{}", + Arc::strong_count(&state), + backends_read.capacity(), + backends_read.len(), + backends_info, + ); + + Response::builder() + .status(StatusCode::OK) + .header(CONTENT_TYPE, HeaderValue::from_static("text/plain")) + .body(Bytes::from(debug).into()) + .unwrap() + } else { + empty_response(StatusCode::NOT_FOUND) + }); + } + }; + + // Copy the incoming request to re-send to the backend. + let mut backend_req = copy_request_empty_body(&req); + if backend_req + .headers_mut() + .insert(FORWARDED_HEADER, HeaderValue::from_static("1")) + .is_some() + { + // The request has been forwarded by us once before. Refuse to get into + // a loop. + error!("request seems to be previously forwarded - refusing to get in a loop!"); + return Ok(empty_response(StatusCode::LOOP_DETECTED)); + } + // TODO: add standard proxy forwarding headers + + // Get the backend address, and set the Routing ID header in the request + let backend = match &mut path.method { + CableMethod::New => { + // TODO: select a backend + path.routing_id.copy_from_slice(&ROUTING_ID); + let headers = backend_req.headers_mut(); + + // Replace the Routing ID header sent to the backend with the + // selected routing ID. + path.insert_routing_id_header(headers); + + // TODO + state + .backends + .read() + .await + .get(&ROUTING_ID) + .unwrap() + .to_owned() + } + + CableMethod::Connect => { + // TODO handle availability + + match state.backends.read().await.get(&path.routing_id) { + Some(a) => a.to_owned(), + None => { + error!("unknown routing ID"); + return Ok(empty_response(StatusCode::NOT_FOUND)); + } + } + } + }; + tracing::Span::current().record("backend", backend.to_string()); + + // Connect to the backend task + info!("connecting to backend"); + let backend_socket = match TcpStream::connect(backend).await { + Ok(s) => s, + Err(e) => { + error!("unable to reach backend: {e}"); + return Ok(empty_response(StatusCode::GATEWAY_TIMEOUT)); + } + }; + + // Set up TLS + let backend_socket = match &state.backend_connector { + None => MaybeTlsStream::Plain(backend_socket), + Some(tls_connector) => { + let backend_ip = backend.ip().to_string(); + let domain = state.tls_domain.as_deref().unwrap_or(backend_ip.as_str()); + + trace!("TLS handshake with backend using domain {domain}"); + match tls_connector.connect(domain, backend_socket).await { + Ok(o) => MaybeTlsStream::NativeTls(o), + Err(e) => { + error!("TLS handshake with backend failed: {e}"); + return Ok(empty_response(StatusCode::BAD_GATEWAY)); + } + } + } + }; + + let (mut sender, conn) = match hyper::client::conn::http1::handshake(backend_socket).await { + Ok(v) => v, + Err(e) => { + error!("unable to handshake with backend: {e}"); + return Ok(empty_response(StatusCode::BAD_GATEWAY)); + } + }; + + // Spawn a task to poll the connection and drive the HTTP state + tokio::task::spawn(async move { + if let Err(e) = conn.await { + error!("backend connection failed: {e:?}"); + } + }); + + // Pass the request to the selected backend + let mut backend_res = match sender.send_request(backend_req).await { + Ok(r) => r, + Err(e) => { + error!("unable to send request to backend: {e}"); + return Ok(empty_response(StatusCode::BAD_GATEWAY)); + } + }; + + if backend_res.status() != StatusCode::SWITCHING_PROTOCOLS { + error!( + "backend returned unexpected status: {}", + backend_res.status() + ); + return Ok(empty_response(backend_res.status())); + } + + // Copy the response for the client + let mut res = copy_response_empty_body(&backend_res).map(|_| Default::default()); + path.insert_routing_id_header(res.headers_mut()); + + // Set up the "upgrade" handler to connect the two sockets together + tokio::task::spawn(async move { + // Upgrade the connection to the backend + let mut backend_upgraded = match hyper::upgrade::on(&mut backend_res).await { + Ok(u) => u, + Err(e) => { + error!("failure upgrading connection to backend: {e}"); + return; + } + }; + + // Upgrade the connection from the client + let mut client_upgraded = match hyper::upgrade::on(&mut req).await { + Ok(u) => u, + Err(e) => { + error!("failure upgrading connection to client: {e}"); + return; + } + }; + + // Connect the two streams together directly. + match tokio::io::copy_bidirectional(&mut backend_upgraded, &mut client_upgraded).await { + Ok((backend_bytes, client_bytes)) => { + info!("connection closed, backend sent {backend_bytes} bytes, client sent {client_bytes} bytes"); + } + Err(e) => { + error!("connection error: {e}"); + backend_upgraded.shutdown().await.ok(); + client_upgraded.shutdown().await.ok(); + } + } + }); + + Ok(res) +} + +#[tokio::main] +async fn main() -> Result<(), Box> { + setup_logging(); + let flags = Flags::parse(); + let server_state = ServerState::try_from(&flags)?; + + // TODO: implement properly + assert_ne!( + flags.bind_address, flags.backend_address, + "--bind-address cannot not be the same as --backend--address" + ); + server_state + .backends + .write() + .await + .insert(ROUTING_ID, flags.backend_address); + let tls_acceptor = flags.protocol.tls_acceptor()?; + info!( + "Starting webauthn-rs cable-tunnel-server-frontend at {}", + flags.protocol.uri(&flags.bind_address)? + ); + + run_server( + flags.bind_address, + tls_acceptor, + server_state, + handle_request, + ) + .await?; + + Ok(()) +}