public inbox for pbs-devel@lists.proxmox.com
 help / color / mirror / Atom feed
* [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection
@ 2023-06-22  9:15 Max Carrara
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder` Max Carrara
                   ` (4 more replies)
  0 siblings, 5 replies; 12+ messages in thread
From: Max Carrara @ 2023-06-22  9:15 UTC (permalink / raw)
  To: pbs-devel

This patch series adds support for redirecting HTTP requests to HTTPS. This is
done by peeking into the incoming TCP stream's buffer in order to check whether
the client initiates a TLS handshake or not.

Secure and insecure connections are then sent along separate channels, allowing
them to be handled by different `hyper::Server`s (if so desired).

Finally, `proxmox-backup-proxy` is adapted correspondingly, making it a little
more convenient to access PBS via its web interface.

proxmox:

Max Carrara (2):
  rest-server: Add `BiAcceptBuilder`
  rest-server: Add `Redirector`

 proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
 proxmox-rest-server/src/lib.rs        |   2 +-
 proxmox-rest-server/src/rest.rs       |  76 ++++++
 3 files changed, 404 insertions(+), 1 deletion(-)


proxmox-backup:

Max Carrara (1):
  proxy: redirect HTTP requests to HTTPS

 src/bin/proxmox-backup-proxy.rs | 45 ++++++++++++++++++++++++++++-----
 1 file changed, 38 insertions(+), 7 deletions(-)

-- 
2.30.2





^ permalink raw reply	[flat|nested] 12+ messages in thread

* [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder`
  2023-06-22  9:15 [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
@ 2023-06-22  9:15 ` Max Carrara
  2023-07-14  9:20   ` Wolfgang Bumiller
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector` Max Carrara
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 12+ messages in thread
From: Max Carrara @ 2023-06-22  9:15 UTC (permalink / raw)
  To: pbs-devel

This builder is similar to `AcceptBuilder`, but is also able to differ
between plain TCP streams and TCP streams running TLS.

It does so by peeking into the stream's buffer and checking whether
the client is initiating a TLS handshake.

Newly accepted plain TCP streams are sent along via a separate channel
in order to clearly distinguish between "secure" and "insecure"
connections.

Signed-off-by: Max Carrara <m.carrara@proxmox.com>
---
 proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
 1 file changed, 327 insertions(+)

diff --git a/proxmox-rest-server/src/connection.rs b/proxmox-rest-server/src/connection.rs
index 7681f00..937b5d7 100644
--- a/proxmox-rest-server/src/connection.rs
+++ b/proxmox-rest-server/src/connection.rs
@@ -302,3 +302,330 @@ impl AcceptBuilder {
         }
     }
 }
