public inbox for pve-devel@lists.proxmox.com
 help / color / mirror / Atom feed
* [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration
@ 2021-11-05 13:03 Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox 1/1] websocket: adapt for client connection Fabian Grünbichler
                   ` (23 more replies)
  0 siblings, 24 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

this series adds remote migration for VMs. there's still some of
TODOs/FIXMEs/stuff that requires discussion left, but this should
hopefully be possible to apply with minor modifications and lots of
future extensions.

both live and offline migration including NBD and storage-migrated disks
should work.

besides lots of rebases, implemented todos and fixed issues the main
difference to the previous RFC is that we no longer define remote
entries in a config file, but just expect the caller/client to give us
all the required information to connect to the remote cluster.

overview over affected repos and changes, see individual patches for
more details.

proxmox:

some compatible changes to make websocket code usable for client-side
connections, required by proxmox-websocket-tunnel

proxmox-websocket-tunnel:

new tunnel helper tool for forwarding commands and data over websocket
connections, required by qemu-server on source side

pve-access-control:

new ticket type, required by qemu-server on target side

pve-common:

bridgepair format akin to storage pair, pve-bridge-id option, required
by qemu-server
remote format for specifying a remote endpoint (including API token,
fingerprint, ..)

pve-guest-common:

handle remote migration (no SSH) in AbstractMigrate,
required by qemu-server

pve-storage:

extend 'pvesm import' to allow import from UNIX socket, required on
target node by qemu-server

qemu-server:

some refactoring, new mtunnel endpoints, new remote_migration endpoints
TODO: handle pending changes and snapshots
TODO: proper CLI for remote migration
potential TODO: precond endpoint?

pve-http-server:

fix for handling unflushed proxy streams

as usual, some of the patches are best viewed with '-w', especially in
qemu-server..

required dependencies are noted, qemu-server also requires a build-dep
on patched pve-common since the required options/formats would be
missing otherwise..




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

* [pve-devel] [PATCH proxmox 1/1] websocket: adapt for client connection
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 1/4] initial commit Fabian Grünbichler
                   ` (22 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

previously, this was only used for the server side handling of web
sockets. by making the mask part of the WebSocket struct and making some
of the fns associated, we can re-use this for client-side connections
such as in proxmox-websocket-tunnel.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    v2:
    - adapt to changes in proxmox(-http)

 proxmox-http/src/websocket/mod.rs | 29 ++++++++++++++++++-----------
 1 file changed, 18 insertions(+), 11 deletions(-)

diff --git a/proxmox-http/src/websocket/mod.rs b/proxmox-http/src/websocket/mod.rs
index 2c53d01..d72c550 100644
--- a/proxmox-http/src/websocket/mod.rs
+++ b/proxmox-http/src/websocket/mod.rs
@@ -650,7 +650,9 @@ impl<R: AsyncRead + Unpin + Send + 'static> AsyncRead for WebSocketReader<R> {
 pub const MAGIC_WEBSOCKET_GUID: &str = "258EAFA5-E914-47DA-95CA-C5AB0DC85B11";
 
 /// Provides methods for connecting a WebSocket endpoint with another
-pub struct WebSocket;
+pub struct WebSocket {
+    pub mask: Option<[u8; 4]>,
+}
 
 impl WebSocket {
     /// Returns a new WebSocket instance and the generates the correct
@@ -702,10 +704,13 @@ impl WebSocket {
 
         let response = response.body(Body::empty())?;
 
-        Ok((Self, response))
+        let mask = None;
+
+        Ok((Self { mask }, response))
     }
 
-    async fn handle_channel_message<W>(
+    pub async fn handle_channel_message<W>(
+        &self,
         result: WebSocketReadResult,
         writer: &mut WebSocketWriter<W>,
     ) -> Result<OpCode, Error>
@@ -714,11 +719,11 @@ impl WebSocket {
     {
         match result {
             Ok((OpCode::Ping, msg)) => {
-                writer.send_control_frame(None, OpCode::Pong, &msg).await?;
+                writer.send_control_frame(self.mask, OpCode::Pong, &msg).await?;
                 Ok(OpCode::Pong)
             }
             Ok((OpCode::Close, msg)) => {
-                writer.send_control_frame(None, OpCode::Close, &msg).await?;
+                writer.send_control_frame(self.mask, OpCode::Close, &msg).await?;
                 Ok(OpCode::Close)
             }
             Ok((opcode, _)) => {
@@ -727,7 +732,7 @@ impl WebSocket {
             }
             Err(err) => {
                 writer
-                    .send_control_frame(None, OpCode::Close, &err.generate_frame_payload())
+                    .send_control_frame(self.mask, OpCode::Close, &err.generate_frame_payload())
                     .await?;
                 Err(Error::from(err))
             }
@@ -735,6 +740,7 @@ impl WebSocket {
     }
 
     async fn copy_to_websocket<R, W>(
+        &self,
         mut reader: &mut R,
         mut writer: &mut WebSocketWriter<W>,
         receiver: &mut mpsc::UnboundedReceiver<WebSocketReadResult>,
@@ -743,7 +749,7 @@ impl WebSocket {
         R: AsyncRead + Unpin + Send,
         W: AsyncWrite + Unpin + Send,
     {
-        let mut buf = ByteBuffer::new();
+        let mut buf = ByteBuffer::with_capacity(16*1024);
         let mut eof = false;
         loop {
             if !buf.is_full() {
@@ -751,7 +757,7 @@ impl WebSocket {
                     res = buf.read_from_async(&mut reader).fuse() => res?,
                     res = receiver.recv().fuse() => {
                         let res = res.ok_or_else(|| format_err!("control channel closed"))?;
-                        match Self::handle_channel_message(res, &mut writer).await? {
+                        match self.handle_channel_message(res, &mut writer).await? {
                             OpCode::Close => return Ok(true),
                             _ => { continue; },
                         }
@@ -771,6 +777,7 @@ impl WebSocket {
             }
 
             if eof && buf.is_empty() {
+                writer.flush().await?;
                 return Ok(false);
             }
         }
@@ -791,10 +798,10 @@ impl WebSocket {
 
         let (tx, mut rx) = mpsc::unbounded_channel();
         let mut wsreader = WebSocketReader::new(usreader, tx);
-        let mut wswriter = WebSocketWriter::new(None, uswriter);
+        let mut wswriter = WebSocketWriter::new(self.mask, uswriter);
 
         let ws_future = tokio::io::copy(&mut wsreader, &mut dswriter);
-        let term_future = Self::copy_to_websocket(&mut dsreader, &mut wswriter, &mut rx);
+        let term_future = self.copy_to_websocket(&mut dsreader, &mut wswriter, &mut rx);
 
         let res = select! {
             res = ws_future.fuse() => match res {
@@ -804,7 +811,7 @@ impl WebSocket {
             res = term_future.fuse() => match res {
                 Ok(sent_close) if !sent_close => {
                     // status code 1000 => 0x03E8
-                    wswriter.send_control_frame(None, OpCode::Close, &WebSocketErrorKind::Normal.to_be_bytes()).await?;
+                    wswriter.send_control_frame(self.mask, OpCode::Close, &WebSocketErrorKind::Normal.to_be_bytes()).await?;
                     Ok(())
                 }
                 Ok(_) => Ok(()),
-- 
2.30.2





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

* [pve-devel] [PATCH proxmox-websocket-tunnel 1/4] initial commit
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox 1/1] websocket: adapt for client connection Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation Fabian Grünbichler
                   ` (21 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 .gitignore    |  1 +
 .cargo/config |  5 +++++
 Cargo.toml    | 11 +++++++++++
 3 files changed, 17 insertions(+)
 create mode 100644 .gitignore
 create mode 100644 .cargo/config
 create mode 100644 Cargo.toml

diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/.cargo/config b/.cargo/config
new file mode 100644
index 0000000..3b5b6e4
--- /dev/null
+++ b/.cargo/config
@@ -0,0 +1,5 @@
+[source]
+[source.debian-packages]
+directory = "/usr/share/cargo/registry"
+[source.crates-io]
+replace-with = "debian-packages"
diff --git a/Cargo.toml b/Cargo.toml
new file mode 100644
index 0000000..939184c
--- /dev/null
+++ b/Cargo.toml
@@ -0,0 +1,11 @@
+[package]
+name = "proxmox-websocket-tunnel"
+version = "0.1.0"
+authors = ["Fabian Grünbichler <f.gruenbichler@proxmox.com>"]
+edition = "2018"
+license = "AGPL-3"
+description = "Proxmox websocket tunneling helper"
+
+exclude = ["debian"]
+
+[dependencies]
-- 
2.30.2





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

* [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox 1/1] websocket: adapt for client connection Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 1/4] initial commit Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-09 12:54   ` Dominik Csapak
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 3/4] add fingerprint validation Fabian Grünbichler
                   ` (20 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

the websocket tunnel helper accepts control commands (encoded as
single-line JSON) on stdin, and prints responses on stdout.

the following commands are available:
- "connect" a 'control' tunnel via a websocket
- "forward" a local unix socket to a remote socket via a websocket
-- if requested, this will ask for a ticket via the control tunnel after
accepting a new connection on the unix socket
- "close" the control tunnel and any forwarded socket

any other json input (without the 'control' flag set) is forwarded as-is
to the remote end of the control tunnel.

internally, the tunnel helper will spawn tokio tasks for
- handling the control tunnel connection (new commands are passed in via
an mpsc channel together with a oneshot channel for the response)
- handling accepting new connections on each forwarded unix socket
- handling forwarding data over accepted forwarded connections

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    requires proxmox-http with changes and bumped version

 Cargo.toml  |  13 ++
 src/main.rs | 410 ++++++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 423 insertions(+)
 create mode 100644 src/main.rs

diff --git a/Cargo.toml b/Cargo.toml
index 939184c..9d2a8c6 100644
--- a/Cargo.toml
+++ b/Cargo.toml
@@ -9,3 +9,16 @@ description = "Proxmox websocket tunneling helper"
 exclude = ["debian"]
 
 [dependencies]
+anyhow = "1.0"
+base64 = "0.12"
+futures = "0.3"
+futures-util = "0.3"
+hyper = { version = "0.14" }
+openssl = "0.10"
+percent-encoding = "2"
+proxmox-http = { version = "0.5.2", path = "../proxmox/proxmox-http", features = ["websocket", "client"] }
+serde = { version = "1.0", features = ["derive"] }
+serde_json = "1.0"
+tokio = { version = "1", features = ["io-std", "io-util", "macros", "rt-multi-thread", "sync"] }
+tokio-stream = { version = "0.1", features = ["io-util"] }
+tokio-util = "0.6"
diff --git a/src/main.rs b/src/main.rs
new file mode 100644
index 0000000..150c1cf
--- /dev/null
+++ b/src/main.rs
@@ -0,0 +1,410 @@
+use anyhow::{bail, format_err, Error};
+
+use std::collections::VecDeque;
+use std::sync::{Arc, Mutex};
+
+use futures::future::FutureExt;
+use futures::select;
+
+use hyper::client::{Client, HttpConnector};
+use hyper::header::{SEC_WEBSOCKET_KEY, SEC_WEBSOCKET_VERSION, UPGRADE};
+use hyper::upgrade::Upgraded;
+use hyper::{Body, Request, StatusCode};
+
+use openssl::ssl::{SslConnector, SslMethod};
+use percent_encoding::{utf8_percent_encode, NON_ALPHANUMERIC};
+
+use serde::{Deserialize, Serialize};
+use serde_json::{Map, Value};
+use tokio::io::{AsyncBufReadExt, AsyncWriteExt, BufReader};
+use tokio::net::{UnixListener, UnixStream};
+use tokio::sync::{mpsc, oneshot};
+use tokio_stream::wrappers::LinesStream;
+use tokio_stream::StreamExt;
+
+use proxmox_http::client::HttpsConnector;
+use proxmox_http::websocket::{OpCode, WebSocket, WebSocketReader, WebSocketWriter};
+
+#[derive(Serialize, Deserialize, Debug)]
+#[serde(rename_all = "kebab-case")]
+enum CmdType {
+    Connect,
+    Forward,
+    CloseCmd,
+    NonControl,
+}
+
+type CmdData = Map<String, Value>;
+
+#[derive(Serialize, Deserialize, Debug)]
+#[serde(rename_all = "kebab-case")]
+struct ConnectCmdData {
+    // target URL for WS connection
+    url: String,
+    // fingerprint of TLS certificate
+    fingerprint: Option<String>,
+    // addition headers such as authorization
+    headers: Option<Vec<(String, String)>>,
+}
+
+#[derive(Serialize, Deserialize, Debug, Clone)]
+#[serde(rename_all = "kebab-case")]
+struct ForwardCmdData {
+    // target URL for WS connection
+    url: String,
+    // addition headers such as authorization
+    headers: Option<Vec<(String, String)>>,
+    // fingerprint of TLS certificate
+    fingerprint: Option<String>,
+    // local UNIX socket path for forwarding
+    unix: String,
+    // request ticket using these parameters
+    ticket: Option<Map<String, Value>>,
+}
+
+struct CtrlTunnel {
+    sender: Option<mpsc::UnboundedSender<(Value, oneshot::Sender<String>)>>,
+    forwarded: Arc<Mutex<Vec<oneshot::Sender<()>>>>,
+}
+
+impl CtrlTunnel {
+    async fn read_cmd_loop(mut self) -> Result<(), Error> {
+        let mut stdin_stream = LinesStream::new(BufReader::new(tokio::io::stdin()).lines());
+        while let Some(res) = stdin_stream.next().await {
+            match res {
+                Ok(line) => {
+                    let (cmd_type, data) = Self::parse_cmd(&line)?;
+                    match cmd_type {
+                        CmdType::Connect => self.handle_connect_cmd(data).await,
+                        CmdType::Forward => {
+                            let res = self.handle_forward_cmd(data).await;
+                            match &res {
+                                Ok(()) => println!("{}", serde_json::json!({"success": true})),
+                                Err(msg) => println!(
+                                    "{}",
+                                    serde_json::json!({"success": false, "msg": msg.to_string()})
+                                ),
+                            };
+                            res
+                        }
+                        CmdType::NonControl => self
+                            .handle_tunnel_cmd(data)
+                            .await
+                            .map(|res| println!("{}", res)),
+                        _ => unimplemented!(),
+                    }
+                }
+                Err(err) => bail!("Failed to read from STDIN - {}", err),
+            }?;
+        }
+
+        Ok(())
+    }
+
+    fn parse_cmd(line: &str) -> Result<(CmdType, CmdData), Error> {
+        let mut json: Map<String, Value> = serde_json::from_str(line)?;
+        match json.remove("control") {
+            Some(Value::Bool(true)) => {
+                match json.remove("cmd").map(serde_json::from_value::<CmdType>) {
+                    None => bail!("input has 'control' flag, but no control 'cmd' set.."),
+                    Some(Err(e)) => bail!("failed to parse control cmd - {}", e),
+                    Some(Ok(cmd_type)) => Ok((cmd_type, json)),
+                }
+            }
+            _ => Ok((CmdType::NonControl, json)),
+        }
+    }
+
+    async fn websocket_connect(
+        url: String,
+        extra_headers: Vec<(String, String)>,
+        fingerprint: Option<String>,
+    ) -> Result<Upgraded, Error> {
+        let ws_key = proxmox::sys::linux::random_data(16)?;
+        let ws_key = base64::encode(&ws_key);
+        let mut req = Request::builder()
+            .uri(url)
+            .header(UPGRADE, "websocket")
+            .header(SEC_WEBSOCKET_VERSION, "13")
+            .header(SEC_WEBSOCKET_KEY, ws_key)
+            .body(Body::empty())
+            .unwrap();
+
+        let headers = req.headers_mut();
+        for (name, value) in extra_headers {
+            let name = hyper::header::HeaderName::from_bytes(name.as_bytes())?;
+            let value = hyper::header::HeaderValue::from_str(&value)?;
+            headers.insert(name, value);
+        }
+
+        let mut ssl_connector_builder = SslConnector::builder(SslMethod::tls()).unwrap();
+        if fingerprint.is_some() {
+            // FIXME actually verify fingerprint via callback!
+            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::NONE);
+        } else {
+            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::PEER);
+        }
+
+        let mut httpc = HttpConnector::new();
+        httpc.enforce_http(false); // we want https...
+        httpc.set_connect_timeout(Some(std::time::Duration::new(10, 0)));
+        let https = HttpsConnector::with_connector(httpc, ssl_connector_builder.build(), 120);
+
+        let client = Client::builder().build::<_, Body>(https);
+        let res = client.request(req).await?;
+        if res.status() != StatusCode::SWITCHING_PROTOCOLS {
+            bail!("server didn't upgrade: {}", res.status());
+        }
+
+        hyper::upgrade::on(res)
+            .await
+            .map_err(|err| format_err!("failed to upgrade - {}", err))
+    }
+
+    async fn handle_connect_cmd(&mut self, mut data: CmdData) -> Result<(), Error> {
+        let mut data: ConnectCmdData = data
+            .remove("data")
+            .ok_or_else(|| format_err!("'connect' command missing 'data'"))
+            .map(serde_json::from_value)??;
+
+        if self.sender.is_some() {
+            bail!("already connected!");
+        }
+
+        let upgraded = Self::websocket_connect(
+            data.url.clone(),
+            data.headers.take().unwrap_or_else(Vec::new),
+            data.fingerprint.take(),
+        )
+        .await?;
+
+        let (tx, rx) = mpsc::unbounded_channel();
+        self.sender = Some(tx);
+        tokio::spawn(async move {
+            if let Err(err) = Self::handle_ctrl_tunnel(upgraded, rx).await {
+                eprintln!("Tunnel to {} failed - {}", data.url, err);
+            }
+        });
+
+        Ok(())
+    }
+
+    async fn handle_forward_cmd(&mut self, mut data: CmdData) -> Result<(), Error> {
+        let data: ForwardCmdData = data
+            .remove("data")
+            .ok_or_else(|| format_err!("'forward' command missing 'data'"))
+            .map(serde_json::from_value)??;
+
+        if self.sender.is_none() && data.ticket.is_some() {
+            bail!("dynamically requesting ticket requires cmd tunnel connection!");
+        }
+
+        let unix_listener = UnixListener::bind(data.unix.clone()).unwrap();
+        let (tx, rx) = oneshot::channel();
+        let data = Arc::new(data);
+
+        self.forwarded.lock().unwrap().push(tx);
+        let cmd_sender = self.sender.clone();
+
+        tokio::spawn(async move {
+            let mut rx = rx.fuse();
+            let mut tasks: Vec<tokio::task::JoinHandle<()>> = Vec::new();
+            loop {
+                let accept = unix_listener.accept().fuse();
+                tokio::pin!(accept);
+                let data2 = data.clone();
+                select! {
+                    _ = rx => {
+                        eprintln!("received shutdown signal, closing unix listener stream and forwarding handlers");
+                        for task in tasks {
+                            task.abort();
+                        }
+                        break;
+                    },
+                    res = accept => match res {
+                        Ok((unix_stream, _)) => {
+                            eprintln!("accepted new connection on '{}'", data2.unix);
+                            let data3: Result<Arc<ForwardCmdData>, Error> = match (&cmd_sender, &data2.ticket) {
+                                (Some(cmd_sender), Some(_)) => Self::get_ticket(cmd_sender, data2.clone()).await,
+                                _ => Ok(data2.clone()),
+                            };
+
+                            match data3 {
+                                Ok(data3) => {
+                                    let task = tokio::spawn(async move {
+                                        if let Err(err) = Self::handle_forward_tunnel(data3.clone(), unix_stream).await {
+                                            eprintln!("Tunnel for {} failed - {}", data3.unix, err);
+                                        }
+                                    });
+                                    tasks.push(task);
+                                },
+                                Err(err) => {
+                                    eprintln!("Failed to accept unix connection - {}", err);
+                                },
+                            };
+                        },
+                        Err(err) => eprintln!("Failed to accept unix connection on {} - {}", data2.unix, err),
+                    },
+                };
+            }
+        });
+
+        Ok(())
+    }
+
+    async fn handle_tunnel_cmd(&mut self, data: CmdData) -> Result<String, Error> {
+        match &mut self.sender {
+            None => bail!("not connected!"),
+            Some(sender) => {
+                let data: Value = data.into();
+                let (tx, rx) = oneshot::channel::<String>();
+                if let Some(cmd) = data.get("cmd") {
+                    eprintln!("-> sending command {} to remote", cmd);
+                } else {
+                    eprintln!("-> sending data line to remote");
+                }
+                sender.send((data, tx))?;
+                let res = rx.await?;
+                eprintln!("<- got reply");
+                Ok(res)
+            }
+        }
+    }
+
+    async fn handle_ctrl_tunnel(
+        websocket: Upgraded,
+        mut cmd_receiver: mpsc::UnboundedReceiver<(Value, oneshot::Sender<String>)>,
+    ) -> Result<(), Error> {
+        let (tunnel_reader, tunnel_writer) = tokio::io::split(websocket);
+        let (ws_close_tx, mut ws_close_rx) = mpsc::unbounded_channel();
+        let ws_reader = WebSocketReader::new(tunnel_reader, ws_close_tx);
+        let mut ws_writer = WebSocketWriter::new(Some([0, 0, 0, 0]), tunnel_writer);
+
+        let mut framed_reader =
+            tokio_util::codec::FramedRead::new(ws_reader, tokio_util::codec::LinesCodec::new());
+
+        let mut resp_tx_queue: VecDeque<oneshot::Sender<String>> = VecDeque::new();
+        let mut shutting_down = false;
+
+        loop {
+            let mut close_future = ws_close_rx.recv().boxed().fuse();
+            let mut frame_future = framed_reader.next().boxed().fuse();
+            let mut cmd_future = cmd_receiver.recv().boxed().fuse();
+
+            select! {
+                res = close_future => {
+                    let res = res.ok_or_else(|| format_err!("WS control channel closed"))?;
+                    eprintln!("WS: received control message: '{:?}'", res);
+                    shutting_down = true;
+                },
+                res = frame_future => {
+                    match res {
+                        None if shutting_down => {
+                            eprintln!("WS closed");
+                            break;
+                        },
+                        None => bail!("WS closed unexpectedly"),
+                        Some(Ok(res)) => {
+                            resp_tx_queue
+                                .pop_front()
+                                .ok_or_else(|| format_err!("no response handler"))?
+                                .send(res)
+                                .map_err(|msg| format_err!("failed to send tunnel response '{}' back to requester - receiver already closed?", msg))?;
+                        },
+                        Some(Err(err)) => {
+                            bail!("reading from control tunnel failed - WS receive failed: {}", err);
+                        },
+                    }
+                },
+                res = cmd_future => {
+                    if shutting_down { continue };
+                    match res {
+                        None => {
+                            eprintln!("CMD channel closed, shutting down");
+                            ws_writer.send_control_frame(Some([1,2,3,4]), OpCode::Close, &[]).await?;
+                            shutting_down = true;
+                        },
+                        Some((msg, resp_tx)) => {
+                            resp_tx_queue.push_back(resp_tx);
+
+                            let line = format!("{}\n", msg);
+                            ws_writer.write_all(line.as_bytes()).await?;
+                            ws_writer.flush().await?;
+                        },
+                    }
+                },
+            };
+        }
+
+        Ok(())
+    }
+
+    async fn handle_forward_tunnel(
+        data: Arc<ForwardCmdData>,
+        unix: UnixStream,
+    ) -> Result<(), Error> {
+        let upgraded = Self::websocket_connect(
+            data.url.clone(),
+            data.headers.clone().unwrap_or_else(Vec::new),
+            data.fingerprint.clone(),
+        )
+        .await?;
+
+        let ws = WebSocket {
+            mask: Some([0, 0, 0, 0]),
+        };
+        eprintln!("established new WS for forwarding '{}'", data.unix);
+        ws.serve_connection(upgraded, unix).await?;
+
+        eprintln!("done handling forwarded connection from '{}'", data.unix);
+
+        Ok(())
+    }
+
+    async fn get_ticket(
+        cmd_sender: &mpsc::UnboundedSender<(Value, oneshot::Sender<String>)>,
+        cmd_data: Arc<ForwardCmdData>,
+    ) -> Result<Arc<ForwardCmdData>, Error> {
+        eprintln!("requesting WS ticket via tunnel");
+        let ticket_cmd = match cmd_data.ticket.clone() {
+            Some(mut ticket_cmd) => {
+                ticket_cmd.insert("cmd".to_string(), serde_json::json!("ticket"));
+                ticket_cmd
+            }
+            None => bail!("can't get ticket without ticket parameters"),
+        };
+        let (tx, rx) = oneshot::channel::<String>();
+        cmd_sender.send((serde_json::json!(ticket_cmd), tx))?;
+        let ticket = rx.await?;
+        let mut ticket: Map<String, Value> = serde_json::from_str(&ticket)?;
+        let ticket = ticket
+            .remove("ticket")
+            .ok_or_else(|| format_err!("failed to retrieve ticket via tunnel"))?;
+
+        let ticket = ticket
+            .as_str()
+            .ok_or_else(|| format_err!("failed to format received ticket"))?;
+        let ticket = utf8_percent_encode(&ticket, NON_ALPHANUMERIC).to_string();
+
+        let mut data = cmd_data.clone();
+        let mut url = data.url.clone();
+        url.push_str("ticket=");
+        url.push_str(&ticket);
+        let mut d = Arc::make_mut(&mut data);
+        d.url = url;
+        Ok(data)
+    }
+}
+
+#[tokio::main]
+async fn main() -> Result<(), Error> {
+    do_main().await
+}
+
+async fn do_main() -> Result<(), Error> {
+    let tunnel = CtrlTunnel {
+        sender: None,
+        forwarded: Arc::new(Mutex::new(Vec::new())),
+    };
+    tunnel.read_cmd_loop().await
+}
-- 
2.30.2





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

* [pve-devel] [PATCH proxmox-websocket-tunnel 3/4] add fingerprint validation
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (2 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 4/4] add packaging Fabian Grünbichler
                   ` (19 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

in case we have no explicit fingerprint, we use openssl's regular "PEER"
verification. if we have a fingerprint, we ignore openssl altogether and
just verify the fingerprint of the presented leaf certificate.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 Cargo.toml  |  1 +
 src/main.rs | 47 ++++++++++++++++++++++++++++++++++++++++++++---
 2 files changed, 45 insertions(+), 3 deletions(-)

diff --git a/Cargo.toml b/Cargo.toml
index 9d2a8c6..adf83f9 100644
--- a/Cargo.toml
+++ b/Cargo.toml
@@ -16,6 +16,7 @@ futures-util = "0.3"
 hyper = { version = "0.14" }
 openssl = "0.10"
 percent-encoding = "2"
+proxmox = { version = "0.15" }
 proxmox-http = { version = "0.5.2", path = "../proxmox/proxmox-http", features = ["websocket", "client"] }
 serde = { version = "1.0", features = ["derive"] }
 serde_json = "1.0"
diff --git a/src/main.rs b/src/main.rs
index 150c1cf..0733141 100644
--- a/src/main.rs
+++ b/src/main.rs
@@ -138,9 +138,50 @@ impl CtrlTunnel {
         }
 
         let mut ssl_connector_builder = SslConnector::builder(SslMethod::tls()).unwrap();
-        if fingerprint.is_some() {
-            // FIXME actually verify fingerprint via callback!
-            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::NONE);
+        if let Some(expected) = fingerprint {
+            ssl_connector_builder.set_verify_callback(
+                openssl::ssl::SslVerifyMode::NONE,
+                move |_valid, ctx| {
+                    let cert = match ctx.current_cert() {
+                        Some(cert) => cert,
+                        None => {
+                            eprintln!("SSL context lacks current certificate.");
+                            return false;
+                        }
+                    };
+
+                    let depth = ctx.error_depth();
+                    if depth != 0 {
+                        return true;
+                    }
+
+                    let fp = match cert.digest(openssl::hash::MessageDigest::sha256()) {
+                        Ok(fp) => fp,
+                        Err(err) => {
+                            // should not happen
+                            eprintln!("failed to calculate certificate FP - {}", err);
+                            return false;
+                        }
+                    };
+                    let fp_string = proxmox::tools::digest_to_hex(&fp);
+                    let fp_string = fp_string
+                        .as_bytes()
+                        .chunks(2)
+                        .map(|v| std::str::from_utf8(v).unwrap())
+                        .collect::<Vec<&str>>()
+                        .join(":");
+
+                    let expected = expected.to_lowercase();
+                    if expected == fp_string {
+                        true
+                    } else {
+                        eprintln!("certificate fingerprint does not match expected fingerprint!");
+                        eprintln!("expected:    {}", expected);
+                        eprintln!("encountered: {}", fp_string);
+                        false
+                    }
+                },
+            );
         } else {
             ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::PEER);
         }
-- 
2.30.2





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

* [pve-devel] [PATCH proxmox-websocket-tunnel 4/4] add packaging
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (3 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 3/4] add fingerprint validation Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH access-control 1/2] tickets: add tunnel ticket Fabian Grünbichler
                   ` (18 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 Makefile             | 44 ++++++++++++++++++++++++++++++++++
 debian/changelog     |  5 ++++
 debian/control       | 57 ++++++++++++++++++++++++++++++++++++++++++++
 debian/copyright     | 16 +++++++++++++
 debian/debcargo.toml | 13 ++++++++++
 5 files changed, 135 insertions(+)
 create mode 100644 Makefile
 create mode 100644 debian/changelog
 create mode 100644 debian/control
 create mode 100644 debian/copyright
 create mode 100644 debian/debcargo.toml

diff --git a/Makefile b/Makefile
new file mode 100644
index 0000000..4f2a799
--- /dev/null
+++ b/Makefile
@@ -0,0 +1,44 @@
+.PHONY: all
+all: check
+
+.PHONY: check
+check:
+	cargo test --all-features
+
+.PHONY: dinstall
+dinstall: deb
+	sudo -k dpkg -i build/librust-*.deb
+
+.PHONY: build
+build:
+	rm -rf build
+	rm -f debian/control
+	mkdir build
+	debcargo package \
+	    --config "$(PWD)/debian/debcargo.toml" \
+	    --changelog-ready \
+	    --no-overlay-write-back \
+	    --directory "$(PWD)/build/proxmox-websocket-tunnel" \
+	    "proxmox-websocket-tunnel" \
+	    "$$(dpkg-parsechangelog -l "debian/changelog" -SVersion | sed -e 's/-.*//')"
+	echo system >build/rust-toolchain
+	rm -f build/proxmox-websocket-tunnel/Cargo.lock
+	find build/proxmox-websocket-tunnel/debian -name '*.hint' -delete
+	cp build/proxmox-websocket-tunnel/debian/control debian/control
+
+.PHONY: deb
+deb: build
+	(cd build/proxmox-websocket-tunnel && dpkg-buildpackage -b -uc -us)
+	lintian build/*.deb
+
+.PHONY: clean
+clean:
+	rm -rf build *.deb *.buildinfo *.changes *.orig.tar.gz
+	cargo clean
+
+upload: deb
+	cd build; \
+	    dcmd --deb rust-proxmox-websocket-tunnel_*.changes \
+	    | grep -v '.changes$$' \
+	    | tar -cf- -T- \
+	    | ssh -X repoman@repo.proxmox.com upload --product pve --dist bullseye
diff --git a/debian/changelog b/debian/changelog
new file mode 100644
index 0000000..04751ce
--- /dev/null
+++ b/debian/changelog
@@ -0,0 +1,5 @@
+rust-proxmox-websocket-tunnel (0.1.0-1) unstable; urgency=medium
+
+  * initial release
+
+ -- Proxmox Support Team <support@proxmox.com>  Tue, 18 May 2021 14:18:14 +0200
diff --git a/debian/control b/debian/control
new file mode 100644
index 0000000..519739b
--- /dev/null
+++ b/debian/control
@@ -0,0 +1,57 @@
+Source: rust-proxmox-websocket-tunnel
+Section: admin
+Priority: optional
+Build-Depends: debhelper (>= 12),
+ dh-cargo (>= 24),
+ cargo:native,
+ rustc:native,
+ libstd-rust-dev,
+ librust-anyhow-1+default-dev,
+ librust-base64-0.12+default-dev,
+ librust-futures-0.3+default-dev,
+ librust-futures-util-0.3+default-dev,
+ librust-hyper-0.14+default-dev,
+ librust-openssl-0.10+default-dev,
+ librust-percent-encoding-2+default-dev,
+ librust-proxmox-0.15+default-dev,
+ librust-proxmox-http-0.5+client-dev (>= 0.5.2-~~),
+ librust-proxmox-http-0.5+default-dev (>= 0.5.2-~~),
+ librust-proxmox-http-0.5+websocket-dev (>= 0.5.2-~~),
+ librust-serde-1+default-dev,
+ librust-serde-1+derive-dev,
+ librust-serde-json-1+default-dev,
+ librust-tokio-1+default-dev,
+ librust-tokio-1+io-std-dev,
+ librust-tokio-1+io-util-dev,
+ librust-tokio-1+macros-dev,
+ librust-tokio-1+rt-multi-thread-dev,
+ librust-tokio-1+sync-dev,
+ librust-tokio-stream-0.1+default-dev,
+ librust-tokio-stream-0.1+io-util-dev,
+ librust-tokio-util-0.6+default-dev
+Maintainer: Proxmox Support Team <support@proxmox.com>
+Standards-Version: 4.5.1
+Vcs-Git: git://git.proxmox.com/git/proxmox-websocket-tunnel.git
+Vcs-Browser: https://git.proxmox.com/?p=proxmox-websocket-tunnel.git
+Rules-Requires-Root: no
+
+Package: proxmox-websocket-tunnel
+Architecture: any
+Multi-Arch: allowed
+Depends:
+ ${misc:Depends},
+ ${shlibs:Depends},
+ ${cargo:Depends}
+Recommends:
+ ${cargo:Recommends}
+Suggests:
+ ${cargo:Suggests}
+Provides:
+ ${cargo:Provides}
+Built-Using: ${cargo:Built-Using}
+XB-X-Cargo-Built-Using: ${cargo:X-Cargo-Built-Using}
+Description: Proxmox websocket tunneling helper
+ Proxmox websocket tunnel helper
+ .
+ This package contains a helper binary for tunnelling UNIX sockets over a
+ websocket connection
diff --git a/debian/copyright b/debian/copyright
new file mode 100644
index 0000000..5661ef6
--- /dev/null
+++ b/debian/copyright
@@ -0,0 +1,16 @@
+Copyright (C) 2021 Proxmox Server Solutions GmbH
+
+This software is written by Proxmox Server Solutions GmbH <support@proxmox.com>
+
+This program is free software: you can redistribute it and/or modify
+it under the terms of the GNU Affero General Public License as published by
+the Free Software Foundation, either version 3 of the License, or
+(at your option) any later version.
+
+This program is distributed in the hope that it will be useful,
+but WITHOUT ANY WARRANTY; without even the implied warranty of
+MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
+GNU Affero General Public License for more details.
+
+You should have received a copy of the GNU Affero General Public License
+along with this program.  If not, see <http://www.gnu.org/licenses/>.
diff --git a/debian/debcargo.toml b/debian/debcargo.toml
new file mode 100644
index 0000000..dfe933e
--- /dev/null
+++ b/debian/debcargo.toml
@@ -0,0 +1,13 @@
+overlay = "."
+crate_src_path = ".."
+maintainer = "Proxmox Support Team <support@proxmox.com>"
+
+[source]
+vcs_git = "git://git.proxmox.com/git/proxmox-websocket-tunnel.git"
+vcs_browser = "https://git.proxmox.com/?p=proxmox-websocket-tunnel.git"
+section = "admin"
+
+[packages.bin]
+description="""Proxmox websocket tunnel helper
+
+This package contains a helper binary for tunnelling UNIX sockets over a websocket connection"""
-- 
2.30.2





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

* [pve-devel] [PATCH access-control 1/2] tickets: add tunnel ticket
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (4 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 4/4] add packaging Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH access-control 2/2] ticket: normalize path for verification Fabian Grünbichler
                   ` (17 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

just like VNC ticket, but different prefix to prevent confusion.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 src/PVE/AccessControl.pm | 50 ++++++++++++++++++++++++++++++----------
 1 file changed, 38 insertions(+), 12 deletions(-)

diff --git a/src/PVE/AccessControl.pm b/src/PVE/AccessControl.pm
index 347c2a8..dfd3af5 100644
--- a/src/PVE/AccessControl.pm
+++ b/src/PVE/AccessControl.pm
@@ -441,12 +441,8 @@ sub verify_token {
     return wantarray ? ($tokenid) : $tokenid;
 }
 
-
-# VNC tickets
-# - they do not contain the username in plain text
-# - they are restricted to a specific resource path (example: '/vms/100')
-sub assemble_vnc_ticket {
-    my ($username, $path) = @_;
+my $assemble_short_lived_ticket = sub {
+    my ($prefix, $username, $path) = @_;
 
     my $rsa_priv = get_privkey();
 
@@ -455,11 +451,11 @@ sub assemble_vnc_ticket {
     my $secret_data = "$username:$path";
 
     return PVE::Ticket::assemble_rsa_ticket(
-	$rsa_priv, 'PVEVNC', undef, $secret_data);
-}
+	$rsa_priv, $prefix, undef, $secret_data);
+};
 
-sub verify_vnc_ticket {
-    my ($ticket, $username, $path, $noerr) = @_;
+my $verify_short_lived_ticket = sub {
+    my ($ticket, $prefix, $username, $path, $noerr) = @_;
 
     my $secret_data = "$username:$path";
 
@@ -469,12 +465,42 @@ sub verify_vnc_ticket {
 	    return undef;
 	} else {
 	    # raise error via undef ticket
-	    PVE::Ticket::verify_rsa_ticket($rsa_pub, 'PVEVNC');
+	    PVE::Ticket::verify_rsa_ticket($rsa_pub, $prefix);
 	}
     }
 
     return PVE::Ticket::verify_rsa_ticket(
-	$rsa_pub, 'PVEVNC', $ticket, $secret_data, -20, 40, $noerr);
+	$rsa_pub, $prefix, $ticket, $secret_data, -20, 40, $noerr);
+};
+
+# VNC tickets
+# - they do not contain the username in plain text
+# - they are restricted to a specific resource path (example: '/vms/100')
+sub assemble_vnc_ticket {
+    my ($username, $path) = @_;
+
+    return $assemble_short_lived_ticket->('PVEVNC', $username, $path);
+}
+
+sub verify_vnc_ticket {
+    my ($ticket, $username, $path, $noerr) = @_;
+
+    return $verify_short_lived_ticket->($ticket, 'PVEVNC', $username, $path, $noerr);
+}
+
+# Tunnel tickets
+# - they do not contain the username in plain text
+# - they are restricted to a specific resource path (example: '/vms/100', '/socket/run/qemu-server/123.storage')
+sub assemble_tunnel_ticket {
+    my ($username, $path) = @_;
+
+    return $assemble_short_lived_ticket->('PVETUNNEL', $username, $path);
+}
+
+sub verify_tunnel_ticket {
+    my ($ticket, $username, $path, $noerr) = @_;
+
+    return $verify_short_lived_ticket->($ticket, 'PVETUNNEL', $username, $path, $noerr);
 }
 
 sub assemble_spice_ticket {
-- 
2.30.2





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

* [pve-devel] [PATCH access-control 2/2] ticket: normalize path for verification
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (5 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH access-control 1/2] tickets: add tunnel ticket Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH common 1/3] schema: rename storagepair to storage-pair Fabian Grünbichler
                   ` (16 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 src/PVE/AccessControl.pm | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/src/PVE/AccessControl.pm b/src/PVE/AccessControl.pm
index dfd3af5..a80ed74 100644
--- a/src/PVE/AccessControl.pm
+++ b/src/PVE/AccessControl.pm
@@ -457,6 +457,8 @@ my $assemble_short_lived_ticket = sub {
 my $verify_short_lived_ticket = sub {
     my ($ticket, $prefix, $username, $path, $noerr) = @_;
 
+    $path = normalize_path($path);
+
     my $secret_data = "$username:$path";
 
     my ($rsa_pub, $rsa_mtime) = get_pubkey();
-- 
2.30.2





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

* [pve-devel] [PATCH common 1/3] schema: rename storagepair to storage-pair
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (6 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH access-control 2/2] ticket: normalize path for verification Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
  2021-11-05 13:03 ` [pve-devel] [PATCH common 2/3] schema: add pve-bridge-id option/format/pair Fabian Grünbichler
                   ` (15 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

more consistent with the other options/formats like pve-storage-id

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    v2: new
    
    this breaks older qemu-server without the patch adapting to the new name

 src/PVE/JSONSchema.pm | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/PVE/JSONSchema.pm b/src/PVE/JSONSchema.pm
index 71df690..52b3125 100644
--- a/src/PVE/JSONSchema.pm
+++ b/src/PVE/JSONSchema.pm
@@ -286,9 +286,9 @@ my $verify_idpair = sub {
 };
 
 # note: this only checks a single list entry
-# when using a storagepair-list map, you need to pass the full parameter to
+# when using a storage-pair-list map, you need to pass the full parameter to
 # parse_idmap
-register_format('storagepair', \&verify_storagepair);
+register_format('storage-pair', \&verify_storagepair);
 sub verify_storagepair {
     my ($storagepair, $noerr) = @_;
     return $verify_idpair->($storagepair, $noerr, 'pve-storage-id');
-- 
2.30.2





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

* [pve-devel] [PATCH common 2/3] schema: add pve-bridge-id option/format/pair
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (7 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH common 1/3] schema: rename storagepair to storage-pair Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
  2021-11-05 13:03 ` [pve-devel] [PATCH common 3/3] schema: add proxmox-remote format/option Fabian Grünbichler
                   ` (14 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

for re-use in qemu-server/pve-container, which already have this option
duplicated. the '-pair' is needed for remote migration, but can also be
a nice addition to regular intra-cluster migration to lift the
restriction of having identically named bridges.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    v2: rename to bridge-pair

 src/PVE/JSONSchema.pm | 26 ++++++++++++++++++++++++++
 1 file changed, 26 insertions(+)

diff --git a/src/PVE/JSONSchema.pm b/src/PVE/JSONSchema.pm
index 52b3125..09c8fc0 100644
--- a/src/PVE/JSONSchema.pm
+++ b/src/PVE/JSONSchema.pm
@@ -82,6 +82,12 @@ register_standard_option('pve-storage-id', {
     type => 'string', format => 'pve-storage-id',
 });
 
+register_standard_option('pve-bridge-id', {
+    description => "Bridge to attach guest network devices to.",
+    type => 'string', format => 'pve-bridge-id',
+    format_description => 'bridge',
+});
+
 register_standard_option('pve-config-digest', {
     description => 'Prevent changes if current configuration file has different SHA1 digest. This can be used to prevent concurrent modifications.',
     type => 'string',
@@ -193,6 +199,17 @@ sub parse_storage_id {
     return parse_id($storeid, 'storage', $noerr);
 }
 
+PVE::JSONSchema::register_format('pve-bridge-id', \&parse_bridge_id);
+sub parse_bridge_id {
+    my ($id, $noerr) = @_;
+
+    if ($id !~ m/^[-_.\w\d]+$/) {
+	return undef if $noerr;
+	die "invalid bridge ID '$id'\n";
+    }
+    return $id;
+}
+
 PVE::JSONSchema::register_format('acme-plugin-id', \&parse_acme_plugin_id);
 sub parse_acme_plugin_id {
     my ($pluginid, $noerr) = @_;
@@ -294,6 +311,15 @@ sub verify_storagepair {
     return $verify_idpair->($storagepair, $noerr, 'pve-storage-id');
 }
 
+# note: this only checks a single list entry
+# when using a bridge-pair-list map, you need to pass the full parameter to
+# parse_idmap
+register_format('bridge-pair', \&verify_bridgepair);
+sub verify_bridgepair {
+    my ($bridgepair, $noerr) = @_;
+    return $verify_idpair->($bridgepair, $noerr, 'pve-bridge-id');
+}
+
 register_format('mac-addr', \&pve_verify_mac_addr);
 sub pve_verify_mac_addr {
     my ($mac_addr, $noerr) = @_;
-- 
2.30.2





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

* [pve-devel] [PATCH common 3/3] schema: add proxmox-remote format/option
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (8 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH common 2/3] schema: add pve-bridge-id option/format/pair Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
  2021-11-05 13:03 ` [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration Fabian Grünbichler
                   ` (13 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

for usage in remote migration/replication.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 src/PVE/JSONSchema.pm | 27 +++++++++++++++++++++++++++
 1 file changed, 27 insertions(+)

diff --git a/src/PVE/JSONSchema.pm b/src/PVE/JSONSchema.pm
index 09c8fc0..38be3f8 100644
--- a/src/PVE/JSONSchema.pm
+++ b/src/PVE/JSONSchema.pm
@@ -653,6 +653,33 @@ register_standard_option('bwlimit', {
     format => $bwlimit_format,
 });
 
+my $remote_format = {
+    host => {
+	type => 'string',
+	format_description => 'Remote Proxmox hostname or IP',
+    },
+    port => {
+	type => 'integer',
+	optional => 1,
+    },
+    apitoken => {
+	type => 'string',
+	format_description => 'A full Proxmox API token including the secret value.',
+    },
+    fingerprint => get_standard_option(
+	'fingerprint-sha256',
+	{
+	    optional => 1,
+	    format_description => 'Remote host\'s certificate fingerprint, if not trusted by system store.',
+	}
+    ),
+};
+register_format('proxmox-remote', $remote_format);
+register_standard_option('proxmox-remote', {
+    description => "Specification of a remote endpoint.",
+    type => 'string', format => 'proxmox-remote',
+});
+
 # used for pve-tag-list in e.g., guest configs
 register_format('pve-tag', \&pve_verify_tag);
 sub pve_verify_tag {
-- 
2.30.2





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

* [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (9 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH common 3/3] schema: add proxmox-remote format/option Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-08 13:50   ` Fabian Ebner
  2021-11-10 12:03   ` Fabian Ebner
  2021-11-05 13:03 ` [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer Fabian Grünbichler
                   ` (12 subsequent siblings)
  23 siblings, 2 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

we only want to use an explicitly provided migration network, not one
for the local cluster.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 src/PVE/AbstractMigrate.pm | 51 +++++++++++++++++++++++---------------
 1 file changed, 31 insertions(+), 20 deletions(-)

diff --git a/src/PVE/AbstractMigrate.pm b/src/PVE/AbstractMigrate.pm
index af2be38..ec60b82 100644
--- a/src/PVE/AbstractMigrate.pm
+++ b/src/PVE/AbstractMigrate.pm
@@ -115,22 +115,27 @@ sub migrate {
 
     $class = ref($class) || $class;
 
-    my $dc_conf = PVE::Cluster::cfs_read_file('datacenter.cfg');
+    my ($ssh_info, $rem_ssh);
+    if (!$opts->{remote}) {
+	my $dc_conf = PVE::Cluster::cfs_read_file('datacenter.cfg');
 
-    my $migration_network = $opts->{migration_network};
-    if (!defined($migration_network)) {
-	$migration_network = $dc_conf->{migration}->{network};
-    }
-    my $ssh_info = PVE::SSHInfo::get_ssh_info($node, $migration_network);
-    $nodeip = $ssh_info->{ip};
-
-    my $migration_type = 'secure';
-    if (defined($opts->{migration_type})) {
-	$migration_type = $opts->{migration_type};
-    } elsif (defined($dc_conf->{migration}->{type})) {
-        $migration_type = $dc_conf->{migration}->{type};
+	my $migration_network = $opts->{migration_network};
+	if (!defined($migration_network)) {
+	    $migration_network = $dc_conf->{migration}->{network};
+	}
+	$ssh_info = PVE::SSHInfo::get_ssh_info($node, $migration_network);
+	$nodeip = $ssh_info->{ip};
+
+	my $migration_type = 'secure';
+	if (defined($opts->{migration_type})) {
+	    $migration_type = $opts->{migration_type};
+	} elsif (defined($dc_conf->{migration}->{type})) {
+	    $migration_type = $dc_conf->{migration}->{type};
+	}
+	$opts->{migration_type} = $migration_type;
+	$opts->{migration_network} = $migration_network;
+	$rem_ssh = PVE::SSHInfo::ssh_info_to_command($ssh_info);
     }
-    $opts->{migration_type} = $migration_type;
 
     my $self = {
 	delayed_interrupt => 0,
@@ -139,7 +144,7 @@ sub migrate {
 	node => $node,
 	ssh_info => $ssh_info,
 	nodeip => $nodeip,
-	rem_ssh => PVE::SSHInfo::ssh_info_to_command($ssh_info)
+	rem_ssh => $rem_ssh,
     };
 
     $self = bless $self, $class;
@@ -162,15 +167,21 @@ sub migrate {
 	&$eval_int($self, sub { $self->{running} = $self->prepare($self->{vmid}); });
 	die $@ if $@;
 
-	if (defined($migration_network)) {
+	if (defined($self->{opts}->{migration_network})) {
 	    $self->log('info', "use dedicated network address for sending " .
 	               "migration traffic ($self->{nodeip})");
 
 	    # test if we can connect to new IP
-	    my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
-	    eval { $self->cmd_quiet($cmd); };
-	    die "Can't connect to destination address ($self->{nodeip}) using " .
-	        "public key authentication\n" if $@;
+	    if ($self->{opts}->{remote}) {
+		eval { $self->{opts}->{remote}->{client}->get("/") };
+		die "Can't connect to destination address ($self->{nodeip}) using " .
+		    "API connection - $@\n" if $@;
+	    } else {
+		my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
+		eval { $self->cmd_quiet($cmd); };
+		die "Can't connect to destination address ($self->{nodeip}) using " .
+		    "public key authentication\n" if $@;
+	    }
 	}
 
 	&$eval_int($self, sub { $self->phase1($self->{vmid}); });
-- 
2.30.2





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

* [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (10 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-08 14:15   ` Fabian Ebner
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 01/10] d/control: add pve-ha-manager to B-D Fabian Grünbichler
                   ` (11 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

for proxied requests, we usually tear down the proxy connection
immediately when closing the source connection. this is not the correct
course of action for bulk one-way data streams that are proxied, where
the source connection might be closed, but the proxy connection might
still have data in the write buffer that needs to be written out.

push_shutdown already handles this case (closing the socket/FH after it
has been fully drained).

one example for such a proxied data stream is the 'migrate' data for a
remote migration, which gets proxied over a websocket connection.
terminating the proxied connection early makes the target VM crash for
obvious reasons.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 src/PVE/APIServer/AnyEvent.pm | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)

diff --git a/src/PVE/APIServer/AnyEvent.pm b/src/PVE/APIServer/AnyEvent.pm
index 86d0e2e..ecf771f 100644
--- a/src/PVE/APIServer/AnyEvent.pm
+++ b/src/PVE/APIServer/AnyEvent.pm
@@ -144,7 +144,8 @@ sub client_do_disconnect {
     };
 
     if (my $proxyhdl = delete $reqstate->{proxyhdl}) {
-	&$shutdown_hdl($proxyhdl);
+	&$shutdown_hdl($proxyhdl)
+		if !$proxyhdl->{block_disconnect};
     }
 
     my $hdl = delete $reqstate->{hdl};
@@ -627,9 +628,10 @@ sub websocket_proxy {
 		    } elsif ($opcode == 8) {
 			my $statuscode = unpack ("n", $payload);
 			$self->dprint("websocket received close. status code: '$statuscode'");
-			if ($reqstate->{proxyhdl}) {
-			    $reqstate->{proxyhdl}->push_shutdown();
-			}
+			if (my $proxyhdl = $reqstate->{proxyhdl}) {
+			    $proxyhdl->{block_disconnect} = 1 if length $proxyhdl->{wbuf} > 0;
+			    $proxyhdl->push_shutdown();
+		        }
 			$hdl->push_shutdown();
 		    } elsif ($opcode == 9) {
 			# ping received, schedule pong
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 01/10] d/control: add pve-ha-manager to B-D
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (11 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 02/10] adapt to renamed storage-pair format Fabian Grünbichler
                   ` (10 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

it's needed for tests to succeed.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 debian/control | 1 +
 1 file changed, 1 insertion(+)

diff --git a/debian/control b/debian/control
index 8349738..8032ae5 100644
--- a/debian/control
+++ b/debian/control
@@ -20,6 +20,7 @@ Build-Depends: debhelper (>= 12~),
                pve-doc-generator (>= 6.2-5),
                pve-edk2-firmware,
                pve-firewall,
+               pve-ha-manager <!nocheck>,
                pve-qemu-kvm,
 Standards-Version: 4.5.1
 Homepage: https://www.proxmox.com
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 02/10] adapt to renamed storage-pair format
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (12 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 01/10] d/control: add pve-ha-manager to B-D Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 03/10] migrate: factor out storage checks Fabian Grünbichler
                   ` (9 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    v2: new
    
    requires bumped dependency on pve-common with renaming patch

 PVE/QemuServer.pm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
index 763c412..a0e1809 100644
--- a/PVE/QemuServer.pm
+++ b/PVE/QemuServer.pm
@@ -137,7 +137,7 @@ sub map_storage {
 PVE::JSONSchema::register_standard_option('pve-targetstorage', {
     description => "Mapping from source to target storages. Providing only a single storage ID maps all source storages to that storage. Providing the special value '1' will map each source storage to itself.",
     type => 'string',
-    format => 'storagepair-list',
+    format => 'storage-pair-list',
     optional => 1,
 });
 
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 03/10] migrate: factor out storage checks
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (13 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 02/10] adapt to renamed storage-pair format Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id Fabian Grünbichler
                   ` (8 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

to re-use them for incoming remote migrations.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 PVE/API2/Qemu.pm | 26 ++++++++++++++------------
 1 file changed, 14 insertions(+), 12 deletions(-)

diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
index b479811..c290a42 100644
--- a/PVE/API2/Qemu.pm
+++ b/PVE/API2/Qemu.pm
@@ -134,6 +134,18 @@ my $check_storage_access_clone = sub {
    return $sharedvm;
 };
 
+my $check_storage_access_migrate = sub {
+    my ($rpcenv, $authuser, $storecfg, $storage, $node) = @_;
+
+    PVE::Storage::storage_check_enabled($storecfg, $storage, $node);
+
+    $rpcenv->check($authuser, "/storage/$storage", ['Datastore.AllocateSpace']);
+
+    my $scfg = PVE::Storage::storage_config($storecfg, $storage);
+    die "storage '$storage' does not support vm images\n"
+	if !$scfg->{content}->{images};
+};
+
 # Note: $pool is only needed when creating a VM, because pool permissions
 # are automatically inherited if VM already exists inside a pool.
 my $create_disks = sub {
@@ -3708,17 +3720,7 @@ __PACKAGE__->register_method({
 	}
 
 	my $storecfg = PVE::Storage::config();
-
 	if (my $targetstorage = $param->{targetstorage}) {
-	    my $check_storage = sub {
-		my ($target_sid) = @_;
-		PVE::Storage::storage_check_enabled($storecfg, $target_sid, $target);
-		$rpcenv->check($authuser, "/storage/$target_sid", ['Datastore.AllocateSpace']);
-		my $scfg = PVE::Storage::storage_config($storecfg, $target_sid);
-		raise_param_exc({ targetstorage => "storage '$target_sid' does not support vm images"})
-		    if !$scfg->{content}->{images};
-	    };
-
 	    my $storagemap = eval { PVE::JSONSchema::parse_idmap($targetstorage, 'pve-storage-id') };
 	    raise_param_exc({ targetstorage => "failed to parse storage map: $@" })
 		if $@;
@@ -3727,10 +3729,10 @@ __PACKAGE__->register_method({
 		if !defined($storagemap->{identity});
 
 	    foreach my $target_sid (values %{$storagemap->{entries}}) {
-		$check_storage->($target_sid);
+		$check_storage_access_migrate->($rpcenv, $authuser, $storecfg, $target_sid, $target);
 	    }
 
-	    $check_storage->($storagemap->{default})
+	    $check_storage_access_migrate->($rpcenv, $authuser, $storecfg, $storagemap->{default}, $target)
 		if $storagemap->{default};
 
 	    PVE::QemuServer::check_storage_availability($storecfg, $conf, $target)
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (14 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 03/10] migrate: factor out storage checks Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-09  9:06   ` Fabian Ebner
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 05/10] schema: use pve-bridge-id Fabian Grünbichler
                   ` (7 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

since we are going to reuse the same mechanism/code for network bridge
mapping.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 PVE/QemuMigrate.pm | 6 +++---
 PVE/QemuServer.pm  | 6 ++++--
 2 files changed, 7 insertions(+), 5 deletions(-)

diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
index ae3eaf1..779f5ee 100644
--- a/PVE/QemuMigrate.pm
+++ b/PVE/QemuMigrate.pm
@@ -342,7 +342,7 @@ sub prepare {
 	my $targetsid = $sid;
 	# NOTE: we currently ignore shared source storages in mappings so skip here too for now
 	if (!$scfg->{shared}) {
-	    $targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $sid);
+	    $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
 	}
 
 	my $target_scfg = PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
@@ -408,7 +408,7 @@ sub scan_local_volumes {
 
 	    next if @{$dl->{$storeid}} == 0;
 
-	    my $targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $storeid);
+	    my $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $storeid);
 	    # check if storage is available on target node
 	    my $target_scfg = PVE::Storage::storage_check_enabled(
 		$storecfg,
@@ -479,7 +479,7 @@ sub scan_local_volumes {
 	    my $targetsid = $sid;
 	    # NOTE: we currently ignore shared source storages in mappings so skip here too for now
 	    if (!$scfg->{shared}) {
-		$targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $sid);
+		$targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
 	    }
 
 	    PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
index a0e1809..23e7b08 100644
--- a/PVE/QemuServer.pm
+++ b/PVE/QemuServer.pm
@@ -120,7 +120,9 @@ PVE::JSONSchema::register_standard_option('pve-qemu-machine', {
 });
 
 
-sub map_storage {
+# maps source to target ID
+# currently used for targetstorage and targetbridge when migrating
+sub map_id {
     my ($map, $source) = @_;
 
     return $source if !defined($map);
@@ -5264,7 +5266,7 @@ sub vm_migrate_alloc_nbd_disks {
 	# volume is not available there, fall back to the default format.
 	# Otherwise use the same format as the original.
 	if (!$storagemap->{identity}) {
-	    $storeid = map_storage($storagemap, $storeid);
+	    $storeid = map_id($storagemap, $storeid);
 	    my ($defFormat, $validFormats) = PVE::Storage::storage_default_format($storecfg, $storeid);
 	    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
 	    my $fileFormat = qemu_img_format($scfg, $volname);
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 05/10] schema: use pve-bridge-id
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (15 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 06/10] update_vm: allow simultaneous setting of boot-order and dev Fabian Grünbichler
                   ` (6 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    requires pve-common with pve-bridge-id

 PVE/QemuServer.pm | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
index 23e7b08..de8c1bb 100644
--- a/PVE/QemuServer.pm
+++ b/PVE/QemuServer.pm
@@ -931,13 +931,10 @@ my $net_fmt = {
         default_key => 1,
     },
     (map { $_ => { keyAlias => 'model', alias => 'macaddr' }} @$nic_model_list),
-    bridge => {
-	type => 'string',
+    bridge => get_standard_option('pve-bridge-id', {
 	description => $net_fmt_bridge_descr,
-	format_description => 'bridge',
-	pattern => '[-_.\w\d]+',
 	optional => 1,
-    },
+    }),
     queues => {
 	type => 'integer',
 	minimum => 0, maximum => 16,
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 06/10] update_vm: allow simultaneous setting of boot-order and dev
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (16 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 05/10] schema: use pve-bridge-id Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints Fabian Grünbichler
                   ` (5 subsequent siblings)
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

else this fails if we check 'boot' before the device was put into
the config or pending section.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    as happens when doing a remote migration and the full config is passed through
    update_vm_api

 PVE/API2/Qemu.pm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
index c290a42..faf028b 100644
--- a/PVE/API2/Qemu.pm
+++ b/PVE/API2/Qemu.pm
@@ -1410,7 +1410,7 @@ my $update_vm_api  = sub {
 			if ($new_bootcfg->{order}) {
 			    my @devs = PVE::Tools::split_list($new_bootcfg->{order});
 			    for my $dev (@devs) {
-				my $exists = $conf->{$dev} || $conf->{pending}->{$dev};
+				my $exists = $conf->{$dev} || $conf->{pending}->{$dev} || $param->{$dev};
 				my $deleted = grep {$_ eq $dev} @delete;
 				die "invalid bootorder: device '$dev' does not exist'\n"
 				    if !$exists || $deleted;
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (17 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 06/10] update_vm: allow simultaneous setting of boot-order and dev Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-09 12:46   ` Fabian Ebner
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 08/10] migrate: refactor remote VM/tunnel start Fabian Grünbichler
                   ` (4 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

the following two endpoints are used for migration on the remote side

POST /nodes/NODE/qemu/VMID/mtunnel

which creates and locks an empty VM config, and spawns the main qmtunnel
worker which binds to a VM-specific UNIX socket.

this worker handles JSON-encoded migration commands coming in via this
UNIX socket:
- config (set target VM config)
-- checks permissions for updating config
-- strips pending changes and snapshots
-- sets (optional) firewall config
- disk (allocate disk for NBD migration)
-- checks permission for target storage
-- returns drive string for allocated volume
- disk-import (import 'pvesm export' stream for offline migration)
-- checks permission for target storage
-- forks a child running 'pvesm import' reading from a UNIX socket
-- only one import allowed to run at any given moment
- query-disk-import
-- checks output of 'pvesm import' for volume ID message
-- returns volid + success, or 'pending', or 'error'
- start (returning migration info)
- fstrim (via agent)
- bwlimit (query bwlimit for storage)
- ticket (creates a ticket for a WS connection to a specific socket)
- resume
- stop
- nbdstop
- unlock
- quit (+ cleanup)

this worker serves as a replacement for both 'qm mtunnel' and various
manual calls via SSH. the API call will return a ticket valid for
connecting to the worker's UNIX socket via a websocket connection.

GET+WebSocket upgrade /nodes/NODE/qemu/VMID/mtunnelwebsocket

gets called for connecting to a UNIX socket via websocket forwarding,
i.e. once for the main command mtunnel, and once each for the memory
migration and each NBD drive-mirror/storage migration.

access is guarded by a short-lived ticket binding the authenticated user
to the socket path. such tickets can be requested over the main mtunnel,
which keeps track of socket paths currently used by that
mtunnel/migration instance.

each command handler should check privileges for the requested action if
necessary.

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    requires
    - pve-storage with UNIX import support
    - pve-access-control with tunnel ticket support
    - pve-http-server with websocket fixes

 PVE/API2/Qemu.pm | 627 +++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 627 insertions(+)

diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
index faf028b..a1a1813 100644
--- a/PVE/API2/Qemu.pm
+++ b/PVE/API2/Qemu.pm
@@ -6,8 +6,13 @@ use Cwd 'abs_path';
 use Net::SSLeay;
 use POSIX;
 use IO::Socket::IP;
+use IO::Socket::UNIX;
+use IPC::Open3;
+use JSON;
+use MIME::Base64;
 use URI::Escape;
 use Crypt::OpenSSL::Random;
+use Socket qw(SOCK_STREAM);
 
 use PVE::Cluster qw (cfs_read_file cfs_write_file);;
 use PVE::RRD;
@@ -856,6 +861,7 @@ __PACKAGE__->register_method({
 	    { subdir => 'spiceproxy' },
 	    { subdir => 'sendkey' },
 	    { subdir => 'firewall' },
+	    { subdir => 'mtunnel' },
 	    ];
 
 	return $res;
@@ -4428,4 +4434,625 @@ __PACKAGE__->register_method({
 	return PVE::QemuServer::Cloudinit::dump_cloudinit_config($conf, $param->{vmid}, $param->{type});
     }});
 
+__PACKAGE__->register_method({
+    name => 'mtunnel',
+    path => '{vmid}/mtunnel',
+    method => 'POST',
+    protected => 1,
+    proxyto => 'node',
+    description => 'Migration tunnel endpoint - only for internal use by VM migration.',
+    permissions => {
+	check => ['perm', '/vms/{vmid}', [ 'VM.Allocate' ]],
+	description => "You need 'VM.Allocate' permissions on /vms/{vmid}. Further permission checks happen during the actual migration.",
+    },
+    parameters => {
+	additionalProperties => 0,
+	properties => {
+	    node => get_standard_option('pve-node'),
+	    vmid => get_standard_option('pve-vmid'),
+	    storages => {
+		type => 'string',
+		format => 'pve-storage-id-list',
+		optional => 1,
+		description => 'List of storages to check permission and availability. Will be checked again for all actually used storages during migration.',
+	    },
+	},
+    },
+    returns => {
+	additionalProperties => 0,
+	properties => {
+	    upid => { type => 'string' },
+	    ticket => { type => 'string' },
+	    socket => { type => 'string' },
+	},
+    },
+    code => sub {
+	my ($param) = @_;
+
+	my $rpcenv = PVE::RPCEnvironment::get();
+	my $authuser = $rpcenv->get_user();
+
+	my $node = extract_param($param, 'node');
+	my $vmid = extract_param($param, 'vmid');
+
+	my $storages = extract_param($param, 'storages');
+
+	my $storecfg = PVE::Storage::config();
+	foreach my $storeid (PVE::Tools::split_list($storages)) {
+	    $check_storage_access_migrate->($rpcenv, $authuser, $storecfg, $storeid, $node);
+	}
+
+	PVE::Cluster::check_cfs_quorum();
+
+	my $socket_addr = "/run/qemu-server/$vmid.mtunnel";
+
+	my $lock = 'create';
+	eval { PVE::QemuConfig->create_and_lock_config($vmid, 0, $lock); };
+
+	raise_param_exc({ vmid => "unable to create empty VM config - $@"})
+	    if $@;
+
+	my $realcmd = sub {
+	    my $pveproxy_uid;
+
+	    my $state = {
+		storecfg => PVE::Storage::config(),
+		lock => $lock,
+	    };
+
+	    my $run_locked = sub {
+		my ($code, $params) = @_;
+		return PVE::QemuConfig->lock_config($vmid, sub {
+		    my $conf = PVE::QemuConfig->load_config($vmid);
+
+		    $state->{conf} = $conf;
+
+		    die "Encountered wrong lock - aborting mtunnel command handling.\n"
+			if $state->{lock} && !PVE::QemuConfig->has_lock($conf, $state->{lock});
+
+		    return $code->($params);
+		});
+	    };
+
+	    my $cmd_desc = {
+		bwlimit => {
+		    storage => {
+			type => 'string',
+			format => 'pve-storage-id',
+			description => "Storage for which bwlimit is queried",
+		    },
+		    bwlimit => {
+			description => "Override I/O bandwidth limit (in KiB/s).",
+			optional => 1,
+			type => 'integer',
+			minimum => '0',
+		    },
+		},
+		config => {
+		    conf => {
+			type => 'string',
+			description => 'Full VM config, adapted for target cluster/node',
+		    },
+		    'firewall-conf' => {
+			type => 'string',
+			description => 'VM firewall config',
+			optional => 1,
+		    },
+		},
+		disk => {
+		    format => PVE::JSONSchema::get_standard_option('pve-qm-image-format'),
+		    storage => {
+			type => 'string',
+			format => 'pve-storage-id',
+		    },
+		    drive => {
+			type => 'object',
+			description => 'parsed drive information without volid and format',
+		    },
+		},
+		'disk-import' => {
+		    volname => {
+			type => 'string',
+			description => 'volume name to use prefered target volume name',
+		    },
+		    format => PVE::JSONSchema::get_standard_option('pve-qm-image-format'),
+		    'export-formats' => {
+			type => 'string',
+			description => 'list of supported export formats',
+		    },
+		    storage => {
+			type => 'string',
+			format => 'pve-storage-id',
+		    },
+		    'with-snapshots' => {
+			description =>
+			    "Whether the stream includes intermediate snapshots",
+			type => 'boolean',
+			optional => 1,
+			default => 0,
+		    },
+		    'allow-rename' => {
+			description => "Choose a new volume ID if the requested " .
+			  "volume ID already exists, instead of throwing an error.",
+			type => 'boolean',
+			optional => 1,
+			default => 0,
+		    },
+		},
+		start => {
+		    start_params => {
+			type => 'object',
+			description => 'params passed to vm_start_nolock',
+		    },
+		    migrate_opts => {
+			type => 'object',
+			description => 'migrate_opts passed to vm_start_nolock',
+		    },
+		},
+		ticket => {
+		    path => {
+			type => 'string',
+			description => 'socket path for which the ticket should be valid. must be known to current mtunnel instance.',
+		    },
+		},
+		quit => {
+		    cleanup => {
+			type => 'boolean',
+			description => 'remove VM config and disks, aborting migration',
+			default => 0,
+		    },
+		},
+	    };
+
+	    my $cmd_handlers = {
+		'version' => sub {
+		    # compared against other end's version
+		    # bump/reset both for breaking changes
+		    # bump tunnel only for opt-in changes
+		    return {
+			api => 2,
+			age => 0,
+		    };
+		},
+		'config' => sub {
+		    my ($params) = @_;
+
+		    # parse and write out VM FW config if given
+		    if (my $fw_conf = $params->{'firewall-config'}) {
+			my ($path, $fh) = PVE::Tools::tempfile_contents($fw_conf, 700);
+
+			my $empty_conf = {
+			    rules => [],
+			    options => {},
+			    aliases => {},
+			    ipset => {} ,
+			    ipset_comments => {},
+			};
+			my $cluster_fw_conf = PVE::Firewall::load_clusterfw_conf();
+
+			# TODO: add flag for strict parsing?
+			# TODO: add import sub that does all this given raw content?
+			my $vmfw_conf = PVE::Firewall::generic_fw_config_parser($path, $cluster_fw_conf, $empty_conf, 'vm');
+			$vmfw_conf->{vmid} = $vmid;
+			PVE::Firewall::save_vmfw_conf($vmid, $vmfw_conf);
+
+			$state->{cleanup}->{fw} = 1;
+		    }
+
+		    PVE::QemuConfig->remove_lock($vmid, 'create');
+
+		    # TODO add flag for strict parsing?
+		    my $new_conf = PVE::QemuServer::parse_vm_config("incoming/qemu-server/$vmid.conf", $params->{conf});
+		    delete $new_conf->{lock};
+		    delete $new_conf->{digest};
+
+		    # TODO handle properly?
+		    delete $new_conf->{snapshots};
+		    delete $new_conf->{pending};
+
+		    # not handled by update_vm_api
+		    my $vmgenid = delete $new_conf->{vmgenid};
+		    my $meta = delete $new_conf->{meta};
+
+		    $new_conf->{vmid} = $vmid;
+		    $new_conf->{node} = $node;
+
+		    $update_vm_api->($new_conf, 1);
+
+		    my $conf = PVE::QemuConfig->load_config($vmid);
+		    $conf->{lock} = 'migrate';
+		    $conf->{vmgenid} = $vmgenid;
+		    $conf->{meta} = $meta;
+		    PVE::QemuConfig->write_config($vmid, $conf);
+
+		    $state->{lock} = 'migrate';
+
+		    return;
+		},
+		'bwlimit' => sub {
+		    my ($params) = @_;
+
+		    my $bwlimit = PVE::Storage::get_bandwidth_limit('migration', [$params->{storage}], $params->{bwlimit});
+		    return { bwlimit => $bwlimit };
+
+		},
+		'disk' => sub {
+		    my ($params) = @_;
+
+		    my $format = $params->{format};
+		    my $storeid = $params->{storage};
+		    my $drive = $params->{drive};
+
+		    $check_storage_access_migrate->($rpcenv, $authuser, $state->{storecfg}, $storeid, $node);
+
+		    my ($default_format, $valid_formats) = PVE::Storage::storage_default_format($state->{storecfg}, $storeid);
+		    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
+		    $format = $default_format
+			if !grep {$format eq $_} @{$valid_formats};
+
+		    my $size = int($drive->{size})/1024;
+		    my $newvolid = PVE::Storage::vdisk_alloc($state->{storecfg}, $storeid, $vmid, $format, undef, $size);
+
+		    my $newdrive = $drive;
+		    $newdrive->{format} = $format;
+		    $newdrive->{file} = $newvolid;
+
+		    $state->{cleanup}->{volumes}->{$newvolid} = 1;
+		    my $drivestr = PVE::QemuServer::print_drive($newdrive);
+		    return {
+			drivestr => $drivestr,
+			volid => $newvolid,
+		    };
+		},
+		'disk-import' => sub {
+		    my ($params) = @_;
+
+		    die "disk import already running as PID '$state->{disk_import}->{pid}'\n"
+			if $state->{disk_import}->{pid};
+
+		    my $format = $params->{format};
+		    my $storeid = $params->{storage};
+		    $check_storage_access_migrate->($rpcenv, $authuser, $state->{storecfg}, $storeid, $node);
+
+		    my $with_snapshots = $params->{'with-snapshots'} ? 1 : 0;
+
+		    my ($default_format, $valid_formats) = PVE::Storage::storage_default_format($state->{storecfg}, $storeid);
+		    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
+		    die "unsupported format '$format' for storage '$storeid'\n"
+			if !grep {$format eq $_} @{$valid_formats};
+
+		    my $volname = $params->{volname};
+
+		    # get target volname, taken from PVE::Storage
+		    (my $name_without_extension = $volname) =~ s/\.$format$//;
+		    if ($scfg->{path}) {
+			$volname = "$vmid/$name_without_extension.$format";
+		    } else {
+			$volname = "$name_without_extension";
+		    }
+
+		    my $migration_snapshot;
+		    if ($scfg->{type} eq 'zfspool' || $scfg->{type} eq 'btrfs') {
+			$migration_snapshot = '__migration__';
+		    }
+
+		    my $volid = "$storeid:$volname";
+
+		    # find common import/export format, taken from PVE::Storage
+		    my @import_formats = PVE::Storage::volume_import_formats($state->{storecfg}, $volid, $migration_snapshot, undef, $with_snapshots);
+		    my @export_formats = PVE::Tools::split_list($params->{'export-formats'});
+		    my %import_hash = map { $_ => 1 } @import_formats;
+		    my @common = grep { $import_hash{$_} } @export_formats;
+		    die "no matching import/export format found for storage '$storeid'\n"
+			if !@common;
+		    $format = $common[0];
+
+		    my $input = IO::File->new();
+		    my $info = IO::File->new();
+		    my $unix = "/run/qemu-server/$vmid.storage";
+
+		    my $import_cmd = ['pvesm', 'import', $volid, $format, "unix://$unix", '-with-snapshots', $with_snapshots];
+		    if ($params->{'allow-rename'}) {
+			push @$import_cmd, '-allow-rename', $params->{'allow-rename'};
+		    }
+		    if ($migration_snapshot) {
+			push @$import_cmd, '-delete-snapshot', $migration_snapshot;
+		    }
+
+		    unlink $unix;
+		    my $cpid = open3($input, $info, $info, @{$import_cmd})
+			or die "failed to spawn disk-import child - $!\n";
+
+		    $state->{disk_import}->{pid} = $cpid;
+		    my $ready;
+		    eval {
+			PVE::Tools::run_with_timeout(5, sub { $ready = <$info>; });
+		    };
+		    die "failed to read readyness from disk import child: $@\n" if $@;
+		    print "$ready\n";
+
+		    chown $pveproxy_uid, -1, $unix;
+
+		    $state->{disk_import}->{fh} = $info;
+		    $state->{disk_import}->{socket} = $unix;
+
+		    $state->{sockets}->{$unix} = 1;
+
+		    return {
+			socket => $unix,
+			format => $format,
+		    };
+		},
+		'query-disk-import' => sub {
+		    my ($params) = @_;
+
+		    die "no disk import running\n"
+			if !$state->{disk_import}->{pid};
+
+		    my $pattern = PVE::Storage::volume_imported_message(undef, 1);
+		    my $result;
+		    eval {
+			my $fh = $state->{disk_import}->{fh};
+			PVE::Tools::run_with_timeout(5, sub { $result = <$fh>; });
+			print "disk-import: $result\n" if $result;
+		    };
+		    if ($result && $result =~ $pattern) {
+			my $volid = $1;
+			waitpid($state->{disk_import}->{pid}, 0);
+
+			my $unix = $state->{disk_import}->{socket};
+			unlink $unix;
+			delete $state->{sockets}->{$unix};
+			delete $state->{disk_import};
+			return {
+			    status => "complete",
+			    volid => $volid,
+			};
+		    } elsif (!$result && waitpid($state->{disk_import}->{pid}, WNOHANG)) {
+			my $unix = $state->{disk_import}->{socket};
+			unlink $unix;
+			delete $state->{sockets}->{$unix};
+			delete $state->{disk_import};
+
+			return {
+			    status => "error",
+			};
+		    } else {
+			return {
+			    status => "pending",
+			};
+		    }
+		},
+		'start' => sub {
+		    my ($params) = @_;
+
+		    my $info = PVE::QemuServer::vm_start_nolock(
+			$state->{storecfg},
+			$vmid,
+			$state->{conf},
+			$params->{start_params},
+			$params->{migrate_opts},
+		    );
+
+
+		    if ($info->{migrate}->{proto} ne 'unix') {
+			PVE::QemuServer::vm_stop(undef, $vmid, 1, 1);
+			die "migration over non-UNIX sockets not possible\n";
+		    }
+
+		    my $socket = $info->{migrate}->{addr};
+		    chown $pveproxy_uid, -1, $socket;
+		    $state->{sockets}->{$socket} = 1;
+
+		    my $unix_sockets = $info->{migrate}->{unix_sockets};
+		    foreach my $socket (@$unix_sockets) {
+			chown $pveproxy_uid, -1, $socket;
+			$state->{sockets}->{$socket} = 1;
+		    }
+		    return $info;
+		},
+		'fstrim' => sub {
+		    if (PVE::QemuServer::qga_check_running($vmid)) {
+			eval { mon_cmd($vmid, "guest-fstrim") };
+			warn "fstrim failed: $@\n" if $@;
+		    }
+		    return;
+		},
+		'stop' => sub {
+		    PVE::QemuServer::vm_stop(undef, $vmid, 1, 1);
+		    return;
+		},
+		'nbdstop' => sub {
+		    PVE::QemuServer::nbd_stop($vmid);
+		    return;
+		},
+		'resume' => sub {
+		    if (PVE::QemuServer::check_running($vmid, 1)) {
+			PVE::QemuServer::vm_resume($vmid, 1, 1);
+		    } else {
+			die "VM $vmid not running\n";
+		    }
+		    return;
+		},
+		'unlock' => sub {
+		    PVE::QemuConfig->remove_lock($vmid, $state->{lock});
+		    delete $state->{lock};
+		    return;
+		},
+		'ticket' => sub {
+		    my ($params) = @_;
+
+		    my $path = $params->{path};
+
+		    die "Not allowed to generate ticket for unknown socket '$path'\n"
+			if !defined($state->{sockets}->{$path});
+
+		    return { ticket => PVE::AccessControl::assemble_tunnel_ticket($authuser, "/socket/$path") };
+		},
+		'quit' => sub {
+		    my ($params) = @_;
+
+		    if ($params->{cleanup}) {
+			if ($state->{cleanup}->{fw}) {
+			    PVE::Firewall::remove_vmfw_conf($vmid);
+			}
+
+			if (my @volumes = keys $state->{cleanup}->{volumes}->$%) {
+			    PVE::Storage::foreach_volid(@volumes, sub {
+				my ($volid, $sid, $volname, $d) = @_;
+
+				print "freeing volume '$volid' as part of cleanup\n";
+				eval { PVE::Storage::vdisk_free($storecfg, $volid) };
+				warn $@ if $@;
+			    });
+			}
+
+			PVE::QemuServer::destroy_vm($state->{storecfg}, $vmid, 1);
+		    }
+
+		    $state->{exit} = 1;
+		    return;
+		},
+	    };
+
+	    $run_locked->(sub {
+		my $socket_addr = "/run/qemu-server/$vmid.mtunnel";
+		unlink $socket_addr;
+
+		$state->{socket} = IO::Socket::UNIX->new(
+	            Type => SOCK_STREAM(),
+		    Local => $socket_addr,
+		    Listen => 1,
+		);
+
+		$pveproxy_uid = getpwnam('www-data')
+		    or die "Failed to resolve user 'www-data' to numeric UID\n";
+		chown $pveproxy_uid, -1, $socket_addr;
+	    });
+
+	    print "mtunnel started\n";
+
+	    my $conn = $state->{socket}->accept();
+
+	    $state->{conn} = $conn;
+
+	    my $reply_err = sub {
+		my ($msg) = @_;
+
+		my $reply = JSON::encode_json({
+		    success => JSON::false,
+		    msg => $msg,
+		});
+		$conn->print("$reply\n");
+		$conn->flush();
+	    };
+
+	    my $reply_ok = sub {
+		my ($res) = @_;
+
+		$res->{success} = JSON::true;
+		my $reply = JSON::encode_json($res);
+		$conn->print("$reply\n");
+		$conn->flush();
+	    };
+
+	    while (my $line = <$conn>) {
+		chomp $line;
+
+		# untaint, we validate below if needed
+		($line) = $line =~ /^(.*)$/;
+		my $parsed = eval { JSON::decode_json($line) };
+		if ($@) {
+		    $reply_err->("failed to parse command - $@");
+		    next;
+		}
+
+		my $cmd = delete $parsed->{cmd};
+		if (!defined($cmd)) {
+		    $reply_err->("'cmd' missing");
+		} elsif (my $handler = $cmd_handlers->{$cmd}) {
+		    print "received command '$cmd'\n";
+		    eval {
+			if ($cmd_desc->{$cmd}) {
+			    PVE::JSONSchema::validate($cmd_desc->{$cmd}, $parsed);
+			} else {
+			    $parsed = {};
+			}
+			my $res = $run_locked->($handler, $parsed);
+			$reply_ok->($res);
+		    };
+		    $reply_err->("failed to handle '$cmd' command - $@")
+			if $@;
+		} else {
+		    $reply_err->("unknown command '$cmd' given");
+		}
+
+		if ($state->{exit}) {
+		    $state->{conn}->close();
+		    $state->{socket}->close();
+		    last;
+		}
+	    }
+
+	    print "mtunnel exited\n";
+	};
+
+	my $ticket = PVE::AccessControl::assemble_tunnel_ticket($authuser, "/socket/$socket_addr");
+	my $upid = $rpcenv->fork_worker('qmtunnel', $vmid, $authuser, $realcmd);
+
+	return {
+	    ticket => $ticket,
+	    upid => $upid,
+	    socket => $socket_addr,
+	};
+    }});
+
+__PACKAGE__->register_method({
+    name => 'mtunnelwebsocket',
+    path => '{vmid}/mtunnelwebsocket',
+    method => 'GET',
+    proxyto => 'node',
+    permissions => {
+	description => "You need to pass a ticket valid for the selected socket. Tickets can be created via the mtunnel API call, which will check permissions accordingly.",
+        user => 'all', # check inside
+    },
+    description => 'Migration tunnel endpoint for websocket upgrade - only for internal use by VM migration.',
+    parameters => {
+	additionalProperties => 0,
+	properties => {
+	    node => get_standard_option('pve-node'),
+	    vmid => get_standard_option('pve-vmid'),
+	    socket => {
+		type => "string",
+		description => "unix socket to forward to",
+	    },
+	    ticket => {
+		type => "string",
+		description => "ticket return by initial 'mtunnel' API call, or retrieved via 'ticket' tunnel command",
+	    },
+	},
+    },
+    returns => {
+	type => "object",
+	properties => {
+	    port => { type => 'string', optional => 1 },
+	    socket => { type => 'string', optional => 1 },
+	},
+    },
+    code => sub {
+	my ($param) = @_;
+
+	my $rpcenv = PVE::RPCEnvironment::get();
+	my $authuser = $rpcenv->get_user();
+
+	my $vmid = $param->{vmid};
+	# check VM exists
+	PVE::QemuConfig->load_config($vmid);
+
+	my $socket = $param->{socket};
+	PVE::AccessControl::verify_tunnel_ticket($param->{ticket}, $authuser, "/socket/$socket");
+
+	return { socket => $socket };
+    }});
+
 1;
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 08/10] migrate: refactor remote VM/tunnel start
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (18 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-09 14:04   ` Fabian Ebner
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling Fabian Grünbichler
                   ` (3 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

no semantic changes intended, except for:
- no longer passing the main migration UNIX socket to SSH twice for
forwarding
- dropping the 'unix:' prefix in start_remote_tunnel's timeout error message

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 PVE/QemuMigrate.pm | 158 ++++++++++++++++++++++++++++-----------------
 PVE/QemuServer.pm  |  32 ++++-----
 2 files changed, 113 insertions(+), 77 deletions(-)

diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
index 779f5ee..07b56eb 100644
--- a/PVE/QemuMigrate.pm
+++ b/PVE/QemuMigrate.pm
@@ -206,19 +206,24 @@ sub finish_tunnel {
     die $err if $err;
 }
 
+# tunnel_info:
+#   proto: unix (secure) or tcp (insecure/legacy compat)
+#   addr: IP or UNIX socket path
+#   port: optional TCP port
+#   unix_sockets: additional UNIX socket paths to forward
 sub start_remote_tunnel {
-    my ($self, $raddr, $rport, $ruri, $unix_socket_info) = @_;
+    my ($self, $tunnel_info) = @_;
 
     my $nodename = PVE::INotify::nodename();
     my $migration_type = $self->{opts}->{migration_type};
 
     if ($migration_type eq 'secure') {
 
-	if ($ruri =~ /^unix:/) {
-	    my $ssh_forward_info = ["$raddr:$raddr"];
-	    $unix_socket_info->{$raddr} = 1;
+	if ($tunnel_info->{proto} eq 'unix') {
+	    my $ssh_forward_info = [];
 
-	    my $unix_sockets = [ keys %$unix_socket_info ];
+	    my $unix_sockets = [ keys %{$tunnel_info->{unix_sockets}} ];
+	    push @$unix_sockets, $tunnel_info->{addr};
 	    for my $sock (@$unix_sockets) {
 		push @$ssh_forward_info, "$sock:$sock";
 		unlink $sock;
@@ -245,23 +250,23 @@ sub start_remote_tunnel {
 	    if ($unix_socket_try > 100) {
 		$self->{errors} = 1;
 		$self->finish_tunnel($self->{tunnel});
-		die "Timeout, migration socket $ruri did not get ready";
+		die "Timeout, migration socket $tunnel_info->{addr} did not get ready";
 	    }
 	    $self->{tunnel}->{unix_sockets} = $unix_sockets if (@$unix_sockets);
 
-	} elsif ($ruri =~ /^tcp:/) {
+	} elsif ($tunnel_info->{proto} eq 'tcp') {
 	    my $ssh_forward_info = [];
-	    if ($raddr eq "localhost") {
+	    if ($tunnel_info->{addr} eq "localhost") {
 		# for backwards compatibility with older qemu-server versions
 		my $pfamily = PVE::Tools::get_host_address_family($nodename);
 		my $lport = PVE::Tools::next_migrate_port($pfamily);
-		push @$ssh_forward_info, "$lport:localhost:$rport";
+		push @$ssh_forward_info, "$lport:localhost:$tunnel_info->{rporyt}";
 	    }
 
 	    $self->{tunnel} = $self->fork_tunnel($ssh_forward_info);
 
 	} else {
-	    die "unsupported protocol in migration URI: $ruri\n";
+	    die "unsupported protocol in migration URI: $tunnel_info->{proto}\n";
 	}
     } else {
 	#fork tunnel for insecure migration, to send faster commands like resume
@@ -813,52 +818,40 @@ sub phase1_cleanup {
     }
 }
 
-sub phase2 {
-    my ($self, $vmid) = @_;
+sub phase2_start_local_cluster {
+    my ($self, $vmid, $params) = @_;
 
     my $conf = $self->{vmconf};
     my $local_volumes = $self->{local_volumes};
     my @online_local_volumes = $self->filter_local_volumes('online');
 
     $self->{storage_migration} = 1 if scalar(@online_local_volumes);
+    my $start = $params->{start_params};
+    my $migrate = $params->{migrate_opts};
 
     $self->log('info', "starting VM $vmid on remote node '$self->{node}'");
 
-    my $raddr;
-    my $rport;
-    my $ruri; # the whole migration dst. URI (protocol:address[:port])
-    my $nodename = PVE::INotify::nodename();
+    my $tunnel_info = {};
 
     ## start on remote node
     my $cmd = [@{$self->{rem_ssh}}];
 
-    my $spice_ticket;
-    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
-	my $res = mon_cmd($vmid, 'query-spice');
-	$spice_ticket = $res->{ticket};
-    }
+    push @$cmd, 'qm', 'start', $vmid, '--skiplock';
+    push @$cmd, '--migratedfrom', $migrate->{migratedfrom};
 
-    push @$cmd , 'qm', 'start', $vmid, '--skiplock', '--migratedfrom', $nodename;
+    push @$cmd, '--migration_type', $migrate->{type};
 
-    my $migration_type = $self->{opts}->{migration_type};
+    push @$cmd, '--migration_network', $migrate->{network}
+      if $migrate->{migration_network};
 
-    push @$cmd, '--migration_type', $migration_type;
+    push @$cmd, '--stateuri', $start->{statefile};
 
-    push @$cmd, '--migration_network', $self->{opts}->{migration_network}
-      if $self->{opts}->{migration_network};
-
-    if ($migration_type eq 'insecure') {
-	push @$cmd, '--stateuri', 'tcp';
-    } else {
-	push @$cmd, '--stateuri', 'unix';
+    if ($start->{forcemachine}) {
+	push @$cmd, '--machine', $start->{forcemachine};
     }
 
-    if ($self->{forcemachine}) {
-	push @$cmd, '--machine', $self->{forcemachine};
-    }
-
-    if ($self->{forcecpu}) {
-	push @$cmd, '--force-cpu', $self->{forcecpu};
+    if ($start->{forcecpu}) {
+	push @$cmd, '--force-cpu', $start->{forcecpu};
     }
 
     if ($self->{storage_migration}) {
@@ -866,11 +859,8 @@ sub phase2 {
     }
 
     my $spice_port;
-    my $unix_socket_info = {};
-    # version > 0 for unix socket support
-    my $nbd_protocol_version = 1;
-    my $input = "nbd_protocol_version: $nbd_protocol_version\n";
-    $input .= "spice_ticket: $spice_ticket\n" if $spice_ticket;
+    my $input = "nbd_protocol_version: $migrate->{nbd_proto_version}\n";
+    $input .= "spice_ticket: $migrate->{spice_ticket}\n" if $migrate->{spice_ticket};
 
     my @online_replicated_volumes = $self->filter_local_volumes('online', 1);
     foreach my $volid (@online_replicated_volumes) {
@@ -900,20 +890,20 @@ sub phase2 {
     my $exitcode = PVE::Tools::run_command($cmd, input => $input, outfunc => sub {
 	my $line = shift;
 
-	if ($line =~ m/^migration listens on tcp:(localhost|[\d\.]+|\[[\d\.:a-fA-F]+\]):(\d+)$/) {
-	    $raddr = $1;
-	    $rport = int($2);
-	    $ruri = "tcp:$raddr:$rport";
+	if ($line =~ m/^migration listens on (tcp):(localhost|[\d\.]+|\[[\d\.:a-fA-F]+\]):(\d+)$/) {
+	    $tunnel_info->{addr} = $2;
+	    $tunnel_info->{port} = int($3);
+	    $tunnel_info->{proto} = $1;
 	}
-	elsif ($line =~ m!^migration listens on unix:(/run/qemu-server/(\d+)\.migrate)$!) {
-	    $raddr = $1;
-	    die "Destination UNIX sockets VMID does not match source VMID" if $vmid ne $2;
-	    $ruri = "unix:$raddr";
+	elsif ($line =~ m!^migration listens on (unix):(/run/qemu-server/(\d+)\.migrate)$!) {
+	    $tunnel_info->{addr} = $2;
+	    die "Destination UNIX sockets VMID does not match source VMID" if $vmid ne $3;
+	    $tunnel_info->{proto} = $1;
 	}
 	elsif ($line =~ m/^migration listens on port (\d+)$/) {
-	    $raddr = "localhost";
-	    $rport = int($1);
-	    $ruri = "tcp:$raddr:$rport";
+	    $tunnel_info->{addr} = "localhost";
+	    $tunnel_info->{port} = int($1);
+	    $tunnel_info->{proto} = "tcp";
 	}
 	elsif ($line =~ m/^spice listens on port (\d+)$/) {
 	    $spice_port = int($1);
@@ -934,7 +924,7 @@ sub phase2 {
 	    $targetdrive =~ s/drive-//g;
 
 	    $handle_storage_migration_listens->($targetdrive, $drivestr, $nbd_uri);
-	    $unix_socket_info->{$nbd_unix_addr} = 1;
+	    $tunnel_info->{unix_sockets}->{$nbd_unix_addr} = 1;
 	} elsif ($line =~ m/^re-using replicated volume: (\S+) - (.*)$/) {
 	    my $drive = $1;
 	    my $volid = $2;
@@ -949,19 +939,65 @@ sub phase2 {
 
     die "remote command failed with exit code $exitcode\n" if $exitcode;
 
-    die "unable to detect remote migration address\n" if !$raddr;
+    die "unable to detect remote migration address\n" if !$tunnel_info->{addr} || !$tunnel_info->{proto};
 
     if (scalar(keys %$target_replicated_volumes) != scalar(@online_replicated_volumes)) {
 	die "number of replicated disks on source and target node do not match - target node too old?\n"
     }
 
+    return ($tunnel_info, $spice_port);
+}
+
+sub phase2 {
+    my ($self, $vmid) = @_;
+
+    my $conf = $self->{vmconf};
+
+    # version > 0 for unix socket support
+    my $nbd_protocol_version = 1;
+
+    my $spice_ticket;
+    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
+	my $res = mon_cmd($vmid, 'query-spice');
+	$spice_ticket = $res->{ticket};
+    }
+
+    my $migration_type = $self->{opts}->{migration_type};
+    my $state_uri = $migration_type eq 'insecure' ? 'tcp' : 'unix';
+
+    my $params = {
+	start_params => {
+	    statefile => $state_uri,
+	    forcemachine => $self->{forcemachine},
+	    forcecpu => $self->{forcecpu},
+	    skiplock => 1,
+	},
+	migrate_opts => {
+	    spice_ticket => $spice_ticket,
+	    type => $migration_type,
+	    network => $self->{opts}->{migration_network},
+	    storagemap => $self->{opts}->{storagemap},
+	    migratedfrom => PVE::INotify::nodename(),
+	    nbd_proto_version => $nbd_protocol_version,
+	    nbd => $self->{nbd},
+	},
+    };
+
+    my ($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
+
     $self->log('info', "start remote tunnel");
-    $self->start_remote_tunnel($raddr, $rport, $ruri, $unix_socket_info);
+    $self->start_remote_tunnel($tunnel_info);
+
+    my $migrate_uri = "$tunnel_info->{proto}:$tunnel_info->{addr}";
+    $migrate_uri .= ":$tunnel_info->{port}"
+	if defined($tunnel_info->{port});
 
     if ($self->{storage_migration}) {
 	$self->{storage_migration_jobs} = {};
 	$self->log('info', "starting storage migration");
 
+	my @online_local_volumes = $self->filter_local_volumes('online');
+
 	die "The number of local disks does not match between the source and the destination.\n"
 	    if (scalar(keys %{$self->{target_drive}}) != scalar(@online_local_volumes));
 	foreach my $drive (keys %{$self->{target_drive}}){
@@ -971,7 +1007,7 @@ sub phase2 {
 	    my $source_drive = PVE::QemuServer::parse_drive($drive, $conf->{$drive});
 	    my $source_volid = $source_drive->{file};
 
-	    my $bwlimit = $local_volumes->{$source_volid}->{bwlimit};
+	    my $bwlimit = $self->{local_volumes}->{$source_volid}->{bwlimit};
 	    my $bitmap = $target->{bitmap};
 
 	    $self->log('info', "$drive: start migration to $nbd_uri");
@@ -979,7 +1015,7 @@ sub phase2 {
 	}
     }
 
-    $self->log('info', "starting online/live migration on $ruri");
+    $self->log('info', "starting online/live migration on $migrate_uri");
     $self->{livemigration} = 1;
 
     # load_defaults
@@ -1056,12 +1092,12 @@ sub phase2 {
 
     my $start = time();
 
-    $self->log('info', "start migrate command to $ruri");
+    $self->log('info', "start migrate command to $migrate_uri");
     eval {
-	mon_cmd($vmid, "migrate", uri => $ruri);
+	mon_cmd($vmid, "migrate", uri => $migrate_uri);
     };
     my $merr = $@;
-    $self->log('info', "migrate uri => $ruri failed: $merr") if $merr;
+    $self->log('info', "migrate uri => $migrate_uri failed: $merr") if $merr;
 
     my $last_mem_transferred = 0;
     my $usleep = 1000000;
diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
index de8c1bb..d494cc0 100644
--- a/PVE/QemuServer.pm
+++ b/PVE/QemuServer.pm
@@ -5431,10 +5431,10 @@ sub vm_start_nolock {
 	return $migration_ip;
     };
 
-    my $migrate_uri;
     if ($statefile) {
 	if ($statefile eq 'tcp') {
-	    my $localip = "localhost";
+	    my $migrate = $res->{migrate} = { proto => 'tcp' };
+	    $migrate->{addr} = "localhost";
 	    my $datacenterconf = PVE::Cluster::cfs_read_file('datacenter.cfg');
 	    my $nodename = nodename();
 
@@ -5447,26 +5447,26 @@ sub vm_start_nolock {
 	    }
 
 	    if ($migration_type eq 'insecure') {
-		$localip = $get_migration_ip->($nodename);
-		$localip = "[$localip]" if Net::IP::ip_is_ipv6($localip);
+		$migrate->{addr} = $get_migration_ip->($nodename);
+		$migrate->{addr} = "[$migrate->{addr}]" if Net::IP::ip_is_ipv6($migrate->{addr});
 	    }
 
 	    my $pfamily = PVE::Tools::get_host_address_family($nodename);
-	    my $migrate_port = PVE::Tools::next_migrate_port($pfamily);
-	    $migrate_uri = "tcp:${localip}:${migrate_port}";
-	    push @$cmd, '-incoming', $migrate_uri;
+	    $migrate->{port} = PVE::Tools::next_migrate_port($pfamily);
+	    $migrate->{uri} = "tcp:$migrate->{addr}:$migrate->{port}";
+	    push @$cmd, '-incoming', $migrate->{uri};
 	    push @$cmd, '-S';
 
 	} elsif ($statefile eq 'unix') {
 	    # should be default for secure migrations as a ssh TCP forward
 	    # tunnel is not deterministic reliable ready and fails regurarly
 	    # to set up in time, so use UNIX socket forwards
-	    my $socket_addr = "/run/qemu-server/$vmid.migrate";
-	    unlink $socket_addr;
+	    my $migrate = $res->{migrate} = { proto => 'unix' };
+	    $migrate->{addr} = "/run/qemu-server/$vmid.migrate";
+	    unlink $migrate->{addr};
 
-	    $migrate_uri = "unix:$socket_addr";
-
-	    push @$cmd, '-incoming', $migrate_uri;
+	    $migrate->{uri} = "unix:$migrate->{addr}";
+	    push @$cmd, '-incoming', $migrate->{uri};
 	    push @$cmd, '-S';
 
 	} elsif (-e $statefile) {
@@ -5608,10 +5608,9 @@ sub vm_start_nolock {
     eval { PVE::QemuServer::PCI::reserve_pci_usage($pci_id_list, $vmid, undef, $pid) };
     warn $@ if $@;
 
-    print "migration listens on $migrate_uri\n" if $migrate_uri;
-    $res->{migrate_uri} = $migrate_uri;
-
-    if ($statefile && $statefile ne 'tcp' && $statefile ne 'unix')  {
+    if (defined($res->{migrate})) {
+	print "migration listens on $res->{migrate}->{uri}\n";
+    } elsif ($statefile) {
 	eval { mon_cmd($vmid, "cont"); };
 	warn $@ if $@;
     }
@@ -5626,6 +5625,7 @@ sub vm_start_nolock {
 	    my $socket_path = "/run/qemu-server/$vmid\_nbd.migrate";
 	    mon_cmd($vmid, "nbd-server-start", addr => { type => 'unix', data => { path => $socket_path } } );
 	    $migrate_storage_uri = "nbd:unix:$socket_path";
+	    $res->{migrate}->{unix_sockets} = [$socket_path];
 	} else {
 	    my $nodename = nodename();
 	    my $localip = $get_migration_ip->($nodename);
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (19 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 08/10] migrate: refactor remote VM/tunnel start Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-10 11:17   ` Fabian Ebner
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint Fabian Grünbichler
                   ` (2 subsequent siblings)
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

remote migration uses a websocket connection to a task worker running on
the target node instead of commands via SSH to control the migration.
this websocket tunnel is started earlier than the SSH tunnel, and allows
adding UNIX-socket forwarding over additional websocket connections
on-demand.

the main differences to regular intra-cluster migration are:
- source VM config and disks are only removed upon request via --delete
- shared storages are treated like local storages, since we can't
assume they are shared across clusters (with potentical to extend this
by marking storages as shared)
- NBD migrated disks are explicitly pre-allocated on the target node via
tunnel command before starting the target VM instance
- in addition to storages, network bridges and the VMID itself is
transformed via a user defined mapping
- all commands and migration data streams are sent via a WS tunnel proxy

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    requires proxmox-websocket-tunnel

 PVE/API2/Qemu.pm   |   4 +-
 PVE/QemuMigrate.pm | 647 +++++++++++++++++++++++++++++++++++++++------
 PVE/QemuServer.pm  |   8 +-
 3 files changed, 575 insertions(+), 84 deletions(-)

diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
index a1a1813..24f5b98 100644
--- a/PVE/API2/Qemu.pm
+++ b/PVE/API2/Qemu.pm
@@ -4610,7 +4610,7 @@ __PACKAGE__->register_method({
 		    # bump/reset both for breaking changes
 		    # bump tunnel only for opt-in changes
 		    return {
-			api => 2,
+			api => $PVE::QemuMigrate::WS_TUNNEL_VERSION,
 			age => 0,
 		    };
 		},
@@ -4897,7 +4897,7 @@ __PACKAGE__->register_method({
 			    PVE::Firewall::remove_vmfw_conf($vmid);
 			}
 
-			if (my @volumes = keys $state->{cleanup}->{volumes}->$%) {
+			if (my @volumes = keys $state->{cleanup}->{volumes}->%*) {
 			    PVE::Storage::foreach_volid(@volumes, sub {
 				my ($volid, $sid, $volname, $d) = @_;
 
diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
index 07b56eb..7378551 100644
--- a/PVE/QemuMigrate.pm
+++ b/PVE/QemuMigrate.pm
@@ -7,9 +7,15 @@ use IO::File;
 use IPC::Open2;
 use POSIX qw( WNOHANG );
 use Time::HiRes qw( usleep );
+use JSON qw(encode_json decode_json);
+use IO::Socket::UNIX;
+use Socket qw(SOCK_STREAM);
+use Storable qw(dclone);
+use URI::Escape;
 
-use PVE::Format qw(render_bytes);
+use PVE::APIClient::LWP;
 use PVE::Cluster;
+use PVE::Format qw(render_bytes);
 use PVE::GuestHelpers qw(safe_boolean_ne safe_string_ne);
 use PVE::INotify;
 use PVE::RPCEnvironment;
@@ -30,6 +36,9 @@ use PVE::QemuServer;
 use PVE::AbstractMigrate;
 use base qw(PVE::AbstractMigrate);
 
+# compared against remote end's minimum version
+our $WS_TUNNEL_VERSION = 2;
+
 sub fork_command_pipe {
     my ($self, $cmd) = @_;
 
@@ -85,7 +94,7 @@ sub finish_command_pipe {
 	}
     }
 
-    $self->log('info', "ssh tunnel still running - terminating now with SIGTERM\n");
+    $self->log('info', "tunnel still running - terminating now with SIGTERM\n");
     kill(15, $cpid);
 
     # wait again
@@ -94,11 +103,11 @@ sub finish_command_pipe {
 	sleep(1);
     }
 
-    $self->log('info', "ssh tunnel still running - terminating now with SIGKILL\n");
+    $self->log('info', "tunnel still running - terminating now with SIGKILL\n");
     kill 9, $cpid;
     sleep 1;
 
-    $self->log('err', "ssh tunnel child process (PID $cpid) couldn't be collected\n")
+    $self->log('err', "tunnel child process (PID $cpid) couldn't be collected\n")
 	if !&$collect_child_process();
 }
 
@@ -115,18 +124,28 @@ sub read_tunnel {
     };
     die "reading from tunnel failed: $@\n" if $@;
 
-    chomp $output;
+    chomp $output if defined($output);
 
     return $output;
 }
 
 sub write_tunnel {
-    my ($self, $tunnel, $timeout, $command) = @_;
+    my ($self, $tunnel, $timeout, $command, $params) = @_;
 
     $timeout = 60 if !defined($timeout);
 
     my $writer = $tunnel->{writer};
 
+    if ($tunnel->{version} && $tunnel->{version} >= 2) {
+	my $object = defined($params) ? dclone($params) : {};
+	$object->{cmd} = $command;
+
+	$command = eval { JSON::encode_json($object) };
+
+	die "failed to encode command as JSON - $@\n"
+	    if $@;
+    }
+
     eval {
 	PVE::Tools::run_with_timeout($timeout, sub {
 	    print $writer "$command\n";
@@ -136,13 +155,29 @@ sub write_tunnel {
     die "writing to tunnel failed: $@\n" if $@;
 
     if ($tunnel->{version} && $tunnel->{version} >= 1) {
-	my $res = eval { $self->read_tunnel($tunnel, 10); };
+	my $res = eval { $self->read_tunnel($tunnel, $timeout); };
 	die "no reply to command '$command': $@\n" if $@;
 
-	if ($res eq 'OK') {
-	    return;
+	if ($tunnel->{version} == 1) {
+	    if ($res eq 'OK') {
+		return;
+	    } else {
+		die "tunnel replied '$res' to command '$command'\n";
+	    }
 	} else {
-	    die "tunnel replied '$res' to command '$command'\n";
+	    my $parsed = eval { JSON::decode_json($res) };
+	    die "failed to decode tunnel reply '$res' (command '$command') - $@\n"
+		if $@;
+
+	    if (!$parsed->{success}) {
+		if (defined($parsed->{msg})) {
+		    die "error - tunnel command '$command' failed - $parsed->{msg}\n";
+		} else {
+		    die "error - tunnel command '$command' failed\n";
+		}
+	    }
+
+	    return $parsed;
 	}
     }
 }
@@ -185,10 +220,150 @@ sub fork_tunnel {
     return $tunnel;
 }
 
+my $forward_unix_socket = sub {
+    my ($self, $local, $remote) = @_;
+
+    my $params = dclone($self->{tunnel}->{params});
+    $params->{unix} = $local;
+    $params->{url} = $params->{url} ."socket=$remote&";
+    $params->{ticket} = { path => $remote };
+
+    my $cmd = encode_json({
+	control => JSON::true,
+	cmd => 'forward',
+	data => $params,
+    });
+
+    my $writer = $self->{tunnel}->{writer};
+    eval {
+	unlink $local;
+	PVE::Tools::run_with_timeout(15, sub {
+	    print $writer "$cmd\n";
+	    $writer->flush();
+	});
+    };
+    die "failed to write forwarding command - $@\n" if $@;
+
+    $self->read_tunnel($self->{tunnel});
+
+    $self->log('info', "Forwarded local unix socket '$local' to remote '$remote' via websocket tunnel");
+};
+
+sub fork_websocket_tunnel {
+    my ($self, $storages) = @_;
+
+    my $remote = $self->{opts}->{remote};
+    my $conn = $remote->{conn};
+
+    my $websocket_url = "https://$conn->{host}:$conn->{port}/api2/json/nodes/$self->{node}/qemu/$remote->{vmid}/mtunnelwebsocket";
+
+    my $params = {
+	url => $websocket_url,
+    };
+
+    if (my $apitoken = $conn->{apitoken}) {
+	$params->{headers} = [["Authorization", "$apitoken"]];
+    } else {
+	die "can't connect to remote host without credentials\n";
+    }
+
+    if (my $fps = $conn->{cached_fingerprints}) {
+	$params->{fingerprint} = (keys %$fps)[0];
+    }
+
+    my $api_client = PVE::APIClient::LWP->new(%$conn);
+    my $storage_list = join(',', keys %$storages);
+    my $res = $api_client->post("/nodes/$self->{node}/qemu/$remote->{vmid}/mtunnel", { storages => $storage_list });
+    $self->log('info', "remote: started migration tunnel worker '$res->{upid}'");
+    $params->{url} .= "?ticket=".uri_escape($res->{ticket});
+    $params->{url} .= "&socket=$res->{socket}";
+
+    my $reader = IO::Pipe->new();
+    my $writer = IO::Pipe->new();
+
+    my $cpid = fork();
+    if ($cpid) {
+	$writer->writer();
+	$reader->reader();
+	my $tunnel = { writer => $writer, reader => $reader, pid => $cpid };
+
+	eval {
+	    my $writer = $tunnel->{writer};
+	    my $cmd = encode_json({
+		control => JSON::true,
+		cmd => 'connect',
+		data => $params,
+	    });
+
+	    eval {
+		PVE::Tools::run_with_timeout(15, sub {
+		    print {$writer} "$cmd\n";
+		    $writer->flush();
+		});
+	    };
+	    die "failed to write tunnel connect command - $@\n" if $@;
+	};
+	die "failed to connect via WS: $@\n" if $@;
+
+	my $err;
+        eval {
+	    my $writer = $tunnel->{writer};
+	    my $cmd = encode_json({
+		cmd => 'version',
+	    });
+
+	    eval {
+		PVE::Tools::run_with_timeout(15, sub {
+		    print {$writer} "$cmd\n";
+		    $writer->flush();
+		});
+	    };
+	    $err = "failed to write tunnel version command - $@\n" if $@;
+	    my $res = $self->read_tunnel($tunnel, 10);
+	    $res = JSON::decode_json($res);
+	    my $version = $res->{api};
+
+	    if ($version =~ /^(\d+)$/) {
+		$tunnel->{version} = $1;
+		$tunnel->{age} = $res->{age};
+		$self->log('info', "tunnel info: $version\n");
+	    } else {
+		$err = "received invalid tunnel version string '$version'\n" if !$err;
+	    }
+	};
+	$err = $@ if !$err;
+
+	if ($err) {
+	    $self->finish_command_pipe($tunnel);
+	    die "can't open migration tunnel - $err";
+	}
+
+	$params->{url} = "$websocket_url?";
+	$tunnel->{params} = $params; # for forwarding
+
+	return $tunnel;
+    } else {
+	eval {
+	    $writer->reader();
+	    $reader->writer();
+	    PVE::Tools::run_command(
+		['proxmox-websocket-tunnel'],
+		input => "<&".fileno($writer),
+		output => ">&".fileno($reader),
+		errfunc => sub { my $line = shift; print "tunnel: $line\n"; },
+	    );
+	};
+	warn "CMD websocket tunnel died: $@\n" if $@;
+	exit 0;
+    }
+}
+
 sub finish_tunnel {
-    my ($self, $tunnel) = @_;
+    my ($self, $tunnel, $cleanup) = @_;
 
-    eval { $self->write_tunnel($tunnel, 30, 'quit'); };
+    $cleanup = $cleanup ? 1 : 0;
+
+    eval { $self->write_tunnel($tunnel, 30, 'quit', { cleanup => $cleanup }); };
     my $err = $@;
 
     $self->finish_command_pipe($tunnel, 30);
@@ -338,23 +513,34 @@ sub prepare {
     }
 
     my $vollist = PVE::QemuServer::get_vm_volumes($conf);
+
+    my $storages = {};
     foreach my $volid (@$vollist) {
 	my ($sid, $volname) = PVE::Storage::parse_volume_id($volid, 1);
 
-	# check if storage is available on both nodes
+	# check if storage is available on source node
 	my $scfg = PVE::Storage::storage_check_enabled($storecfg, $sid);
 
 	my $targetsid = $sid;
-	# NOTE: we currently ignore shared source storages in mappings so skip here too for now
-	if (!$scfg->{shared}) {
+	# NOTE: local ignores shared mappings, remote maps them
+	if (!$scfg->{shared} || $self->{opts}->{remote}) {
 	    $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
 	}
 
-	my $target_scfg = PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
-	my ($vtype) = PVE::Storage::parse_volname($storecfg, $volid);
+	$storages->{$targetsid} = 1;
+
+	if (!$self->{opts}->{remote}) {
+	    # check if storage is available on target node
+	    my $target_scfg = PVE::Storage::storage_check_enabled(
+		$storecfg,
+		$targetsid,
+		$self->{node},
+	    );
+	    my ($vtype) = PVE::Storage::parse_volname($storecfg, $volid);
 
-	die "$volid: content type '$vtype' is not available on storage '$targetsid'\n"
-	    if !$target_scfg->{content}->{$vtype};
+	    die "$volid: content type '$vtype' is not available on storage '$targetsid'\n"
+		if !$target_scfg->{content}->{$vtype};
+	}
 
 	if ($scfg->{shared}) {
 	    # PVE::Storage::activate_storage checks this for non-shared storages
@@ -364,10 +550,23 @@ sub prepare {
 	}
     }
 
-    # test ssh connection
-    my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
-    eval { $self->cmd_quiet($cmd); };
-    die "Can't connect to destination address using public key\n" if $@;
+    if ($self->{opts}->{remote}) {
+	# test & establish websocket connection
+	my $tunnel = $self->fork_websocket_tunnel($storages);
+	my $min_version = $tunnel->{version} - $tunnel->{age};
+	die "Remote tunnel endpoint not compatible, upgrade required (current: $WS_TUNNEL_VERSION, required: $min_version)\n"
+	    if $WS_TUNNEL_VERSION < $min_version;
+	 die "Remote tunnel endpoint too old, upgrade required (local: $WS_TUNNEL_VERSION, remote: $tunnel->{version})"
+	    if $WS_TUNNEL_VERSION > $tunnel->{version};
+
+	print "websocket tunnel started\n";
+	$self->{tunnel} = $tunnel;
+    } else {
+	# test ssh connection
+	my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
+	eval { $self->cmd_quiet($cmd); };
+	die "Can't connect to destination address using public key\n" if $@;
+    }
 
     return $running;
 }
@@ -405,7 +604,7 @@ sub scan_local_volumes {
 	my @sids = PVE::Storage::storage_ids($storecfg);
 	foreach my $storeid (@sids) {
 	    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
-	    next if $scfg->{shared};
+	    next if $scfg->{shared} && !$self->{opts}->{remote};
 	    next if !PVE::Storage::storage_check_enabled($storecfg, $storeid, undef, 1);
 
 	    # get list from PVE::Storage (for unused volumes)
@@ -414,19 +613,24 @@ sub scan_local_volumes {
 	    next if @{$dl->{$storeid}} == 0;
 
 	    my $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $storeid);
-	    # check if storage is available on target node
-	    my $target_scfg = PVE::Storage::storage_check_enabled(
-		$storecfg,
-		$targetsid,
-		$self->{node},
-	    );
-
-	    die "content type 'images' is not available on storage '$targetsid'\n"
-		if !$target_scfg->{content}->{images};
+	    my $bwlimit_sids = [$storeid];
+	    if (!$self->{opts}->{remote}) {
+		# check if storage is available on target node
+		my $target_scfg = PVE::Storage::storage_check_enabled(
+		    $storecfg,
+		    $targetsid,
+		    $self->{node},
+		);
+
+		die "content type 'images' is not available on storage '$targetsid'\n"
+		    if !$target_scfg->{content}->{images};
+
+		push @$bwlimit_sids, $targetsid;
+	    }
 
 	    my $bwlimit = PVE::Storage::get_bandwidth_limit(
 		'migration',
-		[$targetsid, $storeid],
+		$bwlimit_sids,
 		$self->{opts}->{bwlimit},
 	    );
 
@@ -482,14 +686,17 @@ sub scan_local_volumes {
 	    my $scfg = PVE::Storage::storage_check_enabled($storecfg, $sid);
 
 	    my $targetsid = $sid;
-	    # NOTE: we currently ignore shared source storages in mappings so skip here too for now
-	    if (!$scfg->{shared}) {
+	    # NOTE: local ignores shared mappings, remote maps them
+	    if (!$scfg->{shared} || $self->{opts}->{remote}) {
 		$targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
 	    }
 
-	    PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
+	    # check target storage on target node if intra-cluster migration
+	    if (!$self->{opts}->{remote}) {
+		PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
 
-	    return if $scfg->{shared};
+		return if $scfg->{shared};
+	    }
 
 	    $local_volumes->{$volid}->{ref} = $attr->{referenced_in_config} ? 'config' : 'snapshot';
 	    $local_volumes->{$volid}->{ref} = 'storage' if $attr->{is_unused};
@@ -578,6 +785,9 @@ sub scan_local_volumes {
 
 	    my $migratable = $scfg->{type} =~ /^(?:dir|btrfs|zfspool|lvmthin|lvm)$/;
 
+	    # TODO: what is this even here for?
+	    $migratable = 1 if $self->{opts}->{remote};
+
 	    die "can't migrate '$volid' - storage type '$scfg->{type}' not supported\n"
 		if !$migratable;
 
@@ -612,6 +822,10 @@ sub handle_replication {
     my $local_volumes = $self->{local_volumes};
 
     return if !$self->{replication_jobcfg};
+
+    die "can't migrate VM with replicated volumes to remote cluster/node\n"
+	if $self->{opts}->{remote};
+
     if ($self->{running}) {
 
 	my $version = PVE::QemuServer::kvm_user_version();
@@ -709,26 +923,133 @@ sub sync_offline_local_volumes {
     my $opts = $self->{opts};
 
     $self->log('info', "copying local disk images") if scalar(@volids);
-
+    my $forwarded = 0;
     foreach my $volid (@volids) {
 	my $targetsid = $local_volumes->{$volid}->{targetsid};
-	my $bwlimit = $local_volumes->{$volid}->{bwlimit};
-	$bwlimit = $bwlimit * 1024 if defined($bwlimit); # storage_migrate uses bps
-
-	my $storage_migrate_opts = {
-	    'ratelimit_bps' => $bwlimit,
-	    'insecure' => $opts->{migration_type} eq 'insecure',
-	    'with_snapshots' => $local_volumes->{$volid}->{snapshots},
-	    'allow_rename' => !$local_volumes->{$volid}->{is_vmstate},
-	};
 
-	my $logfunc = sub { $self->log('info', $_[0]); };
-	my $new_volid = eval {
-	    PVE::Storage::storage_migrate($storecfg, $volid, $self->{ssh_info},
-					  $targetsid, $storage_migrate_opts, $logfunc);
-	};
-	if (my $err = $@) {
-	    die "storage migration for '$volid' to storage '$targetsid' failed - $err\n";
+	my $new_volid;
+	
+	my $opts = $self->{opts};
+	if (my $remote = $opts->{remote}) {
+	    my $remote_vmid = $remote->{vmid};
+	    my ($sid, undef) = PVE::Storage::parse_volume_id($volid);
+	    my (undef, $name, undef, undef, undef, undef, $format) = PVE::Storage::parse_volname($storecfg, $volid);
+	    my $scfg = PVE::Storage::storage_config($storecfg, $sid);
+	    PVE::Storage::activate_volumes($storecfg, [$volid]);
+
+	    # use 'migrate' limit for transfer to other node
+	    my $bwlimit_opts = {
+		storage => $targetsid,
+		bwlimit => $opts->{bwlimit},
+	    };
+	    my $bwlimit = PVE::Storage::get_bandwidth_limit('migration', [$sid], $opts->{bwlimit});
+	    my $remote_bwlimit = $self->write_tunnel($self->{tunnel}, 10, 'bwlimit', $bwlimit_opts);
+	    $remote_bwlimit = $remote_bwlimit->{bwlimit};
+	    if (defined($remote_bwlimit)) {
+		$bwlimit = $remote_bwlimit if !defined($bwlimit);
+		$bwlimit = $remote_bwlimit if $remote_bwlimit < $bwlimit;
+	    }
+
+	    # JSONSchema and get_bandwidth_limit use kbps - storage_migrate bps
+	    $bwlimit = $bwlimit * 1024 if defined($bwlimit);
+
+	    my $with_snapshots = $local_volumes->{$volid}->{snapshots} ? 1 : 0;
+	    my $snapshot;
+	    if ($scfg->{type} eq 'zfspool') {
+		$snapshot = '__migration__';
+		$with_snapshots = 1;
+		PVE::Storage::volume_snapshot($storecfg, $volid, $snapshot);
+	    }
+
+	    if ($self->{vmid} != $remote_vmid) {
+		$name =~ s/-$self->{vmid}-/-$remote_vmid-/g;
+		$name =~ s/^$self->{vmid}\//$remote_vmid\//;
+	    }
+
+	    my @export_formats = PVE::Storage::volume_export_formats($storecfg, $volid, undef, undef, $with_snapshots);
+
+	    my $storage_migrate_opts = {
+		format => $format,
+		storage => $targetsid,
+		'with-snapshots' => $with_snapshots,
+		'allow-rename' => !$local_volumes->{$volid}->{is_vmstate},
+		'export-formats' => @export_formats,
+		volname => $name,
+	    };
+	    my $res = $self->write_tunnel($self->{tunnel}, 600, 'disk-import', $storage_migrate_opts);
+	    my $local = "/run/qemu-server/$self->{vmid}.storage";
+	    if (!$forwarded) {
+		$forward_unix_socket->($self, $local, $res->{socket});
+		$forwarded = 1;
+	    }
+	    my $socket = IO::Socket::UNIX->new(Peer => $local, Type => SOCK_STREAM())
+		or die "failed to connect to websocket tunnel at $local\n";
+	    # we won't be reading from the socket
+	    shutdown($socket, 0);
+	    my $send = ['pvesm', 'export', $volid, $res->{format}, '-', '-with-snapshots', $with_snapshots];
+	    push @$send, '-snapshot', $snapshot if $snapshot;
+
+	    my @cstream;
+	    if (defined($bwlimit)) {
+		@cstream = ([ '/usr/bin/cstream', '-t', $bwlimit ]);
+		$self->log('info', "using a bandwidth limit of $bwlimit bps for transferring '$volid'");
+	    }
+
+	    eval {
+		PVE::Tools::run_command(
+		    [$send, @cstream],
+		    output => '>&'.fileno($socket),
+		    errfunc => sub { my $line = shift; $self->log('warn', $line); },
+		);
+	    };
+	    my $send_error = $@;
+
+	    # don't close the connection entirely otherwise the
+	    # receiving end might not get all buffered data (and
+	    # fails with 'connection reset by peer')
+	    shutdown($socket, 1);
+
+	    # wait for the remote process to finish
+	    while ($res = $self->write_tunnel($self->{tunnel}, 10, 'query-disk-import')) {
+		if ($res->{status} eq 'pending') {
+		    $self->log('info', "waiting for disk import to finish..\n");
+		    sleep(1)
+		} elsif ($res->{status} eq 'complete') {
+		    $new_volid = $res->{volid};
+		    last;
+		} else {
+		    die "unknown query-disk-import result: $res->{status}\n";
+		}
+	    }
+
+	    # now close the socket
+	    close($socket);
+	    die $send_error if $send_error;
+	} else {
+	    my $bwlimit = $local_volumes->{$volid}->{bwlimit};
+	    $bwlimit = $bwlimit * 1024 if defined($bwlimit); # storage_migrate uses bps
+
+	    my $storage_migrate_opts = {
+		'ratelimit_bps' => $bwlimit,
+		'insecure' => $opts->{migration_type} eq 'insecure',
+		'with_snapshots' => $local_volumes->{$volid}->{snapshots},
+		'allow_rename' => !$local_volumes->{$volid}->{is_vmstate},
+	    };
+
+	    my $logfunc = sub { $self->log('info', $_[0]); };
+	    $new_volid = eval {
+		PVE::Storage::storage_migrate(
+		    $storecfg,
+		    $volid,
+		    $self->{ssh_info},
+		    $targetsid,
+		    $storage_migrate_opts,
+		    $logfunc,
+		);
+	    };
+	    if (my $err = $@) {
+		die "storage migration for '$volid' to storage '$targetsid' failed - $err\n";
+	    }
 	}
 
 	$self->{volume_map}->{$volid} = $new_volid;
@@ -744,6 +1065,12 @@ sub sync_offline_local_volumes {
 sub cleanup_remotedisks {
     my ($self) = @_;
 
+    if ($self->{opts}->{remote}) {
+	$self->finish_tunnel($self->{tunnel}, 1);
+	delete $self->{tunnel};
+	return;
+    }
+
     my $local_volumes = $self->{local_volumes};
 
     foreach my $volid (values %{$self->{volume_map}}) {
@@ -793,8 +1120,84 @@ sub phase1 {
     $self->handle_replication($vmid);
 
     $self->sync_offline_local_volumes();
+    $self->phase1_remote($vmid) if $self->{opts}->{remote};
 };
 
+sub phase1_remote {
+    my ($self, $vmid) = @_;
+
+    my $remote_conf = PVE::QemuConfig->load_config($vmid);
+    PVE::QemuConfig->update_volume_ids($remote_conf, $self->{volume_map});
+
+    # TODO: check bridge availability earlier?
+    my $bridgemap = $self->{opts}->{bridgemap};
+    foreach my $opt (keys %$remote_conf) {
+	next if $opt !~ m/^net\d+$/;
+
+	next if !$remote_conf->{$opt};
+	my $d = PVE::QemuServer::parse_net($remote_conf->{$opt});
+	next if !$d || !$d->{bridge};
+
+	my $target_bridge = PVE::QemuServer::map_id($bridgemap, $d->{bridge});
+	$self->log('info', "mapped: $opt from $d->{bridge} to $target_bridge");
+	$d->{bridge} = $target_bridge;
+	$remote_conf->{$opt} = PVE::QemuServer::print_net($d);
+    }
+
+    my @online_local_volumes = $self->filter_local_volumes('online');
+
+    my $storage_map = $self->{opts}->{storagemap};
+    $self->{nbd} = {};
+    PVE::QemuConfig->foreach_volume($remote_conf, sub {
+	my ($ds, $drive) = @_;
+
+	# TODO eject CDROM?
+	return if PVE::QemuServer::drive_is_cdrom($drive);
+
+	my $volid = $drive->{file};
+	return if !$volid;
+
+	return if !grep { $_ eq $volid} @online_local_volumes;
+
+	my ($storeid, $volname) = PVE::Storage::parse_volume_id($volid);
+	my $scfg = PVE::Storage::storage_config($self->{storecfg}, $storeid);
+	my $source_format = PVE::QemuServer::qemu_img_format($scfg, $volname);
+
+	# set by target cluster
+	my $oldvolid = delete $drive->{file};
+	delete $drive->{format};
+
+	my $targetsid = PVE::QemuServer::map_id($storage_map, $storeid);
+
+	my $params = {
+	    format => $source_format,
+	    storage => $targetsid,
+	    drive => $drive,
+	};
+
+	$self->log('info', "Allocating volume for drive '$ds' on remote storage '$targetsid'..");
+	my $res = $self->write_tunnel($self->{tunnel}, 600, 'disk', $params);
+
+	$self->log('info', "volume '$oldvolid' os '$res->{volid}' on the target\n");
+	$remote_conf->{$ds} = $res->{drivestr};
+	$self->{nbd}->{$ds} = $res;
+    });
+
+    my $conf_str = PVE::QemuServer::write_vm_config("remote", $remote_conf);
+
+    # TODO expose in PVE::Firewall?
+    my $vm_fw_conf_path = "/etc/pve/firewall/$vmid.fw";
+    my $fw_conf_str;
+    $fw_conf_str = PVE::Tools::file_get_contents($vm_fw_conf_path)
+	if -e $vm_fw_conf_path;
+    my $params = {
+	conf => $conf_str,
+	'firewall-config' => $fw_conf_str,
+    };
+
+    $self->write_tunnel($self->{tunnel}, 10, 'config', $params);
+}
+
 sub phase1_cleanup {
     my ($self, $vmid, $err) = @_;
 
@@ -825,7 +1228,6 @@ sub phase2_start_local_cluster {
     my $local_volumes = $self->{local_volumes};
     my @online_local_volumes = $self->filter_local_volumes('online');
 
-    $self->{storage_migration} = 1 if scalar(@online_local_volumes);
     my $start = $params->{start_params};
     my $migrate = $params->{migrate_opts};
 
@@ -948,10 +1350,34 @@ sub phase2_start_local_cluster {
     return ($tunnel_info, $spice_port);
 }
 
+sub phase2_start_remote_cluster {
+    my ($self, $vmid, $params) = @_;
+
+    die "insecure migration to remote cluster not implemented\n"
+	if $params->{migrate_opts}->{type} ne 'websocket';
+
+    my $remote_vmid = $self->{opts}->{remote}->{vmid};
+
+    my $res = $self->write_tunnel($self->{tunnel}, 10, "start", $params);
+
+    foreach my $drive (keys %{$res->{drives}}) {
+	$self->{stopnbd} = 1;
+	$self->{target_drive}->{$drive}->{drivestr} = $res->{drives}->{$drive}->{drivestr};
+	my $nbd_uri = $res->{drives}->{$drive}->{nbd_uri};
+	die "unexpected NBD uri for '$drive': $nbd_uri\n"
+	    if $nbd_uri !~ s!/run/qemu-server/$remote_vmid\_!/run/qemu-server/$vmid\_!;
+
+	$self->{target_drive}->{$drive}->{nbd_uri} = $nbd_uri;
+    }
+
+    return ($res->{migrate}, $res->{spice_port});
+}
+
 sub phase2 {
     my ($self, $vmid) = @_;
 
     my $conf = $self->{vmconf};
+    my $local_volumes = $self->{local_volumes};
 
     # version > 0 for unix socket support
     my $nbd_protocol_version = 1;
@@ -983,10 +1409,42 @@ sub phase2 {
 	},
     };
 
-    my ($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
+    my ($tunnel_info, $spice_port);
+
+    my @online_local_volumes = $self->filter_local_volumes('online');
+    $self->{storage_migration} = 1 if scalar(@online_local_volumes);
+
+    if (my $remote = $self->{opts}->{remote}) {
+	my $remote_vmid = $remote->{vmid};
+	$params->{migrate_opts}->{remote_node} = $self->{node};
+	($tunnel_info, $spice_port) = $self->phase2_start_remote_cluster($vmid, $params);
+	die "only UNIX sockets are supported for remote migration\n"
+	    if $tunnel_info->{proto} ne 'unix';
+
+	my $forwarded = {};
+	my $remote_socket = $tunnel_info->{addr};
+	my $local_socket = $remote_socket;
+	$local_socket =~ s/$remote_vmid/$vmid/g;
+	$tunnel_info->{addr} = $local_socket;
+
+	$self->log('info', "Setting up tunnel for '$local_socket'");
+	$forward_unix_socket->($self, $local_socket, $remote_socket);
+	$forwarded->{$local_socket} = 1;
+
+	foreach my $remote_socket (@{$tunnel_info->{unix_sockets}}) {
+	    my $local_socket = $remote_socket;
+	    $local_socket =~ s/$remote_vmid/$vmid/g;
+	    next if $forwarded->{$local_socket};
+	    $self->log('info', "Setting up tunnel for '$local_socket'");
+	    $forward_unix_socket->($self, $local_socket, $remote_socket);
+	    $forwarded->{$local_socket} = 1;
+	}
+    } else {
+	($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
 
-    $self->log('info', "start remote tunnel");
-    $self->start_remote_tunnel($tunnel_info);
+	$self->log('info', "start remote tunnel");
+	$self->start_remote_tunnel($tunnel_info);
+    }
 
     my $migrate_uri = "$tunnel_info->{proto}:$tunnel_info->{addr}";
     $migrate_uri .= ":$tunnel_info->{port}"
@@ -996,8 +1454,6 @@ sub phase2 {
 	$self->{storage_migration_jobs} = {};
 	$self->log('info', "starting storage migration");
 
-	my @online_local_volumes = $self->filter_local_volumes('online');
-
 	die "The number of local disks does not match between the source and the destination.\n"
 	    if (scalar(keys %{$self->{target_drive}}) != scalar(@online_local_volumes));
 	foreach my $drive (keys %{$self->{target_drive}}){
@@ -1070,7 +1526,7 @@ sub phase2 {
     };
     $self->log('info', "migrate-set-parameters error: $@") if $@;
 
-    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
+    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga} && !$self->{opts}->{remote})) {
 	my $rpcenv = PVE::RPCEnvironment::get();
 	my $authuser = $rpcenv->get_user();
 
@@ -1267,11 +1723,15 @@ sub phase2_cleanup {
 
     my $nodename = PVE::INotify::nodename();
 
-    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'stop', $vmid, '--skiplock', '--migratedfrom', $nodename];
-    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
-    if (my $err = $@) {
-        $self->log('err', $err);
-        $self->{errors} = 1;
+    if ($self->{tunnel} && $self->{tunnel}->{version} >= 2) {
+	$self->write_tunnel($self->{tunnel}, 10, 'stop');
+    } else {
+	my $cmd = [@{$self->{rem_ssh}}, 'qm', 'stop', $vmid, '--skiplock', '--migratedfrom', $nodename];
+	eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
+	if (my $err = $@) {
+	    $self->log('err', $err);
+	    $self->{errors} = 1;
+	}
     }
 
     # cleanup after stopping, otherwise disks might be in-use by target VM!
@@ -1304,7 +1764,7 @@ sub phase3_cleanup {
 
     my $tunnel = $self->{tunnel};
 
-    if ($self->{volume_map}) {
+    if ($self->{volume_map} && !$self->{opts}->{remote}) {
 	my $target_drives = $self->{target_drive};
 
 	# FIXME: for NBD storage migration we now only update the volid, and
@@ -1321,26 +1781,33 @@ sub phase3_cleanup {
 
     # transfer replication state before move config
     $self->transfer_replication_state() if $self->{is_replicated};
-    PVE::QemuConfig->move_config_to_node($vmid, $self->{node});
+    if (!$self->{opts}->{remote}) {
+	PVE::QemuConfig->move_config_to_node($vmid, $self->{node});
+    }
     $self->switch_replication_job_target() if $self->{is_replicated};
 
     if ($self->{livemigration}) {
 	if ($self->{stopnbd}) {
 	    $self->log('info', "stopping NBD storage migration server on target.");
 	    # stop nbd server on remote vm - requirement for resume since 2.9
-	    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'nbdstop', $vmid];
+	    if ($tunnel && $tunnel->{version} && $tunnel->{version} >= 2) {
+		$self->write_tunnel($tunnel, 30, 'nbdstop');
+	    } else {
+		my $cmd = [@{$self->{rem_ssh}}, 'qm', 'nbdstop', $vmid];
 
-	    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
-	    if (my $err = $@) {
-		$self->log('err', $err);
-		$self->{errors} = 1;
+		eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
+		if (my $err = $@) {
+		    $self->log('err', $err);
+		    $self->{errors} = 1;
+		}
 	    }
 	}
 
 	# config moved and nbd server stopped - now we can resume vm on target
 	if ($tunnel && $tunnel->{version} && $tunnel->{version} >= 1) {
+	    my $cmd = $tunnel->{version} == 1 ? "resume $vmid" : "resume";
 	    eval {
-		$self->write_tunnel($tunnel, 30, "resume $vmid");
+		$self->write_tunnel($tunnel, 30, $cmd);
 	    };
 	    if (my $err = $@) {
 		$self->log('err', $err);
@@ -1360,18 +1827,24 @@ sub phase3_cleanup {
 	}
 
 	if ($self->{storage_migration} && PVE::QemuServer::parse_guest_agent($conf)->{fstrim_cloned_disks} && $self->{running}) {
-	    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'guest', 'cmd', $vmid, 'fstrim'];
-	    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
+	    if ($self->{opts}->{remote}) {
+		$self->write_tunnel($self->{tunnel}, 600, 'fstrim');
+	    } else {
+		my $cmd = [@{$self->{rem_ssh}}, 'qm', 'guest', 'cmd', $vmid, 'fstrim'];
+		eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
+	    }
 	}
     }
 
     # close tunnel on successful migration, on error phase2_cleanup closed it
-    if ($tunnel) {
+    if ($tunnel && $tunnel->{version} == 1) {
 	eval { finish_tunnel($self, $tunnel);  };
 	if (my $err = $@) {
 	    $self->log('err', $err);
 	    $self->{errors} = 1;
 	}
+	$tunnel = undef;
+	delete $self->{tunnel};
     }
 
     eval {
@@ -1409,6 +1882,9 @@ sub phase3_cleanup {
 
     # destroy local copies
     foreach my $volid (@not_replicated_volumes) {
+	# remote is cleaned up below
+	next if $self->{opts}->{remote};
+
 	eval { PVE::Storage::vdisk_free($self->{storecfg}, $volid); };
 	if (my $err = $@) {
 	    $self->log('err', "removing local copy of '$volid' failed - $err");
@@ -1418,8 +1894,19 @@ sub phase3_cleanup {
     }
 
     # clear migrate lock
-    my $cmd = [ @{$self->{rem_ssh}}, 'qm', 'unlock', $vmid ];
-    $self->cmd_logerr($cmd, errmsg => "failed to clear migrate lock");
+    if ($tunnel && $tunnel->{version} >= 2) {
+	$self->write_tunnel($tunnel, 10, "unlock");
+
+	$self->finish_tunnel($tunnel);
+    } else {
+	my $cmd = [ @{$self->{rem_ssh}}, 'qm', 'unlock', $vmid ];
+	$self->cmd_logerr($cmd, errmsg => "failed to clear migrate lock");
+    }
+
+    if ($self->{opts}->{remote} && $self->{opts}->{delete}) {
+	eval { PVE::QemuServer::destroy_vm($self->{storecfg}, $vmid, 1, undef, 0) };
+	warn "Failed to remove source VM - $@\n" if $@;
+    }
 }
 
 sub final_cleanup {
diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
index d494cc0..bf05da2 100644
--- a/PVE/QemuServer.pm
+++ b/PVE/QemuServer.pm
@@ -5384,7 +5384,11 @@ sub vm_start_nolock {
     my $defaults = load_defaults();
 
     # set environment variable useful inside network script
-    $ENV{PVE_MIGRATED_FROM} = $migratedfrom if $migratedfrom;
+    if ($migrate_opts->{remote_node}) {
+	$ENV{PVE_MIGRATED_FROM} = $migrate_opts->{remote_node};
+    } elsif ($migratedfrom) {
+	$ENV{PVE_MIGRATED_FROM} = $migratedfrom;
+    }
 
     PVE::GuestHelpers::exec_hookscript($conf, $vmid, 'pre-start', 1);
 
@@ -5621,7 +5625,7 @@ sub vm_start_nolock {
 
 	my $migrate_storage_uri;
 	# nbd_protocol_version > 0 for unix socket support
-	if ($nbd_protocol_version > 0 && $migration_type eq 'secure') {
+	if ($nbd_protocol_version > 0 && ($migration_type eq 'secure' || $migration_type eq 'websocket')) {
 	    my $socket_path = "/run/qemu-server/$vmid\_nbd.migrate";
 	    mon_cmd($vmid, "nbd-server-start", addr => { type => 'unix', data => { path => $socket_path } } );
 	    $migrate_storage_uri = "nbd:unix:$socket_path";
-- 
2.30.2





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

* [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (20 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling Fabian Grünbichler
@ 2021-11-05 13:03 ` Fabian Grünbichler
  2021-11-10 12:29   ` Fabian Ebner
  2021-11-09 16:07 ` [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration DERUMIER, Alexandre
  2021-11-10 12:42 ` Fabian Ebner
  23 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-05 13:03 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---

Notes:
    the checks currently done before the actual migration worker is
    forked could be either moved to the client calling this (that then
    makes the required API calls) or extracted into a precond API call
    like for regular migration.
    
    for testing it helps catch trivial mistakes early on, and the calls shouldn't
    be too expensive, so I left them in for now..
    
    requires
    - pve-common with bridge-pair format
    - pve-guest-common with AbstractMigrate handling remote migration

 PVE/API2/Qemu.pm | 205 ++++++++++++++++++++++++++++++++++++++++++++++-
 debian/control   |   2 +
 2 files changed, 205 insertions(+), 2 deletions(-)

diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
index 24f5b98..b931f04 100644
--- a/PVE/API2/Qemu.pm
+++ b/PVE/API2/Qemu.pm
@@ -14,6 +14,7 @@ use URI::Escape;
 use Crypt::OpenSSL::Random;
 use Socket qw(SOCK_STREAM);
 
+use PVE::APIClient::LWP;
 use PVE::Cluster qw (cfs_read_file cfs_write_file);;
 use PVE::RRD;
 use PVE::SafeSyslog;
@@ -51,8 +52,6 @@ BEGIN {
     }
 }
 
-use Data::Dumper; # fixme: remove
-
 use base qw(PVE::RESTHandler);
 
 my $opt_force_description = "Force physical removal. Without this, we simple remove the disk from the config file and create an additional configuration entry called 'unused[n]', which contains the volume ID. Unlink of unused[n] always cause physical removal.";
@@ -3778,6 +3777,208 @@ __PACKAGE__->register_method({
 
     }});
 
+__PACKAGE__->register_method({
+    name => 'remote_migrate_vm',
+    path => '{vmid}/remote_migrate',
+    method => 'POST',
+    protected => 1,
+    proxyto => 'node',
+    description => "Migrate virtual machine to a remote cluster. Creates a new migration task.",
+    permissions => {
+	check => ['perm', '/vms/{vmid}', [ 'VM.Migrate' ]],
+    },
+    parameters => {
+	additionalProperties => 0,
+	properties => {
+	    node => get_standard_option('pve-node'),
+	    vmid => get_standard_option('pve-vmid', { completion => \&PVE::QemuServer::complete_vmid }),
+	    'target-vmid' => get_standard_option('pve-vmid', { optional => 1 }),
+	    'target-node' => get_standard_option('pve-node', {
+		description => "Target node on remote cluster.",
+            }),
+	    'target-endpoint' => get_standard_option('proxmox-remote', {
+		description => "Remote target endpoint",
+	    }),
+	    online => {
+		type => 'boolean',
+		description => "Use online/live migration if VM is running. Ignored if VM is stopped.",
+		optional => 1,
+	    },
+	    'migration-network' => {
+		type => 'string', format => 'CIDR',
+		description => "CIDR of the (sub) network that is used for migration.",
+		optional => 1,
+	    },
+	    'with-local-disks' => {
+		type => 'boolean',
+		description => "Enable live storage migration for local disk",
+		optional => 1,
+	    },
+	    delete => {
+		type => 'boolean',
+		description => "Delete the original VM and related data after successful migration. By default the original VM is kept on the source cluster in a stopped state.",
+		optional => 1,
+		default => 0,
+	    },
+            'target-storage' => get_standard_option('pve-targetstorage', {
+		completion => \&PVE::QemuServer::complete_migration_storage,
+		optional => 0,
+            }),
+	    'target-bridge' => {
+		type => 'string',
+		description => "Mapping from source to target bridges. Providing only a single bridge ID maps all source bridges to that bridge. Providing the special value '1' will map each source bridge to itself.",
+		format => 'bridge-pair-list',
+	    },
+	    bwlimit => {
+		description => "Override I/O bandwidth limit (in KiB/s).",
+		optional => 1,
+		type => 'integer',
+		minimum => '0',
+		default => 'migrate limit from datacenter or storage config',
+	    },
+	},
+    },
+    returns => {
+	type => 'string',
+	description => "the task ID.",
+    },
+    code => sub {
+	my ($param) = @_;
+
+	my $rpcenv = PVE::RPCEnvironment::get();
+	my $authuser = $rpcenv->get_user();
+
+	my $source_vmid = extract_param($param, 'vmid');
+	my $target_endpoint = extract_param($param, 'target-endpoint');
+	my $target_node = extract_param($param, 'target-node');
+	my $target_vmid = extract_param($param, 'target-vmid') // $source_vmid;
+
+	my $localnode = PVE::INotify::nodename();
+	my $network = extract_param($param, 'migration-network');
+	my $delete = extract_param($param, 'delete') // 0;
+
+	PVE::Cluster::check_cfs_quorum();
+
+	raise_param_exc({ 'migration-network' => "Only root may use this option." })
+	    if $network && $authuser ne 'root@pam';
+
+	# test if VM exists
+	my $conf = PVE::QemuConfig->load_config($source_vmid);
+
+	PVE::QemuConfig->check_lock($conf);
+
+	raise_param_exc({ vmid => "cannot migrate HA-manage VM to remote cluster" })
+	    if PVE::HA::Config::vm_is_ha_managed($source_vmid);
+
+	my $remote = PVE::JSONSchema::parse_property_string('proxmox-remote', $target_endpoint);
+
+	# TODO: move this as helper somewhere appropriate?
+	my $conn_args = {
+	    protocol => 'https',
+	    host => $remote->{host},
+	    port => $remote->{port} // 8006,
+	    apitoken => $remote->{apitoken},
+	};
+
+	my $fp;
+	if ($fp = $remote->{fingerprint}) {
+	    $conn_args->{cached_fingerprints} = { uc($fp) => 1 };
+	}
+
+	print "Establishing API connection with remote at '$remote->{host}'\n";
+
+	my $api_client = PVE::APIClient::LWP->new(%$conn_args);
+	my $version = $api_client->get("/version");
+	print "remote: version '$version->{version}\n";
+
+	if (!defined($fp)) {
+	    my $cert_info = $api_client->get("/nodes/$target_node/certificates/info");
+	    foreach my $cert (@$cert_info) {
+		$fp = $cert->{fingerprint} if $cert->{filename} ne 'pve-root-ca.pem';
+		last if $cert->{filename} eq 'pveproxy-ssl.pem';
+	    }
+	    $conn_args->{cached_fingerprints} = { uc($fp) => 1 }
+		if defined($fp);
+	}
+
+	if (PVE::QemuServer::check_running($source_vmid)) {
+	    die "can't migrate running VM without --online\n" if !$param->{online};
+
+	    my $repl_conf = PVE::ReplicationConfig->new();
+	    my $is_replicated = $repl_conf->check_for_existing_jobs($source_vmid, 1);
+	    die "cannot remote-migrate replicated VM\n" if $is_replicated;
+	} else {
+	    warn "VM isn't running. Doing offline migration instead.\n" if $param->{online};
+	    $param->{online} = 0;
+	}
+
+	# FIXME: fork worker hear to avoid timeout? or poll these periodically
+	# in pvestatd and access cached info here? all of the below is actually
+	# checked at the remote end anyway once we call the mtunnel endpoint,
+	# we could also punt it to the client and not do it here at all..
+	my $resources = $api_client->get("/cluster/resources");
+	if (grep { defined($_->{vmid}) && $_->{vmid} eq $target_vmid } @$resources) {
+	    raise_param_exc({ target_vmid => "Guest with ID '$target_vmid' already exists on remote cluster" });
+	}
+
+	my $storages = [ grep { $_->{type} eq 'storage' && $_->{node} eq $target_node } @$resources ];
+	my $storecfg = PVE::Storage::config();
+	my $target_storage = extract_param($param, 'target-storage');
+	my $storagemap = eval { PVE::JSONSchema::parse_idmap($target_storage, 'pve-storage-id') };
+	raise_param_exc({ 'target-storage' => "failed to parse storage map: $@" })
+	    if $@;
+
+	my $target_bridge = extract_param($param, 'target-bridge');
+	my $bridgemap = eval { PVE::JSONSchema::parse_idmap($target_bridge, 'pve-bridge-id') };
+	raise_param_exc({ 'target-bridge' => "failed to parse bridge map: $@" })
+	    if $@;
+
+	my $check_remote_storage = sub {
+	    my ($storage) = @_;
+	    my $found = [ grep { $_->{storage} eq $storage } @$storages ];
+	    die "remote: storage '$storage' does not exist!\n"
+		if !@$found;
+
+	    $found = @$found[0];
+
+	    my $content_types = [ PVE::Tools::split_list($found->{content}) ];
+	    die "remote: storage '$storage' cannot store images\n"
+		if !grep { $_ eq 'images' } @$content_types;
+	};
+
+	foreach my $target_sid (values %{$storagemap->{entries}}) {
+	    $check_remote_storage->($target_sid);
+	}
+
+	$check_remote_storage->($storagemap->{default})
+	    if $storagemap->{default};
+
+	# TODO: or check all referenced storages?
+	die "remote migration requires explicit storage mapping!\n"
+	    if $storagemap->{identity};
+
+	$param->{storagemap} = $storagemap;
+	$param->{bridgemap} = $bridgemap;
+	$param->{remote} = {
+	    conn => $conn_args, # re-use fingerprint for tunnel
+	    client => $api_client,
+	    vmid => $target_vmid,
+	};
+	$param->{migration_type} = 'websocket';
+	$param->{migration_network} = $network if $network;
+	$param->{delete} = $delete if $delete;
+
+	my $realcmd = sub {
+	    PVE::QemuMigrate->migrate($target_node, $remote->{host}, $source_vmid, $param);
+	};
+
+	my $worker = sub {
+	    return PVE::GuestHelpers::guest_migration_lock($source_vmid, 10, $realcmd);
+	};
+
+	return $rpcenv->fork_worker('qmigrate', $source_vmid, $authuser, $worker);
+    }});
+
 __PACKAGE__->register_method({
     name => 'monitor',
     path => '{vmid}/monitor',
diff --git a/debian/control b/debian/control
index 8032ae5..33e3916 100644
--- a/debian/control
+++ b/debian/control
@@ -6,6 +6,7 @@ Build-Depends: debhelper (>= 12~),
                libglib2.0-dev,
                libio-multiplex-perl,
                libjson-c-dev,
+               libpve-apiclient-perl,
                libpve-cluster-perl,
                libpve-common-perl (>= 6.3-3),
                libpve-guest-common-perl (>= 3.1-3),
@@ -34,6 +35,7 @@ Depends: dbus,
          libjson-xs-perl,
          libnet-ssleay-perl,
          libpve-access-control (>= 5.0-7),
+         libpve-apiclient-perl,
          libpve-cluster-perl,
          libpve-common-perl (>= 7.0-3),
          libpve-guest-common-perl (>= 3.1-3),
-- 
2.30.2





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

* Re: [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration
  2021-11-05 13:03 ` [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration Fabian Grünbichler
@ 2021-11-08 13:50   ` Fabian Ebner
  2021-11-10 12:03   ` Fabian Ebner
  1 sibling, 0 replies; 50+ messages in thread
From: Fabian Ebner @ 2021-11-08 13:50 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> we only want to use an explicitly provided migration network, not one
> for the local cluster.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>   src/PVE/AbstractMigrate.pm | 51 +++++++++++++++++++++++---------------
>   1 file changed, 31 insertions(+), 20 deletions(-)
> 
> diff --git a/src/PVE/AbstractMigrate.pm b/src/PVE/AbstractMigrate.pm
> index af2be38..ec60b82 100644
> --- a/src/PVE/AbstractMigrate.pm
> +++ b/src/PVE/AbstractMigrate.pm
> @@ -115,22 +115,27 @@ sub migrate {

Nit: There is a comment above here that the $nodeip parameter for the 
function is unused, but after this patch, that's not true anymore.

>   
>       $class = ref($class) || $class;
>   
> -    my $dc_conf = PVE::Cluster::cfs_read_file('datacenter.cfg');
> +    my ($ssh_info, $rem_ssh);
> +    if (!$opts->{remote}) {
> +	my $dc_conf = PVE::Cluster::cfs_read_file('datacenter.cfg');
>   
> -    my $migration_network = $opts->{migration_network};
> -    if (!defined($migration_network)) {
> -	$migration_network = $dc_conf->{migration}->{network};
> -    }
> -    my $ssh_info = PVE::SSHInfo::get_ssh_info($node, $migration_network);
> -    $nodeip = $ssh_info->{ip};
> -
> -    my $migration_type = 'secure';
> -    if (defined($opts->{migration_type})) {
> -	$migration_type = $opts->{migration_type};
> -    } elsif (defined($dc_conf->{migration}->{type})) {
> -        $migration_type = $dc_conf->{migration}->{type};
> +	my $migration_network = $opts->{migration_network};
> +	if (!defined($migration_network)) {
> +	    $migration_network = $dc_conf->{migration}->{network};
> +	}

Potential for a small clean-up: get rid of the $migration_network 
variable by using:
     $opts->{migration_network} //= $dc_conf->{migration}->{network};

> +	$ssh_info = PVE::SSHInfo::get_ssh_info($node, $migration_network);
> +	$nodeip = $ssh_info->{ip};
> +
> +	my $migration_type = 'secure';
> +	if (defined($opts->{migration_type})) {
> +	    $migration_type = $opts->{migration_type};
> +	} elsif (defined($dc_conf->{migration}->{type})) {
> +	    $migration_type = $dc_conf->{migration}->{type};
> +	}
> +	$opts->{migration_type} = $migration_type;
> +	$opts->{migration_network} = $migration_network;
> +	$rem_ssh = PVE::SSHInfo::ssh_info_to_command($ssh_info);
>       }
> -    $opts->{migration_type} = $migration_type;
>   
>       my $self = {
>   	delayed_interrupt => 0,
> @@ -139,7 +144,7 @@ sub migrate {
>   	node => $node,
>   	ssh_info => $ssh_info,
>   	nodeip => $nodeip,
> -	rem_ssh => PVE::SSHInfo::ssh_info_to_command($ssh_info)
> +	rem_ssh => $rem_ssh,
>       };
>   
>       $self = bless $self, $class;
> @@ -162,15 +167,21 @@ sub migrate {
>   	&$eval_int($self, sub { $self->{running} = $self->prepare($self->{vmid}); });
>   	die $@ if $@;
>   
> -	if (defined($migration_network)) {
> +	if (defined($self->{opts}->{migration_network})) {
>   	    $self->log('info', "use dedicated network address for sending " .
>   	               "migration traffic ($self->{nodeip})");
>   
>   	    # test if we can connect to new IP
> -	    my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
> -	    eval { $self->cmd_quiet($cmd); };
> -	    die "Can't connect to destination address ($self->{nodeip}) using " .
> -	        "public key authentication\n" if $@;
> +	    if ($self->{opts}->{remote}) {
> +		eval { $self->{opts}->{remote}->{client}->get("/") };
> +		die "Can't connect to destination address ($self->{nodeip}) using " .
> +		    "API connection - $@\n" if $@;
> +	    } else {
> +		my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
> +		eval { $self->cmd_quiet($cmd); };
> +		die "Can't connect to destination address ($self->{nodeip}) using " .
> +		    "public key authentication\n" if $@;
> +	    }
>   	}
>   
>   	&$eval_int($self, sub { $self->phase1($self->{vmid}); });
> 




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

* Re: [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer
  2021-11-05 13:03 ` [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer Fabian Grünbichler
@ 2021-11-08 14:15   ` Fabian Ebner
  2021-11-08 15:45     ` Fabian Grünbichler
  0 siblings, 1 reply; 50+ messages in thread
From: Fabian Ebner @ 2021-11-08 14:15 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> for proxied requests, we usually tear down the proxy connection
> immediately when closing the source connection. this is not the correct
> course of action for bulk one-way data streams that are proxied, where
> the source connection might be closed, but the proxy connection might
> still have data in the write buffer that needs to be written out.
> 
> push_shutdown already handles this case (closing the socket/FH after it
> has been fully drained).
> 
> one example for such a proxied data stream is the 'migrate' data for a
> remote migration, which gets proxied over a websocket connection.
> terminating the proxied connection early makes the target VM crash for
> obvious reasons.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>   src/PVE/APIServer/AnyEvent.pm | 10 ++++++----
>   1 file changed, 6 insertions(+), 4 deletions(-)
> 
> diff --git a/src/PVE/APIServer/AnyEvent.pm b/src/PVE/APIServer/AnyEvent.pm
> index 86d0e2e..ecf771f 100644
> --- a/src/PVE/APIServer/AnyEvent.pm
> +++ b/src/PVE/APIServer/AnyEvent.pm
> @@ -144,7 +144,8 @@ sub client_do_disconnect {
>       };
>   
>       if (my $proxyhdl = delete $reqstate->{proxyhdl}) {
> -	&$shutdown_hdl($proxyhdl);
> +	&$shutdown_hdl($proxyhdl)
> +		if !$proxyhdl->{block_disconnect};

Style nit: fits in one line ;)

I'm not familiar with the code, so I'll just ask: can this be reached 
without going through the code below first, i.e. can it happen that 
block_disconnect is not set, but length $proxyhdl->{wbuf} > 0? Or is it 
not important in other cases (if there are any)?

>       }
>   
>       my $hdl = delete $reqstate->{hdl};
> @@ -627,9 +628,10 @@ sub websocket_proxy {
>   		    } elsif ($opcode == 8) {
>   			my $statuscode = unpack ("n", $payload);
>   			$self->dprint("websocket received close. status code: '$statuscode'");
> -			if ($reqstate->{proxyhdl}) {
> -			    $reqstate->{proxyhdl}->push_shutdown();
> -			}
> +			if (my $proxyhdl = $reqstate->{proxyhdl}) {
> +			    $proxyhdl->{block_disconnect} = 1 if length $proxyhdl->{wbuf} > 0;
> +			    $proxyhdl->push_shutdown();
> +		        }
>   			$hdl->push_shutdown();
>   		    } elsif ($opcode == 9) {
>   			# ping received, schedule pong
> 




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

* Re: [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer
  2021-11-08 14:15   ` Fabian Ebner
@ 2021-11-08 15:45     ` Fabian Grünbichler
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-08 15:45 UTC (permalink / raw)
  To: Fabian Ebner, pve-devel

On November 8, 2021 3:15 pm, Fabian Ebner wrote:
> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
>> for proxied requests, we usually tear down the proxy connection
>> immediately when closing the source connection. this is not the correct
>> course of action for bulk one-way data streams that are proxied, where
>> the source connection might be closed, but the proxy connection might
>> still have data in the write buffer that needs to be written out.
>> 
>> push_shutdown already handles this case (closing the socket/FH after it
>> has been fully drained).
>> 
>> one example for such a proxied data stream is the 'migrate' data for a
>> remote migration, which gets proxied over a websocket connection.
>> terminating the proxied connection early makes the target VM crash for
>> obvious reasons.
>> 
>> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
>> ---
>>   src/PVE/APIServer/AnyEvent.pm | 10 ++++++----
>>   1 file changed, 6 insertions(+), 4 deletions(-)
>> 
>> diff --git a/src/PVE/APIServer/AnyEvent.pm b/src/PVE/APIServer/AnyEvent.pm
>> index 86d0e2e..ecf771f 100644
>> --- a/src/PVE/APIServer/AnyEvent.pm
>> +++ b/src/PVE/APIServer/AnyEvent.pm
>> @@ -144,7 +144,8 @@ sub client_do_disconnect {
>>       };
>>   
>>       if (my $proxyhdl = delete $reqstate->{proxyhdl}) {
>> -	&$shutdown_hdl($proxyhdl);
>> +	&$shutdown_hdl($proxyhdl)
>> +		if !$proxyhdl->{block_disconnect};
> 
> Style nit: fits in one line ;)
> 
> I'm not familiar with the code, so I'll just ask: can this be reached 
> without going through the code below first, i.e. can it happen that 
> block_disconnect is not set, but length $proxyhdl->{wbuf} > 0? Or is it 
> not important in other cases (if there are any)?

in theory, yes. in practice, not likely with anything that matters ;)

we have

- spiceproxy, if the client closes the connection we likely 
  don't care about the last few bytes of input being dropped
- proxied API requests with response_stream - never gets written to
- WS proxy (fixed in this patch ;) technically also used like 
  spiceproxy)

but I have to admit I did think about doing this always and relying on 
timeouts to clear the proxyhdl eventually, but then decided to play it 
safe and just change the one I am sure requires it..

> 
>>       }
>>   
>>       my $hdl = delete $reqstate->{hdl};
>> @@ -627,9 +628,10 @@ sub websocket_proxy {
>>   		    } elsif ($opcode == 8) {
>>   			my $statuscode = unpack ("n", $payload);
>>   			$self->dprint("websocket received close. status code: '$statuscode'");
>> -			if ($reqstate->{proxyhdl}) {
>> -			    $reqstate->{proxyhdl}->push_shutdown();
>> -			}
>> +			if (my $proxyhdl = $reqstate->{proxyhdl}) {
>> +			    $proxyhdl->{block_disconnect} = 1 if length $proxyhdl->{wbuf} > 0;
>> +			    $proxyhdl->push_shutdown();
>> +		        }
>>   			$hdl->push_shutdown();
>>   		    } elsif ($opcode == 9) {
>>   			# ping received, schedule pong
>> 
> 




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

* Re: [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id Fabian Grünbichler
@ 2021-11-09  9:06   ` Fabian Ebner
  2021-11-09 12:44     ` Fabian Grünbichler
  0 siblings, 1 reply; 50+ messages in thread
From: Fabian Ebner @ 2021-11-09  9:06 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

I feel like map_id should be moved to pve-common, so that it can be 
re-used in other places (pve-container will need it too), and so that no 
package boundary needs to be crossed if we ever need to adapt the 
internals of the idmap.

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> since we are going to reuse the same mechanism/code for network bridge
> mapping.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>   PVE/QemuMigrate.pm | 6 +++---
>   PVE/QemuServer.pm  | 6 ++++--
>   2 files changed, 7 insertions(+), 5 deletions(-)
> 
> diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
> index ae3eaf1..779f5ee 100644
> --- a/PVE/QemuMigrate.pm
> +++ b/PVE/QemuMigrate.pm
> @@ -342,7 +342,7 @@ sub prepare {
>   	my $targetsid = $sid;
>   	# NOTE: we currently ignore shared source storages in mappings so skip here too for now
>   	if (!$scfg->{shared}) {
> -	    $targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $sid);
> +	    $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>   	}
>   
>   	my $target_scfg = PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
> @@ -408,7 +408,7 @@ sub scan_local_volumes {
>   
>   	    next if @{$dl->{$storeid}} == 0;
>   
> -	    my $targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $storeid);
> +	    my $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $storeid);
>   	    # check if storage is available on target node
>   	    my $target_scfg = PVE::Storage::storage_check_enabled(
>   		$storecfg,
> @@ -479,7 +479,7 @@ sub scan_local_volumes {
>   	    my $targetsid = $sid;
>   	    # NOTE: we currently ignore shared source storages in mappings so skip here too for now
>   	    if (!$scfg->{shared}) {
> -		$targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $sid);
> +		$targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>   	    }
>   
>   	    PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
> diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
> index a0e1809..23e7b08 100644
> --- a/PVE/QemuServer.pm
> +++ b/PVE/QemuServer.pm
> @@ -120,7 +120,9 @@ PVE::JSONSchema::register_standard_option('pve-qemu-machine', {
>   });
>   
>   
> -sub map_storage {
> +# maps source to target ID
> +# currently used for targetstorage and targetbridge when migrating
> +sub map_id {
>       my ($map, $source) = @_;
>   
>       return $source if !defined($map);
> @@ -5264,7 +5266,7 @@ sub vm_migrate_alloc_nbd_disks {
>   	# volume is not available there, fall back to the default format.
>   	# Otherwise use the same format as the original.
>   	if (!$storagemap->{identity}) {
> -	    $storeid = map_storage($storagemap, $storeid);
> +	    $storeid = map_id($storagemap, $storeid);
>   	    my ($defFormat, $validFormats) = PVE::Storage::storage_default_format($storecfg, $storeid);
>   	    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
>   	    my $fileFormat = qemu_img_format($scfg, $volname);
> 




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

* Re: [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id
  2021-11-09  9:06   ` Fabian Ebner
@ 2021-11-09 12:44     ` Fabian Grünbichler
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-09 12:44 UTC (permalink / raw)
  To: Fabian Ebner, pve-devel

On November 9, 2021 10:06 am, Fabian Ebner wrote:
> I feel like map_id should be moved to pve-common, so that it can be 
> re-used in other places (pve-container will need it too), and so that no 
> package boundary needs to be crossed if we ever need to adapt the 
> internals of the idmap.

yeah, likely some other parts as well (either to guest-common, common or 
storage, once we extend this mechanism to cover ZFS replication and 
container migration)

> 
> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
>> since we are going to reuse the same mechanism/code for network bridge
>> mapping.
>> 
>> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
>> ---
>>   PVE/QemuMigrate.pm | 6 +++---
>>   PVE/QemuServer.pm  | 6 ++++--
>>   2 files changed, 7 insertions(+), 5 deletions(-)
>> 
>> diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
>> index ae3eaf1..779f5ee 100644
>> --- a/PVE/QemuMigrate.pm
>> +++ b/PVE/QemuMigrate.pm
>> @@ -342,7 +342,7 @@ sub prepare {
>>   	my $targetsid = $sid;
>>   	# NOTE: we currently ignore shared source storages in mappings so skip here too for now
>>   	if (!$scfg->{shared}) {
>> -	    $targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $sid);
>> +	    $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>>   	}
>>   
>>   	my $target_scfg = PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
>> @@ -408,7 +408,7 @@ sub scan_local_volumes {
>>   
>>   	    next if @{$dl->{$storeid}} == 0;
>>   
>> -	    my $targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $storeid);
>> +	    my $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $storeid);
>>   	    # check if storage is available on target node
>>   	    my $target_scfg = PVE::Storage::storage_check_enabled(
>>   		$storecfg,
>> @@ -479,7 +479,7 @@ sub scan_local_volumes {
>>   	    my $targetsid = $sid;
>>   	    # NOTE: we currently ignore shared source storages in mappings so skip here too for now
>>   	    if (!$scfg->{shared}) {
>> -		$targetsid = PVE::QemuServer::map_storage($self->{opts}->{storagemap}, $sid);
>> +		$targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>>   	    }
>>   
>>   	    PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
>> diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
>> index a0e1809..23e7b08 100644
>> --- a/PVE/QemuServer.pm
>> +++ b/PVE/QemuServer.pm
>> @@ -120,7 +120,9 @@ PVE::JSONSchema::register_standard_option('pve-qemu-machine', {
>>   });
>>   
>>   
>> -sub map_storage {
>> +# maps source to target ID
>> +# currently used for targetstorage and targetbridge when migrating
>> +sub map_id {
>>       my ($map, $source) = @_;
>>   
>>       return $source if !defined($map);
>> @@ -5264,7 +5266,7 @@ sub vm_migrate_alloc_nbd_disks {
>>   	# volume is not available there, fall back to the default format.
>>   	# Otherwise use the same format as the original.
>>   	if (!$storagemap->{identity}) {
>> -	    $storeid = map_storage($storagemap, $storeid);
>> +	    $storeid = map_id($storagemap, $storeid);
>>   	    my ($defFormat, $validFormats) = PVE::Storage::storage_default_format($storecfg, $storeid);
>>   	    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
>>   	    my $fileFormat = qemu_img_format($scfg, $volname);
>> 
> 




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

* Re: [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints Fabian Grünbichler
@ 2021-11-09 12:46   ` Fabian Ebner
  2021-11-10  7:40     ` Fabian Ebner
  2021-11-11 11:04     ` Fabian Grünbichler
  0 siblings, 2 replies; 50+ messages in thread
From: Fabian Ebner @ 2021-11-09 12:46 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> the following two endpoints are used for migration on the remote side
> 
> POST /nodes/NODE/qemu/VMID/mtunnel
> 
> which creates and locks an empty VM config, and spawns the main qmtunnel
> worker which binds to a VM-specific UNIX socket.
> 
> this worker handles JSON-encoded migration commands coming in via this
> UNIX socket:
> - config (set target VM config)
> -- checks permissions for updating config
> -- strips pending changes and snapshots
> -- sets (optional) firewall config
> - disk (allocate disk for NBD migration)
> -- checks permission for target storage
> -- returns drive string for allocated volume
> - disk-import (import 'pvesm export' stream for offline migration)
> -- checks permission for target storage
> -- forks a child running 'pvesm import' reading from a UNIX socket
> -- only one import allowed to run at any given moment
> - query-disk-import
> -- checks output of 'pvesm import' for volume ID message
> -- returns volid + success, or 'pending', or 'error'
> - start (returning migration info)
> - fstrim (via agent)
> - bwlimit (query bwlimit for storage)
> - ticket (creates a ticket for a WS connection to a specific socket)
> - resume
> - stop
> - nbdstop
> - unlock
> - quit (+ cleanup)
> 
> this worker serves as a replacement for both 'qm mtunnel' and various
> manual calls via SSH. the API call will return a ticket valid for
> connecting to the worker's UNIX socket via a websocket connection.
> 
> GET+WebSocket upgrade /nodes/NODE/qemu/VMID/mtunnelwebsocket
> 
> gets called for connecting to a UNIX socket via websocket forwarding,
> i.e. once for the main command mtunnel, and once each for the memory
> migration and each NBD drive-mirror/storage migration.
> 
> access is guarded by a short-lived ticket binding the authenticated user
> to the socket path. such tickets can be requested over the main mtunnel,
> which keeps track of socket paths currently used by that
> mtunnel/migration instance.
> 
> each command handler should check privileges for the requested action if
> necessary.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>      requires
>      - pve-storage with UNIX import support
>      - pve-access-control with tunnel ticket support
>      - pve-http-server with websocket fixes
> 
>   PVE/API2/Qemu.pm | 627 +++++++++++++++++++++++++++++++++++++++++++++++
>   1 file changed, 627 insertions(+)
> 
> diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
> index faf028b..a1a1813 100644
> --- a/PVE/API2/Qemu.pm
> +++ b/PVE/API2/Qemu.pm
> @@ -6,8 +6,13 @@ use Cwd 'abs_path';
>   use Net::SSLeay;
>   use POSIX;
>   use IO::Socket::IP;
> +use IO::Socket::UNIX;
> +use IPC::Open3;
> +use JSON;
> +use MIME::Base64;
>   use URI::Escape;
>   use Crypt::OpenSSL::Random;
> +use Socket qw(SOCK_STREAM);
>   
>   use PVE::Cluster qw (cfs_read_file cfs_write_file);;
>   use PVE::RRD;
> @@ -856,6 +861,7 @@ __PACKAGE__->register_method({
>   	    { subdir => 'spiceproxy' },
>   	    { subdir => 'sendkey' },
>   	    { subdir => 'firewall' },
> +	    { subdir => 'mtunnel' },
>   	    ];
>   
>   	return $res;
> @@ -4428,4 +4434,625 @@ __PACKAGE__->register_method({
>   	return PVE::QemuServer::Cloudinit::dump_cloudinit_config($conf, $param->{vmid}, $param->{type});
>       }});
>   
> +__PACKAGE__->register_method({
> +    name => 'mtunnel',
> +    path => '{vmid}/mtunnel',
> +    method => 'POST',
> +    protected => 1,
> +    proxyto => 'node',
> +    description => 'Migration tunnel endpoint - only for internal use by VM migration.',
> +    permissions => {
> +	check => ['perm', '/vms/{vmid}', [ 'VM.Allocate' ]],
> +	description => "You need 'VM.Allocate' permissions on /vms/{vmid}. Further permission checks happen during the actual migration.",
> +    },
> +    parameters => {
> +	additionalProperties => 0,
> +	properties => {
> +	    node => get_standard_option('pve-node'),
> +	    vmid => get_standard_option('pve-vmid'),
> +	    storages => {
> +		type => 'string',
> +		format => 'pve-storage-id-list',
> +		optional => 1,
> +		description => 'List of storages to check permission and availability. Will be checked again for all actually used storages during migration.',
> +	    },
> +	},
> +    },
> +    returns => {
> +	additionalProperties => 0,
> +	properties => {
> +	    upid => { type => 'string' },
> +	    ticket => { type => 'string' },
> +	    socket => { type => 'string' },
> +	},
> +    },
> +    code => sub {
> +	my ($param) = @_;
> +
> +	my $rpcenv = PVE::RPCEnvironment::get();
> +	my $authuser = $rpcenv->get_user();
> +
> +	my $node = extract_param($param, 'node');
> +	my $vmid = extract_param($param, 'vmid');
> +
> +	my $storages = extract_param($param, 'storages');
> +
> +	my $storecfg = PVE::Storage::config();
> +	foreach my $storeid (PVE::Tools::split_list($storages)) {
> +	    $check_storage_access_migrate->($rpcenv, $authuser, $storecfg, $storeid, $node);
> +	}
> +
> +	PVE::Cluster::check_cfs_quorum();
> +
> +	my $socket_addr = "/run/qemu-server/$vmid.mtunnel";
> +
> +	my $lock = 'create';
> +	eval { PVE::QemuConfig->create_and_lock_config($vmid, 0, $lock); };
> +
> +	raise_param_exc({ vmid => "unable to create empty VM config - $@"})
> +	    if $@;
> +
> +	my $realcmd = sub {
> +	    my $pveproxy_uid;
> +
> +	    my $state = {
> +		storecfg => PVE::Storage::config(),
> +		lock => $lock,
> +	    };
> +
> +	    my $run_locked = sub {
> +		my ($code, $params) = @_;
> +		return PVE::QemuConfig->lock_config($vmid, sub {
> +		    my $conf = PVE::QemuConfig->load_config($vmid);
> +
> +		    $state->{conf} = $conf;
> +
> +		    die "Encountered wrong lock - aborting mtunnel command handling.\n"
> +			if $state->{lock} && !PVE::QemuConfig->has_lock($conf, $state->{lock});
> +
> +		    return $code->($params);
> +		});
> +	    };
> +
> +	    my $cmd_desc = {
> +		bwlimit => {
> +		    storage => {
> +			type => 'string',
> +			format => 'pve-storage-id',
> +			description => "Storage for which bwlimit is queried",
> +		    },
> +		    bwlimit => {
> +			description => "Override I/O bandwidth limit (in KiB/s).",
> +			optional => 1,
> +			type => 'integer',
> +			minimum => '0',
> +		    },
> +		},
> +		config => {
> +		    conf => {
> +			type => 'string',
> +			description => 'Full VM config, adapted for target cluster/node',
> +		    },
> +		    'firewall-conf' => {

Here and thus for parsing, it's 'firewall-conf', but in the command 
handler 'firewall-config' is accessed.

> +			type => 'string',
> +			description => 'VM firewall config',
> +			optional => 1,
> +		    },
> +		},
> +		disk => {
> +		    format => PVE::JSONSchema::get_standard_option('pve-qm-image-format'),
> +		    storage => {
> +			type => 'string',
> +			format => 'pve-storage-id',
> +		    },
> +		    drive => {
> +			type => 'object',
> +			description => 'parsed drive information without volid and format',
> +		    },
> +		},
> +		'disk-import' => {
> +		    volname => {
> +			type => 'string',
> +			description => 'volume name to use prefered target volume name',

Nit: I wasn't able to parse this description ;) (also missing r in 
preferred)

> +		    },
> +		    format => PVE::JSONSchema::get_standard_option('pve-qm-image-format'),
> +		    'export-formats' => {
> +			type => 'string',
> +			description => 'list of supported export formats',
> +		    },
> +		    storage => {
> +			type => 'string',
> +			format => 'pve-storage-id',
> +		    },
> +		    'with-snapshots' => {
> +			description =>
> +			    "Whether the stream includes intermediate snapshots",
> +			type => 'boolean',
> +			optional => 1,
> +			default => 0,
> +		    },
> +		    'allow-rename' => {
> +			description => "Choose a new volume ID if the requested " .
> +			  "volume ID already exists, instead of throwing an error.",
> +			type => 'boolean',
> +			optional => 1,
> +			default => 0,
> +		    },
> +		},
> +		start => {
> +		    start_params => {
> +			type => 'object',
> +			description => 'params passed to vm_start_nolock',
> +		    },
> +		    migrate_opts => {
> +			type => 'object',
> +			description => 'migrate_opts passed to vm_start_nolock',
> +		    },
> +		},
> +		ticket => {
> +		    path => {
> +			type => 'string',
> +			description => 'socket path for which the ticket should be valid. must be known to current mtunnel instance.',
> +		    },
> +		},
> +		quit => {
> +		    cleanup => {
> +			type => 'boolean',
> +			description => 'remove VM config and disks, aborting migration',
> +			default => 0,
> +		    },
> +		},
> +	    };
> +
> +	    my $cmd_handlers = {
> +		'version' => sub {
> +		    # compared against other end's version
> +		    # bump/reset both for breaking changes
> +		    # bump tunnel only for opt-in changes
> +		    return {
> +			api => 2,
> +			age => 0,
> +		    };
> +		},
> +		'config' => sub {
> +		    my ($params) = @_;
> +
> +		    # parse and write out VM FW config if given
> +		    if (my $fw_conf = $params->{'firewall-config'}) {
> +			my ($path, $fh) = PVE::Tools::tempfile_contents($fw_conf, 700);
> +
> +			my $empty_conf = {
> +			    rules => [],
> +			    options => {},
> +			    aliases => {},
> +			    ipset => {} ,
> +			    ipset_comments => {},
> +			};
> +			my $cluster_fw_conf = PVE::Firewall::load_clusterfw_conf();
> +
> +			# TODO: add flag for strict parsing?
> +			# TODO: add import sub that does all this given raw content?
> +			my $vmfw_conf = PVE::Firewall::generic_fw_config_parser($path, $cluster_fw_conf, $empty_conf, 'vm');
> +			$vmfw_conf->{vmid} = $vmid;
> +			PVE::Firewall::save_vmfw_conf($vmid, $vmfw_conf);
> +
> +			$state->{cleanup}->{fw} = 1;
> +		    }
> +
> +		    PVE::QemuConfig->remove_lock($vmid, 'create');
> +
> +		    # TODO add flag for strict parsing?
> +		    my $new_conf = PVE::QemuServer::parse_vm_config("incoming/qemu-server/$vmid.conf", $params->{conf});
> +		    delete $new_conf->{lock};
> +		    delete $new_conf->{digest};
> +
> +		    # TODO handle properly?
> +		    delete $new_conf->{snapshots};
> +		    delete $new_conf->{pending};

'parent' should also be deleted if the snapshots are.

> +
> +		    # not handled by update_vm_api
> +		    my $vmgenid = delete $new_conf->{vmgenid};
> +		    my $meta = delete $new_conf->{meta};
> +
> +		    $new_conf->{vmid} = $vmid;
> +		    $new_conf->{node} = $node;
> +
> +		    $update_vm_api->($new_conf, 1);
> +
> +		    my $conf = PVE::QemuConfig->load_config($vmid);
> +		    $conf->{lock} = 'migrate';
> +		    $conf->{vmgenid} = $vmgenid;
> +		    $conf->{meta} = $meta;
> +		    PVE::QemuConfig->write_config($vmid, $conf);
> +
> +		    $state->{lock} = 'migrate';
> +
> +		    return;
> +		},
> +		'bwlimit' => sub {
> +		    my ($params) = @_;
> +
> +		    my $bwlimit = PVE::Storage::get_bandwidth_limit('migration', [$params->{storage}], $params->{bwlimit});
> +		    return { bwlimit => $bwlimit };
> +
> +		},
> +		'disk' => sub {
> +		    my ($params) = @_;

Feels like some deduplication between here and 
vm_migrate_alloc_nbd_disks should be possible.

> +
> +		    my $format = $params->{format};
> +		    my $storeid = $params->{storage};
> +		    my $drive = $params->{drive};
> +
> +		    $check_storage_access_migrate->($rpcenv, $authuser, $state->{storecfg}, $storeid, $node);
> +
> +		    my ($default_format, $valid_formats) = PVE::Storage::storage_default_format($state->{storecfg}, $storeid);
> +		    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
> +		    $format = $default_format
> +			if !grep {$format eq $_} @{$valid_formats};
> +
> +		    my $size = int($drive->{size})/1024;
> +		    my $newvolid = PVE::Storage::vdisk_alloc($state->{storecfg}, $storeid, $vmid, $format, undef, $size);
> +
> +		    my $newdrive = $drive;
> +		    $newdrive->{format} = $format;
> +		    $newdrive->{file} = $newvolid;
> +
> +		    $state->{cleanup}->{volumes}->{$newvolid} = 1;
> +		    my $drivestr = PVE::QemuServer::print_drive($newdrive);
> +		    return {
> +			drivestr => $drivestr,
> +			volid => $newvolid,
> +		    };
> +		},
> +		'disk-import' => sub {
> +		    my ($params) = @_;

Similarly here with storage_migrate. Having the checks and deciding on 
name+format be its own function would also make it possible to abort 
early, which is especially useful if there are multiple disks. But would 
require a precondition handler for remote migration of course.

> +
> +		    die "disk import already running as PID '$state->{disk_import}->{pid}'\n"
> +			if $state->{disk_import}->{pid};
> +
> +		    my $format = $params->{format};
> +		    my $storeid = $params->{storage};
> +		    $check_storage_access_migrate->($rpcenv, $authuser, $state->{storecfg}, $storeid, $node);
> +
> +		    my $with_snapshots = $params->{'with-snapshots'} ? 1 : 0;
> +
> +		    my ($default_format, $valid_formats) = PVE::Storage::storage_default_format($state->{storecfg}, $storeid);
> +		    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
> +		    die "unsupported format '$format' for storage '$storeid'\n"
> +			if !grep {$format eq $_} @{$valid_formats};
> +
> +		    my $volname = $params->{volname};
> +
> +		    # get target volname, taken from PVE::Storage
> +		    (my $name_without_extension = $volname) =~ s/\.$format$//;
> +		    if ($scfg->{path}) {
> +			$volname = "$vmid/$name_without_extension.$format";
> +		    } else {
> +			$volname = "$name_without_extension";
> +		    }

This is just a best-effort for guessing a valid volname that was 
intended only as a fall-back when target and source storage have 
different types. If the storage type is the same, the volname should be 
kept, so that e.g. an external plugin with $scfg->{path} and no 
extension also works.

> +
> +		    my $migration_snapshot;
> +		    if ($scfg->{type} eq 'zfspool' || $scfg->{type} eq 'btrfs') {
> +			$migration_snapshot = '__migration__';
> +		    }
> +
> +		    my $volid = "$storeid:$volname";
> +
> +		    # find common import/export format, taken from PVE::Storage
> +		    my @import_formats = PVE::Storage::volume_import_formats($state->{storecfg}, $volid, $migration_snapshot, undef, $with_snapshots);
> +		    my @export_formats = PVE::Tools::split_list($params->{'export-formats'});
> +		    my %import_hash = map { $_ => 1 } @import_formats;
> +		    my @common = grep { $import_hash{$_} } @export_formats;
> +		    die "no matching import/export format found for storage '$storeid'\n"
> +			if !@common;
> +		    $format = $common[0];
> +
> +		    my $input = IO::File->new();
> +		    my $info = IO::File->new();
> +		    my $unix = "/run/qemu-server/$vmid.storage";
> +
> +		    my $import_cmd = ['pvesm', 'import', $volid, $format, "unix://$unix", '-with-snapshots', $with_snapshots];
> +		    if ($params->{'allow-rename'}) {
> +			push @$import_cmd, '-allow-rename', $params->{'allow-rename'};
> +		    }
> +		    if ($migration_snapshot) {
> +			push @$import_cmd, '-delete-snapshot', $migration_snapshot;

Missing '-snapshot $migration_snapshot'? While the parameter is ignored 
by our ZFSPoolPlugin, the BTRFSPlugin aborts if it's not specified 
AFAICS. And external plugins might require it too.

In general, we'll need to be careful not to introduce mismatches between 
the import and the export parameters. Might it be better if the client 
would pass along (most of) the parameters for the import command (which 
basically is how it's done for the existing storage_migrate)?

> +		    }
> +
> +		    unlink $unix;
> +		    my $cpid = open3($input, $info, $info, @{$import_cmd})
> +			or die "failed to spawn disk-import child - $!\n";
> +
> +		    $state->{disk_import}->{pid} = $cpid;
> +		    my $ready;
> +		    eval {
> +			PVE::Tools::run_with_timeout(5, sub { $ready = <$info>; });
> +		    };
> +		    die "failed to read readyness from disk import child: $@\n" if $@;
> +		    print "$ready\n";
> +
> +		    chown $pveproxy_uid, -1, $unix;
> +
> +		    $state->{disk_import}->{fh} = $info;
> +		    $state->{disk_import}->{socket} = $unix;
> +
> +		    $state->{sockets}->{$unix} = 1;
> +
> +		    return {
> +			socket => $unix,
> +			format => $format,
> +		    };
> +		},
> +		'query-disk-import' => sub {
> +		    my ($params) = @_;
> +
> +		    die "no disk import running\n"
> +			if !$state->{disk_import}->{pid};
> +
> +		    my $pattern = PVE::Storage::volume_imported_message(undef, 1);
> +		    my $result;
> +		    eval {
> +			my $fh = $state->{disk_import}->{fh};
> +			PVE::Tools::run_with_timeout(5, sub { $result = <$fh>; });
> +			print "disk-import: $result\n" if $result;
> +		    };
> +		    if ($result && $result =~ $pattern) {
> +			my $volid = $1;
> +			waitpid($state->{disk_import}->{pid}, 0);
> +
> +			my $unix = $state->{disk_import}->{socket};
> +			unlink $unix;
> +			delete $state->{sockets}->{$unix};
> +			delete $state->{disk_import};

$volid should be registered for potential cleanup.

> +			return {
> +			    status => "complete",
> +			    volid => $volid,
> +			};
> +		    } elsif (!$result && waitpid($state->{disk_import}->{pid}, WNOHANG)) {
> +			my $unix = $state->{disk_import}->{socket};
> +			unlink $unix;
> +			delete $state->{sockets}->{$unix};
> +			delete $state->{disk_import};
> +
> +			return {
> +			    status => "error",
> +			};
> +		    } else {
> +			return {
> +			    status => "pending",
> +			};
> +		    }
> +		},
> +		'start' => sub {
> +		    my ($params) = @_;
> +
> +		    my $info = PVE::QemuServer::vm_start_nolock(
> +			$state->{storecfg},
> +			$vmid,
> +			$state->{conf},
> +			$params->{start_params},
> +			$params->{migrate_opts},
> +		    );
> +
> +
> +		    if ($info->{migrate}->{proto} ne 'unix') {
> +			PVE::QemuServer::vm_stop(undef, $vmid, 1, 1);
> +			die "migration over non-UNIX sockets not possible\n";
> +		    }
> +
> +		    my $socket = $info->{migrate}->{addr};
> +		    chown $pveproxy_uid, -1, $socket;
> +		    $state->{sockets}->{$socket} = 1;
> +
> +		    my $unix_sockets = $info->{migrate}->{unix_sockets};
> +		    foreach my $socket (@$unix_sockets) {
> +			chown $pveproxy_uid, -1, $socket;
> +			$state->{sockets}->{$socket} = 1;
> +		    }
> +		    return $info;
> +		},
> +		'fstrim' => sub {
> +		    if (PVE::QemuServer::qga_check_running($vmid)) {
> +			eval { mon_cmd($vmid, "guest-fstrim") };
> +			warn "fstrim failed: $@\n" if $@;
> +		    }
> +		    return;
> +		},
> +		'stop' => sub {
> +		    PVE::QemuServer::vm_stop(undef, $vmid, 1, 1);
> +		    return;
> +		},
> +		'nbdstop' => sub {
> +		    PVE::QemuServer::nbd_stop($vmid);
> +		    return;
> +		},
> +		'resume' => sub {
> +		    if (PVE::QemuServer::check_running($vmid, 1)) {
> +			PVE::QemuServer::vm_resume($vmid, 1, 1);
> +		    } else {
> +			die "VM $vmid not running\n";
> +		    }
> +		    return;
> +		},
> +		'unlock' => sub {
> +		    PVE::QemuConfig->remove_lock($vmid, $state->{lock});
> +		    delete $state->{lock};
> +		    return;
> +		},
> +		'ticket' => sub {
> +		    my ($params) = @_;
> +
> +		    my $path = $params->{path};
> +
> +		    die "Not allowed to generate ticket for unknown socket '$path'\n"
> +			if !defined($state->{sockets}->{$path});
> +
> +		    return { ticket => PVE::AccessControl::assemble_tunnel_ticket($authuser, "/socket/$path") };
> +		},
> +		'quit' => sub {
> +		    my ($params) = @_;
> +
> +		    if ($params->{cleanup}) {
> +			if ($state->{cleanup}->{fw}) {
> +			    PVE::Firewall::remove_vmfw_conf($vmid);
> +			}
> +
> +			if (my @volumes = keys $state->{cleanup}->{volumes}->$%) {

keys on scalar? This is fixed in a later patch, but...

> +			    PVE::Storage::foreach_volid(@volumes, sub {

...PVE::Storage::foreach_volid does not have this signature. It needs 
what vdisk_list returns. A simple 'for' should be good enough here.

> +				my ($volid, $sid, $volname, $d) = @_;
> +
> +				print "freeing volume '$volid' as part of cleanup\n";
> +				eval { PVE::Storage::vdisk_free($storecfg, $volid) };
> +				warn $@ if $@;
> +			    });
> +			}
> +
> +			PVE::QemuServer::destroy_vm($state->{storecfg}, $vmid, 1);
> +		    }
> +
> +		    $state->{exit} = 1;
> +		    return;
> +		},
> +	    };
> +
> +	    $run_locked->(sub {
> +		my $socket_addr = "/run/qemu-server/$vmid.mtunnel";
> +		unlink $socket_addr;
> +
> +		$state->{socket} = IO::Socket::UNIX->new(
> +	            Type => SOCK_STREAM(),
> +		    Local => $socket_addr,
> +		    Listen => 1,
> +		);
> +
> +		$pveproxy_uid = getpwnam('www-data')
> +		    or die "Failed to resolve user 'www-data' to numeric UID\n";
> +		chown $pveproxy_uid, -1, $socket_addr;
> +	    });
> +
> +	    print "mtunnel started\n";
> +
> +	    my $conn = $state->{socket}->accept();
> +
> +	    $state->{conn} = $conn;
> +
> +	    my $reply_err = sub {
> +		my ($msg) = @_;
> +
> +		my $reply = JSON::encode_json({
> +		    success => JSON::false,
> +		    msg => $msg,
> +		});
> +		$conn->print("$reply\n");
> +		$conn->flush();
> +	    };
> +
> +	    my $reply_ok = sub {
> +		my ($res) = @_;
> +
> +		$res->{success} = JSON::true;
> +		my $reply = JSON::encode_json($res);
> +		$conn->print("$reply\n");
> +		$conn->flush();
> +	    };
> +
> +	    while (my $line = <$conn>) {
> +		chomp $line;
> +
> +		# untaint, we validate below if needed
> +		($line) = $line =~ /^(.*)$/;
> +		my $parsed = eval { JSON::decode_json($line) };
> +		if ($@) {
> +		    $reply_err->("failed to parse command - $@");
> +		    next;
> +		}
> +
> +		my $cmd = delete $parsed->{cmd};
> +		if (!defined($cmd)) {
> +		    $reply_err->("'cmd' missing");
> +		} elsif (my $handler = $cmd_handlers->{$cmd}) {
> +		    print "received command '$cmd'\n";
> +		    eval {
> +			if ($cmd_desc->{$cmd}) {
> +			    PVE::JSONSchema::validate($cmd_desc->{$cmd}, $parsed);
> +			} else {
> +			    $parsed = {};
> +			}
> +			my $res = $run_locked->($handler, $parsed);
> +			$reply_ok->($res);
> +		    };
> +		    $reply_err->("failed to handle '$cmd' command - $@")
> +			if $@;
> +		} else {
> +		    $reply_err->("unknown command '$cmd' given");
> +		}
> +
> +		if ($state->{exit}) {
> +		    $state->{conn}->close();
> +		    $state->{socket}->close();
> +		    last;
> +		}
> +	    }
> +
> +	    print "mtunnel exited\n";
> +	};
> +
> +	my $ticket = PVE::AccessControl::assemble_tunnel_ticket($authuser, "/socket/$socket_addr");
> +	my $upid = $rpcenv->fork_worker('qmtunnel', $vmid, $authuser, $realcmd);
> +
> +	return {
> +	    ticket => $ticket,
> +	    upid => $upid,
> +	    socket => $socket_addr,
> +	};
> +    }});
> +
> +__PACKAGE__->register_method({
> +    name => 'mtunnelwebsocket',
> +    path => '{vmid}/mtunnelwebsocket',
> +    method => 'GET',
> +    proxyto => 'node',
> +    permissions => {
> +	description => "You need to pass a ticket valid for the selected socket. Tickets can be created via the mtunnel API call, which will check permissions accordingly.",
> +        user => 'all', # check inside
> +    },
> +    description => 'Migration tunnel endpoint for websocket upgrade - only for internal use by VM migration.',
> +    parameters => {
> +	additionalProperties => 0,
> +	properties => {
> +	    node => get_standard_option('pve-node'),
> +	    vmid => get_standard_option('pve-vmid'),
> +	    socket => {
> +		type => "string",
> +		description => "unix socket to forward to",
> +	    },
> +	    ticket => {
> +		type => "string",
> +		description => "ticket return by initial 'mtunnel' API call, or retrieved via 'ticket' tunnel command",
> +	    },
> +	},
> +    },
> +    returns => {
> +	type => "object",
> +	properties => {
> +	    port => { type => 'string', optional => 1 },
> +	    socket => { type => 'string', optional => 1 },
> +	},
> +    },
> +    code => sub {
> +	my ($param) = @_;
> +
> +	my $rpcenv = PVE::RPCEnvironment::get();
> +	my $authuser = $rpcenv->get_user();
> +
> +	my $vmid = $param->{vmid};
> +	# check VM exists
> +	PVE::QemuConfig->load_config($vmid);
> +
> +	my $socket = $param->{socket};
> +	PVE::AccessControl::verify_tunnel_ticket($param->{ticket}, $authuser, "/socket/$socket");
> +
> +	return { socket => $socket };
> +    }});
> +
>   1;
> 




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

* Re: [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation
  2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation Fabian Grünbichler
@ 2021-11-09 12:54   ` Dominik Csapak
  2021-11-11  9:58     ` Fabian Grünbichler
  0 siblings, 1 reply; 50+ messages in thread
From: Dominik Csapak @ 2021-11-09 12:54 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

looks mostly fine, some comments inline

On 11/5/21 14:03, Fabian Grünbichler wrote:
> the websocket tunnel helper accepts control commands (encoded as
> single-line JSON) on stdin, and prints responses on stdout.
> 
> the following commands are available:
> - "connect" a 'control' tunnel via a websocket
> - "forward" a local unix socket to a remote socket via a websocket
> -- if requested, this will ask for a ticket via the control tunnel after
> accepting a new connection on the unix socket
> - "close" the control tunnel and any forwarded socket
> 
> any other json input (without the 'control' flag set) is forwarded as-is
> to the remote end of the control tunnel.
> 
> internally, the tunnel helper will spawn tokio tasks for
> - handling the control tunnel connection (new commands are passed in via
> an mpsc channel together with a oneshot channel for the response)
> - handling accepting new connections on each forwarded unix socket
> - handling forwarding data over accepted forwarded connections
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>      requires proxmox-http with changes and bumped version
> 
>   Cargo.toml  |  13 ++
>   src/main.rs | 410 ++++++++++++++++++++++++++++++++++++++++++++++++++++
>   2 files changed, 423 insertions(+)
>   create mode 100644 src/main.rs
> 
> diff --git a/Cargo.toml b/Cargo.toml
> index 939184c..9d2a8c6 100644
> --- a/Cargo.toml
> +++ b/Cargo.toml
> @@ -9,3 +9,16 @@ description = "Proxmox websocket tunneling helper"
>   exclude = ["debian"]
>   
>   [dependencies]
> +anyhow = "1.0"
> +base64 = "0.12"
> +futures = "0.3"
> +futures-util = "0.3"
> +hyper = { version = "0.14" }
> +openssl = "0.10"
> +percent-encoding = "2"
> +proxmox-http = { version = "0.5.2", path = "../proxmox/proxmox-http", features = ["websocket", "client"] }
> +serde = { version = "1.0", features = ["derive"] }
> +serde_json = "1.0"
> +tokio = { version = "1", features = ["io-std", "io-util", "macros", "rt-multi-thread", "sync"] }
> +tokio-stream = { version = "0.1", features = ["io-util"] }
> +tokio-util = "0.6"
> diff --git a/src/main.rs b/src/main.rs
> new file mode 100644
> index 0000000..150c1cf
> --- /dev/null
> +++ b/src/main.rs
> @@ -0,0 +1,410 @@
> +use anyhow::{bail, format_err, Error};
> +
> +use std::collections::VecDeque;
> +use std::sync::{Arc, Mutex};
> +
> +use futures::future::FutureExt;
> +use futures::select;
> +
> +use hyper::client::{Client, HttpConnector};
> +use hyper::header::{SEC_WEBSOCKET_KEY, SEC_WEBSOCKET_VERSION, UPGRADE};
> +use hyper::upgrade::Upgraded;
> +use hyper::{Body, Request, StatusCode};
> +
> +use openssl::ssl::{SslConnector, SslMethod};
> +use percent_encoding::{utf8_percent_encode, NON_ALPHANUMERIC};
> +
> +use serde::{Deserialize, Serialize};
> +use serde_json::{Map, Value};
> +use tokio::io::{AsyncBufReadExt, AsyncWriteExt, BufReader};
> +use tokio::net::{UnixListener, UnixStream};
> +use tokio::sync::{mpsc, oneshot};
> +use tokio_stream::wrappers::LinesStream;
> +use tokio_stream::StreamExt;
> +
> +use proxmox_http::client::HttpsConnector;
> +use proxmox_http::websocket::{OpCode, WebSocket, WebSocketReader, WebSocketWriter};
> +
> +#[derive(Serialize, Deserialize, Debug)]
> +#[serde(rename_all = "kebab-case")]
> +enum CmdType {
> +    Connect,
> +    Forward,
> +    CloseCmd,

this is never used

> +    NonControl,
> +}
> +
> +type CmdData = Map<String, Value>;
> +
> +#[derive(Serialize, Deserialize, Debug)]
> +#[serde(rename_all = "kebab-case")]
> +struct ConnectCmdData {
> +    // target URL for WS connection
> +    url: String,
> +    // fingerprint of TLS certificate
> +    fingerprint: Option<String>,
> +    // addition headers such as authorization
> +    headers: Option<Vec<(String, String)>>,
> +}
> +
> +#[derive(Serialize, Deserialize, Debug, Clone)]
> +#[serde(rename_all = "kebab-case")]
> +struct ForwardCmdData {
> +    // target URL for WS connection
> +    url: String,
> +    // addition headers such as authorization
> +    headers: Option<Vec<(String, String)>>,
> +    // fingerprint of TLS certificate
> +    fingerprint: Option<String>,
> +    // local UNIX socket path for forwarding
> +    unix: String,
> +    // request ticket using these parameters
> +    ticket: Option<Map<String, Value>>,
> +}
> +
> +struct CtrlTunnel {
> +    sender: Option<mpsc::UnboundedSender<(Value, oneshot::Sender<String>)>>,
> +    forwarded: Arc<Mutex<Vec<oneshot::Sender<()>>>>,

for now, this is really not used (see my comments further below)

> +}
> +
> +impl CtrlTunnel {
> +    async fn read_cmd_loop(mut self) -> Result<(), Error> {
> +        let mut stdin_stream = LinesStream::new(BufReader::new(tokio::io::stdin()).lines());
> +        while let Some(res) = stdin_stream.next().await {
> +            match res {
> +                Ok(line) => {
> +                    let (cmd_type, data) = Self::parse_cmd(&line)?;
> +                    match cmd_type {
> +                        CmdType::Connect => self.handle_connect_cmd(data).await,
> +                        CmdType::Forward => {
> +                            let res = self.handle_forward_cmd(data).await;
> +                            match &res {
> +                                Ok(()) => println!("{}", serde_json::json!({"success": true})),
> +                                Err(msg) => println!(
> +                                    "{}",
> +                                    serde_json::json!({"success": false, "msg": msg.to_string()})
> +                                ),
> +                            };
> +                            res
> +                        }
> +                        CmdType::NonControl => self
> +                            .handle_tunnel_cmd(data)
> +                            .await
> +                            .map(|res| println!("{}", res)),
> +                        _ => unimplemented!(),
> +                    }
> +                }
> +                Err(err) => bail!("Failed to read from STDIN - {}", err),
> +            }?;
> +        }
> +
> +        Ok(())
> +    }
> +
> +    fn parse_cmd(line: &str) -> Result<(CmdType, CmdData), Error> {
> +        let mut json: Map<String, Value> = serde_json::from_str(line)?;
> +        match json.remove("control") {
> +            Some(Value::Bool(true)) => {
> +                match json.remove("cmd").map(serde_json::from_value::<CmdType>) {
> +                    None => bail!("input has 'control' flag, but no control 'cmd' set.."),
> +                    Some(Err(e)) => bail!("failed to parse control cmd - {}", e),
> +                    Some(Ok(cmd_type)) => Ok((cmd_type, json)),
> +                }
> +            }
> +            _ => Ok((CmdType::NonControl, json)),
> +        }
> +    }
> +
> +    async fn websocket_connect(
> +        url: String,
> +        extra_headers: Vec<(String, String)>,
> +        fingerprint: Option<String>,
> +    ) -> Result<Upgraded, Error> {
> +        let ws_key = proxmox::sys::linux::random_data(16)?;
> +        let ws_key = base64::encode(&ws_key);
> +        let mut req = Request::builder()
> +            .uri(url)
> +            .header(UPGRADE, "websocket")
> +            .header(SEC_WEBSOCKET_VERSION, "13")
> +            .header(SEC_WEBSOCKET_KEY, ws_key)
> +            .body(Body::empty())
> +            .unwrap();
> +
> +        let headers = req.headers_mut();
> +        for (name, value) in extra_headers {
> +            let name = hyper::header::HeaderName::from_bytes(name.as_bytes())?;
> +            let value = hyper::header::HeaderValue::from_str(&value)?;
> +            headers.insert(name, value);
> +        }
> +
> +        let mut ssl_connector_builder = SslConnector::builder(SslMethod::tls()).unwrap();

not sure if this unwrap cannot fail though?

> +        if fingerprint.is_some() {
> +            // FIXME actually verify fingerprint via callback!
> +            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::NONE);
> +        } else {
> +            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::PEER);
> +        }
> +
> +        let mut httpc = HttpConnector::new();
> +        httpc.enforce_http(false); // we want https...
> +        httpc.set_connect_timeout(Some(std::time::Duration::new(10, 0)));
> +        let https = HttpsConnector::with_connector(httpc, ssl_connector_builder.build(), 120);
> +
> +        let client = Client::builder().build::<_, Body>(https);
> +        let res = client.request(req).await?;
> +        if res.status() != StatusCode::SWITCHING_PROTOCOLS {
> +            bail!("server didn't upgrade: {}", res.status());
> +        }
> +
> +        hyper::upgrade::on(res)
> +            .await
> +            .map_err(|err| format_err!("failed to upgrade - {}", err))
> +    }
> +
> +    async fn handle_connect_cmd(&mut self, mut data: CmdData) -> Result<(), Error> {
> +        let mut data: ConnectCmdData = data
> +            .remove("data")
> +            .ok_or_else(|| format_err!("'connect' command missing 'data'"))
> +            .map(serde_json::from_value)??;
> +
> +        if self.sender.is_some() {
> +            bail!("already connected!");
> +        }
> +
> +        let upgraded = Self::websocket_connect(
> +            data.url.clone(),
> +            data.headers.take().unwrap_or_else(Vec::new),
> +            data.fingerprint.take(),
> +        )
> +        .await?;
> +
> +        let (tx, rx) = mpsc::unbounded_channel();
> +        self.sender = Some(tx);
> +        tokio::spawn(async move {
> +            if let Err(err) = Self::handle_ctrl_tunnel(upgraded, rx).await {
> +                eprintln!("Tunnel to {} failed - {}", data.url, err);
> +            }
> +        });
> +
> +        Ok(())
> +    }
> +
> +    async fn handle_forward_cmd(&mut self, mut data: CmdData) -> Result<(), Error> {
> +        let data: ForwardCmdData = data
> +            .remove("data")
> +            .ok_or_else(|| format_err!("'forward' command missing 'data'"))
> +            .map(serde_json::from_value)??;
> +
> +        if self.sender.is_none() && data.ticket.is_some() {
> +            bail!("dynamically requesting ticket requires cmd tunnel connection!");
> +        }
> +
> +        let unix_listener = UnixListener::bind(data.unix.clone()).unwrap();

it would be better to bubble the error up instead of unwrapping here
(AFAICS, the rest of the unwraps are ok, since they cannot really fail?)

> +        let (tx, rx) = oneshot::channel();
> +        let data = Arc::new(data);
> +
> +        self.forwarded.lock().unwrap().push(tx);

we push the 'tx' here into the forwarded vec, but never use it again
(no other 'lock()' call in the file)

> +        let cmd_sender = self.sender.clone();
> +
> +        tokio::spawn(async move {
> +            let mut rx = rx.fuse();
> +            let mut tasks: Vec<tokio::task::JoinHandle<()>> = Vec::new();
> +            loop {
> +                let accept = unix_listener.accept().fuse();
> +                tokio::pin!(accept);
> +                let data2 = data.clone();
> +                select! {
> +                    _ = rx => {
> +                        eprintln!("received shutdown signal, closing unix listener stream and forwarding handlers");
> +                        for task in tasks {
> +                            task.abort();
> +                        }
> +                        break;
> +                    },

which makes this branch dead code

so i'd drop the forwarded part and simplify this to

match unix_listener.accept().await {
...
}

> +                    res = accept => match res {
> +                        Ok((unix_stream, _)) => {
> +                            eprintln!("accepted new connection on '{}'", data2.unix);
> +                            let data3: Result<Arc<ForwardCmdData>, Error> = match (&cmd_sender, &data2.ticket) {
> +                                (Some(cmd_sender), Some(_)) => Self::get_ticket(cmd_sender, data2.clone()).await,\

the get_ticket could probably be inside the 'handle_forward_tunnel' this 
way, another client could connect while the first ticket is checked.
not necessary for now though, since we do not connect in parallel atm

> +                                _ => Ok(data2.clone()),
> +                            };
> +
> +                            match data3 {
> +                                Ok(data3) => {
> +                                    let task = tokio::spawn(async move {
> +                                        if let Err(err) = Self::handle_forward_tunnel(data3.clone(), unix_stream).await {
> +                                            eprintln!("Tunnel for {} failed - {}", data3.unix, err);
> +                                        }
> +                                    });
> +                                    tasks.push(task);
> +                                },
> +                                Err(err) => {
> +                                    eprintln!("Failed to accept unix connection - {}", err);
> +                                },
> +                            };
> +                        },
> +                        Err(err) => eprintln!("Failed to accept unix connection on {} - {}", data2.unix, err),
> +                    },
> +                };
> +            }
> +        });
> +
> +        Ok(())
> +    }
> +
> +    async fn handle_tunnel_cmd(&mut self, data: CmdData) -> Result<String, Error> {
> +        match &mut self.sender {
> +            None => bail!("not connected!"),
> +            Some(sender) => {
> +                let data: Value = data.into();
> +                let (tx, rx) = oneshot::channel::<String>();
> +                if let Some(cmd) = data.get("cmd") {
> +                    eprintln!("-> sending command {} to remote", cmd);
> +                } else {
> +                    eprintln!("-> sending data line to remote");
> +                }
> +                sender.send((data, tx))?;
> +                let res = rx.await?;
> +                eprintln!("<- got reply");
> +                Ok(res)
> +            }
> +        }
> +    }
> +
> +    async fn handle_ctrl_tunnel(
> +        websocket: Upgraded,
> +        mut cmd_receiver: mpsc::UnboundedReceiver<(Value, oneshot::Sender<String>)>,
> +    ) -> Result<(), Error> {
> +        let (tunnel_reader, tunnel_writer) = tokio::io::split(websocket);
> +        let (ws_close_tx, mut ws_close_rx) = mpsc::unbounded_channel();
> +        let ws_reader = WebSocketReader::new(tunnel_reader, ws_close_tx);
> +        let mut ws_writer = WebSocketWriter::new(Some([0, 0, 0, 0]), tunnel_writer);
> +
> +        let mut framed_reader =
> +            tokio_util::codec::FramedRead::new(ws_reader, tokio_util::codec::LinesCodec::new());
> +
> +        let mut resp_tx_queue: VecDeque<oneshot::Sender<String>> = VecDeque::new();
> +        let mut shutting_down = false;
> +
> +        loop {
> +            let mut close_future = ws_close_rx.recv().boxed().fuse();
> +            let mut frame_future = framed_reader.next().boxed().fuse();
> +            let mut cmd_future = cmd_receiver.recv().boxed().fuse();
> +
> +            select! {
> +                res = close_future => {
> +                    let res = res.ok_or_else(|| format_err!("WS control channel closed"))?;
> +                    eprintln!("WS: received control message: '{:?}'", res);
> +                    shutting_down = true;
> +                },
> +                res = frame_future => {
> +                    match res {
> +                        None if shutting_down => {
> +                            eprintln!("WS closed");
> +                            break;
> +                        },
> +                        None => bail!("WS closed unexpectedly"),
> +                        Some(Ok(res)) => {
> +                            resp_tx_queue
> +                                .pop_front()
> +                                .ok_or_else(|| format_err!("no response handler"))?
> +                                .send(res)
> +                                .map_err(|msg| format_err!("failed to send tunnel response '{}' back to requester - receiver already closed?", msg))?;
> +                        },
> +                        Some(Err(err)) => {
> +                            bail!("reading from control tunnel failed - WS receive failed: {}", err);
> +                        },
> +                    }
> +                },
> +                res = cmd_future => {
> +                    if shutting_down { continue };
> +                    match res {
> +                        None => {
> +                            eprintln!("CMD channel closed, shutting down");
> +                            ws_writer.send_control_frame(Some([1,2,3,4]), OpCode::Close, &[]).await?;
> +                            shutting_down = true;
> +                        },
> +                        Some((msg, resp_tx)) => {
> +                            resp_tx_queue.push_back(resp_tx);
> +
> +                            let line = format!("{}\n", msg);
> +                            ws_writer.write_all(line.as_bytes()).await?;
> +                            ws_writer.flush().await?;
> +                        },
> +                    }
> +                },
> +            };
> +        }
> +
> +        Ok(())
> +    }
> +
> +    async fn handle_forward_tunnel(
> +        data: Arc<ForwardCmdData>,
> +        unix: UnixStream,
> +    ) -> Result<(), Error> {
> +        let upgraded = Self::websocket_connect(
> +            data.url.clone(),
> +            data.headers.clone().unwrap_or_else(Vec::new),
> +            data.fingerprint.clone(),
> +        )
> +        .await?;
> +
> +        let ws = WebSocket {
> +            mask: Some([0, 0, 0, 0]),
> +        };
> +        eprintln!("established new WS for forwarding '{}'", data.unix);
> +        ws.serve_connection(upgraded, unix).await?;
> +
> +        eprintln!("done handling forwarded connection from '{}'", data.unix);
> +
> +        Ok(())
> +    }
> +
> +    async fn get_ticket(
> +        cmd_sender: &mpsc::UnboundedSender<(Value, oneshot::Sender<String>)>,
> +        cmd_data: Arc<ForwardCmdData>,
> +    ) -> Result<Arc<ForwardCmdData>, Error> {
> +        eprintln!("requesting WS ticket via tunnel");
> +        let ticket_cmd = match cmd_data.ticket.clone() {
> +            Some(mut ticket_cmd) => {
> +                ticket_cmd.insert("cmd".to_string(), serde_json::json!("ticket"));
> +                ticket_cmd
> +            }
> +            None => bail!("can't get ticket without ticket parameters"),
> +        };
> +        let (tx, rx) = oneshot::channel::<String>();
> +        cmd_sender.send((serde_json::json!(ticket_cmd), tx))?;
> +        let ticket = rx.await?;
> +        let mut ticket: Map<String, Value> = serde_json::from_str(&ticket)?;
> +        let ticket = ticket
> +            .remove("ticket")
> +            .ok_or_else(|| format_err!("failed to retrieve ticket via tunnel"))?;
> +
> +        let ticket = ticket
> +            .as_str()
> +            .ok_or_else(|| format_err!("failed to format received ticket"))?;
> +        let ticket = utf8_percent_encode(&ticket, NON_ALPHANUMERIC).to_string();
> +
> +        let mut data = cmd_data.clone();
> +        let mut url = data.url.clone();
> +        url.push_str("ticket=");
> +        url.push_str(&ticket);
> +        let mut d = Arc::make_mut(&mut data);
> +        d.url = url;
> +        Ok(data)
> +    }
> +}
> +
> +#[tokio::main]
> +async fn main() -> Result<(), Error> {
> +    do_main().await
> +}
> +
> +async fn do_main() -> Result<(), Error> {
> +    let tunnel = CtrlTunnel {
> +        sender: None,
> +        forwarded: Arc::new(Mutex::new(Vec::new())),
> +    };
> +    tunnel.read_cmd_loop().await
> +}
> 





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

* Re: [pve-devel] [PATCH qemu-server 08/10] migrate: refactor remote VM/tunnel start
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 08/10] migrate: refactor remote VM/tunnel start Fabian Grünbichler
@ 2021-11-09 14:04   ` Fabian Ebner
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Ebner @ 2021-11-09 14:04 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> no semantic changes intended, except for:
> - no longer passing the main migration UNIX socket to SSH twice for
> forwarding
> - dropping the 'unix:' prefix in start_remote_tunnel's timeout error message
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>   PVE/QemuMigrate.pm | 158 ++++++++++++++++++++++++++++-----------------
>   PVE/QemuServer.pm  |  32 ++++-----
>   2 files changed, 113 insertions(+), 77 deletions(-)
> 
> diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
> index 779f5ee..07b56eb 100644
> --- a/PVE/QemuMigrate.pm
> +++ b/PVE/QemuMigrate.pm
> @@ -206,19 +206,24 @@ sub finish_tunnel {
>       die $err if $err;
>   }
>   
> +# tunnel_info:
> +#   proto: unix (secure) or tcp (insecure/legacy compat)
> +#   addr: IP or UNIX socket path
> +#   port: optional TCP port
> +#   unix_sockets: additional UNIX socket paths to forward
>   sub start_remote_tunnel {
> -    my ($self, $raddr, $rport, $ruri, $unix_socket_info) = @_;
> +    my ($self, $tunnel_info) = @_;
>   
>       my $nodename = PVE::INotify::nodename();
>       my $migration_type = $self->{opts}->{migration_type};
>   
>       if ($migration_type eq 'secure') {
>   
> -	if ($ruri =~ /^unix:/) {
> -	    my $ssh_forward_info = ["$raddr:$raddr"];
> -	    $unix_socket_info->{$raddr} = 1;
> +	if ($tunnel_info->{proto} eq 'unix') {
> +	    my $ssh_forward_info = [];
>   
> -	    my $unix_sockets = [ keys %$unix_socket_info ];
> +	    my $unix_sockets = [ keys %{$tunnel_info->{unix_sockets}} ];
> +	    push @$unix_sockets, $tunnel_info->{addr};
>   	    for my $sock (@$unix_sockets) {
>   		push @$ssh_forward_info, "$sock:$sock";
>   		unlink $sock;
> @@ -245,23 +250,23 @@ sub start_remote_tunnel {
>   	    if ($unix_socket_try > 100) {
>   		$self->{errors} = 1;
>   		$self->finish_tunnel($self->{tunnel});
> -		die "Timeout, migration socket $ruri did not get ready";
> +		die "Timeout, migration socket $tunnel_info->{addr} did not get ready";
>   	    }
>   	    $self->{tunnel}->{unix_sockets} = $unix_sockets if (@$unix_sockets);
>   
> -	} elsif ($ruri =~ /^tcp:/) {
> +	} elsif ($tunnel_info->{proto} eq 'tcp') {
>   	    my $ssh_forward_info = [];
> -	    if ($raddr eq "localhost") {
> +	    if ($tunnel_info->{addr} eq "localhost") {
>   		# for backwards compatibility with older qemu-server versions
>   		my $pfamily = PVE::Tools::get_host_address_family($nodename);
>   		my $lport = PVE::Tools::next_migrate_port($pfamily);
> -		push @$ssh_forward_info, "$lport:localhost:$rport";
> +		push @$ssh_forward_info, "$lport:localhost:$tunnel_info->{rporyt}";

Typo: s/rporyt/port/

>   	    }
>   
>   	    $self->{tunnel} = $self->fork_tunnel($ssh_forward_info);
>   
>   	} else {
> -	    die "unsupported protocol in migration URI: $ruri\n";
> +	    die "unsupported protocol in migration URI: $tunnel_info->{proto}\n";
>   	}
>       } else {
>   	#fork tunnel for insecure migration, to send faster commands like resume
> @@ -813,52 +818,40 @@ sub phase1_cleanup {
>       }
>   }
>   
> -sub phase2 {
> -    my ($self, $vmid) = @_;
> +sub phase2_start_local_cluster {
> +    my ($self, $vmid, $params) = @_;
>   
>       my $conf = $self->{vmconf};
>       my $local_volumes = $self->{local_volumes};
>       my @online_local_volumes = $self->filter_local_volumes('online');
>   
>       $self->{storage_migration} = 1 if scalar(@online_local_volumes);
> +    my $start = $params->{start_params};
> +    my $migrate = $params->{migrate_opts};
>   
>       $self->log('info', "starting VM $vmid on remote node '$self->{node}'");
>   
> -    my $raddr;
> -    my $rport;
> -    my $ruri; # the whole migration dst. URI (protocol:address[:port])
> -    my $nodename = PVE::INotify::nodename();
> +    my $tunnel_info = {};
>   
>       ## start on remote node
>       my $cmd = [@{$self->{rem_ssh}}];
>   
> -    my $spice_ticket;
> -    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
> -	my $res = mon_cmd($vmid, 'query-spice');
> -	$spice_ticket = $res->{ticket};
> -    }
> +    push @$cmd, 'qm', 'start', $vmid, '--skiplock';
> +    push @$cmd, '--migratedfrom', $migrate->{migratedfrom};
>   
> -    push @$cmd , 'qm', 'start', $vmid, '--skiplock', '--migratedfrom', $nodename;
> +    push @$cmd, '--migration_type', $migrate->{type};
>   
> -    my $migration_type = $self->{opts}->{migration_type};
> +    push @$cmd, '--migration_network', $migrate->{network}
> +      if $migrate->{migration_network};

The key used for the if is wrong.

>   
> -    push @$cmd, '--migration_type', $migration_type;
> +    push @$cmd, '--stateuri', $start->{statefile};
>   
> -    push @$cmd, '--migration_network', $self->{opts}->{migration_network}
> -      if $self->{opts}->{migration_network};
> -
> -    if ($migration_type eq 'insecure') {
> -	push @$cmd, '--stateuri', 'tcp';
> -    } else {
> -	push @$cmd, '--stateuri', 'unix';
> +    if ($start->{forcemachine}) {
> +	push @$cmd, '--machine', $start->{forcemachine};
>       }
>   
> -    if ($self->{forcemachine}) {
> -	push @$cmd, '--machine', $self->{forcemachine};
> -    }
> -
> -    if ($self->{forcecpu}) {
> -	push @$cmd, '--force-cpu', $self->{forcecpu};
> +    if ($start->{forcecpu}) {
> +	push @$cmd, '--force-cpu', $start->{forcecpu};
>       }
>   
>       if ($self->{storage_migration}) {
> @@ -866,11 +859,8 @@ sub phase2 {
>       }
>   
>       my $spice_port;
> -    my $unix_socket_info = {};
> -    # version > 0 for unix socket support
> -    my $nbd_protocol_version = 1;
> -    my $input = "nbd_protocol_version: $nbd_protocol_version\n";
> -    $input .= "spice_ticket: $spice_ticket\n" if $spice_ticket;
> +    my $input = "nbd_protocol_version: $migrate->{nbd_proto_version}\n";
> +    $input .= "spice_ticket: $migrate->{spice_ticket}\n" if $migrate->{spice_ticket};
>   
>       my @online_replicated_volumes = $self->filter_local_volumes('online', 1);
>       foreach my $volid (@online_replicated_volumes) {
> @@ -900,20 +890,20 @@ sub phase2 {
>       my $exitcode = PVE::Tools::run_command($cmd, input => $input, outfunc => sub {
>   	my $line = shift;
>   
> -	if ($line =~ m/^migration listens on tcp:(localhost|[\d\.]+|\[[\d\.:a-fA-F]+\]):(\d+)$/) {
> -	    $raddr = $1;
> -	    $rport = int($2);
> -	    $ruri = "tcp:$raddr:$rport";
> +	if ($line =~ m/^migration listens on (tcp):(localhost|[\d\.]+|\[[\d\.:a-fA-F]+\]):(\d+)$/) {
> +	    $tunnel_info->{addr} = $2;
> +	    $tunnel_info->{port} = int($3);
> +	    $tunnel_info->{proto} = $1;
>   	}
> -	elsif ($line =~ m!^migration listens on unix:(/run/qemu-server/(\d+)\.migrate)$!) {
> -	    $raddr = $1;
> -	    die "Destination UNIX sockets VMID does not match source VMID" if $vmid ne $2;
> -	    $ruri = "unix:$raddr";
> +	elsif ($line =~ m!^migration listens on (unix):(/run/qemu-server/(\d+)\.migrate)$!) {
> +	    $tunnel_info->{addr} = $2;
> +	    die "Destination UNIX sockets VMID does not match source VMID" if $vmid ne $3;
> +	    $tunnel_info->{proto} = $1;
>   	}
>   	elsif ($line =~ m/^migration listens on port (\d+)$/) {
> -	    $raddr = "localhost";
> -	    $rport = int($1);
> -	    $ruri = "tcp:$raddr:$rport";
> +	    $tunnel_info->{addr} = "localhost";
> +	    $tunnel_info->{port} = int($1);
> +	    $tunnel_info->{proto} = "tcp";
>   	}
>   	elsif ($line =~ m/^spice listens on port (\d+)$/) {
>   	    $spice_port = int($1);
> @@ -934,7 +924,7 @@ sub phase2 {
>   	    $targetdrive =~ s/drive-//g;
>   
>   	    $handle_storage_migration_listens->($targetdrive, $drivestr, $nbd_uri);
> -	    $unix_socket_info->{$nbd_unix_addr} = 1;
> +	    $tunnel_info->{unix_sockets}->{$nbd_unix_addr} = 1;
>   	} elsif ($line =~ m/^re-using replicated volume: (\S+) - (.*)$/) {
>   	    my $drive = $1;
>   	    my $volid = $2;
> @@ -949,19 +939,65 @@ sub phase2 {
>   
>       die "remote command failed with exit code $exitcode\n" if $exitcode;
>   
> -    die "unable to detect remote migration address\n" if !$raddr;
> +    die "unable to detect remote migration address\n" if !$tunnel_info->{addr} || !$tunnel_info->{proto};
>   
>       if (scalar(keys %$target_replicated_volumes) != scalar(@online_replicated_volumes)) {
>   	die "number of replicated disks on source and target node do not match - target node too old?\n"
>       }
>   
> +    return ($tunnel_info, $spice_port);
> +}
> +
> +sub phase2 {
> +    my ($self, $vmid) = @_;
> +
> +    my $conf = $self->{vmconf};
> +
> +    # version > 0 for unix socket support
> +    my $nbd_protocol_version = 1;
> +
> +    my $spice_ticket;
> +    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
> +	my $res = mon_cmd($vmid, 'query-spice');
> +	$spice_ticket = $res->{ticket};
> +    }
> +
> +    my $migration_type = $self->{opts}->{migration_type};
> +    my $state_uri = $migration_type eq 'insecure' ? 'tcp' : 'unix';
> +
> +    my $params = {
> +	start_params => {
> +	    statefile => $state_uri,
> +	    forcemachine => $self->{forcemachine},
> +	    forcecpu => $self->{forcecpu},
> +	    skiplock => 1,
> +	},
> +	migrate_opts => {
> +	    spice_ticket => $spice_ticket,
> +	    type => $migration_type,
> +	    network => $self->{opts}->{migration_network},
> +	    storagemap => $self->{opts}->{storagemap},
> +	    migratedfrom => PVE::INotify::nodename(),
> +	    nbd_proto_version => $nbd_protocol_version,
> +	    nbd => $self->{nbd},
> +	},
> +    };
> +
> +    my ($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
> +
>       $self->log('info', "start remote tunnel");
> -    $self->start_remote_tunnel($raddr, $rport, $ruri, $unix_socket_info);
> +    $self->start_remote_tunnel($tunnel_info);
> +
> +    my $migrate_uri = "$tunnel_info->{proto}:$tunnel_info->{addr}";
> +    $migrate_uri .= ":$tunnel_info->{port}"
> +	if defined($tunnel_info->{port});
>   
>       if ($self->{storage_migration}) {
>   	$self->{storage_migration_jobs} = {};
>   	$self->log('info', "starting storage migration");
>   
> +	my @online_local_volumes = $self->filter_local_volumes('online');
> +
>   	die "The number of local disks does not match between the source and the destination.\n"
>   	    if (scalar(keys %{$self->{target_drive}}) != scalar(@online_local_volumes));
>   	foreach my $drive (keys %{$self->{target_drive}}){
> @@ -971,7 +1007,7 @@ sub phase2 {
>   	    my $source_drive = PVE::QemuServer::parse_drive($drive, $conf->{$drive});
>   	    my $source_volid = $source_drive->{file};
>   
> -	    my $bwlimit = $local_volumes->{$source_volid}->{bwlimit};
> +	    my $bwlimit = $self->{local_volumes}->{$source_volid}->{bwlimit};
>   	    my $bitmap = $target->{bitmap};
>   
>   	    $self->log('info', "$drive: start migration to $nbd_uri");
> @@ -979,7 +1015,7 @@ sub phase2 {
>   	}
>       }
>   
> -    $self->log('info', "starting online/live migration on $ruri");
> +    $self->log('info', "starting online/live migration on $migrate_uri");
>       $self->{livemigration} = 1;
>   
>       # load_defaults
> @@ -1056,12 +1092,12 @@ sub phase2 {
>   
>       my $start = time();
>   
> -    $self->log('info', "start migrate command to $ruri");
> +    $self->log('info', "start migrate command to $migrate_uri");
>       eval {
> -	mon_cmd($vmid, "migrate", uri => $ruri);
> +	mon_cmd($vmid, "migrate", uri => $migrate_uri);
>       };
>       my $merr = $@;
> -    $self->log('info', "migrate uri => $ruri failed: $merr") if $merr;
> +    $self->log('info', "migrate uri => $migrate_uri failed: $merr") if $merr;
>   
>       my $last_mem_transferred = 0;
>       my $usleep = 1000000;
> diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
> index de8c1bb..d494cc0 100644
> --- a/PVE/QemuServer.pm
> +++ b/PVE/QemuServer.pm
> @@ -5431,10 +5431,10 @@ sub vm_start_nolock {
>   	return $migration_ip;
>       };
>   
> -    my $migrate_uri;
>       if ($statefile) {
>   	if ($statefile eq 'tcp') {
> -	    my $localip = "localhost";
> +	    my $migrate = $res->{migrate} = { proto => 'tcp' };
> +	    $migrate->{addr} = "localhost";
>   	    my $datacenterconf = PVE::Cluster::cfs_read_file('datacenter.cfg');
>   	    my $nodename = nodename();
>   
> @@ -5447,26 +5447,26 @@ sub vm_start_nolock {
>   	    }
>   
>   	    if ($migration_type eq 'insecure') {
> -		$localip = $get_migration_ip->($nodename);
> -		$localip = "[$localip]" if Net::IP::ip_is_ipv6($localip);
> +		$migrate->{addr} = $get_migration_ip->($nodename);
> +		$migrate->{addr} = "[$migrate->{addr}]" if Net::IP::ip_is_ipv6($migrate->{addr});
>   	    }
>   
>   	    my $pfamily = PVE::Tools::get_host_address_family($nodename);
> -	    my $migrate_port = PVE::Tools::next_migrate_port($pfamily);
> -	    $migrate_uri = "tcp:${localip}:${migrate_port}";
> -	    push @$cmd, '-incoming', $migrate_uri;
> +	    $migrate->{port} = PVE::Tools::next_migrate_port($pfamily);
> +	    $migrate->{uri} = "tcp:$migrate->{addr}:$migrate->{port}";
> +	    push @$cmd, '-incoming', $migrate->{uri};
>   	    push @$cmd, '-S';
>   
>   	} elsif ($statefile eq 'unix') {
>   	    # should be default for secure migrations as a ssh TCP forward
>   	    # tunnel is not deterministic reliable ready and fails regurarly
>   	    # to set up in time, so use UNIX socket forwards
> -	    my $socket_addr = "/run/qemu-server/$vmid.migrate";
> -	    unlink $socket_addr;
> +	    my $migrate = $res->{migrate} = { proto => 'unix' };
> +	    $migrate->{addr} = "/run/qemu-server/$vmid.migrate";
> +	    unlink $migrate->{addr};
>   
> -	    $migrate_uri = "unix:$socket_addr";
> -
> -	    push @$cmd, '-incoming', $migrate_uri;
> +	    $migrate->{uri} = "unix:$migrate->{addr}";
> +	    push @$cmd, '-incoming', $migrate->{uri};
>   	    push @$cmd, '-S';
>   
>   	} elsif (-e $statefile) {
> @@ -5608,10 +5608,9 @@ sub vm_start_nolock {
>       eval { PVE::QemuServer::PCI::reserve_pci_usage($pci_id_list, $vmid, undef, $pid) };
>       warn $@ if $@;
>   
> -    print "migration listens on $migrate_uri\n" if $migrate_uri;
> -    $res->{migrate_uri} = $migrate_uri;

Nit: There's a $res->{migrate_storage_uri} = $migrate_storage_uri; 
further below. If I'm not missing any usages of that, it could also be 
removed.

> -
> -    if ($statefile && $statefile ne 'tcp' && $statefile ne 'unix')  {
> +    if (defined($res->{migrate})) {
> +	print "migration listens on $res->{migrate}->{uri}\n";
> +    } elsif ($statefile) {
>   	eval { mon_cmd($vmid, "cont"); };
>   	warn $@ if $@;
>       }
> @@ -5626,6 +5625,7 @@ sub vm_start_nolock {
>   	    my $socket_path = "/run/qemu-server/$vmid\_nbd.migrate";
>   	    mon_cmd($vmid, "nbd-server-start", addr => { type => 'unix', data => { path => $socket_path } } );
>   	    $migrate_storage_uri = "nbd:unix:$socket_path";
> +	    $res->{migrate}->{unix_sockets} = [$socket_path];
>   	} else {
>   	    my $nodename = nodename();
>   	    my $localip = $get_migration_ip->($nodename);
> 




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

* Re: [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (21 preceding siblings ...)
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint Fabian Grünbichler
@ 2021-11-09 16:07 ` DERUMIER, Alexandre
  2021-11-10 12:42 ` Fabian Ebner
  23 siblings, 0 replies; 50+ messages in thread
From: DERUMIER, Alexandre @ 2021-11-09 16:07 UTC (permalink / raw)
  To: pve-devel

Hi Fabian,

I Didn't have time to test previous patches series (and they didn't
applied on master git some weeks later, I was a bit lazy to find
correct commit).


I'll really to to test it this time as soon a possible.

Thanks again for the hard work !



Le vendredi 05 novembre 2021 à 14:03 +0100, Fabian Grünbichler a
écrit :
> this series adds remote migration for VMs. there's still some of
> TODOs/FIXMEs/stuff that requires discussion left, but this should
> hopefully be possible to apply with minor modifications and lots of
> future extensions.
> 
> both live and offline migration including NBD and storage-migrated
> disks
> should work.
> 
> besides lots of rebases, implemented todos and fixed issues the main
> difference to the previous RFC is that we no longer define remote
> entries in a config file, but just expect the caller/client to give
> us
> all the required information to connect to the remote cluster.
> 
> overview over affected repos and changes, see individual patches for
> more details.
> 
> proxmox:
> 
> some compatible changes to make websocket code usable for client-side
> connections, required by proxmox-websocket-tunnel
> 
> proxmox-websocket-tunnel:
> 
> new tunnel helper tool for forwarding commands and data over
> websocket
> connections, required by qemu-server on source side
> 
> pve-access-control:
> 
> new ticket type, required by qemu-server on target side
> 
> pve-common:
> 
> bridgepair format akin to storage pair, pve-bridge-id option,
> required
> by qemu-server
> remote format for specifying a remote endpoint (including API token,
> fingerprint, ..)
> 
> pve-guest-common:
> 
> handle remote migration (no SSH) in AbstractMigrate,
> required by qemu-server
> 
> pve-storage:
> 
> extend 'pvesm import' to allow import from UNIX socket, required on
> target node by qemu-server
> 
> qemu-server:
> 
> some refactoring, new mtunnel endpoints, new remote_migration
> endpoints
> TODO: handle pending changes and snapshots
> TODO: proper CLI for remote migration
> potential TODO: precond endpoint?
> 
> pve-http-server:
> 
> fix for handling unflushed proxy streams
> 
> as usual, some of the patches are best viewed with '-w', especially
> in
> qemu-server..
> 
> required dependencies are noted, qemu-server also requires a build-
> dep
> on patched pve-common since the required options/formats would be
> missing otherwise..
> 
> 
> _______________________________________________
> pve-devel mailing list
> pve-devel@lists.proxmox.com
> https://antiphishing.cetsi.fr/proxy/v3?i=d09ZU0Z5WTAyTG85WWdYbIX9F1yN
> D7gsvpr6o9NYFYg&r=UTEzTUpQcktwRVdhdEg1TKCFOzhw8CGaAiMfyFTpTR_LTspF9zP
> 2JS-LN0ctA-XBzHeMG-
> sD1OqL3ihNxDMXJg&f=TmtFVlNVNmxSYnFaWFhxYgbJpVVB9ZnN19DehJ0Wa7UDEK3Q5o
> BOIPwujBOSdTuzYd4WFcuo4NETjbVsTIwTiQ&u=https%3A//lists.proxmox.com/cg
> i-bin/mailman/listinfo/pve-devel&k=DWI7
> 


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

* Re: [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints
  2021-11-09 12:46   ` Fabian Ebner
@ 2021-11-10  7:40     ` Fabian Ebner
  2021-11-11 11:07       ` Fabian Grünbichler
  2021-11-11 11:04     ` Fabian Grünbichler
  1 sibling, 1 reply; 50+ messages in thread
From: Fabian Ebner @ 2021-11-10  7:40 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 09.11.21 um 13:46 schrieb Fabian Ebner:
> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:

---snip---

>>   use IO::Socket::IP;
>> +use IO::Socket::UNIX;
>> +use IPC::Open3;
>> +use JSON;
>> +use MIME::Base64;

Forgot to ask: is this import needed or a left-over from development?

---snip---

> 
>> +
>> +            my $migration_snapshot;
>> +            if ($scfg->{type} eq 'zfspool' || $scfg->{type} eq 
>> 'btrfs') {
>> +            $migration_snapshot = '__migration__';
>> +            }
>> +
>> +            my $volid = "$storeid:$volname";
>> +
>> +            # find common import/export format, taken from PVE::Storage
>> +            my @import_formats = 
>> PVE::Storage::volume_import_formats($state->{storecfg}, $volid, 
>> $migration_snapshot, undef, $with_snapshots);
>> +            my @export_formats = 
>> PVE::Tools::split_list($params->{'export-formats'});
>> +            my %import_hash = map { $_ => 1 } @import_formats;
>> +            my @common = grep { $import_hash{$_} } @export_formats;
>> +            die "no matching import/export format found for storage 
>> '$storeid'\n"
>> +            if !@common;
>> +            $format = $common[0];
>> +
>> +            my $input = IO::File->new();
>> +            my $info = IO::File->new();
>> +            my $unix = "/run/qemu-server/$vmid.storage";
>> +
>> +            my $import_cmd = ['pvesm', 'import', $volid, $format, 
>> "unix://$unix", '-with-snapshots', $with_snapshots];
>> +            if ($params->{'allow-rename'}) {
>> +            push @$import_cmd, '-allow-rename', 
>> $params->{'allow-rename'};
>> +            }
>> +            if ($migration_snapshot) {
>> +            push @$import_cmd, '-delete-snapshot', $migration_snapshot;
> 
> Missing '-snapshot $migration_snapshot'? While the parameter is ignored 
> by our ZFSPoolPlugin, the BTRFSPlugin aborts if it's not specified 
> AFAICS. And external plugins might require it too.

That is, for the 'btrfs' format. In the patch with the export command, a 
snapshot is only used for ZFS, so it would already fail on export for 
BTRFS with 'btrfs' format. For external plugins we also don't use a 
migration snapshot in storage_migrate(), so please disregard that part.

> 
> In general, we'll need to be careful not to introduce mismatches between 
> the import and the export parameters. Might it be better if the client 
> would pass along (most of) the parameters for the import command (which 
> basically is how it's done for the existing storage_migrate)?
> 

On the other hand, that would require being very careful with input 
validation.

---snip---




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

* Re: [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling Fabian Grünbichler
@ 2021-11-10 11:17   ` Fabian Ebner
  2021-11-11 12:25     ` Fabian Grünbichler
  0 siblings, 1 reply; 50+ messages in thread
From: Fabian Ebner @ 2021-11-10 11:17 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> remote migration uses a websocket connection to a task worker running on
> the target node instead of commands via SSH to control the migration.
> this websocket tunnel is started earlier than the SSH tunnel, and allows
> adding UNIX-socket forwarding over additional websocket connections
> on-demand.
> 
> the main differences to regular intra-cluster migration are:
> - source VM config and disks are only removed upon request via --delete
> - shared storages are treated like local storages, since we can't
> assume they are shared across clusters (with potentical to extend this
> by marking storages as shared)
> - NBD migrated disks are explicitly pre-allocated on the target node via
> tunnel command before starting the target VM instance
> - in addition to storages, network bridges and the VMID itself is
> transformed via a user defined mapping
> - all commands and migration data streams are sent via a WS tunnel proxy
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>      requires proxmox-websocket-tunnel
> 
>   PVE/API2/Qemu.pm   |   4 +-
>   PVE/QemuMigrate.pm | 647 +++++++++++++++++++++++++++++++++++++++------
>   PVE/QemuServer.pm  |   8 +-
>   3 files changed, 575 insertions(+), 84 deletions(-)
> 
> diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
> index a1a1813..24f5b98 100644
> --- a/PVE/API2/Qemu.pm
> +++ b/PVE/API2/Qemu.pm
> @@ -4610,7 +4610,7 @@ __PACKAGE__->register_method({
>   		    # bump/reset both for breaking changes
>   		    # bump tunnel only for opt-in changes

Sorry for asking about this on this patch: shouldn't opt-in changes bump 
both?

>   		    return {
> -			api => 2,
> +			api => $PVE::QemuMigrate::WS_TUNNEL_VERSION,
>   			age => 0,
>   		    };
>   		},
> @@ -4897,7 +4897,7 @@ __PACKAGE__->register_method({
>   			    PVE::Firewall::remove_vmfw_conf($vmid);
>   			}
>   
> -			if (my @volumes = keys $state->{cleanup}->{volumes}->$%) {
> +			if (my @volumes = keys $state->{cleanup}->{volumes}->%*) {
>   			    PVE::Storage::foreach_volid(@volumes, sub {
>   				my ($volid, $sid, $volname, $d) = @_;
>   
> diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
> index 07b56eb..7378551 100644
> --- a/PVE/QemuMigrate.pm
> +++ b/PVE/QemuMigrate.pm
> @@ -7,9 +7,15 @@ use IO::File;
>   use IPC::Open2;
>   use POSIX qw( WNOHANG );
>   use Time::HiRes qw( usleep );
> +use JSON qw(encode_json decode_json);
> +use IO::Socket::UNIX;
> +use Socket qw(SOCK_STREAM);
> +use Storable qw(dclone);
> +use URI::Escape;
>   
> -use PVE::Format qw(render_bytes);
> +use PVE::APIClient::LWP;
>   use PVE::Cluster;
> +use PVE::Format qw(render_bytes);
>   use PVE::GuestHelpers qw(safe_boolean_ne safe_string_ne);
>   use PVE::INotify;
>   use PVE::RPCEnvironment;
> @@ -30,6 +36,9 @@ use PVE::QemuServer;
>   use PVE::AbstractMigrate;
>   use base qw(PVE::AbstractMigrate);
>   
> +# compared against remote end's minimum version
> +our $WS_TUNNEL_VERSION = 2;
> +
>   sub fork_command_pipe {
>       my ($self, $cmd) = @_;
>   
> @@ -85,7 +94,7 @@ sub finish_command_pipe {
>   	}
>       }
>   
> -    $self->log('info', "ssh tunnel still running - terminating now with SIGTERM\n");
> +    $self->log('info', "tunnel still running - terminating now with SIGTERM\n");
>       kill(15, $cpid);
>   
>       # wait again
> @@ -94,11 +103,11 @@ sub finish_command_pipe {
>   	sleep(1);
>       }
>   
> -    $self->log('info', "ssh tunnel still running - terminating now with SIGKILL\n");
> +    $self->log('info', "tunnel still running - terminating now with SIGKILL\n");
>       kill 9, $cpid;
>       sleep 1;
>   
> -    $self->log('err', "ssh tunnel child process (PID $cpid) couldn't be collected\n")
> +    $self->log('err', "tunnel child process (PID $cpid) couldn't be collected\n")
>   	if !&$collect_child_process();
>   }
>   
> @@ -115,18 +124,28 @@ sub read_tunnel {
>       };
>       die "reading from tunnel failed: $@\n" if $@;
>   
> -    chomp $output;
> +    chomp $output if defined($output);
>   
>       return $output;
>   }
>   
>   sub write_tunnel {
> -    my ($self, $tunnel, $timeout, $command) = @_;
> +    my ($self, $tunnel, $timeout, $command, $params) = @_;
>   
>       $timeout = 60 if !defined($timeout);
>   
>       my $writer = $tunnel->{writer};
>   
> +    if ($tunnel->{version} && $tunnel->{version} >= 2) {
> +	my $object = defined($params) ? dclone($params) : {};
> +	$object->{cmd} = $command;
> +
> +	$command = eval { JSON::encode_json($object) };
> + > +	die "failed to encode command as JSON - $@\n"
> +	    if $@;
> +    }
> +
>       eval {
>   	PVE::Tools::run_with_timeout($timeout, sub {
>   	    print $writer "$command\n";
> @@ -136,13 +155,29 @@ sub write_tunnel {
>       die "writing to tunnel failed: $@\n" if $@;
>   
>       if ($tunnel->{version} && $tunnel->{version} >= 1) {
> -	my $res = eval { $self->read_tunnel($tunnel, 10); };
> +	my $res = eval { $self->read_tunnel($tunnel, $timeout); };
>   	die "no reply to command '$command': $@\n" if $@;
>   
> -	if ($res eq 'OK') {
> -	    return;
> +	if ($tunnel->{version} == 1) {
> +	    if ($res eq 'OK') {
> +		return;
> +	    } else {
> +		die "tunnel replied '$res' to command '$command'\n";
> +	    }
>   	} else {
> -	    die "tunnel replied '$res' to command '$command'\n";
> +	    my $parsed = eval { JSON::decode_json($res) };
> +	    die "failed to decode tunnel reply '$res' (command '$command') - $@\n"
> +		if $@;
> +
> +	    if (!$parsed->{success}) {
> +		if (defined($parsed->{msg})) {
> +		    die "error - tunnel command '$command' failed - $parsed->{msg}\n";
> +		} else {
> +		    die "error - tunnel command '$command' failed\n";
> +		}
> +	    }
> +
> +	    return $parsed;
>   	}
>       }
>   }
> @@ -185,10 +220,150 @@ sub fork_tunnel {
>       return $tunnel;
>   }
>   
> +my $forward_unix_socket = sub {
> +    my ($self, $local, $remote) = @_;
> +
> +    my $params = dclone($self->{tunnel}->{params});
> +    $params->{unix} = $local;
> +    $params->{url} = $params->{url} ."socket=$remote&";
> +    $params->{ticket} = { path => $remote };
> +
> +    my $cmd = encode_json({
> +	control => JSON::true,
> +	cmd => 'forward',
> +	data => $params,
> +    });
> +
> +    my $writer = $self->{tunnel}->{writer};
> +    eval {
> +	unlink $local;
> +	PVE::Tools::run_with_timeout(15, sub {
> +	    print $writer "$cmd\n";
> +	    $writer->flush();
> +	});
> +    };
> +    die "failed to write forwarding command - $@\n" if $@;
> +
> +    $self->read_tunnel($self->{tunnel});
> +
> +    $self->log('info', "Forwarded local unix socket '$local' to remote '$remote' via websocket tunnel");
> +};
> +
> +sub fork_websocket_tunnel {
> +    my ($self, $storages) = @_;
> +
> +    my $remote = $self->{opts}->{remote};
> +    my $conn = $remote->{conn};
> +
> +    my $websocket_url = "https://$conn->{host}:$conn->{port}/api2/json/nodes/$self->{node}/qemu/$remote->{vmid}/mtunnelwebsocket";
> +
> +    my $params = {
> +	url => $websocket_url,
> +    };
> +
> +    if (my $apitoken = $conn->{apitoken}) {
> +	$params->{headers} = [["Authorization", "$apitoken"]];
> +    } else {
> +	die "can't connect to remote host without credentials\n";
> +    }
> +
> +    if (my $fps = $conn->{cached_fingerprints}) {
> +	$params->{fingerprint} = (keys %$fps)[0];
> +    }
> +
> +    my $api_client = PVE::APIClient::LWP->new(%$conn);
> +    my $storage_list = join(',', keys %$storages);
> +    my $res = $api_client->post("/nodes/$self->{node}/qemu/$remote->{vmid}/mtunnel", { storages => $storage_list });
> +    $self->log('info', "remote: started migration tunnel worker '$res->{upid}'");
> +    $params->{url} .= "?ticket=".uri_escape($res->{ticket});
> +    $params->{url} .= "&socket=$res->{socket}";

Nit: could also be escaped.

> +
> +    my $reader = IO::Pipe->new();
> +    my $writer = IO::Pipe->new();
> +
> +    my $cpid = fork();
> +    if ($cpid) {
> +	$writer->writer();
> +	$reader->reader();
> +	my $tunnel = { writer => $writer, reader => $reader, pid => $cpid };
> +
> +	eval {
> +	    my $writer = $tunnel->{writer};
> +	    my $cmd = encode_json({
> +		control => JSON::true,
> +		cmd => 'connect',
> +		data => $params,
> +	    });
> +
> +	    eval {
> +		PVE::Tools::run_with_timeout(15, sub {
> +		    print {$writer} "$cmd\n";
> +		    $writer->flush();
> +		});
> +	    };
> +	    die "failed to write tunnel connect command - $@\n" if $@;
> +	};
> +	die "failed to connect via WS: $@\n" if $@;
> +
> +	my $err;
> +        eval {
> +	    my $writer = $tunnel->{writer};
> +	    my $cmd = encode_json({
> +		cmd => 'version',
> +	    });
> +
> +	    eval {
> +		PVE::Tools::run_with_timeout(15, sub {
> +		    print {$writer} "$cmd\n";
> +		    $writer->flush();
> +		});
> +	    };
> +	    $err = "failed to write tunnel version command - $@\n" if $@;
> +	    my $res = $self->read_tunnel($tunnel, 10);
> +	    $res = JSON::decode_json($res);
> +	    my $version = $res->{api};
> +
> +	    if ($version =~ /^(\d+)$/) {
> +		$tunnel->{version} = $1;
> +		$tunnel->{age} = $res->{age};
> +		$self->log('info', "tunnel info: $version\n");
> +	    } else {
> +		$err = "received invalid tunnel version string '$version'\n" if !$err;
> +	    }
> +	};
> +	$err = $@ if !$err;
> +
> +	if ($err) {
> +	    $self->finish_command_pipe($tunnel);
> +	    die "can't open migration tunnel - $err";
> +	}
> +
> +	$params->{url} = "$websocket_url?";
> +	$tunnel->{params} = $params; # for forwarding
> +
> +	return $tunnel;
> +    } else {
> +	eval {
> +	    $writer->reader();
> +	    $reader->writer();
> +	    PVE::Tools::run_command(
> +		['proxmox-websocket-tunnel'],
> +		input => "<&".fileno($writer),
> +		output => ">&".fileno($reader),
> +		errfunc => sub { my $line = shift; print "tunnel: $line\n"; },
> +	    );
> +	};
> +	warn "CMD websocket tunnel died: $@\n" if $@;
> +	exit 0;
> +    }
> +}
> +
>   sub finish_tunnel {
> -    my ($self, $tunnel) = @_;
> +    my ($self, $tunnel, $cleanup) = @_;
>   
> -    eval { $self->write_tunnel($tunnel, 30, 'quit'); };
> +    $cleanup = $cleanup ? 1 : 0;
> +
> +    eval { $self->write_tunnel($tunnel, 30, 'quit', { cleanup => $cleanup }); };
>       my $err = $@;
>   
>       $self->finish_command_pipe($tunnel, 30);

Nit: below here is
     if (my $unix_sockets = $tunnel->{unix_sockets}) {
         my $cmd = ['rm', '-f', @$unix_sockets];
         PVE::Tools::run_command($cmd);

         # .. and just to be sure check on remote side
         unshift @{$cmd}, @{$self->{rem_ssh}};
         PVE::Tools::run_command($cmd);
     }
and if I'm not mistaken, $self->{rem_ssh} is undef for remote migration, 
resulting in an undef warning and $cmd being executed twice locally.


> @@ -338,23 +513,34 @@ sub prepare {
>       }
>   
>       my $vollist = PVE::QemuServer::get_vm_volumes($conf);
> +
> +    my $storages = {};
>       foreach my $volid (@$vollist) {
>   	my ($sid, $volname) = PVE::Storage::parse_volume_id($volid, 1);
>   
> -	# check if storage is available on both nodes
> +	# check if storage is available on source node
>   	my $scfg = PVE::Storage::storage_check_enabled($storecfg, $sid);
>   
>   	my $targetsid = $sid;
> -	# NOTE: we currently ignore shared source storages in mappings so skip here too for now
> -	if (!$scfg->{shared}) {
> +	# NOTE: local ignores shared mappings, remote maps them
> +	if (!$scfg->{shared} || $self->{opts}->{remote}) {
>   	    $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>   	}
>   
> -	my $target_scfg = PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
> -	my ($vtype) = PVE::Storage::parse_volname($storecfg, $volid);
> +	$storages->{$targetsid} = 1;
> +
> +	if (!$self->{opts}->{remote}) {
> +	    # check if storage is available on target node
> +	    my $target_scfg = PVE::Storage::storage_check_enabled(
> +		$storecfg,
> +		$targetsid,
> +		$self->{node},
> +	    );
> +	    my ($vtype) = PVE::Storage::parse_volname($storecfg, $volid);
>   
> -	die "$volid: content type '$vtype' is not available on storage '$targetsid'\n"
> -	    if !$target_scfg->{content}->{$vtype};
> +	    die "$volid: content type '$vtype' is not available on storage '$targetsid'\n"
> +		if !$target_scfg->{content}->{$vtype};
> +	}
>   
>   	if ($scfg->{shared}) {
>   	    # PVE::Storage::activate_storage checks this for non-shared storages
> @@ -364,10 +550,23 @@ sub prepare {
>   	}
>       }
>   
> -    # test ssh connection
> -    my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
> -    eval { $self->cmd_quiet($cmd); };
> -    die "Can't connect to destination address using public key\n" if $@;
> +    if ($self->{opts}->{remote}) {
> +	# test & establish websocket connection
> +	my $tunnel = $self->fork_websocket_tunnel($storages);
> +	my $min_version = $tunnel->{version} - $tunnel->{age};
> +	die "Remote tunnel endpoint not compatible, upgrade required (current: $WS_TUNNEL_VERSION, required: $min_version)\n"
> +	    if $WS_TUNNEL_VERSION < $min_version;
> +	 die "Remote tunnel endpoint too old, upgrade required (local: $WS_TUNNEL_VERSION, remote: $tunnel->{version})"

Nit: missing '\n' in error, and while we're at it: style nit for >100 
character lines (are not the only instances in the series).

> +	    if $WS_TUNNEL_VERSION > $tunnel->{version};
> +
> +	print "websocket tunnel started\n";
> +	$self->{tunnel} = $tunnel;
> +    } else {
> +	# test ssh connection
> +	my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
> +	eval { $self->cmd_quiet($cmd); };
> +	die "Can't connect to destination address using public key\n" if $@;
> +    }
>   
>       return $running;
>   }
> @@ -405,7 +604,7 @@ sub scan_local_volumes {
>   	my @sids = PVE::Storage::storage_ids($storecfg);
>   	foreach my $storeid (@sids) {
>   	    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
> -	    next if $scfg->{shared};
> +	    next if $scfg->{shared} && !$self->{opts}->{remote};
>   	    next if !PVE::Storage::storage_check_enabled($storecfg, $storeid, undef, 1);
>   
>   	    # get list from PVE::Storage (for unused volumes)
> @@ -414,19 +613,24 @@ sub scan_local_volumes {
>   	    next if @{$dl->{$storeid}} == 0;
>   
>   	    my $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $storeid);
> -	    # check if storage is available on target node
> -	    my $target_scfg = PVE::Storage::storage_check_enabled(
> -		$storecfg,
> -		$targetsid,
> -		$self->{node},
> -	    );
> -
> -	    die "content type 'images' is not available on storage '$targetsid'\n"
> -		if !$target_scfg->{content}->{images};
> +	    my $bwlimit_sids = [$storeid];
> +	    if (!$self->{opts}->{remote}) {
> +		# check if storage is available on target node
> +		my $target_scfg = PVE::Storage::storage_check_enabled(
> +		    $storecfg,
> +		    $targetsid,
> +		    $self->{node},
> +		);
> +
> +		die "content type 'images' is not available on storage '$targetsid'\n"
> +		    if !$target_scfg->{content}->{images};
> +
> +		push @$bwlimit_sids, $targetsid;
> +	    }
>   
>   	    my $bwlimit = PVE::Storage::get_bandwidth_limit(
>   		'migration',
> -		[$targetsid, $storeid],
> +		$bwlimit_sids,
>   		$self->{opts}->{bwlimit},
>   	    );
>   
> @@ -482,14 +686,17 @@ sub scan_local_volumes {
>   	    my $scfg = PVE::Storage::storage_check_enabled($storecfg, $sid);
>   
>   	    my $targetsid = $sid;
> -	    # NOTE: we currently ignore shared source storages in mappings so skip here too for now
> -	    if (!$scfg->{shared}) {
> +	    # NOTE: local ignores shared mappings, remote maps them
> +	    if (!$scfg->{shared} || $self->{opts}->{remote}) {
>   		$targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>   	    }
>   
> -	    PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
> +	    # check target storage on target node if intra-cluster migration
> +	    if (!$self->{opts}->{remote}) {
> +		PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
>   
> -	    return if $scfg->{shared};
> +		return if $scfg->{shared};
> +	    }
>   
>   	    $local_volumes->{$volid}->{ref} = $attr->{referenced_in_config} ? 'config' : 'snapshot';
>   	    $local_volumes->{$volid}->{ref} = 'storage' if $attr->{is_unused};
> @@ -578,6 +785,9 @@ sub scan_local_volumes {
>   
>   	    my $migratable = $scfg->{type} =~ /^(?:dir|btrfs|zfspool|lvmthin|lvm)$/;
>   
> +	    # TODO: what is this even here for?
> +	    $migratable = 1 if $self->{opts}->{remote};
> +
>   	    die "can't migrate '$volid' - storage type '$scfg->{type}' not supported\n"
>   		if !$migratable;
>   
> @@ -612,6 +822,10 @@ sub handle_replication {
>       my $local_volumes = $self->{local_volumes};
>   
>       return if !$self->{replication_jobcfg};
> +
> +    die "can't migrate VM with replicated volumes to remote cluster/node\n"
> +	if $self->{opts}->{remote};

We can add that later, asserting that no local removal will happen ;)
Same for being a base VM referenced by a linked clone.

> +
>       if ($self->{running}) {
>   
>   	my $version = PVE::QemuServer::kvm_user_version();
> @@ -709,26 +923,133 @@ sub sync_offline_local_volumes {
>       my $opts = $self->{opts};
>   
>       $self->log('info', "copying local disk images") if scalar(@volids);
> -
> +    my $forwarded = 0;
>       foreach my $volid (@volids) {
>   	my $targetsid = $local_volumes->{$volid}->{targetsid};
> -	my $bwlimit = $local_volumes->{$volid}->{bwlimit};
> -	$bwlimit = $bwlimit * 1024 if defined($bwlimit); # storage_migrate uses bps
> -
> -	my $storage_migrate_opts = {
> -	    'ratelimit_bps' => $bwlimit,
> -	    'insecure' => $opts->{migration_type} eq 'insecure',
> -	    'with_snapshots' => $local_volumes->{$volid}->{snapshots},
> -	    'allow_rename' => !$local_volumes->{$volid}->{is_vmstate},
> -	};
>   
> -	my $logfunc = sub { $self->log('info', $_[0]); };
> -	my $new_volid = eval {
> -	    PVE::Storage::storage_migrate($storecfg, $volid, $self->{ssh_info},
> -					  $targetsid, $storage_migrate_opts, $logfunc);
> -	};
> -	if (my $err = $@) {
> -	    die "storage migration for '$volid' to storage '$targetsid' failed - $err\n";
> +	my $new_volid;
> +	

Style nit: whitespace error

> +	my $opts = $self->{opts};
> +	if (my $remote = $opts->{remote}) {
> +	    my $remote_vmid = $remote->{vmid};
> +	    my ($sid, undef) = PVE::Storage::parse_volume_id($volid);
> +	    my (undef, $name, undef, undef, undef, undef, $format) = PVE::Storage::parse_volname($storecfg, $volid);
> +	    my $scfg = PVE::Storage::storage_config($storecfg, $sid);
> +	    PVE::Storage::activate_volumes($storecfg, [$volid]);
> +
> +	    # use 'migrate' limit for transfer to other node
> +	    my $bwlimit_opts = {
> +		storage => $targetsid,
> +		bwlimit => $opts->{bwlimit},
> +	    };
> +	    my $bwlimit = PVE::Storage::get_bandwidth_limit('migration', [$sid], $opts->{bwlimit});

Nit: could use
     my $bwlimit = $local_volumes->{$volid}->{bwlimit};

> +	    my $remote_bwlimit = $self->write_tunnel($self->{tunnel}, 10, 'bwlimit', $bwlimit_opts);
> +	    $remote_bwlimit = $remote_bwlimit->{bwlimit};
> +	    if (defined($remote_bwlimit)) {
> +		$bwlimit = $remote_bwlimit if !defined($bwlimit);
> +		$bwlimit = $remote_bwlimit if $remote_bwlimit < $bwlimit;
> +	    }
> +
> +	    # JSONSchema and get_bandwidth_limit use kbps - storage_migrate bps
> +	    $bwlimit = $bwlimit * 1024 if defined($bwlimit);
> +
> +	    my $with_snapshots = $local_volumes->{$volid}->{snapshots} ? 1 : 0;
> +	    my $snapshot;
> +	    if ($scfg->{type} eq 'zfspool') {
> +		$snapshot = '__migration__';
> +		$with_snapshots = 1;
> +		PVE::Storage::volume_snapshot($storecfg, $volid, $snapshot);
> +	    }
> +
> +	    if ($self->{vmid} != $remote_vmid) {
> +		$name =~ s/-$self->{vmid}-/-$remote_vmid-/g;
> +		$name =~ s/^$self->{vmid}\//$remote_vmid\//;
> +	    }
> +
> +	    my @export_formats = PVE::Storage::volume_export_formats($storecfg, $volid, undef, undef, $with_snapshots);
> +
> +	    my $storage_migrate_opts = {

Nit: maybe call it disk_import_opts

> +		format => $format,
> +		storage => $targetsid,
> +		'with-snapshots' => $with_snapshots,
> +		'allow-rename' => !$local_volumes->{$volid}->{is_vmstate},
> +		'export-formats' => @export_formats,

Doesn't this need to be converted to a string?

> +		volname => $name,
> +	    };
> +	    my $res = $self->write_tunnel($self->{tunnel}, 600, 'disk-import', $storage_migrate_opts);
> +	    my $local = "/run/qemu-server/$self->{vmid}.storage";
> +	    if (!$forwarded) {
> +		$forward_unix_socket->($self, $local, $res->{socket});
> +		$forwarded = 1;
> +	    }
> +	    my $socket = IO::Socket::UNIX->new(Peer => $local, Type => SOCK_STREAM())
> +		or die "failed to connect to websocket tunnel at $local\n";
> +	    # we won't be reading from the socket
> +	    shutdown($socket, 0);
> +	    my $send = ['pvesm', 'export', $volid, $res->{format}, '-', '-with-snapshots', $with_snapshots];
> +	    push @$send, '-snapshot', $snapshot if $snapshot;
> +
> +	    my @cstream;
> +	    if (defined($bwlimit)) {
> +		@cstream = ([ '/usr/bin/cstream', '-t', $bwlimit ]);
> +		$self->log('info', "using a bandwidth limit of $bwlimit bps for transferring '$volid'");
> +	    }
> +
> +	    eval {
> +		PVE::Tools::run_command(
> +		    [$send, @cstream],
> +		    output => '>&'.fileno($socket),
> +		    errfunc => sub { my $line = shift; $self->log('warn', $line); },
> +		);
> +	    };
> +	    my $send_error = $@;
> +
> +	    # don't close the connection entirely otherwise the
> +	    # receiving end might not get all buffered data (and
> +	    # fails with 'connection reset by peer')
> +	    shutdown($socket, 1);
> +
> +	    # wait for the remote process to finish
> +	    while ($res = $self->write_tunnel($self->{tunnel}, 10, 'query-disk-import')) {
> +		if ($res->{status} eq 'pending') {
> +		    $self->log('info', "waiting for disk import to finish..\n");
> +		    sleep(1)
> +		} elsif ($res->{status} eq 'complete') {
> +		    $new_volid = $res->{volid};
> +		    last;
> +		} else {
> +		    die "unknown query-disk-import result: $res->{status}\n";
> +		}
> +	    }
> +
> +	    # now close the socket
> +	    close($socket);
> +	    die $send_error if $send_error;
> +	} else {
> +	    my $bwlimit = $local_volumes->{$volid}->{bwlimit};
> +	    $bwlimit = $bwlimit * 1024 if defined($bwlimit); # storage_migrate uses bps
> +
> +	    my $storage_migrate_opts = {
> +		'ratelimit_bps' => $bwlimit,
> +		'insecure' => $opts->{migration_type} eq 'insecure',
> +		'with_snapshots' => $local_volumes->{$volid}->{snapshots},
> +		'allow_rename' => !$local_volumes->{$volid}->{is_vmstate},
> +	    };
> +
> +	    my $logfunc = sub { $self->log('info', $_[0]); };
> +	    $new_volid = eval {
> +		PVE::Storage::storage_migrate(
> +		    $storecfg,
> +		    $volid,
> +		    $self->{ssh_info},
> +		    $targetsid,
> +		    $storage_migrate_opts,
> +		    $logfunc,
> +		);
> +	    };
> +	    if (my $err = $@) {
> +		die "storage migration for '$volid' to storage '$targetsid' failed - $err\n";
> +	    }
>   	}
>   
>   	$self->{volume_map}->{$volid} = $new_volid;
> @@ -744,6 +1065,12 @@ sub sync_offline_local_volumes {
>   sub cleanup_remotedisks {
>       my ($self) = @_;
>   

Nit, not to be taken seriously: cleanup_remotedisks_and_maybe_tunnel ;)

> +    if ($self->{opts}->{remote}) {
> +	$self->finish_tunnel($self->{tunnel}, 1);
> +	delete $self->{tunnel};
> +	return;
> +    }
> +
>       my $local_volumes = $self->{local_volumes};
>   
>       foreach my $volid (values %{$self->{volume_map}}) {
> @@ -793,8 +1120,84 @@ sub phase1 {
>       $self->handle_replication($vmid);
>   
>       $self->sync_offline_local_volumes();
> +    $self->phase1_remote($vmid) if $self->{opts}->{remote};
>   };
>   
> +sub phase1_remote {
> +    my ($self, $vmid) = @_;
> +
> +    my $remote_conf = PVE::QemuConfig->load_config($vmid);
> +    PVE::QemuConfig->update_volume_ids($remote_conf, $self->{volume_map});
> +
> +    # TODO: check bridge availability earlier?
> +    my $bridgemap = $self->{opts}->{bridgemap};
> +    foreach my $opt (keys %$remote_conf) {
> +	next if $opt !~ m/^net\d+$/;
> +
> +	next if !$remote_conf->{$opt};
> +	my $d = PVE::QemuServer::parse_net($remote_conf->{$opt});
> +	next if !$d || !$d->{bridge};
> +
> +	my $target_bridge = PVE::QemuServer::map_id($bridgemap, $d->{bridge});
> +	$self->log('info', "mapped: $opt from $d->{bridge} to $target_bridge");
> +	$d->{bridge} = $target_bridge;
> +	$remote_conf->{$opt} = PVE::QemuServer::print_net($d);
> +    }
> +
> +    my @online_local_volumes = $self->filter_local_volumes('online');
> +
> +    my $storage_map = $self->{opts}->{storagemap};
> +    $self->{nbd} = {};
> +    PVE::QemuConfig->foreach_volume($remote_conf, sub {
> +	my ($ds, $drive) = @_;
> +
> +	# TODO eject CDROM?
> +	return if PVE::QemuServer::drive_is_cdrom($drive);
> +
> +	my $volid = $drive->{file};
> +	return if !$volid;
> +
> +	return if !grep { $_ eq $volid} @online_local_volumes;
> +
> +	my ($storeid, $volname) = PVE::Storage::parse_volume_id($volid);
> +	my $scfg = PVE::Storage::storage_config($self->{storecfg}, $storeid);
> +	my $source_format = PVE::QemuServer::qemu_img_format($scfg, $volname);
> +
> +	# set by target cluster
> +	my $oldvolid = delete $drive->{file};
> +	delete $drive->{format};
> +
> +	my $targetsid = PVE::QemuServer::map_id($storage_map, $storeid);
> +
> +	my $params = {
> +	    format => $source_format,
> +	    storage => $targetsid,
> +	    drive => $drive,
> +	};
> +
> +	$self->log('info', "Allocating volume for drive '$ds' on remote storage '$targetsid'..");
> +	my $res = $self->write_tunnel($self->{tunnel}, 600, 'disk', $params);
> +
> +	$self->log('info', "volume '$oldvolid' os '$res->{volid}' on the target\n");
> +	$remote_conf->{$ds} = $res->{drivestr};
> +	$self->{nbd}->{$ds} = $res;
> +    });
> +
> +    my $conf_str = PVE::QemuServer::write_vm_config("remote", $remote_conf);
> +
> +    # TODO expose in PVE::Firewall?
> +    my $vm_fw_conf_path = "/etc/pve/firewall/$vmid.fw";
> +    my $fw_conf_str;
> +    $fw_conf_str = PVE::Tools::file_get_contents($vm_fw_conf_path)
> +	if -e $vm_fw_conf_path;
> +    my $params = {
> +	conf => $conf_str,
> +	'firewall-config' => $fw_conf_str,
> +    };
> +
> +    $self->write_tunnel($self->{tunnel}, 10, 'config', $params);
> +}
> +
>   sub phase1_cleanup {
>       my ($self, $vmid, $err) = @_;
>   
> @@ -825,7 +1228,6 @@ sub phase2_start_local_cluster {
>       my $local_volumes = $self->{local_volumes};
>       my @online_local_volumes = $self->filter_local_volumes('online');
>   
> -    $self->{storage_migration} = 1 if scalar(@online_local_volumes);
>       my $start = $params->{start_params};
>       my $migrate = $params->{migrate_opts};
>   
> @@ -948,10 +1350,34 @@ sub phase2_start_local_cluster {
>       return ($tunnel_info, $spice_port);
>   }
>   
> +sub phase2_start_remote_cluster {
> +    my ($self, $vmid, $params) = @_;
> +
> +    die "insecure migration to remote cluster not implemented\n"
> +	if $params->{migrate_opts}->{type} ne 'websocket';
> +
> +    my $remote_vmid = $self->{opts}->{remote}->{vmid};
> +
> +    my $res = $self->write_tunnel($self->{tunnel}, 10, "start", $params);
> +
> +    foreach my $drive (keys %{$res->{drives}}) {
> +	$self->{stopnbd} = 1;
> +	$self->{target_drive}->{$drive}->{drivestr} = $res->{drives}->{$drive}->{drivestr};
> +	my $nbd_uri = $res->{drives}->{$drive}->{nbd_uri};
> +	die "unexpected NBD uri for '$drive': $nbd_uri\n"
> +	    if $nbd_uri !~ s!/run/qemu-server/$remote_vmid\_!/run/qemu-server/$vmid\_!;
> +
> +	$self->{target_drive}->{$drive}->{nbd_uri} = $nbd_uri;
> +    }
> +
> +    return ($res->{migrate}, $res->{spice_port});
> +}
> +
>   sub phase2 {
>       my ($self, $vmid) = @_;
>   
>       my $conf = $self->{vmconf};
> +    my $local_volumes = $self->{local_volumes};
>   
>       # version > 0 for unix socket support
>       my $nbd_protocol_version = 1;
> @@ -983,10 +1409,42 @@ sub phase2 {
>   	},
>       };
>   
> -    my ($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
> +    my ($tunnel_info, $spice_port);
> +
> +    my @online_local_volumes = $self->filter_local_volumes('online');
> +    $self->{storage_migration} = 1 if scalar(@online_local_volumes);
> +
> +    if (my $remote = $self->{opts}->{remote}) {
> +	my $remote_vmid = $remote->{vmid};
> +	$params->{migrate_opts}->{remote_node} = $self->{node};
> +	($tunnel_info, $spice_port) = $self->phase2_start_remote_cluster($vmid, $params);
> +	die "only UNIX sockets are supported for remote migration\n"
> +	    if $tunnel_info->{proto} ne 'unix';
> +
> +	my $forwarded = {};
> +	my $remote_socket = $tunnel_info->{addr};
> +	my $local_socket = $remote_socket;
> +	$local_socket =~ s/$remote_vmid/$vmid/g;
> +	$tunnel_info->{addr} = $local_socket;
> +
> +	$self->log('info', "Setting up tunnel for '$local_socket'");
> +	$forward_unix_socket->($self, $local_socket, $remote_socket);
> +	$forwarded->{$local_socket} = 1;
> +
> +	foreach my $remote_socket (@{$tunnel_info->{unix_sockets}}) {
> +	    my $local_socket = $remote_socket;
> +	    $local_socket =~ s/$remote_vmid/$vmid/g;
> +	    next if $forwarded->{$local_socket};
> +	    $self->log('info', "Setting up tunnel for '$local_socket'");
> +	    $forward_unix_socket->($self, $local_socket, $remote_socket);
> +	    $forwarded->{$local_socket} = 1;
> +	}
> +    } else {
> +	($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
>   
> -    $self->log('info', "start remote tunnel");
> -    $self->start_remote_tunnel($tunnel_info);
> +	$self->log('info', "start remote tunnel");
> +	$self->start_remote_tunnel($tunnel_info);
> +    }
>   
>       my $migrate_uri = "$tunnel_info->{proto}:$tunnel_info->{addr}";
>       $migrate_uri .= ":$tunnel_info->{port}"
> @@ -996,8 +1454,6 @@ sub phase2 {
>   	$self->{storage_migration_jobs} = {};
>   	$self->log('info', "starting storage migration");
>   
> -	my @online_local_volumes = $self->filter_local_volumes('online');
> -
>   	die "The number of local disks does not match between the source and the destination.\n"
>   	    if (scalar(keys %{$self->{target_drive}}) != scalar(@online_local_volumes));
>   	foreach my $drive (keys %{$self->{target_drive}}){
> @@ -1070,7 +1526,7 @@ sub phase2 {
>       };
>       $self->log('info', "migrate-set-parameters error: $@") if $@;
>   
> -    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
> +    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga} && !$self->{opts}->{remote})) {
>   	my $rpcenv = PVE::RPCEnvironment::get();
>   	my $authuser = $rpcenv->get_user();
>   
> @@ -1267,11 +1723,15 @@ sub phase2_cleanup {
>   
>       my $nodename = PVE::INotify::nodename();
>   
> -    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'stop', $vmid, '--skiplock', '--migratedfrom', $nodename];
> -    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
> -    if (my $err = $@) {
> -        $self->log('err', $err);
> -        $self->{errors} = 1;
> +    if ($self->{tunnel} && $self->{tunnel}->{version} >= 2) {
> +	$self->write_tunnel($self->{tunnel}, 10, 'stop');
> +    } else {
> +	my $cmd = [@{$self->{rem_ssh}}, 'qm', 'stop', $vmid, '--skiplock', '--migratedfrom', $nodename];
> +	eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
> +	if (my $err = $@) {
> +	    $self->log('err', $err);
> +	    $self->{errors} = 1;
> +	}
>       }
>   
>       # cleanup after stopping, otherwise disks might be in-use by target VM!
> @@ -1304,7 +1764,7 @@ sub phase3_cleanup {
>   
>       my $tunnel = $self->{tunnel};
>   
> -    if ($self->{volume_map}) {
> +    if ($self->{volume_map} && !$self->{opts}->{remote}) {
>   	my $target_drives = $self->{target_drive};
>   
>   	# FIXME: for NBD storage migration we now only update the volid, and
> @@ -1321,26 +1781,33 @@ sub phase3_cleanup {
>   
>       # transfer replication state before move config
>       $self->transfer_replication_state() if $self->{is_replicated};
> -    PVE::QemuConfig->move_config_to_node($vmid, $self->{node});
> +    if (!$self->{opts}->{remote}) {
> +	PVE::QemuConfig->move_config_to_node($vmid, $self->{node});
> +    }
>       $self->switch_replication_job_target() if $self->{is_replicated};

All three lines could/should be guarded by the if.

>   
>       if ($self->{livemigration}) {
>   	if ($self->{stopnbd}) {
>   	    $self->log('info', "stopping NBD storage migration server on target.");
>   	    # stop nbd server on remote vm - requirement for resume since 2.9
> -	    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'nbdstop', $vmid];
> +	    if ($tunnel && $tunnel->{version} && $tunnel->{version} >= 2) {
> +		$self->write_tunnel($tunnel, 30, 'nbdstop');
> +	    } else {
> +		my $cmd = [@{$self->{rem_ssh}}, 'qm', 'nbdstop', $vmid];
>   
> -	    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
> -	    if (my $err = $@) {
> -		$self->log('err', $err);
> -		$self->{errors} = 1;
> +		eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
> +		if (my $err = $@) {
> +		    $self->log('err', $err);
> +		    $self->{errors} = 1;
> +		}
>   	    }
>   	}
>   
>   	# config moved and nbd server stopped - now we can resume vm on target
>   	if ($tunnel && $tunnel->{version} && $tunnel->{version} >= 1) {
> +	    my $cmd = $tunnel->{version} == 1 ? "resume $vmid" : "resume";
>   	    eval {
> -		$self->write_tunnel($tunnel, 30, "resume $vmid");
> +		$self->write_tunnel($tunnel, 30, $cmd);
>   	    };
>   	    if (my $err = $@) {
>   		$self->log('err', $err);
> @@ -1360,18 +1827,24 @@ sub phase3_cleanup {
>   	}
>   
>   	if ($self->{storage_migration} && PVE::QemuServer::parse_guest_agent($conf)->{fstrim_cloned_disks} && $self->{running}) {
> -	    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'guest', 'cmd', $vmid, 'fstrim'];
> -	    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
> +	    if ($self->{opts}->{remote}) {
> +		$self->write_tunnel($self->{tunnel}, 600, 'fstrim');
> +	    } else {
> +		my $cmd = [@{$self->{rem_ssh}}, 'qm', 'guest', 'cmd', $vmid, 'fstrim'];
> +		eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
> +	    }
>   	}
>       }
>   
>       # close tunnel on successful migration, on error phase2_cleanup closed it
> -    if ($tunnel) {
> +    if ($tunnel && $tunnel->{version} == 1) {
>   	eval { finish_tunnel($self, $tunnel);  };
>   	if (my $err = $@) {
>   	    $self->log('err', $err);
>   	    $self->{errors} = 1;
>   	}
> +	$tunnel = undef;
> +	delete $self->{tunnel};
>       }
>   
>       eval {
> @@ -1409,6 +1882,9 @@ sub phase3_cleanup {
>   
>       # destroy local copies
>       foreach my $volid (@not_replicated_volumes) {
> +	# remote is cleaned up below
> +	next if $self->{opts}->{remote};
> +
>   	eval { PVE::Storage::vdisk_free($self->{storecfg}, $volid); };
>   	if (my $err = $@) {
>   	    $self->log('err', "removing local copy of '$volid' failed - $err");
> @@ -1418,8 +1894,19 @@ sub phase3_cleanup {
>       }
>   
>       # clear migrate lock
> -    my $cmd = [ @{$self->{rem_ssh}}, 'qm', 'unlock', $vmid ];
> -    $self->cmd_logerr($cmd, errmsg => "failed to clear migrate lock");
> +    if ($tunnel && $tunnel->{version} >= 2) {
> +	$self->write_tunnel($tunnel, 10, "unlock");
> +
> +	$self->finish_tunnel($tunnel);
> +    } else {
> +	my $cmd = [ @{$self->{rem_ssh}}, 'qm', 'unlock', $vmid ];
> +	$self->cmd_logerr($cmd, errmsg => "failed to clear migrate lock");
> +    }
> +
> +    if ($self->{opts}->{remote} && $self->{opts}->{delete}) {
> +	eval { PVE::QemuServer::destroy_vm($self->{storecfg}, $vmid, 1, undef, 0) };
> +	warn "Failed to remove source VM - $@\n" if $@;
> +    }
>   }
>   
>   sub final_cleanup {
> diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
> index d494cc0..bf05da2 100644
> --- a/PVE/QemuServer.pm
> +++ b/PVE/QemuServer.pm
> @@ -5384,7 +5384,11 @@ sub vm_start_nolock {
>       my $defaults = load_defaults();
>   
>       # set environment variable useful inside network script
> -    $ENV{PVE_MIGRATED_FROM} = $migratedfrom if $migratedfrom;
> +    if ($migrate_opts->{remote_node}) {
> +	$ENV{PVE_MIGRATED_FROM} = $migrate_opts->{remote_node};
> +    } elsif ($migratedfrom) {
> +	$ENV{PVE_MIGRATED_FROM} = $migratedfrom;
> +    }

But the network script tries to load the config from that node and if 
it's not in the cluster that doesn't work?

>   
>       PVE::GuestHelpers::exec_hookscript($conf, $vmid, 'pre-start', 1);
>   
> @@ -5621,7 +5625,7 @@ sub vm_start_nolock {
>   
>   	my $migrate_storage_uri;
>   	# nbd_protocol_version > 0 for unix socket support
> -	if ($nbd_protocol_version > 0 && $migration_type eq 'secure') {
> +	if ($nbd_protocol_version > 0 && ($migration_type eq 'secure' || $migration_type eq 'websocket')) {
>   	    my $socket_path = "/run/qemu-server/$vmid\_nbd.migrate";
>   	    mon_cmd($vmid, "nbd-server-start", addr => { type => 'unix', data => { path => $socket_path } } );
>   	    $migrate_storage_uri = "nbd:unix:$socket_path";
> 




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

* Re: [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration
  2021-11-05 13:03 ` [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration Fabian Grünbichler
  2021-11-08 13:50   ` Fabian Ebner
@ 2021-11-10 12:03   ` Fabian Ebner
  1 sibling, 0 replies; 50+ messages in thread
From: Fabian Ebner @ 2021-11-10 12:03 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> we only want to use an explicitly provided migration network, not one
> for the local cluster.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>   src/PVE/AbstractMigrate.pm | 51 +++++++++++++++++++++++---------------
>   1 file changed, 31 insertions(+), 20 deletions(-)
> 
> diff --git a/src/PVE/AbstractMigrate.pm b/src/PVE/AbstractMigrate.pm
> index af2be38..ec60b82 100644
> --- a/src/PVE/AbstractMigrate.pm
> +++ b/src/PVE/AbstractMigrate.pm
> @@ -115,22 +115,27 @@ sub migrate {
>   
>       $class = ref($class) || $class;
>   
> -    my $dc_conf = PVE::Cluster::cfs_read_file('datacenter.cfg');
> +    my ($ssh_info, $rem_ssh);
> +    if (!$opts->{remote}) {
> +	my $dc_conf = PVE::Cluster::cfs_read_file('datacenter.cfg');
>   
> -    my $migration_network = $opts->{migration_network};
> -    if (!defined($migration_network)) {
> -	$migration_network = $dc_conf->{migration}->{network};
> -    }
> -    my $ssh_info = PVE::SSHInfo::get_ssh_info($node, $migration_network);
> -    $nodeip = $ssh_info->{ip};
> -
> -    my $migration_type = 'secure';
> -    if (defined($opts->{migration_type})) {
> -	$migration_type = $opts->{migration_type};
> -    } elsif (defined($dc_conf->{migration}->{type})) {
> -        $migration_type = $dc_conf->{migration}->{type};
> +	my $migration_network = $opts->{migration_network};
> +	if (!defined($migration_network)) {
> +	    $migration_network = $dc_conf->{migration}->{network};
> +	}
> +	$ssh_info = PVE::SSHInfo::get_ssh_info($node, $migration_network);
> +	$nodeip = $ssh_info->{ip};
> +
> +	my $migration_type = 'secure';
> +	if (defined($opts->{migration_type})) {
> +	    $migration_type = $opts->{migration_type};
> +	} elsif (defined($dc_conf->{migration}->{type})) {
> +	    $migration_type = $dc_conf->{migration}->{type};
> +	}
> +	$opts->{migration_type} = $migration_type;
> +	$opts->{migration_network} = $migration_network;
> +	$rem_ssh = PVE::SSHInfo::ssh_info_to_command($ssh_info);
>       }
> -    $opts->{migration_type} = $migration_type;
>   
>       my $self = {
>   	delayed_interrupt => 0,
> @@ -139,7 +144,7 @@ sub migrate {
>   	node => $node,
>   	ssh_info => $ssh_info,
>   	nodeip => $nodeip,
> -	rem_ssh => PVE::SSHInfo::ssh_info_to_command($ssh_info)
> +	rem_ssh => $rem_ssh,
>       };
>   
>       $self = bless $self, $class;
> @@ -162,15 +167,21 @@ sub migrate {
>   	&$eval_int($self, sub { $self->{running} = $self->prepare($self->{vmid}); });
>   	die $@ if $@;
>   
> -	if (defined($migration_network)) {
> +	if (defined($self->{opts}->{migration_network})) {
>   	    $self->log('info', "use dedicated network address for sending " .
>   	               "migration traffic ($self->{nodeip})");
>   
>   	    # test if we can connect to new IP
> -	    my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
> -	    eval { $self->cmd_quiet($cmd); };
> -	    die "Can't connect to destination address ($self->{nodeip}) using " .
> -	        "public key authentication\n" if $@;
> +	    if ($self->{opts}->{remote}) {
> +		eval { $self->{opts}->{remote}->{client}->get("/") };

Shouldn't the check here happen regardless of whether a 
migration_network is defined? Actually, isn't the same true for the 
existing check too?

> +		die "Can't connect to destination address ($self->{nodeip}) using " .
> +		    "API connection - $@\n" if $@;
> +	    } else {
> +		my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
> +		eval { $self->cmd_quiet($cmd); };
> +		die "Can't connect to destination address ($self->{nodeip}) using " .
> +		    "public key authentication\n" if $@;
> +	    }
>   	}
>   
>   	&$eval_int($self, sub { $self->phase1($self->{vmid}); });
> 




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

* Re: [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint Fabian Grünbichler
@ 2021-11-10 12:29   ` Fabian Ebner
  2021-11-11 12:33     ` Fabian Grünbichler
  0 siblings, 1 reply; 50+ messages in thread
From: Fabian Ebner @ 2021-11-10 12:29 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>      the checks currently done before the actual migration worker is
>      forked could be either moved to the client calling this (that then
>      makes the required API calls) or extracted into a precond API call
>      like for regular migration.
>      
>      for testing it helps catch trivial mistakes early on, and the calls shouldn't
>      be too expensive, so I left them in for now..
>      
>      requires
>      - pve-common with bridge-pair format
>      - pve-guest-common with AbstractMigrate handling remote migration
> 
>   PVE/API2/Qemu.pm | 205 ++++++++++++++++++++++++++++++++++++++++++++++-
>   debian/control   |   2 +
>   2 files changed, 205 insertions(+), 2 deletions(-)
> 
> diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
> index 24f5b98..b931f04 100644
> --- a/PVE/API2/Qemu.pm
> +++ b/PVE/API2/Qemu.pm
> @@ -14,6 +14,7 @@ use URI::Escape;
>   use Crypt::OpenSSL::Random;
>   use Socket qw(SOCK_STREAM);
>   
> +use PVE::APIClient::LWP;
>   use PVE::Cluster qw (cfs_read_file cfs_write_file);;
>   use PVE::RRD;
>   use PVE::SafeSyslog;
> @@ -51,8 +52,6 @@ BEGIN {
>       }
>   }
>   
> -use Data::Dumper; # fixme: remove
> -
>   use base qw(PVE::RESTHandler);
>   
>   my $opt_force_description = "Force physical removal. Without this, we simple remove the disk from the config file and create an additional configuration entry called 'unused[n]', which contains the volume ID. Unlink of unused[n] always cause physical removal.";
> @@ -3778,6 +3777,208 @@ __PACKAGE__->register_method({
>   
>       }});
>   
> +__PACKAGE__->register_method({
> +    name => 'remote_migrate_vm',
> +    path => '{vmid}/remote_migrate',
> +    method => 'POST',
> +    protected => 1,
> +    proxyto => 'node',
> +    description => "Migrate virtual machine to a remote cluster. Creates a new migration task.",
> +    permissions => {
> +	check => ['perm', '/vms/{vmid}', [ 'VM.Migrate' ]],
> +    },
> +    parameters => {
> +	additionalProperties => 0,
> +	properties => {
> +	    node => get_standard_option('pve-node'),
> +	    vmid => get_standard_option('pve-vmid', { completion => \&PVE::QemuServer::complete_vmid }),
> +	    'target-vmid' => get_standard_option('pve-vmid', { optional => 1 }),
> +	    'target-node' => get_standard_option('pve-node', {
> +		description => "Target node on remote cluster.",
> +            }),
> +	    'target-endpoint' => get_standard_option('proxmox-remote', {
> +		description => "Remote target endpoint",
> +	    }),
> +	    online => {
> +		type => 'boolean',
> +		description => "Use online/live migration if VM is running. Ignored if VM is stopped.",
> +		optional => 1,
> +	    },
> +	    'migration-network' => {
> +		type => 'string', format => 'CIDR',
> +		description => "CIDR of the (sub) network that is used for migration.",
> +		optional => 1,
> +	    },
> +	    'with-local-disks' => {
> +		type => 'boolean',
> +		description => "Enable live storage migration for local disk",
> +		optional => 1,
> +	    },
> +	    delete => {
> +		type => 'boolean',
> +		description => "Delete the original VM and related data after successful migration. By default the original VM is kept on the source cluster in a stopped state.",
> +		optional => 1,
> +		default => 0,
> +	    },
> +            'target-storage' => get_standard_option('pve-targetstorage', {
> +		completion => \&PVE::QemuServer::complete_migration_storage,
> +		optional => 0,
> +            }),
> +	    'target-bridge' => {
> +		type => 'string',
> +		description => "Mapping from source to target bridges. Providing only a single bridge ID maps all source bridges to that bridge. Providing the special value '1' will map each source bridge to itself.",
> +		format => 'bridge-pair-list',
> +	    },
> +	    bwlimit => {
> +		description => "Override I/O bandwidth limit (in KiB/s).",
> +		optional => 1,
> +		type => 'integer',
> +		minimum => '0',
> +		default => 'migrate limit from datacenter or storage config',
> +	    },
> +	},
> +    },
> +    returns => {
> +	type => 'string',
> +	description => "the task ID.",
> +    },
> +    code => sub {
> +	my ($param) = @_;
> +
> +	my $rpcenv = PVE::RPCEnvironment::get();
> +	my $authuser = $rpcenv->get_user();
> +
> +	my $source_vmid = extract_param($param, 'vmid');
> +	my $target_endpoint = extract_param($param, 'target-endpoint');
> +	my $target_node = extract_param($param, 'target-node');
> +	my $target_vmid = extract_param($param, 'target-vmid') // $source_vmid;
> +
> +	my $localnode = PVE::INotify::nodename();

Nit: not used (and could've been $param->{node}).

> +	my $network = extract_param($param, 'migration-network');
> +	my $delete = extract_param($param, 'delete') // 0;
> +
> +	PVE::Cluster::check_cfs_quorum();
> +
> +	raise_param_exc({ 'migration-network' => "Only root may use this option." })
> +	    if $network && $authuser ne 'root@pam';

I might be missing something obvious, but where is the migration network 
actually used down the line for the remote migration?

> +
> +	# test if VM exists
> +	my $conf = PVE::QemuConfig->load_config($source_vmid);
> +
> +	PVE::QemuConfig->check_lock($conf);
> +
> +	raise_param_exc({ vmid => "cannot migrate HA-manage VM to remote cluster" })

s/manage/managed/

> +	    if PVE::HA::Config::vm_is_ha_managed($source_vmid);
> +
> +	my $remote = PVE::JSONSchema::parse_property_string('proxmox-remote', $target_endpoint);
> +
> +	# TODO: move this as helper somewhere appropriate?
> +	my $conn_args = {
> +	    protocol => 'https',
> +	    host => $remote->{host},
> +	    port => $remote->{port} // 8006,
> +	    apitoken => $remote->{apitoken},
> +	};
> +
> +	my $fp;
> +	if ($fp = $remote->{fingerprint}) {
> +	    $conn_args->{cached_fingerprints} = { uc($fp) => 1 };
> +	}
> +
> +	print "Establishing API connection with remote at '$remote->{host}'\n";
> +
> +	my $api_client = PVE::APIClient::LWP->new(%$conn_args);
> +	my $version = $api_client->get("/version");
> +	print "remote: version '$version->{version}\n";
> +
> +	if (!defined($fp)) {
> +	    my $cert_info = $api_client->get("/nodes/$target_node/certificates/info");
> +	    foreach my $cert (@$cert_info) {
> +		$fp = $cert->{fingerprint} if $cert->{filename} ne 'pve-root-ca.pem';
> +		last if $cert->{filename} eq 'pveproxy-ssl.pem';

Not future-proof if the API call is ever extended to return an 
additional certificate which is not a valid fall-back here.

> +	    }
> +	    $conn_args->{cached_fingerprints} = { uc($fp) => 1 }
> +		if defined($fp);
> +	}
> +
> +	if (PVE::QemuServer::check_running($source_vmid)) {
> +	    die "can't migrate running VM without --online\n" if !$param->{online};
> +
> +	    my $repl_conf = PVE::ReplicationConfig->new();
> +	    my $is_replicated = $repl_conf->check_for_existing_jobs($source_vmid, 1);
> +	    die "cannot remote-migrate replicated VM\n" if $is_replicated;
> +	} else {
> +	    warn "VM isn't running. Doing offline migration instead.\n" if $param->{online};
> +	    $param->{online} = 0;
> +	}
> +
> +	# FIXME: fork worker hear to avoid timeout? or poll these periodically
> +	# in pvestatd and access cached info here? all of the below is actually
> +	# checked at the remote end anyway once we call the mtunnel endpoint,
> +	# we could also punt it to the client and not do it here at all..
> +	my $resources = $api_client->get("/cluster/resources");
> +	if (grep { defined($_->{vmid}) && $_->{vmid} eq $target_vmid } @$resources) {
> +	    raise_param_exc({ target_vmid => "Guest with ID '$target_vmid' already exists on remote cluster" });
> +	}
> +
> +	my $storages = [ grep { $_->{type} eq 'storage' && $_->{node} eq $target_node } @$resources ];
> +	my $storecfg = PVE::Storage::config();
> +	my $target_storage = extract_param($param, 'target-storage');
> +	my $storagemap = eval { PVE::JSONSchema::parse_idmap($target_storage, 'pve-storage-id') };
> +	raise_param_exc({ 'target-storage' => "failed to parse storage map: $@" })
> +	    if $@;
> +
> +	my $target_bridge = extract_param($param, 'target-bridge');
> +	my $bridgemap = eval { PVE::JSONSchema::parse_idmap($target_bridge, 'pve-bridge-id') };
> +	raise_param_exc({ 'target-bridge' => "failed to parse bridge map: $@" })
> +	    if $@;
> +
> +	my $check_remote_storage = sub {
> +	    my ($storage) = @_;
> +	    my $found = [ grep { $_->{storage} eq $storage } @$storages ];
> +	    die "remote: storage '$storage' does not exist!\n"
> +		if !@$found;
> +
> +	    $found = @$found[0];
> +
> +	    my $content_types = [ PVE::Tools::split_list($found->{content}) ];
> +	    die "remote: storage '$storage' cannot store images\n"
> +		if !grep { $_ eq 'images' } @$content_types;
> +	};
> +
> +	foreach my $target_sid (values %{$storagemap->{entries}}) {
> +	    $check_remote_storage->($target_sid);
> +	}
> +
> +	$check_remote_storage->($storagemap->{default})
> +	    if $storagemap->{default};
> +
> +	# TODO: or check all referenced storages?
> +	die "remote migration requires explicit storage mapping!\n"
> +	    if $storagemap->{identity};
> +
> +	$param->{storagemap} = $storagemap;
> +	$param->{bridgemap} = $bridgemap;
> +	$param->{remote} = {
> +	    conn => $conn_args, # re-use fingerprint for tunnel
> +	    client => $api_client,
> +	    vmid => $target_vmid,
> +	};
> +	$param->{migration_type} = 'websocket';
> +	$param->{migration_network} = $network if $network;
> +	$param->{delete} = $delete if $delete;
> +
> +	my $realcmd = sub {
> +	    PVE::QemuMigrate->migrate($target_node, $remote->{host}, $source_vmid, $param);
> +	};
> +
> +	my $worker = sub {
> +	    return PVE::GuestHelpers::guest_migration_lock($source_vmid, 10, $realcmd);
> +	};
> +
> +	return $rpcenv->fork_worker('qmigrate', $source_vmid, $authuser, $worker);
> +    }});
> +
>   __PACKAGE__->register_method({
>       name => 'monitor',
>       path => '{vmid}/monitor',
> diff --git a/debian/control b/debian/control
> index 8032ae5..33e3916 100644
> --- a/debian/control
> +++ b/debian/control
> @@ -6,6 +6,7 @@ Build-Depends: debhelper (>= 12~),
>                  libglib2.0-dev,
>                  libio-multiplex-perl,
>                  libjson-c-dev,
> +               libpve-apiclient-perl,
>                  libpve-cluster-perl,
>                  libpve-common-perl (>= 6.3-3),
>                  libpve-guest-common-perl (>= 3.1-3),
> @@ -34,6 +35,7 @@ Depends: dbus,
>            libjson-xs-perl,
>            libnet-ssleay-perl,
>            libpve-access-control (>= 5.0-7),
> +         libpve-apiclient-perl,
>            libpve-cluster-perl,
>            libpve-common-perl (>= 7.0-3),
>            libpve-guest-common-perl (>= 3.1-3),
> 




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

* Re: [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration
  2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
                   ` (22 preceding siblings ...)
  2021-11-09 16:07 ` [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration DERUMIER, Alexandre
@ 2021-11-10 12:42 ` Fabian Ebner
  23 siblings, 0 replies; 50+ messages in thread
From: Fabian Ebner @ 2021-11-10 12:42 UTC (permalink / raw)
  To: pve-devel, Fabian Grünbichler

Mostly just nits and suggestions from my side, but there are a handful 
of (easily-fixed) issues too.

For the patches for access-control, common, http-server (although I 
don't feel qualified there) and patches for qemu-server where I didn't 
reply:

Reviewed-by: Fabian Ebner <f.ebner@proxmox.com>

Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> this series adds remote migration for VMs. there's still some of
> TODOs/FIXMEs/stuff that requires discussion left, but this should
> hopefully be possible to apply with minor modifications and lots of
> future extensions.
> 
> both live and offline migration including NBD and storage-migrated disks
> should work.
> 
> besides lots of rebases, implemented todos and fixed issues the main
> difference to the previous RFC is that we no longer define remote
> entries in a config file, but just expect the caller/client to give us
> all the required information to connect to the remote cluster.
> 
> overview over affected repos and changes, see individual patches for
> more details.
> 
> proxmox:
> 
> some compatible changes to make websocket code usable for client-side
> connections, required by proxmox-websocket-tunnel
> 
> proxmox-websocket-tunnel:
> 
> new tunnel helper tool for forwarding commands and data over websocket
> connections, required by qemu-server on source side
> 
> pve-access-control:
> 
> new ticket type, required by qemu-server on target side
> 
> pve-common:
> 
> bridgepair format akin to storage pair, pve-bridge-id option, required
> by qemu-server
> remote format for specifying a remote endpoint (including API token,
> fingerprint, ..)
> 
> pve-guest-common:
> 
> handle remote migration (no SSH) in AbstractMigrate,
> required by qemu-server
> 
> pve-storage:
> 
> extend 'pvesm import' to allow import from UNIX socket, required on
> target node by qemu-server
> 
> qemu-server:
> 
> some refactoring, new mtunnel endpoints, new remote_migration endpoints
> TODO: handle pending changes and snapshots
> TODO: proper CLI for remote migration
> potential TODO: precond endpoint?
> 
> pve-http-server:
> 
> fix for handling unflushed proxy streams
> 
> as usual, some of the patches are best viewed with '-w', especially in
> qemu-server..
> 
> required dependencies are noted, qemu-server also requires a build-dep
> on patched pve-common since the required options/formats would be
> missing otherwise..
> 
> 
> _______________________________________________
> pve-devel mailing list
> pve-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pve-devel
> 
> 




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

* Re: [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation
  2021-11-09 12:54   ` Dominik Csapak
@ 2021-11-11  9:58     ` Fabian Grünbichler
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-11  9:58 UTC (permalink / raw)
  To: Dominik Csapak, Proxmox VE development discussion

On November 9, 2021 1:54 pm, Dominik Csapak wrote:
> looks mostly fine, some comments inline

thanks!

> On 11/5/21 14:03, Fabian Grünbichler wrote:
>> the websocket tunnel helper accepts control commands (encoded as
>> single-line JSON) on stdin, and prints responses on stdout.
>> 
>> the following commands are available:
>> - "connect" a 'control' tunnel via a websocket
>> - "forward" a local unix socket to a remote socket via a websocket
>> -- if requested, this will ask for a ticket via the control tunnel after
>> accepting a new connection on the unix socket
>> - "close" the control tunnel and any forwarded socket
>> 
>> any other json input (without the 'control' flag set) is forwarded as-is
>> to the remote end of the control tunnel.
>> 
>> internally, the tunnel helper will spawn tokio tasks for
>> - handling the control tunnel connection (new commands are passed in via
>> an mpsc channel together with a oneshot channel for the response)
>> - handling accepting new connections on each forwarded unix socket
>> - handling forwarding data over accepted forwarded connections
>> 
>> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
>> ---
>> 
>> Notes:
>>      requires proxmox-http with changes and bumped version
>> 
>>   Cargo.toml  |  13 ++
>>   src/main.rs | 410 ++++++++++++++++++++++++++++++++++++++++++++++++++++
>>   2 files changed, 423 insertions(+)
>>   create mode 100644 src/main.rs
>> 
>> diff --git a/Cargo.toml b/Cargo.toml
>> index 939184c..9d2a8c6 100644
>> --- a/Cargo.toml
>> +++ b/Cargo.toml
>> @@ -9,3 +9,16 @@ description = "Proxmox websocket tunneling helper"
>>   exclude = ["debian"]
>>   
>>   [dependencies]
>> +anyhow = "1.0"
>> +base64 = "0.12"
>> +futures = "0.3"
>> +futures-util = "0.3"
>> +hyper = { version = "0.14" }
>> +openssl = "0.10"
>> +percent-encoding = "2"
>> +proxmox-http = { version = "0.5.2", path = "../proxmox/proxmox-http", features = ["websocket", "client"] }
>> +serde = { version = "1.0", features = ["derive"] }
>> +serde_json = "1.0"
>> +tokio = { version = "1", features = ["io-std", "io-util", "macros", "rt-multi-thread", "sync"] }
>> +tokio-stream = { version = "0.1", features = ["io-util"] }
>> +tokio-util = "0.6"
>> diff --git a/src/main.rs b/src/main.rs
>> new file mode 100644
>> index 0000000..150c1cf
>> --- /dev/null
>> +++ b/src/main.rs
>> @@ -0,0 +1,410 @@
>> +use anyhow::{bail, format_err, Error};
>> +
>> +use std::collections::VecDeque;
>> +use std::sync::{Arc, Mutex};
>> +
>> +use futures::future::FutureExt;
>> +use futures::select;
>> +
>> +use hyper::client::{Client, HttpConnector};
>> +use hyper::header::{SEC_WEBSOCKET_KEY, SEC_WEBSOCKET_VERSION, UPGRADE};
>> +use hyper::upgrade::Upgraded;
>> +use hyper::{Body, Request, StatusCode};
>> +
>> +use openssl::ssl::{SslConnector, SslMethod};
>> +use percent_encoding::{utf8_percent_encode, NON_ALPHANUMERIC};
>> +
>> +use serde::{Deserialize, Serialize};
>> +use serde_json::{Map, Value};
>> +use tokio::io::{AsyncBufReadExt, AsyncWriteExt, BufReader};
>> +use tokio::net::{UnixListener, UnixStream};
>> +use tokio::sync::{mpsc, oneshot};
>> +use tokio_stream::wrappers::LinesStream;
>> +use tokio_stream::StreamExt;
>> +
>> +use proxmox_http::client::HttpsConnector;
>> +use proxmox_http::websocket::{OpCode, WebSocket, WebSocketReader, WebSocketWriter};
>> +
>> +#[derive(Serialize, Deserialize, Debug)]
>> +#[serde(rename_all = "kebab-case")]
>> +enum CmdType {
>> +    Connect,
>> +    Forward,
>> +    CloseCmd,
> 
> this is never used
> 

dropped in v2, we can always re-introduce this if we want to do a more 
elaborate closing dance.

>> +    NonControl,
>> +}
>> +
>> +type CmdData = Map<String, Value>;
>> +
>> +#[derive(Serialize, Deserialize, Debug)]
>> +#[serde(rename_all = "kebab-case")]
>> +struct ConnectCmdData {
>> +    // target URL for WS connection
>> +    url: String,
>> +    // fingerprint of TLS certificate
>> +    fingerprint: Option<String>,
>> +    // addition headers such as authorization
>> +    headers: Option<Vec<(String, String)>>,
>> +}
>> +
>> +#[derive(Serialize, Deserialize, Debug, Clone)]
>> +#[serde(rename_all = "kebab-case")]
>> +struct ForwardCmdData {
>> +    // target URL for WS connection
>> +    url: String,
>> +    // addition headers such as authorization
>> +    headers: Option<Vec<(String, String)>>,
>> +    // fingerprint of TLS certificate
>> +    fingerprint: Option<String>,
>> +    // local UNIX socket path for forwarding
>> +    unix: String,
>> +    // request ticket using these parameters
>> +    ticket: Option<Map<String, Value>>,
>> +}
>> +
>> +struct CtrlTunnel {
>> +    sender: Option<mpsc::UnboundedSender<(Value, oneshot::Sender<String>)>>,
>> +    forwarded: Arc<Mutex<Vec<oneshot::Sender<()>>>>,
> 
> for now, this is really not used (see my comments further below)
> 

same

>> +}
>> +
>> +impl CtrlTunnel {
>> +    async fn read_cmd_loop(mut self) -> Result<(), Error> {
>> +        let mut stdin_stream = LinesStream::new(BufReader::new(tokio::io::stdin()).lines());
>> +        while let Some(res) = stdin_stream.next().await {
>> +            match res {
>> +                Ok(line) => {
>> +                    let (cmd_type, data) = Self::parse_cmd(&line)?;
>> +                    match cmd_type {
>> +                        CmdType::Connect => self.handle_connect_cmd(data).await,
>> +                        CmdType::Forward => {
>> +                            let res = self.handle_forward_cmd(data).await;
>> +                            match &res {
>> +                                Ok(()) => println!("{}", serde_json::json!({"success": true})),
>> +                                Err(msg) => println!(
>> +                                    "{}",
>> +                                    serde_json::json!({"success": false, "msg": msg.to_string()})
>> +                                ),
>> +                            };
>> +                            res
>> +                        }
>> +                        CmdType::NonControl => self
>> +                            .handle_tunnel_cmd(data)
>> +                            .await
>> +                            .map(|res| println!("{}", res)),
>> +                        _ => unimplemented!(),
>> +                    }
>> +                }
>> +                Err(err) => bail!("Failed to read from STDIN - {}", err),
>> +            }?;
>> +        }
>> +
>> +        Ok(())
>> +    }
>> +
>> +    fn parse_cmd(line: &str) -> Result<(CmdType, CmdData), Error> {
>> +        let mut json: Map<String, Value> = serde_json::from_str(line)?;
>> +        match json.remove("control") {
>> +            Some(Value::Bool(true)) => {
>> +                match json.remove("cmd").map(serde_json::from_value::<CmdType>) {
>> +                    None => bail!("input has 'control' flag, but no control 'cmd' set.."),
>> +                    Some(Err(e)) => bail!("failed to parse control cmd - {}", e),
>> +                    Some(Ok(cmd_type)) => Ok((cmd_type, json)),
>> +                }
>> +            }
>> +            _ => Ok((CmdType::NonControl, json)),
>> +        }
>> +    }
>> +
>> +    async fn websocket_connect(
>> +        url: String,
>> +        extra_headers: Vec<(String, String)>,
>> +        fingerprint: Option<String>,
>> +    ) -> Result<Upgraded, Error> {
>> +        let ws_key = proxmox::sys::linux::random_data(16)?;
>> +        let ws_key = base64::encode(&ws_key);
>> +        let mut req = Request::builder()
>> +            .uri(url)
>> +            .header(UPGRADE, "websocket")
>> +            .header(SEC_WEBSOCKET_VERSION, "13")
>> +            .header(SEC_WEBSOCKET_KEY, ws_key)
>> +            .body(Body::empty())
>> +            .unwrap();
>> +
>> +        let headers = req.headers_mut();
>> +        for (name, value) in extra_headers {
>> +            let name = hyper::header::HeaderName::from_bytes(name.as_bytes())?;
>> +            let value = hyper::header::HeaderValue::from_str(&value)?;
>> +            headers.insert(name, value);
>> +        }
>> +
>> +        let mut ssl_connector_builder = SslConnector::builder(SslMethod::tls()).unwrap();
> 
> not sure if this unwrap cannot fail though?
> 

yes, in case the libssl methods it uses fail. changed to bubbling up the 
error (which will probably be rather cryptic, but better than nothing 
;))

>> +        if fingerprint.is_some() {
>> +            // FIXME actually verify fingerprint via callback!
>> +            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::NONE);
>> +        } else {
>> +            ssl_connector_builder.set_verify(openssl::ssl::SslVerifyMode::PEER);
>> +        }
>> +
>> +        let mut httpc = HttpConnector::new();
>> +        httpc.enforce_http(false); // we want https...
>> +        httpc.set_connect_timeout(Some(std::time::Duration::new(10, 0)));
>> +        let https = HttpsConnector::with_connector(httpc, ssl_connector_builder.build(), 120);
>> +
>> +        let client = Client::builder().build::<_, Body>(https);
>> +        let res = client.request(req).await?;
>> +        if res.status() != StatusCode::SWITCHING_PROTOCOLS {
>> +            bail!("server didn't upgrade: {}", res.status());
>> +        }
>> +
>> +        hyper::upgrade::on(res)
>> +            .await
>> +            .map_err(|err| format_err!("failed to upgrade - {}", err))
>> +    }
>> +
>> +    async fn handle_connect_cmd(&mut self, mut data: CmdData) -> Result<(), Error> {
>> +        let mut data: ConnectCmdData = data
>> +            .remove("data")
>> +            .ok_or_else(|| format_err!("'connect' command missing 'data'"))
>> +            .map(serde_json::from_value)??;
>> +
>> +        if self.sender.is_some() {
>> +            bail!("already connected!");
>> +        }
>> +
>> +        let upgraded = Self::websocket_connect(
>> +            data.url.clone(),
>> +            data.headers.take().unwrap_or_else(Vec::new),
>> +            data.fingerprint.take(),
>> +        )
>> +        .await?;
>> +
>> +        let (tx, rx) = mpsc::unbounded_channel();
>> +        self.sender = Some(tx);
>> +        tokio::spawn(async move {
>> +            if let Err(err) = Self::handle_ctrl_tunnel(upgraded, rx).await {
>> +                eprintln!("Tunnel to {} failed - {}", data.url, err);
>> +            }
>> +        });
>> +
>> +        Ok(())
>> +    }
>> +
>> +    async fn handle_forward_cmd(&mut self, mut data: CmdData) -> Result<(), Error> {
>> +        let data: ForwardCmdData = data
>> +            .remove("data")
>> +            .ok_or_else(|| format_err!("'forward' command missing 'data'"))
>> +            .map(serde_json::from_value)??;
>> +
>> +        if self.sender.is_none() && data.ticket.is_some() {
>> +            bail!("dynamically requesting ticket requires cmd tunnel connection!");
>> +        }
>> +
>> +        let unix_listener = UnixListener::bind(data.unix.clone()).unwrap();
> 
> it would be better to bubble the error up instead of unwrapping here
> (AFAICS, the rest of the unwraps are ok, since they cannot really fail?)
> 

changed as well (in addition to the other one you noted, and a third one 
- now only unwrap_or_else remain :)

>> +        let (tx, rx) = oneshot::channel();
>> +        let data = Arc::new(data);
>> +
>> +        self.forwarded.lock().unwrap().push(tx);
> 
> we push the 'tx' here into the forwarded vec, but never use it again
> (no other 'lock()' call in the file)
> 

yeah, dropped together with the rest of the close cmd

>> +        let cmd_sender = self.sender.clone();
>> +
>> +        tokio::spawn(async move {
>> +            let mut rx = rx.fuse();
>> +            let mut tasks: Vec<tokio::task::JoinHandle<()>> = Vec::new();
>> +            loop {
>> +                let accept = unix_listener.accept().fuse();
>> +                tokio::pin!(accept);
>> +                let data2 = data.clone();
>> +                select! {
>> +                    _ = rx => {
>> +                        eprintln!("received shutdown signal, closing unix listener stream and forwarding handlers");
>> +                        for task in tasks {
>> +                            task.abort();
>> +                        }
>> +                        break;
>> +                    },
> 
> which makes this branch dead code
> 
> so i'd drop the forwarded part and simplify this to
> 
> match unix_listener.accept().await {
> ...

done

> }
> 
>> +                    res = accept => match res {
>> +                        Ok((unix_stream, _)) => {
>> +                            eprintln!("accepted new connection on '{}'", data2.unix);
>> +                            let data3: Result<Arc<ForwardCmdData>, Error> = match (&cmd_sender, &data2.ticket) {
>> +                                (Some(cmd_sender), Some(_)) => Self::get_ticket(cmd_sender, data2.clone()).await,\
> 
> the get_ticket could probably be inside the 'handle_forward_tunnel' this 
> way, another client could connect while the first ticket is checked.
> not necessary for now though, since we do not connect in parallel atm
> 

I did it anyway while I was there

>> +                                _ => Ok(data2.clone()),
>> +                            };
>> +
>> +                            match data3 {
>> +                                Ok(data3) => {
>> +                                    let task = tokio::spawn(async move {
>> +                                        if let Err(err) = Self::handle_forward_tunnel(data3.clone(), unix_stream).await {
>> +                                            eprintln!("Tunnel for {} failed - {}", data3.unix, err);
>> +                                        }
>> +                                    });
>> +                                    tasks.push(task);
>> +                                },
>> +                                Err(err) => {
>> +                                    eprintln!("Failed to accept unix connection - {}", err);
>> +                                },
>> +                            };
>> +                        },
>> +                        Err(err) => eprintln!("Failed to accept unix connection on {} - {}", data2.unix, err),
>> +                    },
>> +                };
>> +            }
>> +        });
>> +
>> +        Ok(())
>> +    }
>> +
>> +    async fn handle_tunnel_cmd(&mut self, data: CmdData) -> Result<String, Error> {
>> +        match &mut self.sender {
>> +            None => bail!("not connected!"),
>> +            Some(sender) => {
>> +                let data: Value = data.into();
>> +                let (tx, rx) = oneshot::channel::<String>();
>> +                if let Some(cmd) = data.get("cmd") {
>> +                    eprintln!("-> sending command {} to remote", cmd);
>> +                } else {
>> +                    eprintln!("-> sending data line to remote");
>> +                }
>> +                sender.send((data, tx))?;
>> +                let res = rx.await?;
>> +                eprintln!("<- got reply");
>> +                Ok(res)
>> +            }
>> +        }
>> +    }
>> +
>> +    async fn handle_ctrl_tunnel(
>> +        websocket: Upgraded,
>> +        mut cmd_receiver: mpsc::UnboundedReceiver<(Value, oneshot::Sender<String>)>,
>> +    ) -> Result<(), Error> {
>> +        let (tunnel_reader, tunnel_writer) = tokio::io::split(websocket);
>> +        let (ws_close_tx, mut ws_close_rx) = mpsc::unbounded_channel();
>> +        let ws_reader = WebSocketReader::new(tunnel_reader, ws_close_tx);
>> +        let mut ws_writer = WebSocketWriter::new(Some([0, 0, 0, 0]), tunnel_writer);
>> +
>> +        let mut framed_reader =
>> +            tokio_util::codec::FramedRead::new(ws_reader, tokio_util::codec::LinesCodec::new());
>> +
>> +        let mut resp_tx_queue: VecDeque<oneshot::Sender<String>> = VecDeque::new();
>> +        let mut shutting_down = false;
>> +
>> +        loop {
>> +            let mut close_future = ws_close_rx.recv().boxed().fuse();
>> +            let mut frame_future = framed_reader.next().boxed().fuse();
>> +            let mut cmd_future = cmd_receiver.recv().boxed().fuse();
>> +
>> +            select! {
>> +                res = close_future => {
>> +                    let res = res.ok_or_else(|| format_err!("WS control channel closed"))?;
>> +                    eprintln!("WS: received control message: '{:?}'", res);
>> +                    shutting_down = true;
>> +                },
>> +                res = frame_future => {
>> +                    match res {
>> +                        None if shutting_down => {
>> +                            eprintln!("WS closed");
>> +                            break;
>> +                        },
>> +                        None => bail!("WS closed unexpectedly"),
>> +                        Some(Ok(res)) => {
>> +                            resp_tx_queue
>> +                                .pop_front()
>> +                                .ok_or_else(|| format_err!("no response handler"))?
>> +                                .send(res)
>> +                                .map_err(|msg| format_err!("failed to send tunnel response '{}' back to requester - receiver already closed?", msg))?;
>> +                        },
>> +                        Some(Err(err)) => {
>> +                            bail!("reading from control tunnel failed - WS receive failed: {}", err);
>> +                        },
>> +                    }
>> +                },
>> +                res = cmd_future => {
>> +                    if shutting_down { continue };
>> +                    match res {
>> +                        None => {
>> +                            eprintln!("CMD channel closed, shutting down");
>> +                            ws_writer.send_control_frame(Some([1,2,3,4]), OpCode::Close, &[]).await?;
>> +                            shutting_down = true;
>> +                        },
>> +                        Some((msg, resp_tx)) => {
>> +                            resp_tx_queue.push_back(resp_tx);
>> +
>> +                            let line = format!("{}\n", msg);
>> +                            ws_writer.write_all(line.as_bytes()).await?;
>> +                            ws_writer.flush().await?;
>> +                        },
>> +                    }
>> +                },
>> +            };
>> +        }
>> +
>> +        Ok(())
>> +    }
>> +
>> +    async fn handle_forward_tunnel(
>> +        data: Arc<ForwardCmdData>,
>> +        unix: UnixStream,
>> +    ) -> Result<(), Error> {
>> +        let upgraded = Self::websocket_connect(
>> +            data.url.clone(),
>> +            data.headers.clone().unwrap_or_else(Vec::new),
>> +            data.fingerprint.clone(),
>> +        )
>> +        .await?;
>> +
>> +        let ws = WebSocket {
>> +            mask: Some([0, 0, 0, 0]),
>> +        };
>> +        eprintln!("established new WS for forwarding '{}'", data.unix);
>> +        ws.serve_connection(upgraded, unix).await?;
>> +
>> +        eprintln!("done handling forwarded connection from '{}'", data.unix);
>> +
>> +        Ok(())
>> +    }
>> +
>> +    async fn get_ticket(
>> +        cmd_sender: &mpsc::UnboundedSender<(Value, oneshot::Sender<String>)>,
>> +        cmd_data: Arc<ForwardCmdData>,
>> +    ) -> Result<Arc<ForwardCmdData>, Error> {
>> +        eprintln!("requesting WS ticket via tunnel");
>> +        let ticket_cmd = match cmd_data.ticket.clone() {
>> +            Some(mut ticket_cmd) => {
>> +                ticket_cmd.insert("cmd".to_string(), serde_json::json!("ticket"));
>> +                ticket_cmd
>> +            }
>> +            None => bail!("can't get ticket without ticket parameters"),
>> +        };
>> +        let (tx, rx) = oneshot::channel::<String>();
>> +        cmd_sender.send((serde_json::json!(ticket_cmd), tx))?;
>> +        let ticket = rx.await?;
>> +        let mut ticket: Map<String, Value> = serde_json::from_str(&ticket)?;
>> +        let ticket = ticket
>> +            .remove("ticket")
>> +            .ok_or_else(|| format_err!("failed to retrieve ticket via tunnel"))?;
>> +
>> +        let ticket = ticket
>> +            .as_str()
>> +            .ok_or_else(|| format_err!("failed to format received ticket"))?;
>> +        let ticket = utf8_percent_encode(&ticket, NON_ALPHANUMERIC).to_string();
>> +
>> +        let mut data = cmd_data.clone();
>> +        let mut url = data.url.clone();
>> +        url.push_str("ticket=");
>> +        url.push_str(&ticket);
>> +        let mut d = Arc::make_mut(&mut data);
>> +        d.url = url;
>> +        Ok(data)
>> +    }
>> +}
>> +
>> +#[tokio::main]
>> +async fn main() -> Result<(), Error> {
>> +    do_main().await
>> +}
>> +
>> +async fn do_main() -> Result<(), Error> {
>> +    let tunnel = CtrlTunnel {
>> +        sender: None,
>> +        forwarded: Arc::new(Mutex::new(Vec::new())),
>> +    };
>> +    tunnel.read_cmd_loop().await
>> +}
>> 
> 
> 
> 




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

* Re: [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints
  2021-11-09 12:46   ` Fabian Ebner
  2021-11-10  7:40     ` Fabian Ebner
@ 2021-11-11 11:04     ` Fabian Grünbichler
  1 sibling, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-11 11:04 UTC (permalink / raw)
  To: Fabian Ebner, pve-devel

On November 9, 2021 1:46 pm, Fabian Ebner wrote:
> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
>> the following two endpoints are used for migration on the remote side
>> 
>> POST /nodes/NODE/qemu/VMID/mtunnel
>> 
>> which creates and locks an empty VM config, and spawns the main qmtunnel
>> worker which binds to a VM-specific UNIX socket.
>> 
>> this worker handles JSON-encoded migration commands coming in via this
>> UNIX socket:
>> - config (set target VM config)
>> -- checks permissions for updating config
>> -- strips pending changes and snapshots
>> -- sets (optional) firewall config
>> - disk (allocate disk for NBD migration)
>> -- checks permission for target storage
>> -- returns drive string for allocated volume
>> - disk-import (import 'pvesm export' stream for offline migration)
>> -- checks permission for target storage
>> -- forks a child running 'pvesm import' reading from a UNIX socket
>> -- only one import allowed to run at any given moment
>> - query-disk-import
>> -- checks output of 'pvesm import' for volume ID message
>> -- returns volid + success, or 'pending', or 'error'
>> - start (returning migration info)
>> - fstrim (via agent)
>> - bwlimit (query bwlimit for storage)
>> - ticket (creates a ticket for a WS connection to a specific socket)
>> - resume
>> - stop
>> - nbdstop
>> - unlock
>> - quit (+ cleanup)
>> 
>> this worker serves as a replacement for both 'qm mtunnel' and various
>> manual calls via SSH. the API call will return a ticket valid for
>> connecting to the worker's UNIX socket via a websocket connection.
>> 
>> GET+WebSocket upgrade /nodes/NODE/qemu/VMID/mtunnelwebsocket
>> 
>> gets called for connecting to a UNIX socket via websocket forwarding,
>> i.e. once for the main command mtunnel, and once each for the memory
>> migration and each NBD drive-mirror/storage migration.
>> 
>> access is guarded by a short-lived ticket binding the authenticated user
>> to the socket path. such tickets can be requested over the main mtunnel,
>> which keeps track of socket paths currently used by that
>> mtunnel/migration instance.
>> 
>> each command handler should check privileges for the requested action if
>> necessary.
>> 
>> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
>> ---
>> 
>> Notes:
>>      requires
>>      - pve-storage with UNIX import support
>>      - pve-access-control with tunnel ticket support
>>      - pve-http-server with websocket fixes
>> 
>>   PVE/API2/Qemu.pm | 627 +++++++++++++++++++++++++++++++++++++++++++++++
>>   1 file changed, 627 insertions(+)
>> 
>> diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
>> index faf028b..a1a1813 100644
>> --- a/PVE/API2/Qemu.pm
>> +++ b/PVE/API2/Qemu.pm
>> @@ -6,8 +6,13 @@ use Cwd 'abs_path';
>>   use Net::SSLeay;
>>   use POSIX;
>>   use IO::Socket::IP;
>> +use IO::Socket::UNIX;
>> +use IPC::Open3;
>> +use JSON;
>> +use MIME::Base64;
>>   use URI::Escape;
>>   use Crypt::OpenSSL::Random;
>> +use Socket qw(SOCK_STREAM);
>>   
>>   use PVE::Cluster qw (cfs_read_file cfs_write_file);;
>>   use PVE::RRD;
>> @@ -856,6 +861,7 @@ __PACKAGE__->register_method({
>>   	    { subdir => 'spiceproxy' },
>>   	    { subdir => 'sendkey' },
>>   	    { subdir => 'firewall' },
>> +	    { subdir => 'mtunnel' },
>>   	    ];
>>   
>>   	return $res;
>> @@ -4428,4 +4434,625 @@ __PACKAGE__->register_method({
>>   	return PVE::QemuServer::Cloudinit::dump_cloudinit_config($conf, $param->{vmid}, $param->{type});
>>       }});
>>   
>> +__PACKAGE__->register_method({
>> +    name => 'mtunnel',
>> +    path => '{vmid}/mtunnel',
>> +    method => 'POST',
>> +    protected => 1,
>> +    proxyto => 'node',
>> +    description => 'Migration tunnel endpoint - only for internal use by VM migration.',
>> +    permissions => {
>> +	check => ['perm', '/vms/{vmid}', [ 'VM.Allocate' ]],
>> +	description => "You need 'VM.Allocate' permissions on /vms/{vmid}. Further permission checks happen during the actual migration.",
>> +    },
>> +    parameters => {
>> +	additionalProperties => 0,
>> +	properties => {
>> +	    node => get_standard_option('pve-node'),
>> +	    vmid => get_standard_option('pve-vmid'),
>> +	    storages => {
>> +		type => 'string',
>> +		format => 'pve-storage-id-list',
>> +		optional => 1,
>> +		description => 'List of storages to check permission and availability. Will be checked again for all actually used storages during migration.',
>> +	    },
>> +	},
>> +    },
>> +    returns => {
>> +	additionalProperties => 0,
>> +	properties => {
>> +	    upid => { type => 'string' },
>> +	    ticket => { type => 'string' },
>> +	    socket => { type => 'string' },
>> +	},
>> +    },
>> +    code => sub {
>> +	my ($param) = @_;
>> +
>> +	my $rpcenv = PVE::RPCEnvironment::get();
>> +	my $authuser = $rpcenv->get_user();
>> +
>> +	my $node = extract_param($param, 'node');
>> +	my $vmid = extract_param($param, 'vmid');
>> +
>> +	my $storages = extract_param($param, 'storages');
>> +
>> +	my $storecfg = PVE::Storage::config();
>> +	foreach my $storeid (PVE::Tools::split_list($storages)) {
>> +	    $check_storage_access_migrate->($rpcenv, $authuser, $storecfg, $storeid, $node);
>> +	}
>> +
>> +	PVE::Cluster::check_cfs_quorum();
>> +
>> +	my $socket_addr = "/run/qemu-server/$vmid.mtunnel";
>> +
>> +	my $lock = 'create';
>> +	eval { PVE::QemuConfig->create_and_lock_config($vmid, 0, $lock); };
>> +
>> +	raise_param_exc({ vmid => "unable to create empty VM config - $@"})
>> +	    if $@;
>> +
>> +	my $realcmd = sub {
>> +	    my $pveproxy_uid;
>> +
>> +	    my $state = {
>> +		storecfg => PVE::Storage::config(),
>> +		lock => $lock,
>> +	    };
>> +
>> +	    my $run_locked = sub {
>> +		my ($code, $params) = @_;
>> +		return PVE::QemuConfig->lock_config($vmid, sub {
>> +		    my $conf = PVE::QemuConfig->load_config($vmid);
>> +
>> +		    $state->{conf} = $conf;
>> +
>> +		    die "Encountered wrong lock - aborting mtunnel command handling.\n"
>> +			if $state->{lock} && !PVE::QemuConfig->has_lock($conf, $state->{lock});
>> +
>> +		    return $code->($params);
>> +		});
>> +	    };
>> +
>> +	    my $cmd_desc = {
>> +		bwlimit => {
>> +		    storage => {
>> +			type => 'string',
>> +			format => 'pve-storage-id',
>> +			description => "Storage for which bwlimit is queried",
>> +		    },
>> +		    bwlimit => {
>> +			description => "Override I/O bandwidth limit (in KiB/s).",
>> +			optional => 1,
>> +			type => 'integer',
>> +			minimum => '0',
>> +		    },
>> +		},
>> +		config => {
>> +		    conf => {
>> +			type => 'string',
>> +			description => 'Full VM config, adapted for target cluster/node',
>> +		    },
>> +		    'firewall-conf' => {
> 
> Here and thus for parsing, it's 'firewall-conf', but in the command 
> handler 'firewall-config' is accessed.
> 

thanks! the joys of additionalproperties defaulting to 1 ;)

>> +			type => 'string',
>> +			description => 'VM firewall config',
>> +			optional => 1,
>> +		    },
>> +		},
>> +		disk => {
>> +		    format => PVE::JSONSchema::get_standard_option('pve-qm-image-format'),
>> +		    storage => {
>> +			type => 'string',
>> +			format => 'pve-storage-id',
>> +		    },
>> +		    drive => {
>> +			type => 'object',
>> +			description => 'parsed drive information without volid and format',
>> +		    },
>> +		},
>> +		'disk-import' => {
>> +		    volname => {
>> +			type => 'string',
>> +			description => 'volume name to use prefered target volume name',
> 
> Nit: I wasn't able to parse this description ;) (also missing r in 
> preferred)
> 

probably because it's missing an 'as':

'volume name to use as preferred target volume name'

as in, we try to keep that name, but if it's already taken you get a 
different one if allow-rename is set, or an error other wise ;)

>> +		    },
>> +		    format => PVE::JSONSchema::get_standard_option('pve-qm-image-format'),
>> +		    'export-formats' => {
>> +			type => 'string',
>> +			description => 'list of supported export formats',
>> +		    },
>> +		    storage => {
>> +			type => 'string',
>> +			format => 'pve-storage-id',
>> +		    },
>> +		    'with-snapshots' => {
>> +			description =>
>> +			    "Whether the stream includes intermediate snapshots",
>> +			type => 'boolean',
>> +			optional => 1,
>> +			default => 0,
>> +		    },
>> +		    'allow-rename' => {
>> +			description => "Choose a new volume ID if the requested " .
>> +			  "volume ID already exists, instead of throwing an error.",
>> +			type => 'boolean',
>> +			optional => 1,
>> +			default => 0,
>> +		    },
>> +		},
>> +		start => {
>> +		    start_params => {
>> +			type => 'object',
>> +			description => 'params passed to vm_start_nolock',
>> +		    },
>> +		    migrate_opts => {
>> +			type => 'object',
>> +			description => 'migrate_opts passed to vm_start_nolock',
>> +		    },
>> +		},
>> +		ticket => {
>> +		    path => {
>> +			type => 'string',
>> +			description => 'socket path for which the ticket should be valid. must be known to current mtunnel instance.',
>> +		    },
>> +		},
>> +		quit => {
>> +		    cleanup => {
>> +			type => 'boolean',
>> +			description => 'remove VM config and disks, aborting migration',
>> +			default => 0,
>> +		    },
>> +		},
>> +	    };
>> +
>> +	    my $cmd_handlers = {
>> +		'version' => sub {
>> +		    # compared against other end's version
>> +		    # bump/reset both for breaking changes
>> +		    # bump tunnel only for opt-in changes
>> +		    return {
>> +			api => 2,
>> +			age => 0,
>> +		    };
>> +		},
>> +		'config' => sub {
>> +		    my ($params) = @_;
>> +
>> +		    # parse and write out VM FW config if given
>> +		    if (my $fw_conf = $params->{'firewall-config'}) {
>> +			my ($path, $fh) = PVE::Tools::tempfile_contents($fw_conf, 700);
>> +
>> +			my $empty_conf = {
>> +			    rules => [],
>> +			    options => {},
>> +			    aliases => {},
>> +			    ipset => {} ,
>> +			    ipset_comments => {},
>> +			};
>> +			my $cluster_fw_conf = PVE::Firewall::load_clusterfw_conf();
>> +
>> +			# TODO: add flag for strict parsing?
>> +			# TODO: add import sub that does all this given raw content?
>> +			my $vmfw_conf = PVE::Firewall::generic_fw_config_parser($path, $cluster_fw_conf, $empty_conf, 'vm');
>> +			$vmfw_conf->{vmid} = $vmid;
>> +			PVE::Firewall::save_vmfw_conf($vmid, $vmfw_conf);
>> +
>> +			$state->{cleanup}->{fw} = 1;
>> +		    }
>> +
>> +		    PVE::QemuConfig->remove_lock($vmid, 'create');
>> +
>> +		    # TODO add flag for strict parsing?
>> +		    my $new_conf = PVE::QemuServer::parse_vm_config("incoming/qemu-server/$vmid.conf", $params->{conf});
>> +		    delete $new_conf->{lock};
>> +		    delete $new_conf->{digest};
>> +
>> +		    # TODO handle properly?
>> +		    delete $new_conf->{snapshots};
>> +		    delete $new_conf->{pending};
> 
> 'parent' should also be deleted if the snapshots are.
> 

yes

>> +
>> +		    # not handled by update_vm_api
>> +		    my $vmgenid = delete $new_conf->{vmgenid};
>> +		    my $meta = delete $new_conf->{meta};
>> +
>> +		    $new_conf->{vmid} = $vmid;
>> +		    $new_conf->{node} = $node;
>> +
>> +		    $update_vm_api->($new_conf, 1);
>> +
>> +		    my $conf = PVE::QemuConfig->load_config($vmid);
>> +		    $conf->{lock} = 'migrate';
>> +		    $conf->{vmgenid} = $vmgenid;
>> +		    $conf->{meta} = $meta;
>> +		    PVE::QemuConfig->write_config($vmid, $conf);
>> +
>> +		    $state->{lock} = 'migrate';
>> +
>> +		    return;
>> +		},
>> +		'bwlimit' => sub {
>> +		    my ($params) = @_;
>> +
>> +		    my $bwlimit = PVE::Storage::get_bandwidth_limit('migration', [$params->{storage}], $params->{bwlimit});
>> +		    return { bwlimit => $bwlimit };
>> +
>> +		},
>> +		'disk' => sub {
>> +		    my ($params) = @_;
> 
> Feels like some deduplication between here and 
> vm_migrate_alloc_nbd_disks should be possible.
> 

yes, I seem to have forgotten to do that (this series predates 
vm_migrate_alloc_nbd_disks, but I now remember thinking back then that 
this is a good addition and I should fold it in)

adapted it a bit and merged the two.

>> +
>> +		    my $format = $params->{format};
>> +		    my $storeid = $params->{storage};
>> +		    my $drive = $params->{drive};
>> +
>> +		    $check_storage_access_migrate->($rpcenv, $authuser, $state->{storecfg}, $storeid, $node);
>> +
>> +		    my ($default_format, $valid_formats) = PVE::Storage::storage_default_format($state->{storecfg}, $storeid);
>> +		    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
>> +		    $format = $default_format
>> +			if !grep {$format eq $_} @{$valid_formats};
>> +
>> +		    my $size = int($drive->{size})/1024;
>> +		    my $newvolid = PVE::Storage::vdisk_alloc($state->{storecfg}, $storeid, $vmid, $format, undef, $size);
>> +
>> +		    my $newdrive = $drive;
>> +		    $newdrive->{format} = $format;
>> +		    $newdrive->{file} = $newvolid;
>> +
>> +		    $state->{cleanup}->{volumes}->{$newvolid} = 1;
>> +		    my $drivestr = PVE::QemuServer::print_drive($newdrive);
>> +		    return {
>> +			drivestr => $drivestr,
>> +			volid => $newvolid,
>> +		    };
>> +		},
>> +		'disk-import' => sub {
>> +		    my ($params) = @_;
> 
> Similarly here with storage_migrate. Having the checks and deciding on 
> name+format be its own function would also make it possible to abort 
> early, which is especially useful if there are multiple disks. But would 
> require a precondition handler for remote migration of course.
> 

yeah, this part (and some of the counterpart in QemuMigrate) will move 
to the storage layer one way or another for re-using in pve-container 
and the replication code.

>> +
>> +		    die "disk import already running as PID '$state->{disk_import}->{pid}'\n"
>> +			if $state->{disk_import}->{pid};
>> +
>> +		    my $format = $params->{format};
>> +		    my $storeid = $params->{storage};
>> +		    $check_storage_access_migrate->($rpcenv, $authuser, $state->{storecfg}, $storeid, $node);
>> +
>> +		    my $with_snapshots = $params->{'with-snapshots'} ? 1 : 0;
>> +
>> +		    my ($default_format, $valid_formats) = PVE::Storage::storage_default_format($state->{storecfg}, $storeid);
>> +		    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
>> +		    die "unsupported format '$format' for storage '$storeid'\n"
>> +			if !grep {$format eq $_} @{$valid_formats};
>> +
>> +		    my $volname = $params->{volname};
>> +
>> +		    # get target volname, taken from PVE::Storage
>> +		    (my $name_without_extension = $volname) =~ s/\.$format$//;
>> +		    if ($scfg->{path}) {
>> +			$volname = "$vmid/$name_without_extension.$format";
>> +		    } else {
>> +			$volname = "$name_without_extension";
>> +		    }
> 
> This is just a best-effort for guessing a valid volname that was 
> intended only as a fall-back when target and source storage have 
> different types. If the storage type is the same, the volname should be 
> kept, so that e.g. an external plugin with $scfg->{path} and no 
> extension also works.

but we don't have a guarantee that type foo on cluster A and type foo on 
cluster B are identical, support the same formats, etc. (might be a 
different version with different support, or a different plugin 
altogether). I think this part can improve when we improve our name 
handling in general, but I'd leave it like it is atm..

>> +
>> +		    my $migration_snapshot;
>> +		    if ($scfg->{type} eq 'zfspool' || $scfg->{type} eq 'btrfs') {
>> +			$migration_snapshot = '__migration__';
>> +		    }
>> +
>> +		    my $volid = "$storeid:$volname";
>> +
>> +		    # find common import/export format, taken from PVE::Storage
>> +		    my @import_formats = PVE::Storage::volume_import_formats($state->{storecfg}, $volid, $migration_snapshot, undef, $with_snapshots);
>> +		    my @export_formats = PVE::Tools::split_list($params->{'export-formats'});
>> +		    my %import_hash = map { $_ => 1 } @import_formats;
>> +		    my @common = grep { $import_hash{$_} } @export_formats;
>> +		    die "no matching import/export format found for storage '$storeid'\n"
>> +			if !@common;
>> +		    $format = $common[0];
>> +
>> +		    my $input = IO::File->new();
>> +		    my $info = IO::File->new();
>> +		    my $unix = "/run/qemu-server/$vmid.storage";
>> +
>> +		    my $import_cmd = ['pvesm', 'import', $volid, $format, "unix://$unix", '-with-snapshots', $with_snapshots];
>> +		    if ($params->{'allow-rename'}) {
>> +			push @$import_cmd, '-allow-rename', $params->{'allow-rename'};
>> +		    }
>> +		    if ($migration_snapshot) {
>> +			push @$import_cmd, '-delete-snapshot', $migration_snapshot;
> 
> Missing '-snapshot $migration_snapshot'? While the parameter is ignored 
> by our ZFSPoolPlugin, the BTRFSPlugin aborts if it's not specified 
> AFAICS. And external plugins might require it too.

done

> 
> In general, we'll need to be careful not to introduce mismatches between 
> the import and the export parameters. Might it be better if the client 
> would pass along (most of) the parameters for the import command (which 
> basically is how it's done for the existing storage_migrate)?
> 

see next mail

>> +		    }
>> +
>> +		    unlink $unix;
>> +		    my $cpid = open3($input, $info, $info, @{$import_cmd})
>> +			or die "failed to spawn disk-import child - $!\n";
>> +
>> +		    $state->{disk_import}->{pid} = $cpid;
>> +		    my $ready;
>> +		    eval {
>> +			PVE::Tools::run_with_timeout(5, sub { $ready = <$info>; });
>> +		    };
>> +		    die "failed to read readyness from disk import child: $@\n" if $@;
>> +		    print "$ready\n";
>> +
>> +		    chown $pveproxy_uid, -1, $unix;
>> +
>> +		    $state->{disk_import}->{fh} = $info;
>> +		    $state->{disk_import}->{socket} = $unix;
>> +
>> +		    $state->{sockets}->{$unix} = 1;
>> +
>> +		    return {
>> +			socket => $unix,
>> +			format => $format,
>> +		    };
>> +		},
>> +		'query-disk-import' => sub {
>> +		    my ($params) = @_;
>> +
>> +		    die "no disk import running\n"
>> +			if !$state->{disk_import}->{pid};
>> +
>> +		    my $pattern = PVE::Storage::volume_imported_message(undef, 1);
>> +		    my $result;
>> +		    eval {
>> +			my $fh = $state->{disk_import}->{fh};
>> +			PVE::Tools::run_with_timeout(5, sub { $result = <$fh>; });
>> +			print "disk-import: $result\n" if $result;
>> +		    };
>> +		    if ($result && $result =~ $pattern) {
>> +			my $volid = $1;
>> +			waitpid($state->{disk_import}->{pid}, 0);
>> +
>> +			my $unix = $state->{disk_import}->{socket};
>> +			unlink $unix;
>> +			delete $state->{sockets}->{$unix};
>> +			delete $state->{disk_import};
> 
> $volid should be registered for potential cleanup.
> 

done

>> +			return {
>> +			    status => "complete",
>> +			    volid => $volid,
>> +			};
>> +		    } elsif (!$result && waitpid($state->{disk_import}->{pid}, WNOHANG)) {
>> +			my $unix = $state->{disk_import}->{socket};
>> +			unlink $unix;
>> +			delete $state->{sockets}->{$unix};
>> +			delete $state->{disk_import};
>> +
>> +			return {
>> +			    status => "error",
>> +			};
>> +		    } else {
>> +			return {
>> +			    status => "pending",
>> +			};
>> +		    }
>> +		},
>> +		'start' => sub {
>> +		    my ($params) = @_;
>> +
>> +		    my $info = PVE::QemuServer::vm_start_nolock(
>> +			$state->{storecfg},
>> +			$vmid,
>> +			$state->{conf},
>> +			$params->{start_params},
>> +			$params->{migrate_opts},
>> +		    );
>> +
>> +
>> +		    if ($info->{migrate}->{proto} ne 'unix') {
>> +			PVE::QemuServer::vm_stop(undef, $vmid, 1, 1);
>> +			die "migration over non-UNIX sockets not possible\n";
>> +		    }
>> +
>> +		    my $socket = $info->{migrate}->{addr};
>> +		    chown $pveproxy_uid, -1, $socket;
>> +		    $state->{sockets}->{$socket} = 1;
>> +
>> +		    my $unix_sockets = $info->{migrate}->{unix_sockets};
>> +		    foreach my $socket (@$unix_sockets) {
>> +			chown $pveproxy_uid, -1, $socket;
>> +			$state->{sockets}->{$socket} = 1;
>> +		    }
>> +		    return $info;
>> +		},
>> +		'fstrim' => sub {
>> +		    if (PVE::QemuServer::qga_check_running($vmid)) {
>> +			eval { mon_cmd($vmid, "guest-fstrim") };
>> +			warn "fstrim failed: $@\n" if $@;
>> +		    }
>> +		    return;
>> +		},
>> +		'stop' => sub {
>> +		    PVE::QemuServer::vm_stop(undef, $vmid, 1, 1);
>> +		    return;
>> +		},
>> +		'nbdstop' => sub {
>> +		    PVE::QemuServer::nbd_stop($vmid);
>> +		    return;
>> +		},
>> +		'resume' => sub {
>> +		    if (PVE::QemuServer::check_running($vmid, 1)) {
>> +			PVE::QemuServer::vm_resume($vmid, 1, 1);
>> +		    } else {
>> +			die "VM $vmid not running\n";
>> +		    }
>> +		    return;
>> +		},
>> +		'unlock' => sub {
>> +		    PVE::QemuConfig->remove_lock($vmid, $state->{lock});
>> +		    delete $state->{lock};
>> +		    return;
>> +		},
>> +		'ticket' => sub {
>> +		    my ($params) = @_;
>> +
>> +		    my $path = $params->{path};
>> +
>> +		    die "Not allowed to generate ticket for unknown socket '$path'\n"
>> +			if !defined($state->{sockets}->{$path});
>> +
>> +		    return { ticket => PVE::AccessControl::assemble_tunnel_ticket($authuser, "/socket/$path") };
>> +		},
>> +		'quit' => sub {
>> +		    my ($params) = @_;
>> +
>> +		    if ($params->{cleanup}) {
>> +			if ($state->{cleanup}->{fw}) {
>> +			    PVE::Firewall::remove_vmfw_conf($vmid);
>> +			}
>> +
>> +			if (my @volumes = keys $state->{cleanup}->{volumes}->$%) {
> 
> keys on scalar? This is fixed in a later patch, but...

yeah, that was a rebase gone wrong ;)

> 
>> +			    PVE::Storage::foreach_volid(@volumes, sub {
> 
> ...PVE::Storage::foreach_volid does not have this signature. It needs 
> what vdisk_list returns. A simple 'for' should be good enough here.
> 

ack, I guess that was the source of a stray volume I had in one of my 
last tests..

>> +				my ($volid, $sid, $volname, $d) = @_;
>> +
>> +				print "freeing volume '$volid' as part of cleanup\n";
>> +				eval { PVE::Storage::vdisk_free($storecfg, $volid) };
>> +				warn $@ if $@;
>> +			    });
>> +			}
>> +
>> +			PVE::QemuServer::destroy_vm($state->{storecfg}, $vmid, 1);
>> +		    }
>> +
>> +		    $state->{exit} = 1;
>> +		    return;
>> +		},
>> +	    };
>> +
>> +	    $run_locked->(sub {
>> +		my $socket_addr = "/run/qemu-server/$vmid.mtunnel";
>> +		unlink $socket_addr;
>> +
>> +		$state->{socket} = IO::Socket::UNIX->new(
>> +	            Type => SOCK_STREAM(),
>> +		    Local => $socket_addr,
>> +		    Listen => 1,
>> +		);
>> +
>> +		$pveproxy_uid = getpwnam('www-data')
>> +		    or die "Failed to resolve user 'www-data' to numeric UID\n";
>> +		chown $pveproxy_uid, -1, $socket_addr;
>> +	    });
>> +
>> +	    print "mtunnel started\n";
>> +
>> +	    my $conn = $state->{socket}->accept();
>> +
>> +	    $state->{conn} = $conn;
>> +
>> +	    my $reply_err = sub {
>> +		my ($msg) = @_;
>> +
>> +		my $reply = JSON::encode_json({
>> +		    success => JSON::false,
>> +		    msg => $msg,
>> +		});
>> +		$conn->print("$reply\n");
>> +		$conn->flush();
>> +	    };
>> +
>> +	    my $reply_ok = sub {
>> +		my ($res) = @_;
>> +
>> +		$res->{success} = JSON::true;
>> +		my $reply = JSON::encode_json($res);
>> +		$conn->print("$reply\n");
>> +		$conn->flush();
>> +	    };
>> +
>> +	    while (my $line = <$conn>) {
>> +		chomp $line;
>> +
>> +		# untaint, we validate below if needed
>> +		($line) = $line =~ /^(.*)$/;
>> +		my $parsed = eval { JSON::decode_json($line) };
>> +		if ($@) {
>> +		    $reply_err->("failed to parse command - $@");
>> +		    next;
>> +		}
>> +
>> +		my $cmd = delete $parsed->{cmd};
>> +		if (!defined($cmd)) {
>> +		    $reply_err->("'cmd' missing");
>> +		} elsif (my $handler = $cmd_handlers->{$cmd}) {
>> +		    print "received command '$cmd'\n";
>> +		    eval {
>> +			if ($cmd_desc->{$cmd}) {
>> +			    PVE::JSONSchema::validate($cmd_desc->{$cmd}, $parsed);
>> +			} else {
>> +			    $parsed = {};
>> +			}
>> +			my $res = $run_locked->($handler, $parsed);
>> +			$reply_ok->($res);
>> +		    };
>> +		    $reply_err->("failed to handle '$cmd' command - $@")
>> +			if $@;
>> +		} else {
>> +		    $reply_err->("unknown command '$cmd' given");
>> +		}
>> +
>> +		if ($state->{exit}) {
>> +		    $state->{conn}->close();
>> +		    $state->{socket}->close();
>> +		    last;
>> +		}
>> +	    }
>> +
>> +	    print "mtunnel exited\n";
>> +	};
>> +
>> +	my $ticket = PVE::AccessControl::assemble_tunnel_ticket($authuser, "/socket/$socket_addr");
>> +	my $upid = $rpcenv->fork_worker('qmtunnel', $vmid, $authuser, $realcmd);
>> +
>> +	return {
>> +	    ticket => $ticket,
>> +	    upid => $upid,
>> +	    socket => $socket_addr,
>> +	};
>> +    }});
>> +
>> +__PACKAGE__->register_method({
>> +    name => 'mtunnelwebsocket',
>> +    path => '{vmid}/mtunnelwebsocket',
>> +    method => 'GET',
>> +    proxyto => 'node',
>> +    permissions => {
>> +	description => "You need to pass a ticket valid for the selected socket. Tickets can be created via the mtunnel API call, which will check permissions accordingly.",
>> +        user => 'all', # check inside
>> +    },
>> +    description => 'Migration tunnel endpoint for websocket upgrade - only for internal use by VM migration.',
>> +    parameters => {
>> +	additionalProperties => 0,
>> +	properties => {
>> +	    node => get_standard_option('pve-node'),
>> +	    vmid => get_standard_option('pve-vmid'),
>> +	    socket => {
>> +		type => "string",
>> +		description => "unix socket to forward to",
>> +	    },
>> +	    ticket => {
>> +		type => "string",
>> +		description => "ticket return by initial 'mtunnel' API call, or retrieved via 'ticket' tunnel command",
>> +	    },
>> +	},
>> +    },
>> +    returns => {
>> +	type => "object",
>> +	properties => {
>> +	    port => { type => 'string', optional => 1 },
>> +	    socket => { type => 'string', optional => 1 },
>> +	},
>> +    },
>> +    code => sub {
>> +	my ($param) = @_;
>> +
>> +	my $rpcenv = PVE::RPCEnvironment::get();
>> +	my $authuser = $rpcenv->get_user();
>> +
>> +	my $vmid = $param->{vmid};
>> +	# check VM exists
>> +	PVE::QemuConfig->load_config($vmid);
>> +
>> +	my $socket = $param->{socket};
>> +	PVE::AccessControl::verify_tunnel_ticket($param->{ticket}, $authuser, "/socket/$socket");
>> +
>> +	return { socket => $socket };
>> +    }});
>> +
>>   1;
>> 
> 




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

* Re: [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints
  2021-11-10  7:40     ` Fabian Ebner
@ 2021-11-11 11:07       ` Fabian Grünbichler
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-11 11:07 UTC (permalink / raw)
  To: Fabian Ebner, pve-devel

On November 10, 2021 8:40 am, Fabian Ebner wrote:
> Am 09.11.21 um 13:46 schrieb Fabian Ebner:
>> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
> 
> ---snip---
> 
>>>   use IO::Socket::IP;
>>> +use IO::Socket::UNIX;
>>> +use IPC::Open3;
>>> +use JSON;
>>> +use MIME::Base64;
> 
> Forgot to ask: is this import needed or a left-over from development?

yes

> 
> ---snip---
> 
>> 
>>> +
>>> +            my $migration_snapshot;
>>> +            if ($scfg->{type} eq 'zfspool' || $scfg->{type} eq 
>>> 'btrfs') {
>>> +            $migration_snapshot = '__migration__';
>>> +            }
>>> +
>>> +            my $volid = "$storeid:$volname";
>>> +
>>> +            # find common import/export format, taken from PVE::Storage
>>> +            my @import_formats = 
>>> PVE::Storage::volume_import_formats($state->{storecfg}, $volid, 
>>> $migration_snapshot, undef, $with_snapshots);
>>> +            my @export_formats = 
>>> PVE::Tools::split_list($params->{'export-formats'});
>>> +            my %import_hash = map { $_ => 1 } @import_formats;
>>> +            my @common = grep { $import_hash{$_} } @export_formats;
>>> +            die "no matching import/export format found for storage 
>>> '$storeid'\n"
>>> +            if !@common;
>>> +            $format = $common[0];
>>> +
>>> +            my $input = IO::File->new();
>>> +            my $info = IO::File->new();
>>> +            my $unix = "/run/qemu-server/$vmid.storage";
>>> +
>>> +            my $import_cmd = ['pvesm', 'import', $volid, $format, 
>>> "unix://$unix", '-with-snapshots', $with_snapshots];
>>> +            if ($params->{'allow-rename'}) {
>>> +            push @$import_cmd, '-allow-rename', 
>>> $params->{'allow-rename'};
>>> +            }
>>> +            if ($migration_snapshot) {
>>> +            push @$import_cmd, '-delete-snapshot', $migration_snapshot;
>> 
>> Missing '-snapshot $migration_snapshot'? While the parameter is ignored 
>> by our ZFSPoolPlugin, the BTRFSPlugin aborts if it's not specified 
>> AFAICS. And external plugins might require it too.
> 
> That is, for the 'btrfs' format. In the patch with the export command, a 
> snapshot is only used for ZFS, so it would already fail on export for 
> BTRFS with 'btrfs' format. For external plugins we also don't use a 
> migration snapshot in storage_migrate(), so please disregard that part.

done

> 
>> 
>> In general, we'll need to be careful not to introduce mismatches between 
>> the import and the export parameters. Might it be better if the client 
>> would pass along (most of) the parameters for the import command (which 
>> basically is how it's done for the existing storage_migrate)?
>> 
> 
> On the other hand, that would require being very careful with input 
> validation.

yeah, and since we are crossing a trust boundary here (between two 
clusters) we have to be careful. if we change the export/import code, we 
can always also bump the tunnel API if needed (either to selectively use 
new features only if supported, or to error out early if there was a 
breaking change). just passing in "pass this to `pvesm import`" is 
potentially dangerous if we don't carefully validate the 'this', and 
that is easier if it's structured data ;) so I'd rather do this explicit 
even if it means extending two places when we change the interface.

> 
> ---snip---
> 




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

* Re: [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling
  2021-11-10 11:17   ` Fabian Ebner
@ 2021-11-11 12:25     ` Fabian Grünbichler
  2021-11-11 12:57       ` Fabian Ebner
  0 siblings, 1 reply; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-11 12:25 UTC (permalink / raw)
  To: Fabian Ebner, pve-devel

On November 10, 2021 12:17 pm, Fabian Ebner wrote:
> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
>> remote migration uses a websocket connection to a task worker running on
>> the target node instead of commands via SSH to control the migration.
>> this websocket tunnel is started earlier than the SSH tunnel, and allows
>> adding UNIX-socket forwarding over additional websocket connections
>> on-demand.
>> 
>> the main differences to regular intra-cluster migration are:
>> - source VM config and disks are only removed upon request via --delete
>> - shared storages are treated like local storages, since we can't
>> assume they are shared across clusters (with potentical to extend this
>> by marking storages as shared)
>> - NBD migrated disks are explicitly pre-allocated on the target node via
>> tunnel command before starting the target VM instance
>> - in addition to storages, network bridges and the VMID itself is
>> transformed via a user defined mapping
>> - all commands and migration data streams are sent via a WS tunnel proxy
>> 
>> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
>> ---
>> 
>> Notes:
>>      requires proxmox-websocket-tunnel
>> 
>>   PVE/API2/Qemu.pm   |   4 +-
>>   PVE/QemuMigrate.pm | 647 +++++++++++++++++++++++++++++++++++++++------
>>   PVE/QemuServer.pm  |   8 +-
>>   3 files changed, 575 insertions(+), 84 deletions(-)
>> 
>> diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
>> index a1a1813..24f5b98 100644
>> --- a/PVE/API2/Qemu.pm
>> +++ b/PVE/API2/Qemu.pm
>> @@ -4610,7 +4610,7 @@ __PACKAGE__->register_method({
>>   		    # bump/reset both for breaking changes
>>   		    # bump tunnel only for opt-in changes
> 
> Sorry for asking about this on this patch: shouldn't opt-in changes bump 
> both?

yes. this was initially version and min version, then changed to version 
and age like with the storage plugins..

> 
>>   		    return {
>> -			api => 2,
>> +			api => $PVE::QemuMigrate::WS_TUNNEL_VERSION,
>>   			age => 0,
>>   		    };
>>   		},
>> @@ -4897,7 +4897,7 @@ __PACKAGE__->register_method({
>>   			    PVE::Firewall::remove_vmfw_conf($vmid);
>>   			}
>>   
>> -			if (my @volumes = keys $state->{cleanup}->{volumes}->$%) {
>> +			if (my @volumes = keys $state->{cleanup}->{volumes}->%*) {
>>   			    PVE::Storage::foreach_volid(@volumes, sub {
>>   				my ($volid, $sid, $volname, $d) = @_;
>>   
>> diff --git a/PVE/QemuMigrate.pm b/PVE/QemuMigrate.pm
>> index 07b56eb..7378551 100644
>> --- a/PVE/QemuMigrate.pm
>> +++ b/PVE/QemuMigrate.pm
>> @@ -7,9 +7,15 @@ use IO::File;
>>   use IPC::Open2;
>>   use POSIX qw( WNOHANG );
>>   use Time::HiRes qw( usleep );
>> +use JSON qw(encode_json decode_json);
>> +use IO::Socket::UNIX;
>> +use Socket qw(SOCK_STREAM);
>> +use Storable qw(dclone);
>> +use URI::Escape;
>>   
>> -use PVE::Format qw(render_bytes);
>> +use PVE::APIClient::LWP;
>>   use PVE::Cluster;
>> +use PVE::Format qw(render_bytes);
>>   use PVE::GuestHelpers qw(safe_boolean_ne safe_string_ne);
>>   use PVE::INotify;
>>   use PVE::RPCEnvironment;
>> @@ -30,6 +36,9 @@ use PVE::QemuServer;
>>   use PVE::AbstractMigrate;
>>   use base qw(PVE::AbstractMigrate);
>>   
>> +# compared against remote end's minimum version
>> +our $WS_TUNNEL_VERSION = 2;
>> +
>>   sub fork_command_pipe {
>>       my ($self, $cmd) = @_;
>>   
>> @@ -85,7 +94,7 @@ sub finish_command_pipe {
>>   	}
>>       }
>>   
>> -    $self->log('info', "ssh tunnel still running - terminating now with SIGTERM\n");
>> +    $self->log('info', "tunnel still running - terminating now with SIGTERM\n");
>>       kill(15, $cpid);
>>   
>>       # wait again
>> @@ -94,11 +103,11 @@ sub finish_command_pipe {
>>   	sleep(1);
>>       }
>>   
>> -    $self->log('info', "ssh tunnel still running - terminating now with SIGKILL\n");
>> +    $self->log('info', "tunnel still running - terminating now with SIGKILL\n");
>>       kill 9, $cpid;
>>       sleep 1;
>>   
>> -    $self->log('err', "ssh tunnel child process (PID $cpid) couldn't be collected\n")
>> +    $self->log('err', "tunnel child process (PID $cpid) couldn't be collected\n")
>>   	if !&$collect_child_process();
>>   }
>>   
>> @@ -115,18 +124,28 @@ sub read_tunnel {
>>       };
>>       die "reading from tunnel failed: $@\n" if $@;
>>   
>> -    chomp $output;
>> +    chomp $output if defined($output);
>>   
>>       return $output;
>>   }
>>   
>>   sub write_tunnel {
>> -    my ($self, $tunnel, $timeout, $command) = @_;
>> +    my ($self, $tunnel, $timeout, $command, $params) = @_;
>>   
>>       $timeout = 60 if !defined($timeout);
>>   
>>       my $writer = $tunnel->{writer};
>>   
>> +    if ($tunnel->{version} && $tunnel->{version} >= 2) {
>> +	my $object = defined($params) ? dclone($params) : {};
>> +	$object->{cmd} = $command;
>> +
>> +	$command = eval { JSON::encode_json($object) };
>> + > +	die "failed to encode command as JSON - $@\n"
>> +	    if $@;
>> +    }
>> +
>>       eval {
>>   	PVE::Tools::run_with_timeout($timeout, sub {
>>   	    print $writer "$command\n";
>> @@ -136,13 +155,29 @@ sub write_tunnel {
>>       die "writing to tunnel failed: $@\n" if $@;
>>   
>>       if ($tunnel->{version} && $tunnel->{version} >= 1) {
>> -	my $res = eval { $self->read_tunnel($tunnel, 10); };
>> +	my $res = eval { $self->read_tunnel($tunnel, $timeout); };
>>   	die "no reply to command '$command': $@\n" if $@;
>>   
>> -	if ($res eq 'OK') {
>> -	    return;
>> +	if ($tunnel->{version} == 1) {
>> +	    if ($res eq 'OK') {
>> +		return;
>> +	    } else {
>> +		die "tunnel replied '$res' to command '$command'\n";
>> +	    }
>>   	} else {
>> -	    die "tunnel replied '$res' to command '$command'\n";
>> +	    my $parsed = eval { JSON::decode_json($res) };
>> +	    die "failed to decode tunnel reply '$res' (command '$command') - $@\n"
>> +		if $@;
>> +
>> +	    if (!$parsed->{success}) {
>> +		if (defined($parsed->{msg})) {
>> +		    die "error - tunnel command '$command' failed - $parsed->{msg}\n";
>> +		} else {
>> +		    die "error - tunnel command '$command' failed\n";
>> +		}
>> +	    }
>> +
>> +	    return $parsed;
>>   	}
>>       }
>>   }
>> @@ -185,10 +220,150 @@ sub fork_tunnel {
>>       return $tunnel;
>>   }
>>   
>> +my $forward_unix_socket = sub {
>> +    my ($self, $local, $remote) = @_;
>> +
>> +    my $params = dclone($self->{tunnel}->{params});
>> +    $params->{unix} = $local;
>> +    $params->{url} = $params->{url} ."socket=$remote&";
>> +    $params->{ticket} = { path => $remote };
>> +
>> +    my $cmd = encode_json({
>> +	control => JSON::true,
>> +	cmd => 'forward',
>> +	data => $params,
>> +    });
>> +
>> +    my $writer = $self->{tunnel}->{writer};
>> +    eval {
>> +	unlink $local;
>> +	PVE::Tools::run_with_timeout(15, sub {
>> +	    print $writer "$cmd\n";
>> +	    $writer->flush();
>> +	});
>> +    };
>> +    die "failed to write forwarding command - $@\n" if $@;
>> +
>> +    $self->read_tunnel($self->{tunnel});
>> +
>> +    $self->log('info', "Forwarded local unix socket '$local' to remote '$remote' via websocket tunnel");
>> +};
>> +
>> +sub fork_websocket_tunnel {
>> +    my ($self, $storages) = @_;
>> +
>> +    my $remote = $self->{opts}->{remote};
>> +    my $conn = $remote->{conn};
>> +
>> +    my $websocket_url = "https://$conn->{host}:$conn->{port}/api2/json/nodes/$self->{node}/qemu/$remote->{vmid}/mtunnelwebsocket";
>> +
>> +    my $params = {
>> +	url => $websocket_url,
>> +    };
>> +
>> +    if (my $apitoken = $conn->{apitoken}) {
>> +	$params->{headers} = [["Authorization", "$apitoken"]];
>> +    } else {
>> +	die "can't connect to remote host without credentials\n";
>> +    }
>> +
>> +    if (my $fps = $conn->{cached_fingerprints}) {
>> +	$params->{fingerprint} = (keys %$fps)[0];
>> +    }
>> +
>> +    my $api_client = PVE::APIClient::LWP->new(%$conn);
>> +    my $storage_list = join(',', keys %$storages);
>> +    my $res = $api_client->post("/nodes/$self->{node}/qemu/$remote->{vmid}/mtunnel", { storages => $storage_list });
>> +    $self->log('info', "remote: started migration tunnel worker '$res->{upid}'");
>> +    $params->{url} .= "?ticket=".uri_escape($res->{ticket});
>> +    $params->{url} .= "&socket=$res->{socket}";
> 
> Nit: could also be escaped.
> 

done

>> +
>> +    my $reader = IO::Pipe->new();
>> +    my $writer = IO::Pipe->new();
>> +
>> +    my $cpid = fork();
>> +    if ($cpid) {
>> +	$writer->writer();
>> +	$reader->reader();
>> +	my $tunnel = { writer => $writer, reader => $reader, pid => $cpid };
>> +
>> +	eval {
>> +	    my $writer = $tunnel->{writer};
>> +	    my $cmd = encode_json({
>> +		control => JSON::true,
>> +		cmd => 'connect',
>> +		data => $params,
>> +	    });
>> +
>> +	    eval {
>> +		PVE::Tools::run_with_timeout(15, sub {
>> +		    print {$writer} "$cmd\n";
>> +		    $writer->flush();
>> +		});
>> +	    };
>> +	    die "failed to write tunnel connect command - $@\n" if $@;
>> +	};
>> +	die "failed to connect via WS: $@\n" if $@;
>> +
>> +	my $err;
>> +        eval {
>> +	    my $writer = $tunnel->{writer};
>> +	    my $cmd = encode_json({
>> +		cmd => 'version',
>> +	    });
>> +
>> +	    eval {
>> +		PVE::Tools::run_with_timeout(15, sub {
>> +		    print {$writer} "$cmd\n";
>> +		    $writer->flush();
>> +		});
>> +	    };
>> +	    $err = "failed to write tunnel version command - $@\n" if $@;
>> +	    my $res = $self->read_tunnel($tunnel, 10);
>> +	    $res = JSON::decode_json($res);
>> +	    my $version = $res->{api};
>> +
>> +	    if ($version =~ /^(\d+)$/) {
>> +		$tunnel->{version} = $1;
>> +		$tunnel->{age} = $res->{age};
>> +		$self->log('info', "tunnel info: $version\n");
>> +	    } else {
>> +		$err = "received invalid tunnel version string '$version'\n" if !$err;
>> +	    }
>> +	};
>> +	$err = $@ if !$err;
>> +
>> +	if ($err) {
>> +	    $self->finish_command_pipe($tunnel);
>> +	    die "can't open migration tunnel - $err";
>> +	}
>> +
>> +	$params->{url} = "$websocket_url?";
>> +	$tunnel->{params} = $params; # for forwarding
>> +
>> +	return $tunnel;
>> +    } else {
>> +	eval {
>> +	    $writer->reader();
>> +	    $reader->writer();
>> +	    PVE::Tools::run_command(
>> +		['proxmox-websocket-tunnel'],
>> +		input => "<&".fileno($writer),
>> +		output => ">&".fileno($reader),
>> +		errfunc => sub { my $line = shift; print "tunnel: $line\n"; },
>> +	    );
>> +	};
>> +	warn "CMD websocket tunnel died: $@\n" if $@;
>> +	exit 0;
>> +    }
>> +}
>> +
>>   sub finish_tunnel {
>> -    my ($self, $tunnel) = @_;
>> +    my ($self, $tunnel, $cleanup) = @_;
>>   
>> -    eval { $self->write_tunnel($tunnel, 30, 'quit'); };
>> +    $cleanup = $cleanup ? 1 : 0;
>> +
>> +    eval { $self->write_tunnel($tunnel, 30, 'quit', { cleanup => $cleanup }); };
>>       my $err = $@;
>>   
>>       $self->finish_command_pipe($tunnel, 30);
> 
> Nit: below here is
>      if (my $unix_sockets = $tunnel->{unix_sockets}) {
>          my $cmd = ['rm', '-f', @$unix_sockets];
>          PVE::Tools::run_command($cmd);
> 
>          # .. and just to be sure check on remote side
>          unshift @{$cmd}, @{$self->{rem_ssh}};
>          PVE::Tools::run_command($cmd);
>      }
> and if I'm not mistaken, $self->{rem_ssh} is undef for remote migration, 
> resulting in an undef warning and $cmd being executed twice locally.

doesn't hurt, but also not optimal. conditionalized in v2

>> @@ -338,23 +513,34 @@ sub prepare {
>>       }
>>   
>>       my $vollist = PVE::QemuServer::get_vm_volumes($conf);
>> +
>> +    my $storages = {};
>>       foreach my $volid (@$vollist) {
>>   	my ($sid, $volname) = PVE::Storage::parse_volume_id($volid, 1);
>>   
>> -	# check if storage is available on both nodes
>> +	# check if storage is available on source node
>>   	my $scfg = PVE::Storage::storage_check_enabled($storecfg, $sid);
>>   
>>   	my $targetsid = $sid;
>> -	# NOTE: we currently ignore shared source storages in mappings so skip here too for now
>> -	if (!$scfg->{shared}) {
>> +	# NOTE: local ignores shared mappings, remote maps them
>> +	if (!$scfg->{shared} || $self->{opts}->{remote}) {
>>   	    $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>>   	}
>>   
>> -	my $target_scfg = PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
>> -	my ($vtype) = PVE::Storage::parse_volname($storecfg, $volid);
>> +	$storages->{$targetsid} = 1;
>> +
>> +	if (!$self->{opts}->{remote}) {
>> +	    # check if storage is available on target node
>> +	    my $target_scfg = PVE::Storage::storage_check_enabled(
>> +		$storecfg,
>> +		$targetsid,
>> +		$self->{node},
>> +	    );
>> +	    my ($vtype) = PVE::Storage::parse_volname($storecfg, $volid);
>>   
>> -	die "$volid: content type '$vtype' is not available on storage '$targetsid'\n"
>> -	    if !$target_scfg->{content}->{$vtype};
>> +	    die "$volid: content type '$vtype' is not available on storage '$targetsid'\n"
>> +		if !$target_scfg->{content}->{$vtype};
>> +	}
>>   
>>   	if ($scfg->{shared}) {
>>   	    # PVE::Storage::activate_storage checks this for non-shared storages
>> @@ -364,10 +550,23 @@ sub prepare {
>>   	}
>>       }
>>   
>> -    # test ssh connection
>> -    my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
>> -    eval { $self->cmd_quiet($cmd); };
>> -    die "Can't connect to destination address using public key\n" if $@;
>> +    if ($self->{opts}->{remote}) {
>> +	# test & establish websocket connection
>> +	my $tunnel = $self->fork_websocket_tunnel($storages);
>> +	my $min_version = $tunnel->{version} - $tunnel->{age};
>> +	die "Remote tunnel endpoint not compatible, upgrade required (current: $WS_TUNNEL_VERSION, required: $min_version)\n"
>> +	    if $WS_TUNNEL_VERSION < $min_version;
>> +	 die "Remote tunnel endpoint too old, upgrade required (local: $WS_TUNNEL_VERSION, remote: $tunnel->{version})"
> 
> Nit: missing '\n' in error, and while we're at it: style nit for >100 
> character lines (are not the only instances in the series).
> 

fixed by switching to three info log statements, and shorter error 
messages without the versions

>> +	    if $WS_TUNNEL_VERSION > $tunnel->{version};
>> +
>> +	print "websocket tunnel started\n";
>> +	$self->{tunnel} = $tunnel;
>> +    } else {
>> +	# test ssh connection
>> +	my $cmd = [ @{$self->{rem_ssh}}, '/bin/true' ];
>> +	eval { $self->cmd_quiet($cmd); };
>> +	die "Can't connect to destination address using public key\n" if $@;
>> +    }
>>   
>>       return $running;
>>   }
>> @@ -405,7 +604,7 @@ sub scan_local_volumes {
>>   	my @sids = PVE::Storage::storage_ids($storecfg);
>>   	foreach my $storeid (@sids) {
>>   	    my $scfg = PVE::Storage::storage_config($storecfg, $storeid);
>> -	    next if $scfg->{shared};
>> +	    next if $scfg->{shared} && !$self->{opts}->{remote};
>>   	    next if !PVE::Storage::storage_check_enabled($storecfg, $storeid, undef, 1);
>>   
>>   	    # get list from PVE::Storage (for unused volumes)
>> @@ -414,19 +613,24 @@ sub scan_local_volumes {
>>   	    next if @{$dl->{$storeid}} == 0;
>>   
>>   	    my $targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $storeid);
>> -	    # check if storage is available on target node
>> -	    my $target_scfg = PVE::Storage::storage_check_enabled(
>> -		$storecfg,
>> -		$targetsid,
>> -		$self->{node},
>> -	    );
>> -
>> -	    die "content type 'images' is not available on storage '$targetsid'\n"
>> -		if !$target_scfg->{content}->{images};
>> +	    my $bwlimit_sids = [$storeid];
>> +	    if (!$self->{opts}->{remote}) {
>> +		# check if storage is available on target node
>> +		my $target_scfg = PVE::Storage::storage_check_enabled(
>> +		    $storecfg,
>> +		    $targetsid,
>> +		    $self->{node},
>> +		);
>> +
>> +		die "content type 'images' is not available on storage '$targetsid'\n"
>> +		    if !$target_scfg->{content}->{images};
>> +
>> +		push @$bwlimit_sids, $targetsid;
>> +	    }
>>   
>>   	    my $bwlimit = PVE::Storage::get_bandwidth_limit(
>>   		'migration',
>> -		[$targetsid, $storeid],
>> +		$bwlimit_sids,
>>   		$self->{opts}->{bwlimit},
>>   	    );
>>   
>> @@ -482,14 +686,17 @@ sub scan_local_volumes {
>>   	    my $scfg = PVE::Storage::storage_check_enabled($storecfg, $sid);
>>   
>>   	    my $targetsid = $sid;
>> -	    # NOTE: we currently ignore shared source storages in mappings so skip here too for now
>> -	    if (!$scfg->{shared}) {
>> +	    # NOTE: local ignores shared mappings, remote maps them
>> +	    if (!$scfg->{shared} || $self->{opts}->{remote}) {
>>   		$targetsid = PVE::QemuServer::map_id($self->{opts}->{storagemap}, $sid);
>>   	    }
>>   
>> -	    PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
>> +	    # check target storage on target node if intra-cluster migration
>> +	    if (!$self->{opts}->{remote}) {
>> +		PVE::Storage::storage_check_enabled($storecfg, $targetsid, $self->{node});
>>   
>> -	    return if $scfg->{shared};
>> +		return if $scfg->{shared};
>> +	    }
>>   
>>   	    $local_volumes->{$volid}->{ref} = $attr->{referenced_in_config} ? 'config' : 'snapshot';
>>   	    $local_volumes->{$volid}->{ref} = 'storage' if $attr->{is_unused};
>> @@ -578,6 +785,9 @@ sub scan_local_volumes {
>>   
>>   	    my $migratable = $scfg->{type} =~ /^(?:dir|btrfs|zfspool|lvmthin|lvm)$/;
>>   
>> +	    # TODO: what is this even here for?
>> +	    $migratable = 1 if $self->{opts}->{remote};
>> +
>>   	    die "can't migrate '$volid' - storage type '$scfg->{type}' not supported\n"
>>   		if !$migratable;
>>   
>> @@ -612,6 +822,10 @@ sub handle_replication {
>>       my $local_volumes = $self->{local_volumes};
>>   
>>       return if !$self->{replication_jobcfg};
>> +
>> +    die "can't migrate VM with replicated volumes to remote cluster/node\n"
>> +	if $self->{opts}->{remote};
> 
> We can add that later, asserting that no local removal will happen ;)
> Same for being a base VM referenced by a linked clone.
> 
>> +
>>       if ($self->{running}) {
>>   
>>   	my $version = PVE::QemuServer::kvm_user_version();
>> @@ -709,26 +923,133 @@ sub sync_offline_local_volumes {
>>       my $opts = $self->{opts};
>>   
>>       $self->log('info', "copying local disk images") if scalar(@volids);
>> -
>> +    my $forwarded = 0;
>>       foreach my $volid (@volids) {
>>   	my $targetsid = $local_volumes->{$volid}->{targetsid};
>> -	my $bwlimit = $local_volumes->{$volid}->{bwlimit};
>> -	$bwlimit = $bwlimit * 1024 if defined($bwlimit); # storage_migrate uses bps
>> -
>> -	my $storage_migrate_opts = {
>> -	    'ratelimit_bps' => $bwlimit,
>> -	    'insecure' => $opts->{migration_type} eq 'insecure',
>> -	    'with_snapshots' => $local_volumes->{$volid}->{snapshots},
>> -	    'allow_rename' => !$local_volumes->{$volid}->{is_vmstate},
>> -	};
>>   
>> -	my $logfunc = sub { $self->log('info', $_[0]); };
>> -	my $new_volid = eval {
>> -	    PVE::Storage::storage_migrate($storecfg, $volid, $self->{ssh_info},
>> -					  $targetsid, $storage_migrate_opts, $logfunc);
>> -	};
>> -	if (my $err = $@) {
>> -	    die "storage migration for '$volid' to storage '$targetsid' failed - $err\n";
>> +	my $new_volid;
>> +	
> 
> Style nit: whitespace error
> 
>> +	my $opts = $self->{opts};
>> +	if (my $remote = $opts->{remote}) {
>> +	    my $remote_vmid = $remote->{vmid};
>> +	    my ($sid, undef) = PVE::Storage::parse_volume_id($volid);
>> +	    my (undef, $name, undef, undef, undef, undef, $format) = PVE::Storage::parse_volname($storecfg, $volid);
>> +	    my $scfg = PVE::Storage::storage_config($storecfg, $sid);
>> +	    PVE::Storage::activate_volumes($storecfg, [$volid]);
>> +
>> +	    # use 'migrate' limit for transfer to other node
>> +	    my $bwlimit_opts = {
>> +		storage => $targetsid,
>> +		bwlimit => $opts->{bwlimit},
>> +	    };
>> +	    my $bwlimit = PVE::Storage::get_bandwidth_limit('migration', [$sid], $opts->{bwlimit});
> 
> Nit: could use
>      my $bwlimit = $local_volumes->{$volid}->{bwlimit};
> 
>> +	    my $remote_bwlimit = $self->write_tunnel($self->{tunnel}, 10, 'bwlimit', $bwlimit_opts);
>> +	    $remote_bwlimit = $remote_bwlimit->{bwlimit};
>> +	    if (defined($remote_bwlimit)) {
>> +		$bwlimit = $remote_bwlimit if !defined($bwlimit);
>> +		$bwlimit = $remote_bwlimit if $remote_bwlimit < $bwlimit;
>> +	    }
>> +
>> +	    # JSONSchema and get_bandwidth_limit use kbps - storage_migrate bps
>> +	    $bwlimit = $bwlimit * 1024 if defined($bwlimit);
>> +
>> +	    my $with_snapshots = $local_volumes->{$volid}->{snapshots} ? 1 : 0;
>> +	    my $snapshot;
>> +	    if ($scfg->{type} eq 'zfspool') {
>> +		$snapshot = '__migration__';
>> +		$with_snapshots = 1;
>> +		PVE::Storage::volume_snapshot($storecfg, $volid, $snapshot);
>> +	    }
>> +
>> +	    if ($self->{vmid} != $remote_vmid) {
>> +		$name =~ s/-$self->{vmid}-/-$remote_vmid-/g;
>> +		$name =~ s/^$self->{vmid}\//$remote_vmid\//;
>> +	    }
>> +
>> +	    my @export_formats = PVE::Storage::volume_export_formats($storecfg, $volid, undef, undef, $with_snapshots);
>> +
>> +	    my $storage_migrate_opts = {
> 
> Nit: maybe call it disk_import_opts
> 
>> +		format => $format,
>> +		storage => $targetsid,
>> +		'with-snapshots' => $with_snapshots,
>> +		'allow-rename' => !$local_volumes->{$volid}->{is_vmstate},
>> +		'export-formats' => @export_formats,
> 
> Doesn't this need to be converted to a string?
> 

yes, but it seems it works without when there's just a single entry, 
which is true for everything but btrfs :)

>> +		volname => $name,
>> +	    };
>> +	    my $res = $self->write_tunnel($self->{tunnel}, 600, 'disk-import', $storage_migrate_opts);
>> +	    my $local = "/run/qemu-server/$self->{vmid}.storage";
>> +	    if (!$forwarded) {
>> +		$forward_unix_socket->($self, $local, $res->{socket});
>> +		$forwarded = 1;
>> +	    }
>> +	    my $socket = IO::Socket::UNIX->new(Peer => $local, Type => SOCK_STREAM())
>> +		or die "failed to connect to websocket tunnel at $local\n";
>> +	    # we won't be reading from the socket
>> +	    shutdown($socket, 0);
>> +	    my $send = ['pvesm', 'export', $volid, $res->{format}, '-', '-with-snapshots', $with_snapshots];
>> +	    push @$send, '-snapshot', $snapshot if $snapshot;
>> +
>> +	    my @cstream;
>> +	    if (defined($bwlimit)) {
>> +		@cstream = ([ '/usr/bin/cstream', '-t', $bwlimit ]);
>> +		$self->log('info', "using a bandwidth limit of $bwlimit bps for transferring '$volid'");
>> +	    }
>> +
>> +	    eval {
>> +		PVE::Tools::run_command(
>> +		    [$send, @cstream],
>> +		    output => '>&'.fileno($socket),
>> +		    errfunc => sub { my $line = shift; $self->log('warn', $line); },
>> +		);
>> +	    };
>> +	    my $send_error = $@;
>> +
>> +	    # don't close the connection entirely otherwise the
>> +	    # receiving end might not get all buffered data (and
>> +	    # fails with 'connection reset by peer')
>> +	    shutdown($socket, 1);
>> +
>> +	    # wait for the remote process to finish
>> +	    while ($res = $self->write_tunnel($self->{tunnel}, 10, 'query-disk-import')) {
>> +		if ($res->{status} eq 'pending') {
>> +		    $self->log('info', "waiting for disk import to finish..\n");
>> +		    sleep(1)
>> +		} elsif ($res->{status} eq 'complete') {
>> +		    $new_volid = $res->{volid};
>> +		    last;
>> +		} else {
>> +		    die "unknown query-disk-import result: $res->{status}\n";
>> +		}
>> +	    }
>> +
>> +	    # now close the socket
>> +	    close($socket);
>> +	    die $send_error if $send_error;
>> +	} else {
>> +	    my $bwlimit = $local_volumes->{$volid}->{bwlimit};
>> +	    $bwlimit = $bwlimit * 1024 if defined($bwlimit); # storage_migrate uses bps
>> +
>> +	    my $storage_migrate_opts = {
>> +		'ratelimit_bps' => $bwlimit,
>> +		'insecure' => $opts->{migration_type} eq 'insecure',
>> +		'with_snapshots' => $local_volumes->{$volid}->{snapshots},
>> +		'allow_rename' => !$local_volumes->{$volid}->{is_vmstate},
>> +	    };
>> +
>> +	    my $logfunc = sub { $self->log('info', $_[0]); };
>> +	    $new_volid = eval {
>> +		PVE::Storage::storage_migrate(
>> +		    $storecfg,
>> +		    $volid,
>> +		    $self->{ssh_info},
>> +		    $targetsid,
>> +		    $storage_migrate_opts,
>> +		    $logfunc,
>> +		);
>> +	    };
>> +	    if (my $err = $@) {
>> +		die "storage migration for '$volid' to storage '$targetsid' failed - $err\n";
>> +	    }
>>   	}
>>   
>>   	$self->{volume_map}->{$volid} = $new_volid;
>> @@ -744,6 +1065,12 @@ sub sync_offline_local_volumes {
>>   sub cleanup_remotedisks {
>>       my ($self) = @_;
>>   
> 
> Nit, not to be taken seriously: cleanup_remotedisks_and_maybe_tunnel ;)
> 
>> +    if ($self->{opts}->{remote}) {
>> +	$self->finish_tunnel($self->{tunnel}, 1);
>> +	delete $self->{tunnel};
>> +	return;
>> +    }
>> +
>>       my $local_volumes = $self->{local_volumes};
>>   
>>       foreach my $volid (values %{$self->{volume_map}}) {
>> @@ -793,8 +1120,84 @@ sub phase1 {
>>       $self->handle_replication($vmid);
>>   
>>       $self->sync_offline_local_volumes();
>> +    $self->phase1_remote($vmid) if $self->{opts}->{remote};
>>   };
>>   
>> +sub phase1_remote {
>> +    my ($self, $vmid) = @_;
>> +
>> +    my $remote_conf = PVE::QemuConfig->load_config($vmid);
>> +    PVE::QemuConfig->update_volume_ids($remote_conf, $self->{volume_map});
>> +
>> +    # TODO: check bridge availability earlier?
>> +    my $bridgemap = $self->{opts}->{bridgemap};
>> +    foreach my $opt (keys %$remote_conf) {
>> +	next if $opt !~ m/^net\d+$/;
>> +
>> +	next if !$remote_conf->{$opt};
>> +	my $d = PVE::QemuServer::parse_net($remote_conf->{$opt});
>> +	next if !$d || !$d->{bridge};
>> +
>> +	my $target_bridge = PVE::QemuServer::map_id($bridgemap, $d->{bridge});
>> +	$self->log('info', "mapped: $opt from $d->{bridge} to $target_bridge");
>> +	$d->{bridge} = $target_bridge;
>> +	$remote_conf->{$opt} = PVE::QemuServer::print_net($d);
>> +    }
>> +
>> +    my @online_local_volumes = $self->filter_local_volumes('online');
>> +
>> +    my $storage_map = $self->{opts}->{storagemap};
>> +    $self->{nbd} = {};
>> +    PVE::QemuConfig->foreach_volume($remote_conf, sub {
>> +	my ($ds, $drive) = @_;
>> +
>> +	# TODO eject CDROM?
>> +	return if PVE::QemuServer::drive_is_cdrom($drive);
>> +
>> +	my $volid = $drive->{file};
>> +	return if !$volid;
>> +
>> +	return if !grep { $_ eq $volid} @online_local_volumes;
>> +
>> +	my ($storeid, $volname) = PVE::Storage::parse_volume_id($volid);
>> +	my $scfg = PVE::Storage::storage_config($self->{storecfg}, $storeid);
>> +	my $source_format = PVE::QemuServer::qemu_img_format($scfg, $volname);
>> +
>> +	# set by target cluster
>> +	my $oldvolid = delete $drive->{file};
>> +	delete $drive->{format};
>> +
>> +	my $targetsid = PVE::QemuServer::map_id($storage_map, $storeid);
>> +
>> +	my $params = {
>> +	    format => $source_format,
>> +	    storage => $targetsid,
>> +	    drive => $drive,
>> +	};
>> +
>> +	$self->log('info', "Allocating volume for drive '$ds' on remote storage '$targetsid'..");
>> +	my $res = $self->write_tunnel($self->{tunnel}, 600, 'disk', $params);
>> +
>> +	$self->log('info', "volume '$oldvolid' os '$res->{volid}' on the target\n");
>> +	$remote_conf->{$ds} = $res->{drivestr};
>> +	$self->{nbd}->{$ds} = $res;
>> +    });
>> +
>> +    my $conf_str = PVE::QemuServer::write_vm_config("remote", $remote_conf);
>> +
>> +    # TODO expose in PVE::Firewall?
>> +    my $vm_fw_conf_path = "/etc/pve/firewall/$vmid.fw";
>> +    my $fw_conf_str;
>> +    $fw_conf_str = PVE::Tools::file_get_contents($vm_fw_conf_path)
>> +	if -e $vm_fw_conf_path;
>> +    my $params = {
>> +	conf => $conf_str,
>> +	'firewall-config' => $fw_conf_str,
>> +    };
>> +
>> +    $self->write_tunnel($self->{tunnel}, 10, 'config', $params);
>> +}
>> +
>>   sub phase1_cleanup {
>>       my ($self, $vmid, $err) = @_;
>>   
>> @@ -825,7 +1228,6 @@ sub phase2_start_local_cluster {
>>       my $local_volumes = $self->{local_volumes};
>>       my @online_local_volumes = $self->filter_local_volumes('online');
>>   
>> -    $self->{storage_migration} = 1 if scalar(@online_local_volumes);
>>       my $start = $params->{start_params};
>>       my $migrate = $params->{migrate_opts};
>>   
>> @@ -948,10 +1350,34 @@ sub phase2_start_local_cluster {
>>       return ($tunnel_info, $spice_port);
>>   }
>>   
>> +sub phase2_start_remote_cluster {
>> +    my ($self, $vmid, $params) = @_;
>> +
>> +    die "insecure migration to remote cluster not implemented\n"
>> +	if $params->{migrate_opts}->{type} ne 'websocket';
>> +
>> +    my $remote_vmid = $self->{opts}->{remote}->{vmid};
>> +
>> +    my $res = $self->write_tunnel($self->{tunnel}, 10, "start", $params);
>> +
>> +    foreach my $drive (keys %{$res->{drives}}) {
>> +	$self->{stopnbd} = 1;
>> +	$self->{target_drive}->{$drive}->{drivestr} = $res->{drives}->{$drive}->{drivestr};
>> +	my $nbd_uri = $res->{drives}->{$drive}->{nbd_uri};
>> +	die "unexpected NBD uri for '$drive': $nbd_uri\n"
>> +	    if $nbd_uri !~ s!/run/qemu-server/$remote_vmid\_!/run/qemu-server/$vmid\_!;
>> +
>> +	$self->{target_drive}->{$drive}->{nbd_uri} = $nbd_uri;
>> +    }
>> +
>> +    return ($res->{migrate}, $res->{spice_port});
>> +}
>> +
>>   sub phase2 {
>>       my ($self, $vmid) = @_;
>>   
>>       my $conf = $self->{vmconf};
>> +    my $local_volumes = $self->{local_volumes};
>>   
>>       # version > 0 for unix socket support
>>       my $nbd_protocol_version = 1;
>> @@ -983,10 +1409,42 @@ sub phase2 {
>>   	},
>>       };
>>   
>> -    my ($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
>> +    my ($tunnel_info, $spice_port);
>> +
>> +    my @online_local_volumes = $self->filter_local_volumes('online');
>> +    $self->{storage_migration} = 1 if scalar(@online_local_volumes);
>> +
>> +    if (my $remote = $self->{opts}->{remote}) {
>> +	my $remote_vmid = $remote->{vmid};
>> +	$params->{migrate_opts}->{remote_node} = $self->{node};
>> +	($tunnel_info, $spice_port) = $self->phase2_start_remote_cluster($vmid, $params);
>> +	die "only UNIX sockets are supported for remote migration\n"
>> +	    if $tunnel_info->{proto} ne 'unix';
>> +
>> +	my $forwarded = {};
>> +	my $remote_socket = $tunnel_info->{addr};
>> +	my $local_socket = $remote_socket;
>> +	$local_socket =~ s/$remote_vmid/$vmid/g;
>> +	$tunnel_info->{addr} = $local_socket;
>> +
>> +	$self->log('info', "Setting up tunnel for '$local_socket'");
>> +	$forward_unix_socket->($self, $local_socket, $remote_socket);
>> +	$forwarded->{$local_socket} = 1;
>> +
>> +	foreach my $remote_socket (@{$tunnel_info->{unix_sockets}}) {
>> +	    my $local_socket = $remote_socket;
>> +	    $local_socket =~ s/$remote_vmid/$vmid/g;
>> +	    next if $forwarded->{$local_socket};
>> +	    $self->log('info', "Setting up tunnel for '$local_socket'");
>> +	    $forward_unix_socket->($self, $local_socket, $remote_socket);
>> +	    $forwarded->{$local_socket} = 1;
>> +	}
>> +    } else {
>> +	($tunnel_info, $spice_port) = $self->phase2_start_local_cluster($vmid, $params);
>>   
>> -    $self->log('info', "start remote tunnel");
>> -    $self->start_remote_tunnel($tunnel_info);
>> +	$self->log('info', "start remote tunnel");
>> +	$self->start_remote_tunnel($tunnel_info);
>> +    }
>>   
>>       my $migrate_uri = "$tunnel_info->{proto}:$tunnel_info->{addr}";
>>       $migrate_uri .= ":$tunnel_info->{port}"
>> @@ -996,8 +1454,6 @@ sub phase2 {
>>   	$self->{storage_migration_jobs} = {};
>>   	$self->log('info', "starting storage migration");
>>   
>> -	my @online_local_volumes = $self->filter_local_volumes('online');
>> -
>>   	die "The number of local disks does not match between the source and the destination.\n"
>>   	    if (scalar(keys %{$self->{target_drive}}) != scalar(@online_local_volumes));
>>   	foreach my $drive (keys %{$self->{target_drive}}){
>> @@ -1070,7 +1526,7 @@ sub phase2 {
>>       };
>>       $self->log('info', "migrate-set-parameters error: $@") if $@;
>>   
>> -    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga})) {
>> +    if (PVE::QemuServer::vga_conf_has_spice($conf->{vga} && !$self->{opts}->{remote})) {
>>   	my $rpcenv = PVE::RPCEnvironment::get();
>>   	my $authuser = $rpcenv->get_user();
>>   
>> @@ -1267,11 +1723,15 @@ sub phase2_cleanup {
>>   
>>       my $nodename = PVE::INotify::nodename();
>>   
>> -    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'stop', $vmid, '--skiplock', '--migratedfrom', $nodename];
>> -    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
>> -    if (my $err = $@) {
>> -        $self->log('err', $err);
>> -        $self->{errors} = 1;
>> +    if ($self->{tunnel} && $self->{tunnel}->{version} >= 2) {
>> +	$self->write_tunnel($self->{tunnel}, 10, 'stop');
>> +    } else {
>> +	my $cmd = [@{$self->{rem_ssh}}, 'qm', 'stop', $vmid, '--skiplock', '--migratedfrom', $nodename];
>> +	eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
>> +	if (my $err = $@) {
>> +	    $self->log('err', $err);
>> +	    $self->{errors} = 1;
>> +	}
>>       }
>>   
>>       # cleanup after stopping, otherwise disks might be in-use by target VM!
>> @@ -1304,7 +1764,7 @@ sub phase3_cleanup {
>>   
>>       my $tunnel = $self->{tunnel};
>>   
>> -    if ($self->{volume_map}) {
>> +    if ($self->{volume_map} && !$self->{opts}->{remote}) {
>>   	my $target_drives = $self->{target_drive};
>>   
>>   	# FIXME: for NBD storage migration we now only update the volid, and
>> @@ -1321,26 +1781,33 @@ sub phase3_cleanup {
>>   
>>       # transfer replication state before move config
>>       $self->transfer_replication_state() if $self->{is_replicated};
>> -    PVE::QemuConfig->move_config_to_node($vmid, $self->{node});
>> +    if (!$self->{opts}->{remote}) {
>> +	PVE::QemuConfig->move_config_to_node($vmid, $self->{node});
>> +    }
>>       $self->switch_replication_job_target() if $self->{is_replicated};
> 
> All three lines could/should be guarded by the if.
> 

true (doesn't matter now since we error out on replicated VMs anyway, 
but makes this more obvious when we change that face ;))

>>   
>>       if ($self->{livemigration}) {
>>   	if ($self->{stopnbd}) {
>>   	    $self->log('info', "stopping NBD storage migration server on target.");
>>   	    # stop nbd server on remote vm - requirement for resume since 2.9
>> -	    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'nbdstop', $vmid];
>> +	    if ($tunnel && $tunnel->{version} && $tunnel->{version} >= 2) {
>> +		$self->write_tunnel($tunnel, 30, 'nbdstop');
>> +	    } else {
>> +		my $cmd = [@{$self->{rem_ssh}}, 'qm', 'nbdstop', $vmid];
>>   
>> -	    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
>> -	    if (my $err = $@) {
>> -		$self->log('err', $err);
>> -		$self->{errors} = 1;
>> +		eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
>> +		if (my $err = $@) {
>> +		    $self->log('err', $err);
>> +		    $self->{errors} = 1;
>> +		}
>>   	    }
>>   	}
>>   
>>   	# config moved and nbd server stopped - now we can resume vm on target
>>   	if ($tunnel && $tunnel->{version} && $tunnel->{version} >= 1) {
>> +	    my $cmd = $tunnel->{version} == 1 ? "resume $vmid" : "resume";
>>   	    eval {
>> -		$self->write_tunnel($tunnel, 30, "resume $vmid");
>> +		$self->write_tunnel($tunnel, 30, $cmd);
>>   	    };
>>   	    if (my $err = $@) {
>>   		$self->log('err', $err);
>> @@ -1360,18 +1827,24 @@ sub phase3_cleanup {
>>   	}
>>   
>>   	if ($self->{storage_migration} && PVE::QemuServer::parse_guest_agent($conf)->{fstrim_cloned_disks} && $self->{running}) {
>> -	    my $cmd = [@{$self->{rem_ssh}}, 'qm', 'guest', 'cmd', $vmid, 'fstrim'];
>> -	    eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
>> +	    if ($self->{opts}->{remote}) {
>> +		$self->write_tunnel($self->{tunnel}, 600, 'fstrim');
>> +	    } else {
>> +		my $cmd = [@{$self->{rem_ssh}}, 'qm', 'guest', 'cmd', $vmid, 'fstrim'];
>> +		eval{ PVE::Tools::run_command($cmd, outfunc => sub {}, errfunc => sub {}) };
>> +	    }
>>   	}
>>       }
>>   
>>       # close tunnel on successful migration, on error phase2_cleanup closed it
>> -    if ($tunnel) {
>> +    if ($tunnel && $tunnel->{version} == 1) {
>>   	eval { finish_tunnel($self, $tunnel);  };
>>   	if (my $err = $@) {
>>   	    $self->log('err', $err);
>>   	    $self->{errors} = 1;
>>   	}
>> +	$tunnel = undef;
>> +	delete $self->{tunnel};
>>       }
>>   
>>       eval {
>> @@ -1409,6 +1882,9 @@ sub phase3_cleanup {
>>   
>>       # destroy local copies
>>       foreach my $volid (@not_replicated_volumes) {
>> +	# remote is cleaned up below
>> +	next if $self->{opts}->{remote};
>> +
>>   	eval { PVE::Storage::vdisk_free($self->{storecfg}, $volid); };
>>   	if (my $err = $@) {
>>   	    $self->log('err', "removing local copy of '$volid' failed - $err");
>> @@ -1418,8 +1894,19 @@ sub phase3_cleanup {
>>       }
>>   
>>       # clear migrate lock
>> -    my $cmd = [ @{$self->{rem_ssh}}, 'qm', 'unlock', $vmid ];
>> -    $self->cmd_logerr($cmd, errmsg => "failed to clear migrate lock");
>> +    if ($tunnel && $tunnel->{version} >= 2) {
>> +	$self->write_tunnel($tunnel, 10, "unlock");
>> +
>> +	$self->finish_tunnel($tunnel);
>> +    } else {
>> +	my $cmd = [ @{$self->{rem_ssh}}, 'qm', 'unlock', $vmid ];
>> +	$self->cmd_logerr($cmd, errmsg => "failed to clear migrate lock");
>> +    }
>> +
>> +    if ($self->{opts}->{remote} && $self->{opts}->{delete}) {
>> +	eval { PVE::QemuServer::destroy_vm($self->{storecfg}, $vmid, 1, undef, 0) };
>> +	warn "Failed to remove source VM - $@\n" if $@;
>> +    }
>>   }
>>   
>>   sub final_cleanup {
>> diff --git a/PVE/QemuServer.pm b/PVE/QemuServer.pm
>> index d494cc0..bf05da2 100644
>> --- a/PVE/QemuServer.pm
>> +++ b/PVE/QemuServer.pm
>> @@ -5384,7 +5384,11 @@ sub vm_start_nolock {
>>       my $defaults = load_defaults();
>>   
>>       # set environment variable useful inside network script
>> -    $ENV{PVE_MIGRATED_FROM} = $migratedfrom if $migratedfrom;
>> +    if ($migrate_opts->{remote_node}) {
>> +	$ENV{PVE_MIGRATED_FROM} = $migrate_opts->{remote_node};
>> +    } elsif ($migratedfrom) {
>> +	$ENV{PVE_MIGRATED_FROM} = $migratedfrom;
>> +    }
> 
> But the network script tries to load the config from that node and if 
> it's not in the cluster that doesn't work?
> 

this is a bit confusing, yeah.

$migratedfrom contains the source node, which is unusable on the remote 
cluster
remote_node contains the target node, which actually has the full config 
when we start the VM there over the tunnel (in contrast to a local 
migration, where the target node doesn't yet have the config!)

so this should be correct? but even easier would be to just not set it 
(for remote migrations), since the start MUST happen on the node where 
mtunnel is running/the config is located.

>>   
>>       PVE::GuestHelpers::exec_hookscript($conf, $vmid, 'pre-start', 1);
>>   
>> @@ -5621,7 +5625,7 @@ sub vm_start_nolock {
>>   
>>   	my $migrate_storage_uri;
>>   	# nbd_protocol_version > 0 for unix socket support
>> -	if ($nbd_protocol_version > 0 && $migration_type eq 'secure') {
>> +	if ($nbd_protocol_version > 0 && ($migration_type eq 'secure' || $migration_type eq 'websocket')) {
>>   	    my $socket_path = "/run/qemu-server/$vmid\_nbd.migrate";
>>   	    mon_cmd($vmid, "nbd-server-start", addr => { type => 'unix', data => { path => $socket_path } } );
>>   	    $migrate_storage_uri = "nbd:unix:$socket_path";
>> 
> 




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

* Re: [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint
  2021-11-10 12:29   ` Fabian Ebner
@ 2021-11-11 12:33     ` Fabian Grünbichler
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-11-11 12:33 UTC (permalink / raw)
  To: Fabian Ebner, pve-devel

On November 10, 2021 1:29 pm, Fabian Ebner wrote:
> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
>> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
>> ---
>> 
>> Notes:
>>      the checks currently done before the actual migration worker is
>>      forked could be either moved to the client calling this (that then
>>      makes the required API calls) or extracted into a precond API call
>>      like for regular migration.
>>      
>>      for testing it helps catch trivial mistakes early on, and the calls shouldn't
>>      be too expensive, so I left them in for now..
>>      
>>      requires
>>      - pve-common with bridge-pair format
>>      - pve-guest-common with AbstractMigrate handling remote migration
>> 
>>   PVE/API2/Qemu.pm | 205 ++++++++++++++++++++++++++++++++++++++++++++++-
>>   debian/control   |   2 +
>>   2 files changed, 205 insertions(+), 2 deletions(-)
>> 
>> diff --git a/PVE/API2/Qemu.pm b/PVE/API2/Qemu.pm
>> index 24f5b98..b931f04 100644
>> --- a/PVE/API2/Qemu.pm
>> +++ b/PVE/API2/Qemu.pm
>> @@ -14,6 +14,7 @@ use URI::Escape;
>>   use Crypt::OpenSSL::Random;
>>   use Socket qw(SOCK_STREAM);
>>   
>> +use PVE::APIClient::LWP;
>>   use PVE::Cluster qw (cfs_read_file cfs_write_file);;
>>   use PVE::RRD;
>>   use PVE::SafeSyslog;
>> @@ -51,8 +52,6 @@ BEGIN {
>>       }
>>   }
>>   
>> -use Data::Dumper; # fixme: remove
>> -
>>   use base qw(PVE::RESTHandler);
>>   
>>   my $opt_force_description = "Force physical removal. Without this, we simple remove the disk from the config file and create an additional configuration entry called 'unused[n]', which contains the volume ID. Unlink of unused[n] always cause physical removal.";
>> @@ -3778,6 +3777,208 @@ __PACKAGE__->register_method({
>>   
>>       }});
>>   
>> +__PACKAGE__->register_method({
>> +    name => 'remote_migrate_vm',
>> +    path => '{vmid}/remote_migrate',
>> +    method => 'POST',
>> +    protected => 1,
>> +    proxyto => 'node',
>> +    description => "Migrate virtual machine to a remote cluster. Creates a new migration task.",
>> +    permissions => {
>> +	check => ['perm', '/vms/{vmid}', [ 'VM.Migrate' ]],
>> +    },
>> +    parameters => {
>> +	additionalProperties => 0,
>> +	properties => {
>> +	    node => get_standard_option('pve-node'),
>> +	    vmid => get_standard_option('pve-vmid', { completion => \&PVE::QemuServer::complete_vmid }),
>> +	    'target-vmid' => get_standard_option('pve-vmid', { optional => 1 }),
>> +	    'target-node' => get_standard_option('pve-node', {
>> +		description => "Target node on remote cluster.",
>> +            }),
>> +	    'target-endpoint' => get_standard_option('proxmox-remote', {
>> +		description => "Remote target endpoint",
>> +	    }),
>> +	    online => {
>> +		type => 'boolean',
>> +		description => "Use online/live migration if VM is running. Ignored if VM is stopped.",
>> +		optional => 1,
>> +	    },
>> +	    'migration-network' => {
>> +		type => 'string', format => 'CIDR',
>> +		description => "CIDR of the (sub) network that is used for migration.",
>> +		optional => 1,
>> +	    },
>> +	    'with-local-disks' => {
>> +		type => 'boolean',
>> +		description => "Enable live storage migration for local disk",
>> +		optional => 1,
>> +	    },
>> +	    delete => {
>> +		type => 'boolean',
>> +		description => "Delete the original VM and related data after successful migration. By default the original VM is kept on the source cluster in a stopped state.",
>> +		optional => 1,
>> +		default => 0,
>> +	    },
>> +            'target-storage' => get_standard_option('pve-targetstorage', {
>> +		completion => \&PVE::QemuServer::complete_migration_storage,
>> +		optional => 0,
>> +            }),
>> +	    'target-bridge' => {
>> +		type => 'string',
>> +		description => "Mapping from source to target bridges. Providing only a single bridge ID maps all source bridges to that bridge. Providing the special value '1' will map each source bridge to itself.",
>> +		format => 'bridge-pair-list',
>> +	    },
>> +	    bwlimit => {
>> +		description => "Override I/O bandwidth limit (in KiB/s).",
>> +		optional => 1,
>> +		type => 'integer',
>> +		minimum => '0',
>> +		default => 'migrate limit from datacenter or storage config',
>> +	    },
>> +	},
>> +    },
>> +    returns => {
>> +	type => 'string',
>> +	description => "the task ID.",
>> +    },
>> +    code => sub {
>> +	my ($param) = @_;
>> +
>> +	my $rpcenv = PVE::RPCEnvironment::get();
>> +	my $authuser = $rpcenv->get_user();
>> +
>> +	my $source_vmid = extract_param($param, 'vmid');
>> +	my $target_endpoint = extract_param($param, 'target-endpoint');
>> +	my $target_node = extract_param($param, 'target-node');
>> +	my $target_vmid = extract_param($param, 'target-vmid') // $source_vmid;
>> +
>> +	my $localnode = PVE::INotify::nodename();
> 
> Nit: not used (and could've been $param->{node}).
> 
>> +	my $network = extract_param($param, 'migration-network');
>> +	my $delete = extract_param($param, 'delete') // 0;
>> +
>> +	PVE::Cluster::check_cfs_quorum();
>> +
>> +	raise_param_exc({ 'migration-network' => "Only root may use this option." })
>> +	    if $network && $authuser ne 'root@pam';
> 
> I might be missing something obvious, but where is the migration network 
> actually used down the line for the remote migration?
> 

ha - no. this is leftover from the previous version, where we had a 
remote config file specifying endpoints, and those might be reachable 
over another network that could be specified here. since we now specify 
the API endpoint info directly, that network selection can simply happen 
with that (a client can connect however, query for fingerprint and 
address, then call this endpoint with the already correct info).

we might need to re-introduce it if we ever want to support 'insecure' 
migration over websocket tunnels, since the insecure migration might 
than be over another network than the API traffic/control tunnel. but 
for now, I'll drop it in v2!

>> +
>> +	# test if VM exists
>> +	my $conf = PVE::QemuConfig->load_config($source_vmid);
>> +
>> +	PVE::QemuConfig->check_lock($conf);
>> +
>> +	raise_param_exc({ vmid => "cannot migrate HA-manage VM to remote cluster" })
> 
> s/manage/managed/
> 
>> +	    if PVE::HA::Config::vm_is_ha_managed($source_vmid);
>> +
>> +	my $remote = PVE::JSONSchema::parse_property_string('proxmox-remote', $target_endpoint);
>> +
>> +	# TODO: move this as helper somewhere appropriate?
>> +	my $conn_args = {
>> +	    protocol => 'https',
>> +	    host => $remote->{host},
>> +	    port => $remote->{port} // 8006,
>> +	    apitoken => $remote->{apitoken},
>> +	};
>> +
>> +	my $fp;
>> +	if ($fp = $remote->{fingerprint}) {
>> +	    $conn_args->{cached_fingerprints} = { uc($fp) => 1 };
>> +	}
>> +
>> +	print "Establishing API connection with remote at '$remote->{host}'\n";
>> +
>> +	my $api_client = PVE::APIClient::LWP->new(%$conn_args);
>> +	my $version = $api_client->get("/version");
>> +	print "remote: version '$version->{version}\n";
>> +
>> +	if (!defined($fp)) {
>> +	    my $cert_info = $api_client->get("/nodes/$target_node/certificates/info");
>> +	    foreach my $cert (@$cert_info) {
>> +		$fp = $cert->{fingerprint} if $cert->{filename} ne 'pve-root-ca.pem';
>> +		last if $cert->{filename} eq 'pveproxy-ssl.pem';
> 
> Not future-proof if the API call is ever extended to return an 
> additional certificate which is not a valid fall-back here.

switched it to only look at pveproxy-ssl.pem and pve-ssl.pem

> 
>> +	    }
>> +	    $conn_args->{cached_fingerprints} = { uc($fp) => 1 }
>> +		if defined($fp);
>> +	}
>> +
>> +	if (PVE::QemuServer::check_running($source_vmid)) {
>> +	    die "can't migrate running VM without --online\n" if !$param->{online};
>> +
>> +	    my $repl_conf = PVE::ReplicationConfig->new();
>> +	    my $is_replicated = $repl_conf->check_for_existing_jobs($source_vmid, 1);
>> +	    die "cannot remote-migrate replicated VM\n" if $is_replicated;
>> +	} else {
>> +	    warn "VM isn't running. Doing offline migration instead.\n" if $param->{online};
>> +	    $param->{online} = 0;
>> +	}
>> +
>> +	# FIXME: fork worker hear to avoid timeout? or poll these periodically
>> +	# in pvestatd and access cached info here? all of the below is actually
>> +	# checked at the remote end anyway once we call the mtunnel endpoint,
>> +	# we could also punt it to the client and not do it here at all..
>> +	my $resources = $api_client->get("/cluster/resources");
>> +	if (grep { defined($_->{vmid}) && $_->{vmid} eq $target_vmid } @$resources) {
>> +	    raise_param_exc({ target_vmid => "Guest with ID '$target_vmid' already exists on remote cluster" });
>> +	}
>> +
>> +	my $storages = [ grep { $_->{type} eq 'storage' && $_->{node} eq $target_node } @$resources ];
>> +	my $storecfg = PVE::Storage::config();
>> +	my $target_storage = extract_param($param, 'target-storage');
>> +	my $storagemap = eval { PVE::JSONSchema::parse_idmap($target_storage, 'pve-storage-id') };
>> +	raise_param_exc({ 'target-storage' => "failed to parse storage map: $@" })
>> +	    if $@;
>> +
>> +	my $target_bridge = extract_param($param, 'target-bridge');
>> +	my $bridgemap = eval { PVE::JSONSchema::parse_idmap($target_bridge, 'pve-bridge-id') };
>> +	raise_param_exc({ 'target-bridge' => "failed to parse bridge map: $@" })
>> +	    if $@;
>> +
>> +	my $check_remote_storage = sub {
>> +	    my ($storage) = @_;
>> +	    my $found = [ grep { $_->{storage} eq $storage } @$storages ];
>> +	    die "remote: storage '$storage' does not exist!\n"
>> +		if !@$found;
>> +
>> +	    $found = @$found[0];
>> +
>> +	    my $content_types = [ PVE::Tools::split_list($found->{content}) ];
>> +	    die "remote: storage '$storage' cannot store images\n"
>> +		if !grep { $_ eq 'images' } @$content_types;
>> +	};
>> +
>> +	foreach my $target_sid (values %{$storagemap->{entries}}) {
>> +	    $check_remote_storage->($target_sid);
>> +	}
>> +
>> +	$check_remote_storage->($storagemap->{default})
>> +	    if $storagemap->{default};
>> +
>> +	# TODO: or check all referenced storages?
>> +	die "remote migration requires explicit storage mapping!\n"
>> +	    if $storagemap->{identity};
>> +
>> +	$param->{storagemap} = $storagemap;
>> +	$param->{bridgemap} = $bridgemap;
>> +	$param->{remote} = {
>> +	    conn => $conn_args, # re-use fingerprint for tunnel
>> +	    client => $api_client,
>> +	    vmid => $target_vmid,
>> +	};
>> +	$param->{migration_type} = 'websocket';
>> +	$param->{migration_network} = $network if $network;
>> +	$param->{delete} = $delete if $delete;
>> +
>> +	my $realcmd = sub {
>> +	    PVE::QemuMigrate->migrate($target_node, $remote->{host}, $source_vmid, $param);
>> +	};
>> +
>> +	my $worker = sub {
>> +	    return PVE::GuestHelpers::guest_migration_lock($source_vmid, 10, $realcmd);
>> +	};
>> +
>> +	return $rpcenv->fork_worker('qmigrate', $source_vmid, $authuser, $worker);
>> +    }});
>> +
>>   __PACKAGE__->register_method({
>>       name => 'monitor',
>>       path => '{vmid}/monitor',
>> diff --git a/debian/control b/debian/control
>> index 8032ae5..33e3916 100644
>> --- a/debian/control
>> +++ b/debian/control
>> @@ -6,6 +6,7 @@ Build-Depends: debhelper (>= 12~),
>>                  libglib2.0-dev,
>>                  libio-multiplex-perl,
>>                  libjson-c-dev,
>> +               libpve-apiclient-perl,
>>                  libpve-cluster-perl,
>>                  libpve-common-perl (>= 6.3-3),
>>                  libpve-guest-common-perl (>= 3.1-3),
>> @@ -34,6 +35,7 @@ Depends: dbus,
>>            libjson-xs-perl,
>>            libnet-ssleay-perl,
>>            libpve-access-control (>= 5.0-7),
>> +         libpve-apiclient-perl,
>>            libpve-cluster-perl,
>>            libpve-common-perl (>= 7.0-3),
>>            libpve-guest-common-perl (>= 3.1-3),
>> 
> 




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

* Re: [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling
  2021-11-11 12:25     ` Fabian Grünbichler
@ 2021-11-11 12:57       ` Fabian Ebner
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Ebner @ 2021-11-11 12:57 UTC (permalink / raw)
  To: Fabian Grünbichler, pve-devel

Am 11.11.21 um 13:25 schrieb Fabian Grünbichler:
> On November 10, 2021 12:17 pm, Fabian Ebner wrote:
>> Am 05.11.21 um 14:03 schrieb Fabian Grünbichler:
>>>    
>>>        # set environment variable useful inside network script
>>> -    $ENV{PVE_MIGRATED_FROM} = $migratedfrom if $migratedfrom;
>>> +    if ($migrate_opts->{remote_node}) {
>>> +	$ENV{PVE_MIGRATED_FROM} = $migrate_opts->{remote_node};
>>> +    } elsif ($migratedfrom) {
>>> +	$ENV{PVE_MIGRATED_FROM} = $migratedfrom;
>>> +    }
>>
>> But the network script tries to load the config from that node and if
>> it's not in the cluster that doesn't work?
>>
> 
> this is a bit confusing, yeah.
> 
> $migratedfrom contains the source node, which is unusable on the remote
> cluster
> remote_node contains the target node, which actually has the full config
> when we start the VM there over the tunnel (in contrast to a local
> migration, where the target node doesn't yet have the config!)
> 
> so this should be correct? but even easier would be to just not set it
> (for remote migrations), since the start MUST happen on the node where
> mtunnel is running/the config is located.

Yes, sorry, I think I might've misread the code yesterday ;)

> 
>>>    
>>>        PVE::GuestHelpers::exec_hookscript($conf, $vmid, 'pre-start', 1);
>>>    
>>> @@ -5621,7 +5625,7 @@ sub vm_start_nolock {
>>>    
>>>    	my $migrate_storage_uri;
>>>    	# nbd_protocol_version > 0 for unix socket support
>>> -	if ($nbd_protocol_version > 0 && $migration_type eq 'secure') {
>>> +	if ($nbd_protocol_version > 0 && ($migration_type eq 'secure' || $migration_type eq 'websocket')) {
>>>    	    my $socket_path = "/run/qemu-server/$vmid\_nbd.migrate";
>>>    	    mon_cmd($vmid, "nbd-server-start", addr => { type => 'unix', data => { path => $socket_path } } );
>>>    	    $migrate_storage_uri = "nbd:unix:$socket_path";
>>>
>>




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

* [pve-devel] applied: [PATCH common 1/3] schema: rename storagepair to storage-pair
  2021-11-05 13:03 ` [pve-devel] [PATCH common 1/3] schema: rename storagepair to storage-pair Fabian Grünbichler
@ 2021-11-11 13:18   ` Thomas Lamprecht
  0 siblings, 0 replies; 50+ messages in thread
From: Thomas Lamprecht @ 2021-11-11 13:18 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

On 05.11.21 14:03, Fabian Grünbichler wrote:
> more consistent with the other options/formats like pve-storage-id
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>     v2: new
>     
>     this breaks older qemu-server without the patch adapting to the new name
> 
>  src/PVE/JSONSchema.pm | 4 ++--
>  1 file changed, 2 insertions(+), 2 deletions(-)
> 
>

applied, thanks!




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

* [pve-devel] applied: [PATCH common 2/3] schema: add pve-bridge-id option/format/pair
  2021-11-05 13:03 ` [pve-devel] [PATCH common 2/3] schema: add pve-bridge-id option/format/pair Fabian Grünbichler
@ 2021-11-11 13:18   ` Thomas Lamprecht
  0 siblings, 0 replies; 50+ messages in thread
From: Thomas Lamprecht @ 2021-11-11 13:18 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

On 05.11.21 14:03, Fabian Grünbichler wrote:
> for re-use in qemu-server/pve-container, which already have this option
> duplicated. the '-pair' is needed for remote migration, but can also be
> a nice addition to regular intra-cluster migration to lift the
> restriction of having identically named bridges.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>     v2: rename to bridge-pair
> 
>  src/PVE/JSONSchema.pm | 26 ++++++++++++++++++++++++++
>  1 file changed, 26 insertions(+)
> 
>

applied, thanks!




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

* [pve-devel] applied: [PATCH common 3/3] schema: add proxmox-remote format/option
  2021-11-05 13:03 ` [pve-devel] [PATCH common 3/3] schema: add proxmox-remote format/option Fabian Grünbichler
@ 2021-11-11 13:18   ` Thomas Lamprecht
  0 siblings, 0 replies; 50+ messages in thread
From: Thomas Lamprecht @ 2021-11-11 13:18 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

On 05.11.21 14:03, Fabian Grünbichler wrote:
> for usage in remote migration/replication.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>  src/PVE/JSONSchema.pm | 27 +++++++++++++++++++++++++++
>  1 file changed, 27 insertions(+)
> 
>

applied, thanks!




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

* [pve-devel] applied: [PATCH qemu-server 01/10] d/control: add pve-ha-manager to B-D
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 01/10] d/control: add pve-ha-manager to B-D Fabian Grünbichler
@ 2021-11-11 13:18   ` Thomas Lamprecht
  0 siblings, 0 replies; 50+ messages in thread
From: Thomas Lamprecht @ 2021-11-11 13:18 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

On 05.11.21 14:03, Fabian Grünbichler wrote:
> it's needed for tests to succeed.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>  debian/control | 1 +
>  1 file changed, 1 insertion(+)
> 
>

applied, thanks!




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

* [pve-devel] applied: [PATCH qemu-server 02/10] adapt to renamed storage-pair format
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 02/10] adapt to renamed storage-pair format Fabian Grünbichler
@ 2021-11-11 13:18   ` Thomas Lamprecht
  0 siblings, 0 replies; 50+ messages in thread
From: Thomas Lamprecht @ 2021-11-11 13:18 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

On 05.11.21 14:03, Fabian Grünbichler wrote:
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
> 
> Notes:
>     v2: new
>     
>     requires bumped dependency on pve-common with renaming patch
> 
>  PVE/QemuServer.pm | 2 +-
>  1 file changed, 1 insertion(+), 1 deletion(-)
> 
>

applied, thanks!




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

* [pve-devel] applied: [PATCH qemu-server 03/10] migrate: factor out storage checks
  2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 03/10] migrate: factor out storage checks Fabian Grünbichler
@ 2021-11-11 13:18   ` Thomas Lamprecht
  0 siblings, 0 replies; 50+ messages in thread
From: Thomas Lamprecht @ 2021-11-11 13:18 UTC (permalink / raw)
  To: Proxmox VE development discussion, Fabian Grünbichler

On 05.11.21 14:03, Fabian Grünbichler wrote:
> to re-use them for incoming remote migrations.
> 
> Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
> ---
>  PVE/API2/Qemu.pm | 26 ++++++++++++++------------
>  1 file changed, 14 insertions(+), 12 deletions(-)
> 
>

applied, thanks!




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

* [pve-devel] [PATCH access-control 2/2] ticket: normalize path for verification
  2021-04-13 12:16 [pve-devel] [RFC " Fabian Grünbichler
@ 2021-04-13 12:16 ` Fabian Grünbichler
  0 siblings, 0 replies; 50+ messages in thread
From: Fabian Grünbichler @ 2021-04-13 12:16 UTC (permalink / raw)
  To: pve-devel

Signed-off-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
---
 PVE/AccessControl.pm | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/PVE/AccessControl.pm b/PVE/AccessControl.pm
index 9d9a4bd..7949fde 100644
--- a/PVE/AccessControl.pm
+++ b/PVE/AccessControl.pm
@@ -461,6 +461,8 @@ my $assemble_short_lived_ticket = sub {
 my $verify_short_lived_ticket = sub {
     my ($ticket, $prefix, $username, $path, $noerr) = @_;
 
+    $path = normalize_path($path);
+
     my $secret_data = "$username:$path";
 
     my ($rsa_pub, $rsa_mtime) = get_pubkey();
-- 
2.20.1





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

end of thread, other threads:[~2021-11-11 13:19 UTC | newest]

Thread overview: 50+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-11-05 13:03 [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH proxmox 1/1] websocket: adapt for client connection Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 1/4] initial commit Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 2/4] add tunnel implementation Fabian Grünbichler
2021-11-09 12:54   ` Dominik Csapak
2021-11-11  9:58     ` Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 3/4] add fingerprint validation Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH proxmox-websocket-tunnel 4/4] add packaging Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH access-control 1/2] tickets: add tunnel ticket Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH access-control 2/2] ticket: normalize path for verification Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH common 1/3] schema: rename storagepair to storage-pair Fabian Grünbichler
2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
2021-11-05 13:03 ` [pve-devel] [PATCH common 2/3] schema: add pve-bridge-id option/format/pair Fabian Grünbichler
2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
2021-11-05 13:03 ` [pve-devel] [PATCH common 3/3] schema: add proxmox-remote format/option Fabian Grünbichler
2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
2021-11-05 13:03 ` [pve-devel] [PATCH guest-common 1/1] migrate: handle migration_network with remote migration Fabian Grünbichler
2021-11-08 13:50   ` Fabian Ebner
2021-11-10 12:03   ` Fabian Ebner
2021-11-05 13:03 ` [pve-devel] [PATCH http-server 1/1] webproxy: handle unflushed write buffer Fabian Grünbichler
2021-11-08 14:15   ` Fabian Ebner
2021-11-08 15:45     ` Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 01/10] d/control: add pve-ha-manager to B-D Fabian Grünbichler
2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 02/10] adapt to renamed storage-pair format Fabian Grünbichler
2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 03/10] migrate: factor out storage checks Fabian Grünbichler
2021-11-11 13:18   ` [pve-devel] applied: " Thomas Lamprecht
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 04/10] refactor map_storage to map_id Fabian Grünbichler
2021-11-09  9:06   ` Fabian Ebner
2021-11-09 12:44     ` Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 05/10] schema: use pve-bridge-id Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 06/10] update_vm: allow simultaneous setting of boot-order and dev Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 07/10] mtunnel: add API endpoints Fabian Grünbichler
2021-11-09 12:46   ` Fabian Ebner
2021-11-10  7:40     ` Fabian Ebner
2021-11-11 11:07       ` Fabian Grünbichler
2021-11-11 11:04     ` Fabian Grünbichler
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 08/10] migrate: refactor remote VM/tunnel start Fabian Grünbichler
2021-11-09 14:04   ` Fabian Ebner
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 09/10] migrate: add remote migration handling Fabian Grünbichler
2021-11-10 11:17   ` Fabian Ebner
2021-11-11 12:25     ` Fabian Grünbichler
2021-11-11 12:57       ` Fabian Ebner
2021-11-05 13:03 ` [pve-devel] [PATCH qemu-server 10/10] api: add remote migrate endpoint Fabian Grünbichler
2021-11-10 12:29   ` Fabian Ebner
2021-11-11 12:33     ` Fabian Grünbichler
2021-11-09 16:07 ` [pve-devel] [PATCH-series qemu-server++ 0/22] remote migration DERUMIER, Alexandre
2021-11-10 12:42 ` Fabian Ebner
  -- strict thread matches above, loose matches on Subject: below --
2021-04-13 12:16 [pve-devel] [RFC " Fabian Grünbichler
2021-04-13 12:16 ` [pve-devel] [PATCH access-control 2/2] ticket: normalize path for verification Fabian Grünbichler

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