+
+#[cfg(feature = "rate-limited-stream")]
+type InsecureClientStreamResult = Pin<Box<RateLimitedStream<TcpStream>>>;
+#[cfg(not(feature = "rate-limited-stream"))]
+type InsecureClientStreamResult = Pin<Box<TcpStream>>;
+
+#[cfg(feature = "rate-limited-stream")]
+type ClientStream = RateLimitedStream<TcpStream>;
+
+#[cfg(not(feature = "rate-limited-stream"))]
+type ClientStream = TcpStream;
+
+pub struct BiAcceptBuilder {
+    acceptor: Option<Arc<Mutex<SslAcceptor>>>,
+    debug: bool,
+    tcp_keepalive_time: u32,
+    max_pending_accepts: usize,
+
+    #[cfg(feature = "rate-limited-stream")]
+    lookup_rate_limiter: Option<Arc<LookupRateLimiter>>,
+}
+
+impl Default for BiAcceptBuilder {
+    fn default() -> Self {
+        Self {
+            acceptor: None,
+            debug: false,
+            tcp_keepalive_time: 120,
+            max_pending_accepts: 1024,
+            #[cfg(feature = "rate-limited-stream")]
+            lookup_rate_limiter: None,
+        }
+    }
+}
+
+impl BiAcceptBuilder {
+    pub fn new() -> Self {
+        Default::default()
+    }
+
+    pub fn with_acceptor(acceptor: Arc<Mutex<SslAcceptor>>) -> Self {
+        Self {
+            acceptor: Some(acceptor),
+            ..Default::default()
+        }
+    }
+
+    pub fn debug(mut self, debug: bool) -> Self {
+        self.debug = debug;
+        self
+    }
+
+    pub fn tcp_keepalive_time(mut self, time: u32) -> Self {
+        self.tcp_keepalive_time = time;
+        self
+    }
+
+    pub fn max_pending_accepts(mut self, count: usize) -> Self {
+        self.max_pending_accepts = count;
+        self
+    }
+
+    #[cfg(feature = "rate-limited-stream")]
+    pub fn rate_limiter_lookup(mut self, lookup_rate_limiter: Arc<LookupRateLimiter>) -> Self {
+        self.lookup_rate_limiter = Some(lookup_rate_limiter);
+        self
+    }
+
+    pub fn accept(
+        self,
+        listener: TcpListener,
+    ) -> (
+        impl hyper::server::accept::Accept<Conn = ClientStreamResult, Error = Error>,
+        impl hyper::server::accept::Accept<Conn = InsecureClientStreamResult, Error = Error>,
+    ) {
+        use hyper::server::accept;
+        use tokio::sync::mpsc::channel;
+
+        let (secure_sender, secure_receiver) = channel(self.max_pending_accepts);
+        let (insecure_sender, insecure_receiver) = channel(self.max_pending_accepts);
+
+        tokio::spawn(self.accept_connections(listener, secure_sender, insecure_sender));
+
+        (
+            accept::from_stream(ReceiverStream::new(secure_receiver)),
+            accept::from_stream(ReceiverStream::new(insecure_receiver)),
+        )
+    }
+
+    async fn accept_connections(
+        self,
+        listener: TcpListener,
+        secure_sender: tokio::sync::mpsc::Sender<Result<ClientStreamResult, Error>>,
+        insecure_sender: tokio::sync::mpsc::Sender<Result<InsecureClientStreamResult, Error>>,
+    ) {
+        let accept_counter = Arc::new(());
+        let mut shutdown_future = crate::shutdown_future().fuse();
+
+        loop {
+            let (sock, peer) = futures::select! {
+                res = listener.accept().fuse() => match res {
+                    Ok(conn) => conn,
+                    Err(err) => {
+                        log::error!("error accepting tcp connection: {err}");
+                        continue;
+                    }
+                },
+                _ = shutdown_future => break,
+            };
+
+            #[cfg(not(feature = "rate-limited-stream"))]
+            drop(peer);
+
+            sock.set_nodelay(true).unwrap();
+
+            let _ = proxmox_sys::linux::socket::set_tcp_keepalive(
+                sock.as_raw_fd(),
+                self.tcp_keepalive_time,
+            );
+
+            #[cfg(feature = "rate-limited-stream")]
+            let sock = match self.lookup_rate_limiter.clone() {
+                Some(lookup) => {
+                    RateLimitedStream::with_limiter_update_cb(sock, move || lookup(peer))
+                }
+                None => RateLimitedStream::with_limiter(sock, None, None),
+            };
+
+            let accept_counter = Arc::clone(&accept_counter);
+
+            if Arc::strong_count(&accept_counter) > self.max_pending_accepts {
+                log::error!("connection rejected - too many open connections");
+                continue;
+            }
+
+            let acceptor = self.acceptor.clone();
+
+            // prevents the loop from being blocked if many plain TCP
+            // connections are being opened
+            let accept_future = Self::do_accept(
+                sock,
+                acceptor,
+                accept_counter,
+                secure_sender.clone(),
+                insecure_sender.clone(),
+                self.debug,
+            );
+
+            tokio::spawn(accept_future);
+        }
+    }
+
+    async fn do_accept(
+        sock: ClientStream,
+        acceptor: Option<Arc<Mutex<SslAcceptor>>>,
+        accept_counter: Arc<()>,
+        secure_sender: tokio::sync::mpsc::Sender<Result<ClientStreamResult, Error>>,
+        insecure_sender: tokio::sync::mpsc::Sender<Result<InsecureClientStreamResult, Error>>,
+        debug: bool,
+    ) {
+        #[inline(always)]
+        fn send_insecure(
+            sock: ClientStream,
+            insecure_sender: tokio::sync::mpsc::Sender<Result<InsecureClientStreamResult, Error>>,
+            accept_counter: Arc<()>,
+            debug: bool,
+        ) {
+            let insecure_stream = Box::pin(sock);
+
+            tokio::spawn(async move {
+                if insecure_sender.send(Ok(insecure_stream)).await.is_err() && debug {
+                    log::error!("detected closed connection channel")
+                };
+
+                drop(accept_counter);
+            });
+        }
+
+        if acceptor.is_none() {
+            send_insecure(sock, insecure_sender, accept_counter, debug);
+            return;
+        }
+
+        let client_initiates_handshake = {
+            #[cfg(feature = "rate-limited-stream")]
+            let sock = sock.inner();
+
+            #[cfg(not(feature = "rate-limited-stream"))]
+            let sock = &sock;
+
+            match Self::wait_for_client_tls_handshake(sock).await {
+                Ok(initiates_handshake) => initiates_handshake,
+                Err(err) => {
+                    log::error!("error checking for TLS handshake: {err}");
+                    return;
+                }
+            }
+        };
+
+        if !client_initiates_handshake {
+            send_insecure(sock, insecure_sender, accept_counter, debug);
+            return;
+        }
+
+        let ssl = {
+            // limit acceptor_guard scope
+            // Acceptor can be reloaded using the command socket "reload-certificate" command
+            let acceptor_guard = acceptor.as_ref().unwrap().lock().unwrap();
+
+            match openssl::ssl::Ssl::new(acceptor_guard.context()) {
+                Ok(ssl) => ssl,
+                Err(err) => {
+                    log::error!("failed to create Ssl object from Acceptor context - {err}");
+                    return;
+                }
+            }
+        };
+
+        let secure_stream = match tokio_openssl::SslStream::new(ssl, sock) {
+            Ok(stream) => stream,
+            Err(err) => {
+                log::error!("failed to create SslStream using ssl and connection socket - {err}");
+                return;
+            }
+        };
+
+        let mut secure_stream = Box::pin(secure_stream);
+        let secure_sender = secure_sender.clone();
+
+        tokio::spawn(async move {
+            let accept_future =
+                tokio::time::timeout(Duration::new(10, 0), secure_stream.as_mut().accept());
+
+            let result = accept_future.await;
+
+            match result {
+                Ok(Ok(())) => {
+                    if secure_sender.send(Ok(secure_stream)).await.is_err() && debug {
+                        log::error!("detected closed connection channel");
+                    }
+                }
+                Ok(Err(err)) => {
+                    if debug {
+                        log::error!("https handshake failed - {err}");
+                    }
+                }
+                Err(_) => {
+                    if debug {
+                        log::error!("https handshake timeout");
+                    }
+                }
+            }
+
+            drop(accept_counter); // decrease reference count
+        });
+    }
+
+    async fn wait_for_client_tls_handshake(incoming_stream: &TcpStream) -> Result<bool, Error> {
+        const MS_TIMEOUT: u64 = 1000;
+        const BYTES_BUF_SIZE: usize = 128;
+
+        let mut buf = [0; BYTES_BUF_SIZE];
+        let mut last_peek_size = 0;
+
+        let future = async {
+            loop {
+                let peek_size = incoming_stream
+                    .peek(&mut buf)
+                    .await
+                    .context("couldn't peek into incoming tcp stream")?;
+
+                if contains_tls_handshake_fragment(&buf) {
+                    return Ok(true);
+                }
+
+                // No more new data came in
+                if peek_size == last_peek_size {
+                    return Ok(false);
+                }
+
+                last_peek_size = peek_size;
+
+                // yields to event loop; this future blocks otherwise ad infinitum
+                tokio::time::sleep(Duration::from_millis(0)).await;
+            }
+        };
+
+        tokio::time::timeout(Duration::from_millis(MS_TIMEOUT), future)
+            .await
+            .unwrap_or(Ok(false))
+    }
+}
+
+/// Checks whether an [SSL 3.0 / TLS plaintext fragment][0] being part of a
+/// SSL / TLS handshake is contained in the given buffer.
+///
+/// Such a fragment might look as follows:
+/// ```ignore
+/// [0x16, 0x3, 0x1, 0x02, 0x00, ...]
+/// //  |    |    |     |_____|
+/// //  |    |    |            \__ content length interpreted as u16
+/// //  |    |    |                must not exceed 0x4000 (2^14) bytes
+/// //  |    |    |
+/// //  |    |     \__ any minor version
+/// //  |    |
+/// //  |     \__ major version 3
+/// //  |
+/// //   \__ content type is handshake(22)
+/// ```
+///
+/// If a slice like this is detected at the beginning of the given buffer,
+/// a TLS handshake is most definitely being made.
+///
+/// [0]: https://datatracker.ietf.org/doc/html/rfc6101#section-5.2
+#[inline]
+fn contains_tls_handshake_fragment(buf: &[u8]) -> bool {
+    const SLICE_LENGTH: usize = 5;
+    const CONTENT_SIZE: u16 = 1 << 14; // max length of a TLS plaintext fragment
+
+    if buf.len() < SLICE_LENGTH {
+        return false;
+    }
+
+    return buf[0] == 0x16
+        && buf[1] == 0x3
+        && (((buf[3] as u16) << 8) + buf[4] as u16) <= CONTENT_SIZE;
+}
-- 
2.30.2





^ permalink raw reply	[flat|nested] 12+ messages in thread

* [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector`
  2023-06-22  9:15 [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder` Max Carrara
@ 2023-06-22  9:15 ` Max Carrara
  2023-07-14  9:24   ` Wolfgang Bumiller
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox-backup 3/3] proxy: redirect HTTP requests to HTTPS Max Carrara
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 12+ messages in thread
From: Max Carrara @ 2023-06-22  9:15 UTC (permalink / raw)
  To: pbs-devel

The `Redirector` is a simple `Service` that redirects HTTP requests
to HTTPS and can be served by a `hyper::Server`.

Signed-off-by: Max Carrara <m.carrara@proxmox.com>
---
 proxmox-rest-server/src/lib.rs  |  2 +-
 proxmox-rest-server/src/rest.rs | 76 +++++++++++++++++++++++++++++++++
 2 files changed, 77 insertions(+), 1 deletion(-)

diff --git a/proxmox-rest-server/src/lib.rs b/proxmox-rest-server/src/lib.rs
index bc5be01..1c64ffb 100644
--- a/proxmox-rest-server/src/lib.rs
+++ b/proxmox-rest-server/src/lib.rs
@@ -48,7 +48,7 @@ mod api_config;
 pub use api_config::{ApiConfig, AuthError, AuthHandler, IndexHandler};
 
 mod rest;
-pub use rest::RestServer;
+pub use rest::{Redirector, RestServer};
 
 pub mod connection;
 
diff --git a/proxmox-rest-server/src/rest.rs b/proxmox-rest-server/src/rest.rs
index 100c93c..2584e96 100644
--- a/proxmox-rest-server/src/rest.rs
+++ b/proxmox-rest-server/src/rest.rs
@@ -97,6 +97,82 @@ impl<T: PeerAddress> Service<&T> for RestServer {
     }
 }
 
+pub struct Redirector {}
+
+impl Redirector {
+    pub fn new() -> Self {
+        Self {}
+    }
+}
+
+impl<T: PeerAddress> Service<&T> for Redirector {
+    type Response = RedirectService;
+    type Error = Error;
+    type Future = std::future::Ready<Result<Self::Response, Self::Error>>;
+
+    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
+        Poll::Ready(Ok(()))
+    }
+
+    fn call(&mut self, ctx: &T) -> Self::Future {
+        std::future::ready(match ctx.peer_addr() {
+            Err(err) => Err(format_err!("unable to get peer address - {err}")),
+            Ok(peer) => Ok(RedirectService { peer }),
+        })
+    }
+}
+
+pub struct RedirectService {
+    pub peer: std::net::SocketAddr,
+}
+
+impl Service<Request<Body>> for RedirectService {
+    type Response = Response<Body>;
+    type Error = anyhow::Error;
+    type Future = Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send>>;
+
+    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
+        Poll::Ready(Ok(()))
+    }
+
+    fn call(&mut self, req: Request<Body>) -> Self::Future {
+        lazy_static! {
+            static ref RE: Regex = Regex::new(r"((http(s)?://)|^)").unwrap();
+        }
+
+        let future = async move {
+            let header_host_value = req
+                .headers()
+                .get("host")
+                .and_then(|value| value.to_str().ok());
+
+            let response = if let Some(value) = header_host_value {
+                let location_value = RE.replace(value, "https://");
+
+                let status_code = if matches!(*req.method(), http::Method::GET | http::Method::HEAD)
+                {
+                    StatusCode::MOVED_PERMANENTLY
+                } else {
+                    StatusCode::PERMANENT_REDIRECT
+                };
+
+                Response::builder()
+                    .status(status_code)
+                    .header("Location", String::from(location_value))
+                    .body(Body::empty())?
+            } else {
+                Response::builder()
+                    .status(StatusCode::BAD_REQUEST)
+                    .body(Body::empty())?
+            };
+
+            Ok(response)
+        };
+
+        future.boxed()
+    }
+}
+
 pub trait PeerAddress {
     fn peer_addr(&self) -> Result<std::net::SocketAddr, Error>;
 }
-- 
2.30.2





^ permalink raw reply	[flat|nested] 12+ messages in thread

* [pbs-devel] [PATCH proxmox-backup 3/3] proxy: redirect HTTP requests to HTTPS
  2023-06-22  9:15 [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder` Max Carrara
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector` Max Carrara
@ 2023-06-22  9:15 ` Max Carrara
  2023-06-23 10:15 ` [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
  2023-06-23 11:40 ` Lukas Wagner
  4 siblings, 0 replies; 12+ messages in thread
From: Max Carrara @ 2023-06-22  9:15 UTC (permalink / raw)
  To: pbs-devel

Signed-off-by: Max Carrara <m.carrara@proxmox.com>
---
 src/bin/proxmox-backup-proxy.rs | 45 ++++++++++++++++++++++++++++-----
 1 file changed, 38 insertions(+), 7 deletions(-)

diff --git a/src/bin/proxmox-backup-proxy.rs b/src/bin/proxmox-backup-proxy.rs
index f38a02bd..a5187bcb 100644
--- a/src/bin/proxmox-backup-proxy.rs
+++ b/src/bin/proxmox-backup-proxy.rs
@@ -23,8 +23,8 @@ use proxmox_sys::{task_log, task_warn};
 use pbs_datastore::DataStore;
 
 use proxmox_rest_server::{
-    cleanup_old_tasks, cookie_from_header, rotate_task_log_archive, ApiConfig, RestEnvironment,
-    RestServer, WorkerTask,
+    cleanup_old_tasks, cookie_from_header, rotate_task_log_archive, ApiConfig, Redirector,
+    RestEnvironment, RestServer, WorkerTask,
 };
 
 use proxmox_backup::rrd_cache::{
@@ -253,6 +253,7 @@ async fn run() -> Result<(), Error> {
         )?;
 
     let rest_server = RestServer::new(config);
+    let redirector = Redirector::new();
     proxmox_rest_server::init_worker_tasks(
         pbs_buildcfg::PROXMOX_BACKUP_LOG_DIR_M!().into(),
         file_opts.clone(),
@@ -288,23 +289,53 @@ async fn run() -> Result<(), Error> {
         Ok(Value::Null)
     })?;
 
-    let connections = proxmox_rest_server::connection::AcceptBuilder::with_acceptor(acceptor)
+    let connections = proxmox_rest_server::connection::BiAcceptBuilder::with_acceptor(acceptor)
         .debug(debug)
         .rate_limiter_lookup(Arc::new(lookup_rate_limiter))
         .tcp_keepalive_time(PROXMOX_BACKUP_TCP_KEEPALIVE_TIME);
+
     let server = daemon::create_daemon(
         ([0, 0, 0, 0, 0, 0, 0, 0], 8007).into(),
         move |listener| {
-            let connections = connections.accept(listener);
+            let (secure_connections, insecure_connections) = connections.accept(listener);
 
             Ok(async {
                 daemon::systemd_notify(daemon::SystemdNotify::Ready)?;
 
-                hyper::Server::builder(connections)
+                let secure_server = hyper::Server::builder(secure_connections)
                     .serve(rest_server)
                     .with_graceful_shutdown(proxmox_rest_server::shutdown_future())
-                    .map_err(Error::from)
-                    .await
+                    .map_err(Error::from);
+
+                let insecure_server = hyper::Server::builder(insecure_connections)
+                    .serve(redirector)
+                    .with_graceful_shutdown(proxmox_rest_server::shutdown_future())
+                    .map_err(Error::from);
+
+                let handles = vec![tokio::spawn(secure_server), tokio::spawn(insecure_server)];
+
+                let mut results: Vec<Result<(), Error>> = vec![];
+
+                for res_handle in futures::future::join_all(handles).await.into_iter() {
+                    let flattened_res = match res_handle {
+                        Ok(inner) => inner,
+                        Err(err) => Err(format_err!(err)),
+                    };
+
+                    results.push(flattened_res);
+                }
+
+                if results.iter().any(Result::is_err) {
+                    let cat_errors = results
+                        .into_iter()
+                        .filter_map(|res| res.err().map(|err| err.to_string()))
+                        .collect::<Vec<_>>()
+                        .join("\n");
+
+                    return Err(format_err!(cat_errors));
+                }
+
+                Ok(())
             })
         },
         Some(pbs_buildcfg::PROXMOX_BACKUP_PROXY_PID_FN),
-- 
2.30.2





^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection
  2023-06-22  9:15 [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
                   ` (2 preceding siblings ...)
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox-backup 3/3] proxy: redirect HTTP requests to HTTPS Max Carrara
@ 2023-06-23 10:15 ` Max Carrara
  2023-06-23 10:55   ` Thomas Lamprecht
  2023-06-23 11:40 ` Lukas Wagner
  4 siblings, 1 reply; 12+ messages in thread
From: Max Carrara @ 2023-06-23 10:15 UTC (permalink / raw)
  To: pbs-devel

On 6/22/23 11:15, Max Carrara wrote:
> This patch series adds support for redirecting HTTP requests to HTTPS. This is
> done by peeking into the incoming TCP stream's buffer in order to check whether
> the client initiates a TLS handshake or not.
> 
> Secure and insecure connections are then sent along separate channels, allowing
> them to be handled by different `hyper::Server`s (if so desired).
> 
> Finally, `proxmox-backup-proxy` is adapted correspondingly, making it a little
> more convenient to access PBS via its web interface.
> 
> proxmox:
> 
> Max Carrara (2):
>   rest-server: Add `BiAcceptBuilder`
>   rest-server: Add `Redirector`
> 
>  proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
>  proxmox-rest-server/src/lib.rs        |   2 +-
>  proxmox-rest-server/src/rest.rs       |  76 ++++++
>  3 files changed, 404 insertions(+), 1 deletion(-)
> 
> 
> proxmox-backup:
> 
> Max Carrara (1):
>   proxy: redirect HTTP requests to HTTPS
> 
>  src/bin/proxmox-backup-proxy.rs | 45 ++++++++++++++++++++++++++++-----
>  1 file changed, 38 insertions(+), 7 deletions(-)
> 

Note: This series was built and tested on the `stable-2` branches of
either repo, but applies on both `master` branches as well.




^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection
  2023-06-23 10:15 ` [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
@ 2023-06-23 10:55   ` Thomas Lamprecht
  2023-06-27  9:39     ` Max Carrara
  0 siblings, 1 reply; 12+ messages in thread
From: Thomas Lamprecht @ 2023-06-23 10:55 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion, Max Carrara

Am 23/06/2023 um 12:15 schrieb Max Carrara:
> On 6/22/23 11:15, Max Carrara wrote:
>> This patch series adds support for redirecting HTTP requests to HTTPS. This is
>> done by peeking into the incoming TCP stream's buffer in order to check whether
>> the client initiates a TLS handshake or not.
>>
>> Secure and insecure connections are then sent along separate channels, allowing
>> them to be handled by different `hyper::Server`s (if so desired).
>>
>> Finally, `proxmox-backup-proxy` is adapted correspondingly, making it a little
>> more convenient to access PBS via its web interface.
>>
>> proxmox:
>>
>> Max Carrara (2):
>>   rest-server: Add `BiAcceptBuilder`
>>   rest-server: Add `Redirector`
>>
>>  proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
>>  proxmox-rest-server/src/lib.rs        |   2 +-
>>  proxmox-rest-server/src/rest.rs       |  76 ++++++
>>  3 files changed, 404 insertions(+), 1 deletion(-)
>>
>>
>> proxmox-backup:
>>
>> Max Carrara (1):
>>   proxy: redirect HTTP requests to HTTPS
>>
>>  src/bin/proxmox-backup-proxy.rs | 45 ++++++++++++++++++++++++++++-----
>>  1 file changed, 38 insertions(+), 7 deletions(-)
>>
> 
> Note: This series was built and tested on the `stable-2` branches of
> either repo, but applies on both `master` branches as well.
> 

Ok but I'd rather only apply this to master. This isn't a clear targeted bug fix,
so rather would keep away any potential regression from the stable branch.





^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection
  2023-06-22  9:15 [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
                   ` (3 preceding siblings ...)
  2023-06-23 10:15 ` [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
@ 2023-06-23 11:40 ` Lukas Wagner
  4 siblings, 0 replies; 12+ messages in thread
From: Lukas Wagner @ 2023-06-23 11:40 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion, Max Carrara

Hello,

tested this on the current master branch.
Navigated to `http://<ip>:8007`, redirect to `https://<ip>:8007` works as expected.

Code LGTM.

Consider this:

Tested-by: Lukas Wagner <l.wagner@proxmox.com>
Reviewed-by: Lukas Wagner <l.wagner@proxmox.com>

On 6/22/23 11:15, Max Carrara wrote:
> This patch series adds support for redirecting HTTP requests to HTTPS. This is
> done by peeking into the incoming TCP stream's buffer in order to check whether
> the client initiates a TLS handshake or not.
> 
> Secure and insecure connections are then sent along separate channels, allowing
> them to be handled by different `hyper::Server`s (if so desired).
> 
> Finally, `proxmox-backup-proxy` is adapted correspondingly, making it a little
> more convenient to access PBS via its web interface.
> 
> proxmox:
> 
> Max Carrara (2):
>    rest-server: Add `BiAcceptBuilder`
>    rest-server: Add `Redirector`
> 
>   proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
>   proxmox-rest-server/src/lib.rs        |   2 +-
>   proxmox-rest-server/src/rest.rs       |  76 ++++++
>   3 files changed, 404 insertions(+), 1 deletion(-)
> 
> 
> proxmox-backup:
> 
> Max Carrara (1):
>    proxy: redirect HTTP requests to HTTPS
> 
>   src/bin/proxmox-backup-proxy.rs | 45 ++++++++++++++++++++++++++++-----
>   1 file changed, 38 insertions(+), 7 deletions(-)
> 

-- 
- Lukas




^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection
  2023-06-23 10:55   ` Thomas Lamprecht
@ 2023-06-27  9:39     ` Max Carrara
  0 siblings, 0 replies; 12+ messages in thread
From: Max Carrara @ 2023-06-27  9:39 UTC (permalink / raw)
  To: Thomas Lamprecht, Proxmox Backup Server development discussion

On 6/23/23 12:55, Thomas Lamprecht wrote:
> Am 23/06/2023 um 12:15 schrieb Max Carrara:
>> On 6/22/23 11:15, Max Carrara wrote:
>>> This patch series adds support for redirecting HTTP requests to HTTPS. This is
>>> done by peeking into the incoming TCP stream's buffer in order to check whether
>>> the client initiates a TLS handshake or not.
>>>
>>> Secure and insecure connections are then sent along separate channels, allowing
>>> them to be handled by different `hyper::Server`s (if so desired).
>>>
>>> Finally, `proxmox-backup-proxy` is adapted correspondingly, making it a little
>>> more convenient to access PBS via its web interface.
>>>
>>> proxmox:
>>>
>>> Max Carrara (2):
>>>   rest-server: Add `BiAcceptBuilder`
>>>   rest-server: Add `Redirector`
>>>
>>>  proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
>>>  proxmox-rest-server/src/lib.rs        |   2 +-
>>>  proxmox-rest-server/src/rest.rs       |  76 ++++++
>>>  3 files changed, 404 insertions(+), 1 deletion(-)
>>>
>>>
>>> proxmox-backup:
>>>
>>> Max Carrara (1):
>>>   proxy: redirect HTTP requests to HTTPS
>>>
>>>  src/bin/proxmox-backup-proxy.rs | 45 ++++++++++++++++++++++++++++-----
>>>  1 file changed, 38 insertions(+), 7 deletions(-)
>>>
>>
>> Note: This series was built and tested on the `stable-2` branches of
>> either repo, but applies on both `master` branches as well.
>>
> 
> Ok but I'd rather only apply this to master. This isn't a clear targeted bug fix,
> so rather would keep away any potential regression from the stable branch.
> 

FYI: just built and tested it on master; works as as of commit
54cb9be8edba5e78071275df892516742fa26a75

Just wanted to make sure myself, even though Lukas already tested it
(thanks!)




^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder`
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder` Max Carrara
@ 2023-07-14  9:20   ` Wolfgang Bumiller
  2023-07-18  5:46     ` Max Carrara
  0 siblings, 1 reply; 12+ messages in thread
From: Wolfgang Bumiller @ 2023-07-14  9:20 UTC (permalink / raw)
  To: Max Carrara; +Cc: pbs-devel

On Thu, Jun 22, 2023 at 11:15:24AM +0200, Max Carrara wrote:
> This builder is similar to `AcceptBuilder`, but is also able to differ
> between plain TCP streams and TCP streams running TLS.
> 
> It does so by peeking into the stream's buffer and checking whether
> the client is initiating a TLS handshake.
> 
> Newly accepted plain TCP streams are sent along via a separate channel
> in order to clearly distinguish between "secure" and "insecure"
> connections.
> 
> Signed-off-by: Max Carrara <m.carrara@proxmox.com>
> ---
>  proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
>  1 file changed, 327 insertions(+)
> 
> diff --git a/proxmox-rest-server/src/connection.rs b/proxmox-rest-server/src/connection.rs
> index 7681f00..937b5d7 100644
> --- a/proxmox-rest-server/src/connection.rs
> +++ b/proxmox-rest-server/src/connection.rs
> @@ -302,3 +302,330 @@ impl AcceptBuilder {
>          }
>      }
>  }
> +
> +#[cfg(feature = "rate-limited-stream")]
> +type InsecureClientStreamResult = Pin<Box<RateLimitedStream<TcpStream>>>;
> +#[cfg(not(feature = "rate-limited-stream"))]
> +type InsecureClientStreamResult = Pin<Box<TcpStream>>;

^ You can drop one set of `#[cfg]`s by using `Pin<Box<ClientStream>>` ;-)

> +
> +#[cfg(feature = "rate-limited-stream")]
> +type ClientStream = RateLimitedStream<TcpStream>;
> +
> +#[cfg(not(feature = "rate-limited-stream"))]
> +type ClientStream = TcpStream;
> +
> +pub struct BiAcceptBuilder {
> +    acceptor: Option<Arc<Mutex<SslAcceptor>>>,
> +    debug: bool,
> +    tcp_keepalive_time: u32,
> +    max_pending_accepts: usize,
> +
> +    #[cfg(feature = "rate-limited-stream")]
> +    lookup_rate_limiter: Option<Arc<LookupRateLimiter>>,
> +}

90% of this whole thing is a copy of `AcceptBuilder`.
I'd argue that we should be able to instead add this version's
`accept()` method to the regular `AcceptBuilder` as another variant with
a different name, eg. `accept_with_tls_optional()`.

The `accept_connections()` task AFAICT is also just the original split
in 2 with the tls check in between. It should be fine to just change the
original to this with the tls check covered by whether an
`Option<Sender<Insecure...>>` is `Some`.

Otherwise we're just duplicating too much.

The only other change is that the tls acceptor is now optional.
Do we even have a use case for where we need potentially-rate-limited
non-tls streams?
If so, this could also be another accept method.

In fact, given the point where the acceptor is actually used, perhaps we
should drop it from the struct entirely and instead pass it along to the
`accept()` methods:
- accept_tls(acceptor) -> stream
- accept_optional_tls(acceptor) -> (tls stream, insecure stream)
- accept_direct(acceptor) -> nontls-stream

^ dropping the original `accept` on purpose ensure all crate users get
updated accordingly




^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector`
  2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector` Max Carrara
@ 2023-07-14  9:24   ` Wolfgang Bumiller
  2023-07-18  5:59     ` Max Carrara
  0 siblings, 1 reply; 12+ messages in thread
From: Wolfgang Bumiller @ 2023-07-14  9:24 UTC (permalink / raw)
  To: Max Carrara; +Cc: pbs-devel

On Thu, Jun 22, 2023 at 11:15:25AM +0200, Max Carrara wrote:
> The `Redirector` is a simple `Service` that redirects HTTP requests
> to HTTPS and can be served by a `hyper::Server`.
> 
> Signed-off-by: Max Carrara <m.carrara@proxmox.com>
> ---
>  proxmox-rest-server/src/lib.rs  |  2 +-
>  proxmox-rest-server/src/rest.rs | 76 +++++++++++++++++++++++++++++++++
>  2 files changed, 77 insertions(+), 1 deletion(-)
> 
> diff --git a/proxmox-rest-server/src/lib.rs b/proxmox-rest-server/src/lib.rs
> index bc5be01..1c64ffb 100644
> --- a/proxmox-rest-server/src/lib.rs
> +++ b/proxmox-rest-server/src/lib.rs
> @@ -48,7 +48,7 @@ mod api_config;
>  pub use api_config::{ApiConfig, AuthError, AuthHandler, IndexHandler};
>  
>  mod rest;
> -pub use rest::RestServer;
> +pub use rest::{Redirector, RestServer};
>  
>  pub mod connection;
>  
> diff --git a/proxmox-rest-server/src/rest.rs b/proxmox-rest-server/src/rest.rs
> index 100c93c..2584e96 100644
> --- a/proxmox-rest-server/src/rest.rs
> +++ b/proxmox-rest-server/src/rest.rs
> @@ -97,6 +97,82 @@ impl<T: PeerAddress> Service<&T> for RestServer {
>      }
>  }
>  
> +pub struct Redirector {}
> +
> +impl Redirector {
> +    pub fn new() -> Self {
> +        Self {}
> +    }
> +}
> +
> +impl<T: PeerAddress> Service<&T> for Redirector {
> +    type Response = RedirectService;
> +    type Error = Error;
> +    type Future = std::future::Ready<Result<Self::Response, Self::Error>>;
> +
> +    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
> +        Poll::Ready(Ok(()))
> +    }
> +
> +    fn call(&mut self, ctx: &T) -> Self::Future {
> +        std::future::ready(match ctx.peer_addr() {

^ In theory we don't even need to bother with the address (although it's
unlikely to fail), since we never use it in the RedirectService?
Shouldn't RedirectService work just fine as a ZST? :-)

> +            Err(err) => Err(format_err!("unable to get peer address - {err}")),
> +            Ok(peer) => Ok(RedirectService { peer }),
> +        })
> +    }
> +}
> +
> +pub struct RedirectService {
> +    pub peer: std::net::SocketAddr,
> +}
> +
> +impl Service<Request<Body>> for RedirectService {
> +    type Response = Response<Body>;
> +    type Error = anyhow::Error;
> +    type Future = Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send>>;
> +
> +    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
> +        Poll::Ready(Ok(()))
> +    }
> +
> +    fn call(&mut self, req: Request<Body>) -> Self::Future {
> +        lazy_static! {
> +            static ref RE: Regex = Regex::new(r"((http(s)?://)|^)").unwrap();
> +        }
> +
> +        let future = async move {
> +            let header_host_value = req
> +                .headers()
> +                .get("host")
> +                .and_then(|value| value.to_str().ok());
> +
> +            let response = if let Some(value) = header_host_value {
> +                let location_value = RE.replace(value, "https://");
> +
> +                let status_code = if matches!(*req.method(), http::Method::GET | http::Method::HEAD)
> +                {
> +                    StatusCode::MOVED_PERMANENTLY
> +                } else {
> +                    StatusCode::PERMANENT_REDIRECT
> +                };
> +
> +                Response::builder()
> +                    .status(status_code)
> +                    .header("Location", String::from(location_value))
> +                    .body(Body::empty())?
> +            } else {
> +                Response::builder()
> +                    .status(StatusCode::BAD_REQUEST)
> +                    .body(Body::empty())?
> +            };
> +
> +            Ok(response)
> +        };
> +
> +        future.boxed()
> +    }
> +}
> +
>  pub trait PeerAddress {
>      fn peer_addr(&self) -> Result<std::net::SocketAddr, Error>;
>  }
> -- 
> 2.30.2
> 
> 
> 
> _______________________________________________
> pbs-devel mailing list
> pbs-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
> 
> 




^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder`
  2023-07-14  9:20   ` Wolfgang Bumiller
@ 2023-07-18  5:46     ` Max Carrara
  0 siblings, 0 replies; 12+ messages in thread
From: Max Carrara @ 2023-07-18  5:46 UTC (permalink / raw)
  To: Wolfgang Bumiller; +Cc: pbs-devel

On 7/14/23 11:20, Wolfgang Bumiller wrote:
> On Thu, Jun 22, 2023 at 11:15:24AM +0200, Max Carrara wrote:
>> This builder is similar to `AcceptBuilder`, but is also able to differ
>> between plain TCP streams and TCP streams running TLS.
>>
>> It does so by peeking into the stream's buffer and checking whether
>> the client is initiating a TLS handshake.
>>
>> Newly accepted plain TCP streams are sent along via a separate channel
>> in order to clearly distinguish between "secure" and "insecure"
>> connections.
>>
>> Signed-off-by: Max Carrara <m.carrara@proxmox.com>
>> ---
>>  proxmox-rest-server/src/connection.rs | 327 ++++++++++++++++++++++++++
>>  1 file changed, 327 insertions(+)
>>
>> diff --git a/proxmox-rest-server/src/connection.rs b/proxmox-rest-server/src/connection.rs
>> index 7681f00..937b5d7 100644
>> --- a/proxmox-rest-server/src/connection.rs
>> +++ b/proxmox-rest-server/src/connection.rs
>> @@ -302,3 +302,330 @@ impl AcceptBuilder {
>>          }
>>      }
>>  }
>> +
>> +#[cfg(feature = "rate-limited-stream")]
>> +type InsecureClientStreamResult = Pin<Box<RateLimitedStream<TcpStream>>>;
>> +#[cfg(not(feature = "rate-limited-stream"))]
>> +type InsecureClientStreamResult = Pin<Box<TcpStream>>;
> 
> ^ You can drop one set of `#[cfg]`s by using `Pin<Box<ClientStream>>` ;-)
> 
>> +
>> +#[cfg(feature = "rate-limited-stream")]
>> +type ClientStream = RateLimitedStream<TcpStream>;
>> +
>> +#[cfg(not(feature = "rate-limited-stream"))]
>> +type ClientStream = TcpStream;
>> +
>> +pub struct BiAcceptBuilder {
>> +    acceptor: Option<Arc<Mutex<SslAcceptor>>>,
>> +    debug: bool,
>> +    tcp_keepalive_time: u32,
>> +    max_pending_accepts: usize,
>> +
>> +    #[cfg(feature = "rate-limited-stream")]
>> +    lookup_rate_limiter: Option<Arc<LookupRateLimiter>>,
>> +}
> 
> 90% of this whole thing is a copy of `AcceptBuilder`.
> I'd argue that we should be able to instead add this version's
> `accept()` method to the regular `AcceptBuilder` as another variant with
> a different name, eg. `accept_with_tls_optional()`.
> 
> The `accept_connections()` task AFAICT is also just the original split
> in 2 with the tls check in between. It should be fine to just change the
> original to this with the tls check covered by whether an
> `Option<Sender<Insecure...>>` is `Some`.
> 
> Otherwise we're just duplicating too much.
> 

I had realized this while writing this series; I decided to go for a
separate type because I wasn't sure whether I'd be breaking an API or
not (but as you had already mentioned off list, this isn't the case here).

> The only other change is that the tls acceptor is now optional.
> Do we even have a use case for where we need potentially-rate-limited
> non-tls streams?
> If so, this could also be another accept method.
> 
> In fact, given the point where the acceptor is actually used, perhaps we
> should drop it from the struct entirely and instead pass it along to the
> `accept()` methods:
> - accept_tls(acceptor) -> stream
> - accept_optional_tls(acceptor) -> (tls stream, insecure stream)
> - accept_direct(acceptor) -> nontls-stream
> 
> ^ dropping the original `accept` on purpose ensure all crate users get
> updated accordingly

Good point actually! This seems like the right way to go.




^ permalink raw reply	[flat|nested] 12+ messages in thread

* Re: [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector`
  2023-07-14  9:24   ` Wolfgang Bumiller
@ 2023-07-18  5:59     ` Max Carrara
  0 siblings, 0 replies; 12+ messages in thread
From: Max Carrara @ 2023-07-18  5:59 UTC (permalink / raw)
  To: Wolfgang Bumiller; +Cc: pbs-devel

On 7/14/23 11:24, Wolfgang Bumiller wrote:
> On Thu, Jun 22, 2023 at 11:15:25AM +0200, Max Carrara wrote:
>> The `Redirector` is a simple `Service` that redirects HTTP requests
>> to HTTPS and can be served by a `hyper::Server`.
>>
>> Signed-off-by: Max Carrara <m.carrara@proxmox.com>
>> ---
>>  proxmox-rest-server/src/lib.rs  |  2 +-
>>  proxmox-rest-server/src/rest.rs | 76 +++++++++++++++++++++++++++++++++
>>  2 files changed, 77 insertions(+), 1 deletion(-)
>>
>> diff --git a/proxmox-rest-server/src/lib.rs b/proxmox-rest-server/src/lib.rs
>> index bc5be01..1c64ffb 100644
>> --- a/proxmox-rest-server/src/lib.rs
>> +++ b/proxmox-rest-server/src/lib.rs
>> @@ -48,7 +48,7 @@ mod api_config;
>>  pub use api_config::{ApiConfig, AuthError, AuthHandler, IndexHandler};
>>  
>>  mod rest;
>> -pub use rest::RestServer;
>> +pub use rest::{Redirector, RestServer};
>>  
>>  pub mod connection;
>>  
>> diff --git a/proxmox-rest-server/src/rest.rs b/proxmox-rest-server/src/rest.rs
>> index 100c93c..2584e96 100644
>> --- a/proxmox-rest-server/src/rest.rs
>> +++ b/proxmox-rest-server/src/rest.rs
>> @@ -97,6 +97,82 @@ impl<T: PeerAddress> Service<&T> for RestServer {
>>      }
>>  }
>>  
>> +pub struct Redirector {}
>> +
>> +impl Redirector {
>> +    pub fn new() -> Self {
>> +        Self {}
>> +    }
>> +}
>> +
>> +impl<T: PeerAddress> Service<&T> for Redirector {
>> +    type Response = RedirectService;
>> +    type Error = Error;
>> +    type Future = std::future::Ready<Result<Self::Response, Self::Error>>;
>> +
>> +    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
>> +        Poll::Ready(Ok(()))
>> +    }
>> +
>> +    fn call(&mut self, ctx: &T) -> Self::Future {
>> +        std::future::ready(match ctx.peer_addr() {
> 
> ^ In theory we don't even need to bother with the address (although it's
> unlikely to fail), since we never use it in the RedirectService?
> Shouldn't RedirectService work just fine as a ZST? :-)
> 

It does work as ZST indeed!

This is (yet) another story of me wanting to conform to existing
patterns / not breaking anything. :-) I'll throw it out for now, if we
need it again later I can always put it back in.

>> +            Err(err) => Err(format_err!("unable to get peer address - {err}")),
>> +            Ok(peer) => Ok(RedirectService { peer }),
>> +        })
>> +    }
>> +}
>> +
>> +pub struct RedirectService {
>> +    pub peer: std::net::SocketAddr,
>> +}
>> +
>> +impl Service<Request<Body>> for RedirectService {
>> +    type Response = Response<Body>;
>> +    type Error = anyhow::Error;
>> +    type Future = Pin<Box<dyn Future<Output = Result<Self::Response, Self::Error>> + Send>>;
>> +
>> +    fn poll_ready(&mut self, _cx: &mut Context<'_>) -> Poll<Result<(), Self::Error>> {
>> +        Poll::Ready(Ok(()))
>> +    }
>> +
>> +    fn call(&mut self, req: Request<Body>) -> Self::Future {
>> +        lazy_static! {
>> +            static ref RE: Regex = Regex::new(r"((http(s)?://)|^)").unwrap();
>> +        }
>> +
>> +        let future = async move {
>> +            let header_host_value = req
>> +                .headers()
>> +                .get("host")
>> +                .and_then(|value| value.to_str().ok());
>> +
>> +            let response = if let Some(value) = header_host_value {
>> +                let location_value = RE.replace(value, "https://");
>> +
>> +                let status_code = if matches!(*req.method(), http::Method::GET | http::Method::HEAD)
>> +                {
>> +                    StatusCode::MOVED_PERMANENTLY
>> +                } else {
>> +                    StatusCode::PERMANENT_REDIRECT
>> +                };
>> +
>> +                Response::builder()
>> +                    .status(status_code)
>> +                    .header("Location", String::from(location_value))
>> +                    .body(Body::empty())?
>> +            } else {
>> +                Response::builder()
>> +                    .status(StatusCode::BAD_REQUEST)
>> +                    .body(Body::empty())?
>> +            };
>> +
>> +            Ok(response)
>> +        };
>> +
>> +        future.boxed()
>> +    }
>> +}
>> +
>>  pub trait PeerAddress {
>>      fn peer_addr(&self) -> Result<std::net::SocketAddr, Error>;
>>  }
>> -- 
>> 2.30.2
>>
>>
>>
>> _______________________________________________
>> pbs-devel mailing list
>> pbs-devel@lists.proxmox.com
>> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
>>
>>





^ permalink raw reply	[flat|nested] 12+ messages in thread

end of thread, other threads:[~2023-07-18  6:00 UTC | newest]

Thread overview: 12+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-06-22  9:15 [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 1/3] rest-server: Add `BiAcceptBuilder` Max Carrara
2023-07-14  9:20   ` Wolfgang Bumiller
2023-07-18  5:46     ` Max Carrara
2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox 2/3] rest-server: Add `Redirector` Max Carrara
2023-07-14  9:24   ` Wolfgang Bumiller
2023-07-18  5:59     ` Max Carrara
2023-06-22  9:15 ` [pbs-devel] [PATCH proxmox-backup 3/3] proxy: redirect HTTP requests to HTTPS Max Carrara
2023-06-23 10:15 ` [pbs-devel] [PATCH proxmox, proxmox-backup 0/3] Add support for HTTP to HTTPS redirection Max Carrara
2023-06-23 10:55   ` Thomas Lamprecht
2023-06-27  9:39     ` Max Carrara
2023-06-23 11:40 ` Lukas Wagner

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox
Service provided by Proxmox Server Solutions GmbH | Privacy | Legal