public inbox for pbs-devel@lists.proxmox.com
 help / color / mirror / Atom feed
* [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target
@ 2024-10-31 12:14 Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 01/29] client: backup writer: refactor backup and upload stats counters Christian Ebner
                   ` (28 more replies)
  0 siblings, 29 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

This patch series implements the functionality to extend the current
sync jobs in pull direction by an additional push direction, allowing
to push contents of a local source datastore to a remote target.

The series implements this by using the REST API of the remote target
for fetching, creating and/or deleting namespaces, groups and backups,
and reuses the clients backup writer functionality to create snapshots
by writing a manifeset on the remote target and sync the fixed index,
dynamic index or blobs contained in the source manifest to the remote,
preserving also encryption information.

Thanks to Fabian and Dominik for further feedback to the previous
patch series version.

Changes since version 5 of the patch series
- Split roles and permissions for separate remote datastore prune and remote
  datastoe modify roles.
- Fetch target groups filtered by ownership, so to not try to push or remove
  unowned groups.
- Add documentation, highlight the caveats of conflicting push jobs when using
  shared remotes.
- Check also for optional `PRIV_DATASTORE_BACKUP` as opposed to only
  `PRIV_DATASTORE_READ` on source datastore namespace, that user can read the
  contents from there as well.
- Drop `sync-direction` parameter from API endpoints where not needed, determine
  it from the corresponding jobs configuration instead.
- Adapt layout of split job view in WebUI to use more general, less component
  specific values
- Introduce `remote_acl_path` helpers for `BackupNamespace` and `SyncJobConfig`.
- Refactor upload counters to bundle and update counters by chunk variant.
- Rework `version` endpoint and supported api feature check to be based on
  `supported_features` rather than a hardcoded version, allowing for more
  flexibility.
- `PushParameters` now always have the remote version for check stored
  unconditionally.
- Renamed `igonre-protected` to a less misinterpretable `error-on-protected` and
  inverted boolean logic.
- Squashed and reorderd patches, the delete stats are not followup patches as
  they are now fully backwards compatible.

Changes since version 4 of the patch series:
- Rebased onto current master

Most notable changes since version 3 of the patch series include:
- Rework access control permission checks to resemble the pull based
  logic more closely.
  In order to perform a full sync in push direction, including
  permissions for pruning contents with remove vansished, a acl.cfg
  looks like below:
  ```
  acl:1:/datastore/source-store:syncoperator@pbs:DatastoreReader
  acl:1:/remote:syncoperator@pbs:RemoteAudit
  acl:1:/remote/remote-target/target-store:syncoperator@pbs:RemoteDatastorePrune,RemoteSyncPushOperator
  ```
- Modify access to sync jobs now requires `DatastoreAudit` for both,
  pull and push sync jobs
- Fix previously incorrect privs required for removing target
  namespaces
- Fix performance bottleneck by not reading known chunks from source,
  by sending `MergedChunkInfo` instead of `ChunkInfo` over to the
  upload stream
- Factor upload statistic counters and structs out into their own
  module and provide methods for easy conversion
- Implement `map_to_target` helper for easier/more readable source to
  target mapping for namespaces
- Optimize namespace creation on target, only try creating non
  pre-existing namespace components.
- Avoid temp file for manifest and upload source manifest directly
- Not failing on deletion for protected snapshots is now opt-in
- Refactor api endpoint `version` in order to be able to fetch api
  version for target
- Reworked `SyncDirection` api type, use `api` macro to reduce code

Most notable changes since version 2 of the patch series include:
- Add checks and extend roles and privs to allow for restricting a local
  users access to remote datastore operations. In order to perform a
  full sync in push direction, including permissions for namespace
  creation and deleting contents with remove vansished, a acl.cfg looks
  like below:
  ```
  acl:1:/datastore/datastore:syncoperator@pbs:DatastoreAudit
  acl:1:/remote:syncoperator@pbs:RemoteSyncOperator
  acl:1:/remote/local/pushme:syncoperator@pbs:RemoteDatastoreModify,RemoteDatastorePrune,RemoteSyncPushOperator
  ```
  Based on further feedback, privs might get further grouped or an
  additional role containing most of these can be created.
- Drop patch introducing `no-timestamp-check` flag for backup client, as pointed
  out by Fabian this is not needed, as only backups newer than the currently
  last available will be pushed.
- Fix read snapshots from source by using the correct namespace.
- Rename PullParameters `owner` to more fitting `local_user`.
- Fix typos in remote sync push operator comment.
- Fix comments not matching the functionality for the cli implementations.

Link to issue on bugtracker:
https://bugzilla.proxmox.com/show_bug.cgi?id=3044

Christian Ebner (29):
  client: backup writer: refactor backup and upload stats counters
  client: backup writer: factor out merged chunk stream upload
  client: backup writer: allow push uploading index and chunks
  config: acl: refactor acl path component check for datastore
  config: acl: allow namespace components for remote datastores
  api types: add remote acl path method for `BackupNamespace`
  api types: implement remote acl path method for sync job
  api types: define remote permissions and roles for push sync
  datastore: move `BackupGroupDeleteStats` to api types
  api types: implement api type for `BackupGroupDeleteStats`
  datastore: increment deleted group counter when removing group
  api/api-types: refactor api endpoint version, add api types
  fix #3044: server: implement push support for sync operations
  api types/config: add `sync-push` config type for push sync jobs
  api: push: implement endpoint for sync in push direction
  api: sync: move sync job invocation to server sync module
  api: config: Require PRIV_DATASTORE_AUDIT to modify sync job
  api: config: factor out sync job owner check
  api: sync jobs: expose optional `sync-direction` parameter
  api: admin: avoid duplicate name for list sync jobs api method
  bin: manager: add datastore push cli command
  ui: group filter: allow to set namespace for local datastore
  ui: sync edit: source group filters based on sync direction
  ui: add view with separate grids for pull and push sync jobs
  ui: sync job: adapt edit window to be used for pull and push
  ui: sync view: set proxy on view instead of model
  api: datastore/namespace: return backup groups delete stats on remove
  api: version: add 'prune-delete-stats' as supported feature
  docs: add section for sync jobs in push direction

 docs/managing-remotes.rst              |  36 +
 pbs-api-types/src/acl.rs               |  31 +
 pbs-api-types/src/datastore.rs         |  76 +-
 pbs-api-types/src/jobs.rs              |  46 ++
 pbs-api-types/src/lib.rs               |   3 +
 pbs-api-types/src/version.rs           | 109 +++
 pbs-client/src/backup_stats.rs         | 119 +++
 pbs-client/src/backup_writer.rs        | 318 +++++---
 pbs-client/src/inject_reused_chunks.rs |  14 +-
 pbs-client/src/lib.rs                  |   4 +
 pbs-config/src/acl.rs                  |  11 +-
 pbs-config/src/sync.rs                 |  16 +-
 pbs-datastore/src/backup_info.rs       |  34 +-
 pbs-datastore/src/datastore.rs         |  27 +-
 src/api2/admin/datastore.rs            |  29 +-
 src/api2/admin/namespace.rs            |  31 +-
 src/api2/admin/sync.rs                 |  43 +-
 src/api2/config/datastore.rs           |  15 +-
 src/api2/config/notifications/mod.rs   |  21 +-
 src/api2/config/sync.rs                | 298 ++++++--
 src/api2/mod.rs                        |   2 +
 src/api2/pull.rs                       | 108 ---
 src/api2/push.rs                       | 183 +++++
 src/api2/version.rs                    |  42 +-
 src/bin/proxmox-backup-manager.rs      | 216 ++++--
 src/bin/proxmox-backup-proxy.rs        |  24 +-
 src/server/mod.rs                      |   2 +
 src/server/pull.rs                     |   6 +-
 src/server/push.rs                     | 980 +++++++++++++++++++++++++
 src/server/sync.rs                     | 150 +++-
 www/Makefile                           |   1 +
 www/config/SyncPullPushView.js         |  61 ++
 www/config/SyncView.js                 |  18 +-
 www/datastore/DataStoreList.js         |   2 +-
 www/datastore/Panel.js                 |   2 +-
 www/form/GroupFilter.js                |  21 +-
 www/window/SyncJobEdit.js              |  45 +-
 37 files changed, 2644 insertions(+), 500 deletions(-)
 create mode 100644 pbs-api-types/src/version.rs
 create mode 100644 pbs-client/src/backup_stats.rs
 create mode 100644 src/api2/push.rs
 create mode 100644 src/server/push.rs
 create mode 100644 www/config/SyncPullPushView.js

-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 01/29] client: backup writer: refactor backup and upload stats counters
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 02/29] client: backup writer: factor out merged chunk stream upload Christian Ebner
                   ` (27 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

In preparation for push support in sync jobs.

Extend and move `BackupStats` into `backup_stats` submodule and add
method to create them from `UploadStats`.

Further, introduce `UploadCounters` struct to hold the Arc clones of
the chunk upload statistics counters, simplifying the house keeping.

By bundling the counters into the struct, they can be passed as
single function parameter when factoring out the common stream future
in the subsequent implementation of the chunk upload for sync jobs
in push direction.

Co-developed-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
Changes since version 5:
- refactor and combine counter helpers based on chunk variant

 pbs-client/src/backup_stats.rs         | 119 ++++++++++++++++++++
 pbs-client/src/backup_writer.rs        | 145 +++++++++----------------
 pbs-client/src/inject_reused_chunks.rs |  14 +--
 pbs-client/src/lib.rs                  |   3 +
 4 files changed, 180 insertions(+), 101 deletions(-)
 create mode 100644 pbs-client/src/backup_stats.rs

diff --git a/pbs-client/src/backup_stats.rs b/pbs-client/src/backup_stats.rs
new file mode 100644
index 000000000..f0563a001
--- /dev/null
+++ b/pbs-client/src/backup_stats.rs
@@ -0,0 +1,119 @@
+//! Implements counters to generate statistics for log outputs during uploads with backup writer
+
+use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
+use std::sync::Arc;
+use std::time::Duration;
+
+use crate::pxar::create::ReusableDynamicEntry;
+
+/// Basic backup run statistics and archive checksum
+pub struct BackupStats {
+    pub size: u64,
+    pub csum: [u8; 32],
+    pub duration: Duration,
+    pub chunk_count: u64,
+}
+
+/// Extended backup run statistics and archive checksum
+pub(crate) struct UploadStats {
+    pub(crate) chunk_count: usize,
+    pub(crate) chunk_reused: usize,
+    pub(crate) chunk_injected: usize,
+    pub(crate) size: usize,
+    pub(crate) size_reused: usize,
+    pub(crate) size_injected: usize,
+    pub(crate) size_compressed: usize,
+    pub(crate) duration: Duration,
+    pub(crate) csum: [u8; 32],
+}
+
+impl UploadStats {
+    /// Convert the upload stats to the more concise [`BackupStats`]
+    #[inline(always)]
+    pub(crate) fn to_backup_stats(&self) -> BackupStats {
+        BackupStats {
+            chunk_count: self.chunk_count as u64,
+            size: self.size as u64,
+            duration: self.duration,
+            csum: self.csum,
+        }
+    }
+}
+
+/// Atomic counters for accounting upload stream progress information
+#[derive(Clone)]
+pub(crate) struct UploadCounters {
+    injected_chunk_count: Arc<AtomicUsize>,
+    known_chunk_count: Arc<AtomicUsize>,
+    total_chunk_count: Arc<AtomicUsize>,
+    compressed_stream_len: Arc<AtomicU64>,
+    injected_stream_len: Arc<AtomicUsize>,
+    reused_stream_len: Arc<AtomicUsize>,
+    total_stream_len: Arc<AtomicUsize>,
+}
+
+impl UploadCounters {
+    /// Create and zero init new upload counters
+    pub(crate) fn new() -> Self {
+        Self {
+            total_chunk_count: Arc::new(AtomicUsize::new(0)),
+            injected_chunk_count: Arc::new(AtomicUsize::new(0)),
+            known_chunk_count: Arc::new(AtomicUsize::new(0)),
+            compressed_stream_len: Arc::new(AtomicU64::new(0)),
+            injected_stream_len: Arc::new(AtomicUsize::new(0)),
+            reused_stream_len: Arc::new(AtomicUsize::new(0)),
+            total_stream_len: Arc::new(AtomicUsize::new(0)),
+        }
+    }
+
+    #[inline(always)]
+    pub(crate) fn add_known_chunk(&mut self, chunk_len: usize) -> usize {
+        self.known_chunk_count.fetch_add(1, Ordering::SeqCst);
+        self.total_chunk_count.fetch_add(1, Ordering::SeqCst);
+        self.reused_stream_len
+            .fetch_add(chunk_len, Ordering::SeqCst);
+        self.total_stream_len.fetch_add(chunk_len, Ordering::SeqCst)
+    }
+
+    #[inline(always)]
+    pub(crate) fn add_new_chunk(&mut self, chunk_len: usize, chunk_raw_size: u64) -> usize {
+        self.total_chunk_count.fetch_add(1, Ordering::SeqCst);
+        self.compressed_stream_len
+            .fetch_add(chunk_raw_size, Ordering::SeqCst);
+        self.total_stream_len.fetch_add(chunk_len, Ordering::SeqCst)
+    }
+
+    #[inline(always)]
+    pub(crate) fn add_injected_chunk(&mut self, chunk: &ReusableDynamicEntry) -> usize {
+        self.total_chunk_count.fetch_add(1, Ordering::SeqCst);
+        self.injected_chunk_count.fetch_add(1, Ordering::SeqCst);
+
+        self.reused_stream_len
+            .fetch_add(chunk.size() as usize, Ordering::SeqCst);
+        self.injected_stream_len
+            .fetch_add(chunk.size() as usize, Ordering::SeqCst);
+        self.total_stream_len
+            .fetch_add(chunk.size() as usize, Ordering::SeqCst)
+    }
+
+    #[inline(always)]
+    pub(crate) fn total_stream_len(&self) -> usize {
+        self.total_stream_len.load(Ordering::SeqCst)
+    }
+
+    /// Convert the counters to [`UploadStats`], including given archive checksum and runtime.
+    #[inline(always)]
+    pub(crate) fn to_upload_stats(&self, csum: [u8; 32], duration: Duration) -> UploadStats {
+        UploadStats {
+            chunk_count: self.total_chunk_count.load(Ordering::SeqCst),
+            chunk_reused: self.known_chunk_count.load(Ordering::SeqCst),
+            chunk_injected: self.injected_chunk_count.load(Ordering::SeqCst),
+            size: self.total_stream_len.load(Ordering::SeqCst),
+            size_reused: self.reused_stream_len.load(Ordering::SeqCst),
+            size_injected: self.injected_stream_len.load(Ordering::SeqCst),
+            size_compressed: self.compressed_stream_len.load(Ordering::SeqCst) as usize,
+            duration,
+            csum,
+        }
+    }
+}
diff --git a/pbs-client/src/backup_writer.rs b/pbs-client/src/backup_writer.rs
index 4d2e8a801..8b9afdb95 100644
--- a/pbs-client/src/backup_writer.rs
+++ b/pbs-client/src/backup_writer.rs
@@ -1,7 +1,8 @@
 use std::collections::HashSet;
 use std::future::Future;
-use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
+use std::sync::atomic::{AtomicUsize, Ordering};
 use std::sync::{Arc, Mutex};
+use std::time::Instant;
 
 use anyhow::{bail, format_err, Error};
 use futures::future::{self, AbortHandle, Either, FutureExt, TryFutureExt};
@@ -23,6 +24,7 @@ use pbs_tools::crypt_config::CryptConfig;
 use proxmox_human_byte::HumanByte;
 use proxmox_time::TimeSpan;
 
+use super::backup_stats::{BackupStats, UploadCounters, UploadStats};
 use super::inject_reused_chunks::{InjectChunks, InjectReusedChunks, InjectedChunksInfo};
 use super::merge_known_chunks::{MergeKnownChunks, MergedChunkInfo};
 
@@ -40,11 +42,6 @@ impl Drop for BackupWriter {
     }
 }
 
-pub struct BackupStats {
-    pub size: u64,
-    pub csum: [u8; 32],
-}
-
 /// Options for uploading blobs/streams to the server
 #[derive(Default, Clone)]
 pub struct UploadOptions {
@@ -54,18 +51,6 @@ pub struct UploadOptions {
     pub fixed_size: Option<u64>,
 }
 
-struct UploadStats {
-    chunk_count: usize,
-    chunk_reused: usize,
-    chunk_injected: usize,
-    size: usize,
-    size_reused: usize,
-    size_injected: usize,
-    size_compressed: usize,
-    duration: std::time::Duration,
-    csum: [u8; 32],
-}
-
 struct ChunkUploadResponse {
     future: h2::client::ResponseFuture,
     size: usize,
@@ -194,6 +179,7 @@ impl BackupWriter {
         mut reader: R,
         file_name: &str,
     ) -> Result<BackupStats, Error> {
+        let start_time = Instant::now();
         let mut raw_data = Vec::new();
         // fixme: avoid loading into memory
         reader.read_to_end(&mut raw_data)?;
@@ -211,7 +197,12 @@ impl BackupWriter {
                 raw_data,
             )
             .await?;
-        Ok(BackupStats { size, csum })
+        Ok(BackupStats {
+            size,
+            csum,
+            duration: start_time.elapsed(),
+            chunk_count: 0,
+        })
     }
 
     pub async fn upload_blob_from_data(
@@ -220,6 +211,7 @@ impl BackupWriter {
         file_name: &str,
         options: UploadOptions,
     ) -> Result<BackupStats, Error> {
+        let start_time = Instant::now();
         let blob = match (options.encrypt, &self.crypt_config) {
             (false, _) => DataBlob::encode(&data, None, options.compress)?,
             (true, None) => bail!("requested encryption without a crypt config"),
@@ -243,7 +235,12 @@ impl BackupWriter {
                 raw_data,
             )
             .await?;
-        Ok(BackupStats { size, csum })
+        Ok(BackupStats {
+            size,
+            csum,
+            duration: start_time.elapsed(),
+            chunk_count: 0,
+        })
     }
 
     pub async fn upload_blob_from_file<P: AsRef<std::path::Path>>(
@@ -421,10 +418,7 @@ impl BackupWriter {
             "csum": hex::encode(upload_stats.csum),
         });
         let _value = self.h2.post(&close_path, Some(param)).await?;
-        Ok(BackupStats {
-            size: upload_stats.size as u64,
-            csum: upload_stats.csum,
-        })
+        Ok(upload_stats.to_backup_stats())
     }
 
     fn response_queue() -> (
@@ -653,23 +647,9 @@ impl BackupWriter {
         injections: Option<std::sync::mpsc::Receiver<InjectChunks>>,
         archive: &str,
     ) -> impl Future<Output = Result<UploadStats, Error>> {
-        let total_chunks = Arc::new(AtomicUsize::new(0));
-        let total_chunks2 = total_chunks.clone();
-        let known_chunk_count = Arc::new(AtomicUsize::new(0));
-        let known_chunk_count2 = known_chunk_count.clone();
-        let injected_chunk_count = Arc::new(AtomicUsize::new(0));
-        let injected_chunk_count2 = injected_chunk_count.clone();
-
-        let stream_len = Arc::new(AtomicUsize::new(0));
-        let stream_len2 = stream_len.clone();
-        let stream_len3 = stream_len.clone();
-        let compressed_stream_len = Arc::new(AtomicU64::new(0));
-        let compressed_stream_len2 = compressed_stream_len.clone();
-        let reused_len = Arc::new(AtomicUsize::new(0));
-        let reused_len2 = reused_len.clone();
-        let injected_len = Arc::new(AtomicUsize::new(0));
-        let injected_len2 = injected_len.clone();
-        let uploaded_len = Arc::new(AtomicUsize::new(0));
+        let mut counters = UploadCounters::new();
+        let uploaded_len = Arc::new(std::sync::atomic::AtomicUsize::new(0));
+        let counters_readonly = counters.clone();
 
         let append_chunk_path = format!("{}_index", prefix);
         let upload_chunk_path = format!("{}_chunk", prefix);
@@ -687,11 +667,12 @@ impl BackupWriter {
             || archive.ends_with(".pxar")
             || archive.ends_with(".ppxar")
         {
+            let counters = counters.clone();
             Some(tokio::spawn(async move {
                 loop {
                     tokio::time::sleep(tokio::time::Duration::from_secs(60)).await;
 
-                    let size = HumanByte::from(stream_len3.load(Ordering::SeqCst));
+                    let size = HumanByte::from(counters.total_stream_len());
                     let size_uploaded = HumanByte::from(uploaded_len.load(Ordering::SeqCst));
                     let elapsed = TimeSpan::from(start_time.elapsed());
 
@@ -703,22 +684,15 @@ impl BackupWriter {
         };
 
         stream
-            .inject_reused_chunks(injections, stream_len.clone())
+            .inject_reused_chunks(injections, counters.clone())
             .and_then(move |chunk_info| match chunk_info {
                 InjectedChunksInfo::Known(chunks) => {
                     // account for injected chunks
-                    let count = chunks.len();
-                    total_chunks.fetch_add(count, Ordering::SeqCst);
-                    injected_chunk_count.fetch_add(count, Ordering::SeqCst);
-
                     let mut known = Vec::new();
                     let mut guard = index_csum.lock().unwrap();
                     let csum = guard.as_mut().unwrap();
                     for chunk in chunks {
-                        let offset =
-                            stream_len.fetch_add(chunk.size() as usize, Ordering::SeqCst) as u64;
-                        reused_len.fetch_add(chunk.size() as usize, Ordering::SeqCst);
-                        injected_len.fetch_add(chunk.size() as usize, Ordering::SeqCst);
+                        let offset = counters.add_injected_chunk(&chunk) as u64;
                         let digest = chunk.digest();
                         known.push((offset, digest));
                         let end_offset = offset + chunk.size();
@@ -731,9 +705,6 @@ impl BackupWriter {
                     // account for not injected chunks (new and known)
                     let chunk_len = data.len();
 
-                    total_chunks.fetch_add(1, Ordering::SeqCst);
-                    let offset = stream_len.fetch_add(chunk_len, Ordering::SeqCst) as u64;
-
                     let mut chunk_builder = DataChunkBuilder::new(data.as_ref()).compress(compress);
 
                     if let Some(ref crypt_config) = crypt_config {
@@ -741,7 +712,29 @@ impl BackupWriter {
                     }
 
                     let mut known_chunks = known_chunks.lock().unwrap();
-                    let digest = chunk_builder.digest();
+                    let digest = *chunk_builder.digest();
+                    let (offset, res) = if known_chunks.contains(&digest) {
+                        let offset = counters.add_known_chunk(chunk_len) as u64;
+                        (offset, MergedChunkInfo::Known(vec![(offset, digest)]))
+                    } else {
+                        match chunk_builder.build() {
+                            Ok((chunk, digest)) => {
+                                let offset =
+                                    counters.add_new_chunk(chunk_len, chunk.raw_size()) as u64;
+                                known_chunks.insert(digest);
+                                (
+                                    offset,
+                                    MergedChunkInfo::New(ChunkInfo {
+                                        chunk,
+                                        digest,
+                                        chunk_len: chunk_len as u64,
+                                        offset,
+                                    }),
+                                )
+                            }
+                            Err(err) => return future::err(err),
+                        }
+                    };
 
                     let mut guard = index_csum.lock().unwrap();
                     let csum = guard.as_mut().unwrap();
@@ -751,26 +744,9 @@ impl BackupWriter {
                     if !is_fixed_chunk_size {
                         csum.update(&chunk_end.to_le_bytes());
                     }
-                    csum.update(digest);
+                    csum.update(&digest);
 
-                    let chunk_is_known = known_chunks.contains(digest);
-                    if chunk_is_known {
-                        known_chunk_count.fetch_add(1, Ordering::SeqCst);
-                        reused_len.fetch_add(chunk_len, Ordering::SeqCst);
-                        future::ok(MergedChunkInfo::Known(vec![(offset, *digest)]))
-                    } else {
-                        let compressed_stream_len2 = compressed_stream_len.clone();
-                        known_chunks.insert(*digest);
-                        future::ready(chunk_builder.build().map(move |(chunk, digest)| {
-                            compressed_stream_len2.fetch_add(chunk.raw_size(), Ordering::SeqCst);
-                            MergedChunkInfo::New(ChunkInfo {
-                                chunk,
-                                digest,
-                                chunk_len: chunk_len as u64,
-                                offset,
-                            })
-                        }))
-                    }
+                    future::ok(res)
                 }
             })
             .merge_known_chunks()
@@ -837,15 +813,6 @@ impl BackupWriter {
             })
             .then(move |result| async move { upload_result.await?.and(result) }.boxed())
             .and_then(move |_| {
-                let duration = start_time.elapsed();
-                let chunk_count = total_chunks2.load(Ordering::SeqCst);
-                let chunk_reused = known_chunk_count2.load(Ordering::SeqCst);
-                let chunk_injected = injected_chunk_count2.load(Ordering::SeqCst);
-                let size = stream_len2.load(Ordering::SeqCst);
-                let size_reused = reused_len2.load(Ordering::SeqCst);
-                let size_injected = injected_len2.load(Ordering::SeqCst);
-                let size_compressed = compressed_stream_len2.load(Ordering::SeqCst) as usize;
-
                 let mut guard = index_csum_2.lock().unwrap();
                 let csum = guard.take().unwrap().finish();
 
@@ -853,17 +820,7 @@ impl BackupWriter {
                     handle.abort();
                 }
 
-                futures::future::ok(UploadStats {
-                    chunk_count,
-                    chunk_reused,
-                    chunk_injected,
-                    size,
-                    size_reused,
-                    size_injected,
-                    size_compressed,
-                    duration,
-                    csum,
-                })
+                futures::future::ok(counters_readonly.to_upload_stats(csum, start_time.elapsed()))
             })
     }
 
diff --git a/pbs-client/src/inject_reused_chunks.rs b/pbs-client/src/inject_reused_chunks.rs
index 4b2922012..6da2bcd16 100644
--- a/pbs-client/src/inject_reused_chunks.rs
+++ b/pbs-client/src/inject_reused_chunks.rs
@@ -1,13 +1,13 @@
 use std::cmp;
 use std::pin::Pin;
-use std::sync::atomic::{AtomicUsize, Ordering};
-use std::sync::{mpsc, Arc};
+use std::sync::mpsc;
 use std::task::{Context, Poll};
 
 use anyhow::{anyhow, Error};
 use futures::{ready, Stream};
 use pin_project_lite::pin_project;
 
+use crate::backup_stats::UploadCounters;
 use crate::pxar::create::ReusableDynamicEntry;
 
 pin_project! {
@@ -16,7 +16,7 @@ pin_project! {
         input: S,
         next_injection: Option<InjectChunks>,
         injections: Option<mpsc::Receiver<InjectChunks>>,
-        stream_len: Arc<AtomicUsize>,
+        counters: UploadCounters,
     }
 }
 
@@ -42,7 +42,7 @@ pub trait InjectReusedChunks: Sized {
     fn inject_reused_chunks(
         self,
         injections: Option<mpsc::Receiver<InjectChunks>>,
-        stream_len: Arc<AtomicUsize>,
+        counters: UploadCounters,
     ) -> InjectReusedChunksQueue<Self>;
 }
 
@@ -53,13 +53,13 @@ where
     fn inject_reused_chunks(
         self,
         injections: Option<mpsc::Receiver<InjectChunks>>,
-        stream_len: Arc<AtomicUsize>,
+        counters: UploadCounters,
     ) -> InjectReusedChunksQueue<Self> {
         InjectReusedChunksQueue {
             input: self,
             next_injection: None,
             injections,
-            stream_len,
+            counters,
         }
     }
 }
@@ -85,7 +85,7 @@ where
 
             if let Some(inject) = this.next_injection.take() {
                 // got reusable dynamic entries to inject
-                let offset = this.stream_len.load(Ordering::SeqCst) as u64;
+                let offset = this.counters.total_stream_len() as u64;
 
                 match inject.boundary.cmp(&offset) {
                     // inject now
diff --git a/pbs-client/src/lib.rs b/pbs-client/src/lib.rs
index 3d2da27b9..b875347bb 100644
--- a/pbs-client/src/lib.rs
+++ b/pbs-client/src/lib.rs
@@ -41,4 +41,7 @@ pub use backup_specification::*;
 mod chunk_stream;
 pub use chunk_stream::{ChunkStream, FixedChunkStream, InjectionData};
 
+mod backup_stats;
+pub use backup_stats::BackupStats;
+
 pub const PROXMOX_BACKUP_TCP_KEEPALIVE_TIME: u32 = 120;
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel

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

* [pbs-devel] [PATCH v6 proxmox-backup 02/29] client: backup writer: factor out merged chunk stream upload
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 01/29] client: backup writer: refactor backup and upload stats counters Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 03/29] client: backup writer: allow push uploading index and chunks Christian Ebner
                   ` (26 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

In preparation for implementing push support for sync jobs.

Factor out the upload stream for merged chunks, which can be reused
to upload the local chunks to a remote target datastore during a
snapshot sync operation in push direction.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
Changes since version 5:
- refactor to adapt to UploadCounter changes

 pbs-client/src/backup_writer.rs | 88 +++++++++++++++++++++------------
 1 file changed, 56 insertions(+), 32 deletions(-)

diff --git a/pbs-client/src/backup_writer.rs b/pbs-client/src/backup_writer.rs
index 8b9afdb95..f1bad4128 100644
--- a/pbs-client/src/backup_writer.rs
+++ b/pbs-client/src/backup_writer.rs
@@ -7,6 +7,7 @@ use std::time::Instant;
 use anyhow::{bail, format_err, Error};
 use futures::future::{self, AbortHandle, Either, FutureExt, TryFutureExt};
 use futures::stream::{Stream, StreamExt, TryStreamExt};
+use openssl::sha::Sha256;
 use serde_json::{json, Value};
 use tokio::io::AsyncReadExt;
 use tokio::sync::{mpsc, oneshot};
@@ -648,42 +649,14 @@ impl BackupWriter {
         archive: &str,
     ) -> impl Future<Output = Result<UploadStats, Error>> {
         let mut counters = UploadCounters::new();
-        let uploaded_len = Arc::new(std::sync::atomic::AtomicUsize::new(0));
         let counters_readonly = counters.clone();
 
-        let append_chunk_path = format!("{}_index", prefix);
-        let upload_chunk_path = format!("{}_chunk", prefix);
         let is_fixed_chunk_size = prefix == "fixed";
 
-        let (upload_queue, upload_result) =
-            Self::append_chunk_queue(h2.clone(), wid, append_chunk_path, uploaded_len.clone());
-
-        let start_time = std::time::Instant::now();
-
         let index_csum = Arc::new(Mutex::new(Some(openssl::sha::Sha256::new())));
         let index_csum_2 = index_csum.clone();
 
-        let progress_handle = if archive.ends_with(".img")
-            || archive.ends_with(".pxar")
-            || archive.ends_with(".ppxar")
-        {
-            let counters = counters.clone();
-            Some(tokio::spawn(async move {
-                loop {
-                    tokio::time::sleep(tokio::time::Duration::from_secs(60)).await;
-
-                    let size = HumanByte::from(counters.total_stream_len());
-                    let size_uploaded = HumanByte::from(uploaded_len.load(Ordering::SeqCst));
-                    let elapsed = TimeSpan::from(start_time.elapsed());
-
-                    log::info!("processed {size} in {elapsed}, uploaded {size_uploaded}");
-                }
-            }))
-        } else {
-            None
-        };
-
-        stream
+        let stream = stream
             .inject_reused_chunks(injections, counters.clone())
             .and_then(move |chunk_info| match chunk_info {
                 InjectedChunksInfo::Known(chunks) => {
@@ -749,7 +722,58 @@ impl BackupWriter {
                     future::ok(res)
                 }
             })
-            .merge_known_chunks()
+            .merge_known_chunks();
+
+        Self::upload_merged_chunk_stream(
+            h2,
+            wid,
+            archive,
+            prefix,
+            stream,
+            index_csum_2,
+            counters_readonly,
+        )
+    }
+
+    fn upload_merged_chunk_stream(
+        h2: H2Client,
+        wid: u64,
+        archive: &str,
+        prefix: &str,
+        stream: impl Stream<Item = Result<MergedChunkInfo, Error>>,
+        index_csum: Arc<Mutex<Option<Sha256>>>,
+        counters: UploadCounters,
+    ) -> impl Future<Output = Result<UploadStats, Error>> {
+        let append_chunk_path = format!("{prefix}_index");
+        let upload_chunk_path = format!("{prefix}_chunk");
+
+        let start_time = std::time::Instant::now();
+        let uploaded_len = Arc::new(AtomicUsize::new(0));
+
+        let (upload_queue, upload_result) =
+            Self::append_chunk_queue(h2.clone(), wid, append_chunk_path, uploaded_len.clone());
+
+        let progress_handle = if archive.ends_with(".img")
+            || archive.ends_with(".pxar")
+            || archive.ends_with(".ppxar")
+        {
+            let counters = counters.clone();
+            Some(tokio::spawn(async move {
+                loop {
+                    tokio::time::sleep(tokio::time::Duration::from_secs(60)).await;
+
+                    let size = HumanByte::from(counters.total_stream_len());
+                    let size_uploaded = HumanByte::from(uploaded_len.load(Ordering::SeqCst));
+                    let elapsed = TimeSpan::from(start_time.elapsed());
+
+                    log::info!("processed {size} in {elapsed}, uploaded {size_uploaded}");
+                }
+            }))
+        } else {
+            None
+        };
+
+        stream
             .try_for_each(move |merged_chunk_info| {
                 let upload_queue = upload_queue.clone();
 
@@ -813,14 +837,14 @@ impl BackupWriter {
             })
             .then(move |result| async move { upload_result.await?.and(result) }.boxed())
             .and_then(move |_| {
-                let mut guard = index_csum_2.lock().unwrap();
+                let mut guard = index_csum.lock().unwrap();
                 let csum = guard.take().unwrap().finish();
 
                 if let Some(handle) = progress_handle {
                     handle.abort();
                 }
 
-                futures::future::ok(counters_readonly.to_upload_stats(csum, start_time.elapsed()))
+                futures::future::ok(counters.to_upload_stats(csum, start_time.elapsed()))
             })
     }
 
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 03/29] client: backup writer: allow push uploading index and chunks
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 01/29] client: backup writer: refactor backup and upload stats counters Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 02/29] client: backup writer: factor out merged chunk stream upload Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 04/29] config: acl: refactor acl path component check for datastore Christian Ebner
                   ` (25 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

Add a method `upload_index_chunk_info` to be used for uploading an
existing index and the corresponding chunk stream.
Instead of taking an input stream of raw bytes as the
`upload_stream`, this takes a stream of `MergedChunkInfo` object
provided by the local chunk reader of the sync jobs source.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
Changes since version 5:
- refactor to adapt to UploadCounter changes
- fixes incorrect compressed stream counter

 pbs-client/src/backup_writer.rs | 93 +++++++++++++++++++++++++++++++++
 pbs-client/src/lib.rs           |  1 +
 2 files changed, 94 insertions(+)

diff --git a/pbs-client/src/backup_writer.rs b/pbs-client/src/backup_writer.rs
index f1bad4128..685510da3 100644
--- a/pbs-client/src/backup_writer.rs
+++ b/pbs-client/src/backup_writer.rs
@@ -266,6 +266,99 @@ impl BackupWriter {
             .await
     }
 
+    /// Upload chunks and index
+    pub async fn upload_index_chunk_info(
+        &self,
+        archive_name: &str,
+        stream: impl Stream<Item = Result<MergedChunkInfo, Error>>,
+        options: UploadOptions,
+    ) -> Result<BackupStats, Error> {
+        let mut param = json!({ "archive-name": archive_name });
+        let prefix = if let Some(size) = options.fixed_size {
+            param["size"] = size.into();
+            "fixed"
+        } else {
+            "dynamic"
+        };
+
+        if options.encrypt && self.crypt_config.is_none() {
+            bail!("requested encryption without a crypt config");
+        }
+
+        let wid = self
+            .h2
+            .post(&format!("{prefix}_index"), Some(param))
+            .await?
+            .as_u64()
+            .unwrap();
+
+        let mut counters = UploadCounters::new();
+        let counters_readonly = counters.clone();
+
+        let is_fixed_chunk_size = prefix == "fixed";
+
+        let index_csum = Arc::new(Mutex::new(Some(Sha256::new())));
+        let index_csum_2 = index_csum.clone();
+
+        let stream = stream
+            .and_then(move |mut merged_chunk_info| {
+                match merged_chunk_info {
+                    MergedChunkInfo::New(ref chunk_info) => {
+                        let chunk_len = chunk_info.chunk_len;
+                        let offset =
+                            counters.add_new_chunk(chunk_len as usize, chunk_info.chunk.raw_size());
+                        let end_offset = offset as u64 + chunk_len;
+                        let mut guard = index_csum.lock().unwrap();
+                        let csum = guard.as_mut().unwrap();
+                        if !is_fixed_chunk_size {
+                            csum.update(&end_offset.to_le_bytes());
+                        }
+                        csum.update(&chunk_info.digest);
+                    }
+                    MergedChunkInfo::Known(ref mut known_chunk_list) => {
+                        for (chunk_len, digest) in known_chunk_list {
+                            let offset = counters.add_known_chunk(*chunk_len as usize);
+                            let end_offset = offset as u64 + *chunk_len;
+                            let mut guard = index_csum.lock().unwrap();
+                            let csum = guard.as_mut().unwrap();
+                            if !is_fixed_chunk_size {
+                                csum.update(&end_offset.to_le_bytes());
+                            }
+                            csum.update(digest);
+                            // Replace size with offset, expected by further stream
+                            *chunk_len = offset as u64;
+                        }
+                    }
+                }
+                future::ok(merged_chunk_info)
+            })
+            .merge_known_chunks();
+
+        let upload_stats = Self::upload_merged_chunk_stream(
+            self.h2.clone(),
+            wid,
+            archive_name,
+            prefix,
+            stream,
+            index_csum_2,
+            counters_readonly,
+        )
+        .await?;
+
+        let param = json!({
+            "wid": wid ,
+            "chunk-count": upload_stats.chunk_count,
+            "size": upload_stats.size,
+            "csum": hex::encode(upload_stats.csum),
+        });
+        let _value = self
+            .h2
+            .post(&format!("{prefix}_close"), Some(param))
+            .await?;
+
+        Ok(upload_stats.to_backup_stats())
+    }
+
     pub async fn upload_stream(
         &self,
         archive_name: &str,
diff --git a/pbs-client/src/lib.rs b/pbs-client/src/lib.rs
index b875347bb..4b8e4e4f4 100644
--- a/pbs-client/src/lib.rs
+++ b/pbs-client/src/lib.rs
@@ -9,6 +9,7 @@ pub mod tools;
 
 mod inject_reused_chunks;
 mod merge_known_chunks;
+pub use merge_known_chunks::MergedChunkInfo;
 pub mod pipe_to_stream;
 
 mod http_client;
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 04/29] config: acl: refactor acl path component check for datastore
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (2 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 03/29] client: backup writer: allow push uploading index and chunks Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 05/29] config: acl: allow namespace components for remote datastores Christian Ebner
                   ` (24 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

Combine the two if statements checking the datastores ACL path
components, which can be represented more concisely as one.

Further, extend the pre-existing comment to clarify that `datastore`
ACL paths are not limited to the datastore name, but might have
further sub-components specifying the namespace.

Suggested-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 pbs-config/src/acl.rs | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

diff --git a/pbs-config/src/acl.rs b/pbs-config/src/acl.rs
index 4ce4c13c0..29ad3e8c9 100644
--- a/pbs-config/src/acl.rs
+++ b/pbs-config/src/acl.rs
@@ -80,11 +80,8 @@ pub fn check_acl_path(path: &str) -> Result<(), Error> {
             }
         }
         "datastore" => {
-            // /datastore/{store}
-            if components_len <= 2 {
-                return Ok(());
-            }
-            if components_len > 2 && components_len <= 2 + pbs_api_types::MAX_NAMESPACE_DEPTH {
+            // /datastore/{store}/{namespace}
+            if components_len <= 2 + pbs_api_types::MAX_NAMESPACE_DEPTH {
                 return Ok(());
             }
         }
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel

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

* [pbs-devel] [PATCH v6 proxmox-backup 05/29] config: acl: allow namespace components for remote datastores
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (3 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 04/29] config: acl: refactor acl path component check for datastore Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 06/29] api types: add remote acl path method for `BackupNamespace` Christian Ebner
                   ` (23 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

Extend the component limit for ACL paths of `remote` to include
possible namespace components.

This allows to limit the permissions for sync jobs in push direction
to a namespace subset on the remote datastore.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 pbs-config/src/acl.rs | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/pbs-config/src/acl.rs b/pbs-config/src/acl.rs
index 29ad3e8c9..a06b918ad 100644
--- a/pbs-config/src/acl.rs
+++ b/pbs-config/src/acl.rs
@@ -86,8 +86,8 @@ pub fn check_acl_path(path: &str) -> Result<(), Error> {
             }
         }
         "remote" => {
-            // /remote/{remote}/{store}
-            if components_len <= 3 {
+            // /remote/{remote}/{store}/{namespace}
+            if components_len <= 3 + pbs_api_types::MAX_NAMESPACE_DEPTH {
                 return Ok(());
             }
         }
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 06/29] api types: add remote acl path method for `BackupNamespace`
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (4 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 05/29] config: acl: allow namespace components for remote datastores Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 07/29] api types: implement remote acl path method for sync job Christian Ebner
                   ` (22 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

Add a `remote_acl_path` helper method for creating acl paths for
remote namespaces, to be used by the priv checks on remote datastore
namespaces for e.g. the sync job in push direction.

Factor out the common path extension into a dedicated method.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- not present in previous version

 pbs-api-types/src/datastore.rs | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)

diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
index 31767417a..b037b6fec 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -717,9 +717,7 @@ impl BackupNamespace {
         Ok(())
     }
 
-    pub fn acl_path<'a>(&'a self, store: &'a str) -> Vec<&'a str> {
-        let mut path: Vec<&str> = vec!["datastore", store];
-
+    fn acl_path_extend<'a>(&'a self, mut path: Vec<&'a str>) -> Vec<&'a str> {
         if self.is_root() {
             path
         } else {
@@ -728,6 +726,14 @@ impl BackupNamespace {
         }
     }
 
+    pub fn acl_path<'a>(&'a self, store: &'a str) -> Vec<&'a str> {
+        self.acl_path_extend(vec!["datastore", store])
+    }
+
+    pub fn remote_acl_path<'a>(&'a self, remote: &'a str, store: &'a str) -> Vec<&'a str> {
+        self.acl_path_extend(vec!["remote", remote, store])
+    }
+
     /// Check whether this namespace contains another namespace.
     ///
     /// If so, the depth is returned.
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 07/29] api types: implement remote acl path method for sync job
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (5 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 06/29] api types: add remote acl path method for `BackupNamespace` Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 08/29] api types: define remote permissions and roles for push sync Christian Ebner
                   ` (21 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

Add `remote_acl_path` method which generates the acl path from the sync
job configuration. This helper allows to easily generate the acl path
from a given sync job config for privilege checks.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- use BackupNamespace::remote_acl_path helper

 pbs-api-types/src/jobs.rs | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/pbs-api-types/src/jobs.rs b/pbs-api-types/src/jobs.rs
index 868702bc0..bf7a6bd5a 100644
--- a/pbs-api-types/src/jobs.rs
+++ b/pbs-api-types/src/jobs.rs
@@ -594,6 +594,14 @@ impl SyncJobConfig {
             None => vec!["datastore", &self.store],
         }
     }
+
+    pub fn remote_acl_path(&self) -> Option<Vec<&str>> {
+        let remote = self.remote.as_ref()?;
+        match &self.remote_ns {
+            Some(remote_ns) => Some(remote_ns.remote_acl_path(remote, &self.remote_store)),
+            None => Some(vec!["remote", remote, &self.remote_store]),
+        }
+    }
 }
 
 #[api(
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 08/29] api types: define remote permissions and roles for push sync
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (6 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 07/29] api types: implement remote acl path method for sync job Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 09/29] datastore: move `BackupGroupDeleteStats` to api types Christian Ebner
                   ` (20 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

Adding the privileges to allow backup, namespace creation and prune
on remote targets, to be used for sync jobs in push direction.

Also adds dedicated roles setting the required privileges.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- dedicated remote datastore modify and remote datastore prune roles
- remove local datastore read access for sync job push operator

 pbs-api-types/src/acl.rs | 31 +++++++++++++++++++++++++++++++
 1 file changed, 31 insertions(+)

diff --git a/pbs-api-types/src/acl.rs b/pbs-api-types/src/acl.rs
index a8ae57a9d..68ed59105 100644
--- a/pbs-api-types/src/acl.rs
+++ b/pbs-api-types/src/acl.rs
@@ -58,6 +58,12 @@ constnamedbitmap! {
         PRIV_REMOTE_MODIFY("Remote.Modify");
         /// Remote.Read allows reading data from a configured `Remote`
         PRIV_REMOTE_READ("Remote.Read");
+        /// Remote.DatastoreBackup allows creating new snapshots on remote datastores
+        PRIV_REMOTE_DATASTORE_BACKUP("Remote.DatastoreBackup");
+        /// Remote.DatastoreModify allows to modify remote datastores
+        PRIV_REMOTE_DATASTORE_MODIFY("Remote.DatastoreModify");
+        /// Remote.DatastorePrune allows deleting snapshots on remote datastores
+        PRIV_REMOTE_DATASTORE_PRUNE("Remote.DatastorePrune");
 
         /// Sys.Console allows access to the system's console
         PRIV_SYS_CONSOLE("Sys.Console");
@@ -160,6 +166,25 @@ pub const ROLE_REMOTE_SYNC_OPERATOR: u64 = 0
     | PRIV_REMOTE_AUDIT
     | PRIV_REMOTE_READ;
 
+#[rustfmt::skip]
+#[allow(clippy::identity_op)]
+/// Remote.SyncPushOperator can do read and push snapshots to the remote.
+pub const ROLE_REMOTE_SYNC_PUSH_OPERATOR: u64 = 0
+    | PRIV_REMOTE_AUDIT
+    | PRIV_REMOTE_DATASTORE_BACKUP;
+
+#[rustfmt::skip]
+#[allow(clippy::identity_op)]
+/// Remote.DatastorePrune can prune owned snapshots and groups.
+pub const ROLE_REMOTE_DATASTORE_PRUNE: u64 = 0
+    | PRIV_REMOTE_DATASTORE_PRUNE;
+
+#[rustfmt::skip]
+#[allow(clippy::identity_op)]
+/// Remote.DatastoreModify can create and remove namespaces on the remote.
+pub const ROLE_REMOTE_DATASTORE_MODIFY: u64 = 0
+    | PRIV_REMOTE_DATASTORE_MODIFY;
+
 #[rustfmt::skip]
 #[allow(clippy::identity_op)]
 /// Tape.Audit can audit the tape backup configuration and media content
@@ -225,6 +250,12 @@ pub enum Role {
     RemoteAdmin = ROLE_REMOTE_ADMIN,
     /// Synchronization Operator
     RemoteSyncOperator = ROLE_REMOTE_SYNC_OPERATOR,
+    /// Synchronisation Operator (push direction)
+    RemoteSyncPushOperator = ROLE_REMOTE_SYNC_PUSH_OPERATOR,
+    /// Remote Datastore Prune
+    RemoteDatastorePrune = ROLE_REMOTE_DATASTORE_PRUNE,
+    /// Remote Datastore Modify
+    RemoteDatastoreModify = ROLE_REMOTE_DATASTORE_MODIFY,
     /// Tape Auditor
     TapeAudit = ROLE_TAPE_AUDIT,
     /// Tape Administrator
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 09/29] datastore: move `BackupGroupDeleteStats` to api types
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (7 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 08/29] api types: define remote permissions and roles for push sync Christian Ebner
@ 2024-10-31 12:14 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 10/29] api types: implement api type for `BackupGroupDeleteStats` Christian Ebner
                   ` (19 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:14 UTC (permalink / raw)
  To: pbs-devel

In preparation for the delete stats to be exposed as return type to
the backup group delete api endpoint.

Also, rename the private field `unremoved_protected` to a better
fitting `protected_snapshots` to be in line with the method names.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- reordered patch, so BackupGroupDeleteStats can be used already in
  earlier patches, since delete stats are not considered followup
  patches anymore as there is no breaking api change
- no changes

 pbs-api-types/src/datastore.rs   | 30 +++++++++++++++++++++++++++++
 pbs-datastore/src/backup_info.rs | 33 ++------------------------------
 pbs-datastore/src/datastore.rs   |  7 ++++---
 3 files changed, 36 insertions(+), 34 deletions(-)

diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
index b037b6fec..9e866bef2 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -1575,3 +1575,33 @@ pub fn print_store_and_ns(store: &str, ns: &BackupNamespace) -> String {
         format!("datastore '{}', namespace '{}'", store, ns)
     }
 }
+
+#[derive(Default)]
+pub struct BackupGroupDeleteStats {
+    // Count of protected snapshots, therefore not removed
+    protected_snapshots: usize,
+    // Count of deleted snapshots
+    removed_snapshots: usize,
+}
+
+impl BackupGroupDeleteStats {
+    pub fn all_removed(&self) -> bool {
+        self.protected_snapshots == 0
+    }
+
+    pub fn removed_snapshots(&self) -> usize {
+        self.removed_snapshots
+    }
+
+    pub fn protected_snapshots(&self) -> usize {
+        self.protected_snapshots
+    }
+
+    pub fn increment_removed_snapshots(&mut self) {
+        self.removed_snapshots += 1;
+    }
+
+    pub fn increment_protected_snapshots(&mut self) {
+        self.protected_snapshots += 1;
+    }
+}
diff --git a/pbs-datastore/src/backup_info.rs b/pbs-datastore/src/backup_info.rs
index 414ec878d..222134074 100644
--- a/pbs-datastore/src/backup_info.rs
+++ b/pbs-datastore/src/backup_info.rs
@@ -8,7 +8,8 @@ use anyhow::{bail, format_err, Error};
 use proxmox_sys::fs::{lock_dir_noblock, replace_file, CreateOptions};
 
 use pbs_api_types::{
-    Authid, BackupNamespace, BackupType, GroupFilter, BACKUP_DATE_REGEX, BACKUP_FILE_REGEX,
+    Authid, BackupGroupDeleteStats, BackupNamespace, BackupType, GroupFilter, BACKUP_DATE_REGEX,
+    BACKUP_FILE_REGEX,
 };
 use pbs_config::{open_backup_lockfile, BackupLockGuard};
 
@@ -17,36 +18,6 @@ use crate::manifest::{
 };
 use crate::{DataBlob, DataStore};
 
-#[derive(Default)]
-pub struct BackupGroupDeleteStats {
-    // Count of protected snapshots, therefore not removed
-    unremoved_protected: usize,
-    // Count of deleted snapshots
-    removed_snapshots: usize,
-}
-
-impl BackupGroupDeleteStats {
-    pub fn all_removed(&self) -> bool {
-        self.unremoved_protected == 0
-    }
-
-    pub fn removed_snapshots(&self) -> usize {
-        self.removed_snapshots
-    }
-
-    pub fn protected_snapshots(&self) -> usize {
-        self.unremoved_protected
-    }
-
-    fn increment_removed_snapshots(&mut self) {
-        self.removed_snapshots += 1;
-    }
-
-    fn increment_protected_snapshots(&mut self) {
-        self.unremoved_protected += 1;
-    }
-}
-
 /// BackupGroup is a directory containing a list of BackupDir
 #[derive(Clone)]
 pub struct BackupGroup {
diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index d0f3c53ac..c8701d2dd 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -18,11 +18,12 @@ use proxmox_sys::process_locker::ProcessLockSharedGuard;
 use proxmox_worker_task::WorkerTaskContext;
 
 use pbs_api_types::{
-    Authid, BackupNamespace, BackupType, ChunkOrder, DataStoreConfig, DatastoreFSyncLevel,
-    DatastoreTuning, GarbageCollectionStatus, MaintenanceMode, MaintenanceType, Operation, UPID,
+    Authid, BackupGroupDeleteStats, BackupNamespace, BackupType, ChunkOrder, DataStoreConfig,
+    DatastoreFSyncLevel, DatastoreTuning, GarbageCollectionStatus, MaintenanceMode,
+    MaintenanceType, Operation, UPID,
 };
 
-use crate::backup_info::{BackupDir, BackupGroup, BackupGroupDeleteStats};
+use crate::backup_info::{BackupDir, BackupGroup};
 use crate::chunk_store::ChunkStore;
 use crate::dynamic_index::{DynamicIndexReader, DynamicIndexWriter};
 use crate::fixed_index::{FixedIndexReader, FixedIndexWriter};
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 10/29] api types: implement api type for `BackupGroupDeleteStats`
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (8 preceding siblings ...)
  2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 09/29] datastore: move `BackupGroupDeleteStats` to api types Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 11/29] datastore: increment deleted group counter when removing group Christian Ebner
                   ` (18 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Make the `BackupGroupDeleteStats` exposable via the API by implementing
the ApiTypes trait via the api macro invocation and add an additional
field to account for the number of deleted groups.
Further, add a method to add up the statistics.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- reordered patch for delete stats counters to be used earlier in the
  patch series
- no changes

 pbs-api-types/src/datastore.rs | 36 +++++++++++++++++++++++++++++++++-
 1 file changed, 35 insertions(+), 1 deletion(-)

diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
index 9e866bef2..ec100a703 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -1576,8 +1576,28 @@ pub fn print_store_and_ns(store: &str, ns: &BackupNamespace) -> String {
     }
 }
 
-#[derive(Default)]
+pub const DELETE_STATS_COUNT_SCHEMA: Schema =
+    IntegerSchema::new("Number of entities").minimum(0).schema();
+
+#[api(
+    properties: {
+        "removed-groups": {
+            schema: DELETE_STATS_COUNT_SCHEMA,
+        },
+        "protected-snapshots": {
+            schema: DELETE_STATS_COUNT_SCHEMA,
+        },
+        "removed-snapshots": {
+            schema: DELETE_STATS_COUNT_SCHEMA,
+        },
+     },
+)]
+#[derive(Default, Deserialize, Serialize)]
+#[serde(rename_all = "kebab-case")]
+/// Statistics for removed backup groups
 pub struct BackupGroupDeleteStats {
+    // Count of removed groups
+    removed_groups: usize,
     // Count of protected snapshots, therefore not removed
     protected_snapshots: usize,
     // Count of deleted snapshots
@@ -1589,6 +1609,10 @@ impl BackupGroupDeleteStats {
         self.protected_snapshots == 0
     }
 
+    pub fn removed_groups(&self) -> usize {
+        self.removed_groups
+    }
+
     pub fn removed_snapshots(&self) -> usize {
         self.removed_snapshots
     }
@@ -1597,6 +1621,16 @@ impl BackupGroupDeleteStats {
         self.protected_snapshots
     }
 
+    pub fn add(&mut self, rhs: &Self) {
+        self.removed_groups += rhs.removed_groups;
+        self.protected_snapshots += rhs.protected_snapshots;
+        self.removed_snapshots += rhs.removed_snapshots;
+    }
+
+    pub fn increment_removed_groups(&mut self) {
+        self.removed_groups += 1;
+    }
+
     pub fn increment_removed_snapshots(&mut self) {
         self.removed_snapshots += 1;
     }
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 11/29] datastore: increment deleted group counter when removing group
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (9 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 10/29] api types: implement api type for `BackupGroupDeleteStats` Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 12/29] api/api-types: refactor api endpoint version, add api types Christian Ebner
                   ` (17 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

To correctly account also for the number of deleted backup groups, in
preparation to correctly return the delete statistics when removing
contents via the REST API.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- reordered
- no changes

 pbs-datastore/src/backup_info.rs | 1 +
 1 file changed, 1 insertion(+)

diff --git a/pbs-datastore/src/backup_info.rs b/pbs-datastore/src/backup_info.rs
index 222134074..62d12b118 100644
--- a/pbs-datastore/src/backup_info.rs
+++ b/pbs-datastore/src/backup_info.rs
@@ -221,6 +221,7 @@ impl BackupGroup {
             std::fs::remove_dir_all(&path).map_err(|err| {
                 format_err!("removing group directory {:?} failed - {}", path, err)
             })?;
+            delete_stats.increment_removed_groups();
         }
 
         Ok(delete_stats)
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 12/29] api/api-types: refactor api endpoint version, add api types
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (10 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 11/29] datastore: increment deleted group counter when removing group Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 13/29] fix #3044: server: implement push support for sync operations Christian Ebner
                   ` (16 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Add a dedicated api type for the `version` api endpoint and helper
methods for supported feature comparison.
This will be used to detect api incompatibility of older hosts, not
supporting some features.

Use the new api type to refactor the version endpoint and set it as
return type.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- add `features` vector to store supported feature strings
- drop `min_version_check`, introduce `supported_feature` check

 pbs-api-types/src/lib.rs     |   3 +
 pbs-api-types/src/version.rs | 109 +++++++++++++++++++++++++++++++++++
 src/api2/version.rs          |  42 ++++++++------
 3 files changed, 137 insertions(+), 17 deletions(-)
 create mode 100644 pbs-api-types/src/version.rs

diff --git a/pbs-api-types/src/lib.rs b/pbs-api-types/src/lib.rs
index 460c7da7c..6bae4a52b 100644
--- a/pbs-api-types/src/lib.rs
+++ b/pbs-api-types/src/lib.rs
@@ -155,6 +155,9 @@ pub use zfs::*;
 mod metrics;
 pub use metrics::*;
 
+mod version;
+pub use version::*;
+
 const_regex! {
     // just a rough check - dummy acceptor is used before persisting
     pub OPENSSL_CIPHERS_REGEX = r"^[0-9A-Za-z_:, +!\-@=.]+$";
diff --git a/pbs-api-types/src/version.rs b/pbs-api-types/src/version.rs
new file mode 100644
index 000000000..c7c91a53a
--- /dev/null
+++ b/pbs-api-types/src/version.rs
@@ -0,0 +1,109 @@
+//! Defines the types for the api version info endpoint
+use std::convert::TryFrom;
+
+use anyhow::Context;
+
+use proxmox_schema::api;
+
+#[api(
+    description: "Api version information",
+    properties: {
+        "version": {
+            description: "Version 'major.minor'",
+            type: String,
+        },
+        "release": {
+            description: "Version release",
+            type: String,
+        },
+        "repoid": {
+            description: "Version repository id",
+            type: String,
+        },
+        "features": {
+            description: "List of supported features",
+            type: Array,
+            items: {
+                type: String,
+                description: "Feature id",
+            },
+        },
+    }
+)]
+#[derive(serde::Deserialize, serde::Serialize)]
+pub struct ApiVersionInfo {
+    pub version: String,
+    pub release: String,
+    pub repoid: String,
+    #[serde(default, skip_serializing_if = "Vec::is_empty")]
+    pub features: Vec<String>,
+}
+
+pub type ApiVersionMajor = u64;
+pub type ApiVersionMinor = u64;
+pub type ApiVersionRelease = u64;
+
+#[allow(dead_code)]
+pub struct ApiVersion {
+    major: ApiVersionMajor,
+    minor: ApiVersionMinor,
+    release: ApiVersionRelease,
+    features: Vec<String>,
+}
+
+impl TryFrom<ApiVersionInfo> for ApiVersion {
+    type Error = anyhow::Error;
+
+    fn try_from(value: ApiVersionInfo) -> Result<Self, Self::Error> {
+        let mut parts = value.version.split('.');
+        let major: ApiVersionMajor = if let Some(val) = parts.next() {
+            val.parse()
+                .with_context(|| "failed to parse major version")?
+        } else {
+            ApiVersionMajor::default()
+        };
+        let minor: ApiVersionMinor = if let Some(val) = parts.next() {
+            val.parse()
+                .with_context(|| "failed to parse minor version")?
+        } else {
+            ApiVersionMinor::default()
+        };
+
+        let release: ApiVersionMinor = value
+            .release
+            .parse()
+            .with_context(|| "failed to parse release version")?;
+
+        Ok(Self {
+            major,
+            minor,
+            release,
+            features: value.features.to_vec(),
+        })
+    }
+}
+
+impl ApiVersion {
+    pub fn new(
+        major: ApiVersionMajor,
+        minor: ApiVersionMinor,
+        release: ApiVersionRelease,
+        features: Vec<String>,
+    ) -> Self {
+        Self {
+            major,
+            minor,
+            release,
+            features,
+        }
+    }
+
+    pub fn supports_feature(&self, feature: &str) -> bool {
+        for supported_feature in &self.features {
+            if *supported_feature == feature {
+                return true;
+            }
+        }
+        false
+    }
+}
diff --git a/src/api2/version.rs b/src/api2/version.rs
index 0e91688b5..a6cec5216 100644
--- a/src/api2/version.rs
+++ b/src/api2/version.rs
@@ -1,27 +1,35 @@
 //! Version information
 
 use anyhow::Error;
-use serde_json::{json, Value};
+use serde_json::Value;
 
-use proxmox_router::{ApiHandler, ApiMethod, Permission, Router, RpcEnvironment};
-use proxmox_schema::ObjectSchema;
+use proxmox_router::{ApiMethod, Permission, Router, RpcEnvironment};
+use proxmox_schema::api;
 
-fn get_version(
+use pbs_api_types::ApiVersionInfo;
+
+const FEATURES: &'static [&'static str] = &[];
+
+#[api(
+    returns: {
+        type: ApiVersionInfo,
+    },
+    access: {
+        permission: &Permission::Anybody,
+    }
+)]
+///Proxmox Backup Server API version.
+fn version(
     _param: Value,
     _info: &ApiMethod,
     _rpcenv: &mut dyn RpcEnvironment,
-) -> Result<Value, Error> {
-    Ok(json!({
-        "version": pbs_buildcfg::PROXMOX_PKG_VERSION,
-        "release": pbs_buildcfg::PROXMOX_PKG_RELEASE,
-        "repoid": pbs_buildcfg::PROXMOX_PKG_REPOID
-    }))
+) -> Result<ApiVersionInfo, Error> {
+    Ok(ApiVersionInfo {
+        version: pbs_buildcfg::PROXMOX_PKG_VERSION.to_string(),
+        release: pbs_buildcfg::PROXMOX_PKG_RELEASE.to_string(),
+        repoid: pbs_buildcfg::PROXMOX_PKG_REPOID.to_string(),
+        features: FEATURES.iter().map(|feature| feature.to_string()).collect(),
+    })
 }
 
-pub const ROUTER: Router = Router::new().get(
-    &ApiMethod::new(
-        &ApiHandler::Sync(&get_version),
-        &ObjectSchema::new("Proxmox Backup Server API version.", &[]),
-    )
-    .access(None, &Permission::Anybody),
-);
+pub const ROUTER: Router = Router::new().get(&API_METHOD_VERSION);
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 13/29] fix #3044: server: implement push support for sync operations
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (11 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 12/29] api/api-types: refactor api endpoint version, add api types Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 14/29] api types/config: add `sync-push` config type for push sync jobs Christian Ebner
                   ` (15 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Adds the functionality required to push datastore contents from a
source to a remote target.
This includes syncing of the namespaces, backup groups and snapshots
based on the provided filters as well as removing vanished contents
from the target when requested.

While trying to mimic the pull direction of sync jobs, the
implementation is different as access to the remote must be performed
via the REST API, not needed for the pull job which can access the
local datastore via the filesystem directly.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- fetch backup groups split by owned and not owned, only allow to push
  to owned groups, don't allow to prune not owned groups.
- store remote api version unconditionally
- check for supported feature instead of api version to include conditional
  parameters for api calls
- directly use delete stats from api calls, since these are not followup
  patches anymore

 src/server/mod.rs  |   1 +
 src/server/push.rs | 980 +++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 981 insertions(+)
 create mode 100644 src/server/push.rs

diff --git a/src/server/mod.rs b/src/server/mod.rs
index 2e40bde3c..7c14ed4b8 100644
--- a/src/server/mod.rs
+++ b/src/server/mod.rs
@@ -36,6 +36,7 @@ pub mod auth;
 pub mod metric_collection;
 
 pub(crate) mod pull;
+pub(crate) mod push;
 pub(crate) mod sync;
 
 pub(crate) async fn reload_proxy_certificate() -> Result<(), Error> {
diff --git a/src/server/push.rs b/src/server/push.rs
new file mode 100644
index 000000000..c38e9c96b
--- /dev/null
+++ b/src/server/push.rs
@@ -0,0 +1,980 @@
+//! Sync datastore by pushing contents to remote server
+
+use std::cmp::Ordering;
+use std::collections::HashSet;
+use std::sync::{Arc, Mutex};
+
+use anyhow::{bail, format_err, Error};
+use futures::stream::{self, StreamExt, TryStreamExt};
+use tokio::sync::mpsc;
+use tokio_stream::wrappers::ReceiverStream;
+use tracing::{info, warn};
+
+use pbs_api_types::{
+    print_store_and_ns, ApiVersion, ApiVersionInfo, Authid, BackupDir, BackupGroup,
+    BackupGroupDeleteStats, BackupNamespace, CryptMode, GroupFilter, GroupListItem,
+    NamespaceListItem, Operation, RateLimitConfig, Remote, SnapshotListItem,
+    PRIV_REMOTE_DATASTORE_BACKUP, PRIV_REMOTE_DATASTORE_MODIFY, PRIV_REMOTE_DATASTORE_PRUNE,
+};
+use pbs_client::{BackupRepository, BackupWriter, HttpClient, MergedChunkInfo, UploadOptions};
+use pbs_config::CachedUserInfo;
+use pbs_datastore::data_blob::ChunkInfo;
+use pbs_datastore::dynamic_index::DynamicIndexReader;
+use pbs_datastore::fixed_index::FixedIndexReader;
+use pbs_datastore::index::IndexFile;
+use pbs_datastore::manifest::{ArchiveType, CLIENT_LOG_BLOB_NAME, MANIFEST_BLOB_NAME};
+use pbs_datastore::read_chunk::AsyncReadChunk;
+use pbs_datastore::{BackupManifest, DataStore, StoreProgress};
+
+use super::sync::{
+    check_namespace_depth_limit, LocalSource, RemovedVanishedStats, SkipInfo, SkipReason,
+    SyncSource, SyncStats,
+};
+use crate::api2::config::remote;
+
+/// Target for backups to be pushed to
+pub(crate) struct PushTarget {
+    // Remote as found in remote.cfg
+    remote: Remote,
+    // Target repository on remote
+    repo: BackupRepository,
+    // Target namespace on remote
+    ns: BackupNamespace,
+    // Http client to connect to remote
+    client: HttpClient,
+    // Api version reported by the target
+    api_version: ApiVersion,
+}
+
+impl PushTarget {
+    fn remote_user(&self) -> Authid {
+        self.remote.config.auth_id.clone()
+    }
+}
+
+/// Parameters for a push operation
+pub(crate) struct PushParameters {
+    /// Source of backups to be pushed to remote
+    source: Arc<LocalSource>,
+    /// Target for backups to be pushed to
+    target: PushTarget,
+    /// Local user limiting the accessible source contents, makes sure that the sync job sees the
+    /// same source content when executed by different users with different privileges
+    /// User as which the job gets executed, requires the permissions on the remote
+    local_user: Authid,
+    /// Whether to remove groups which exist locally, but not on the remote end
+    remove_vanished: bool,
+    /// How many levels of sub-namespaces to push (0 == no recursion, None == maximum recursion)
+    max_depth: Option<usize>,
+    /// Filters for reducing the push scope
+    group_filter: Vec<GroupFilter>,
+    /// How many snapshots should be transferred at most (taking the newest N snapshots)
+    transfer_last: Option<usize>,
+}
+
+impl PushParameters {
+    /// Creates a new instance of `PushParameters`.
+    #[allow(clippy::too_many_arguments)]
+    pub(crate) async fn new(
+        store: &str,
+        ns: BackupNamespace,
+        remote_id: &str,
+        remote_store: &str,
+        remote_ns: BackupNamespace,
+        local_user: Authid,
+        remove_vanished: Option<bool>,
+        max_depth: Option<usize>,
+        group_filter: Option<Vec<GroupFilter>>,
+        limit: RateLimitConfig,
+        transfer_last: Option<usize>,
+    ) -> Result<Self, Error> {
+        if let Some(max_depth) = max_depth {
+            ns.check_max_depth(max_depth)?;
+            remote_ns.check_max_depth(max_depth)?;
+        };
+        let remove_vanished = remove_vanished.unwrap_or(false);
+
+        let source = Arc::new(LocalSource {
+            store: DataStore::lookup_datastore(store, Some(Operation::Read))?,
+            ns,
+        });
+
+        let (remote_config, _digest) = pbs_config::remote::config()?;
+        let remote: Remote = remote_config.lookup("remote", remote_id)?;
+
+        let repo = BackupRepository::new(
+            Some(remote.config.auth_id.clone()),
+            Some(remote.config.host.clone()),
+            remote.config.port,
+            remote_store.to_string(),
+        );
+
+        let client = remote::remote_client_config(&remote, Some(limit))?;
+
+        let mut result = client.get("api2/json/version", None).await?;
+        let data = result["data"].take();
+        let version_info: ApiVersionInfo = serde_json::from_value(data)?;
+        let api_version = ApiVersion::try_from(version_info)?;
+        let target = PushTarget {
+            remote,
+            repo,
+            ns: remote_ns,
+            client,
+            api_version,
+        };
+        let group_filter = group_filter.unwrap_or_default();
+
+        Ok(Self {
+            source,
+            target,
+            local_user,
+            remove_vanished,
+            max_depth,
+            group_filter,
+            transfer_last,
+        })
+    }
+
+    // Map the given namespace from source to target by adapting the prefix
+    fn map_to_target(&self, namespace: &BackupNamespace) -> Result<BackupNamespace, Error> {
+        namespace.map_prefix(&self.source.ns, &self.target.ns)
+    }
+}
+
+// Check if the job user given in the push parameters has the provided privs on the remote
+// datastore namespace
+fn check_ns_remote_datastore_privs(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    privs: u64,
+) -> Result<(), Error> {
+    let user_info = CachedUserInfo::new()?;
+    let mut acl_path: Vec<&str> = vec![
+        "remote",
+        &params.target.remote.name,
+        params.target.repo.store(),
+    ];
+
+    if !namespace.is_root() {
+        let ns_components: Vec<&str> = namespace.components().collect();
+        acl_path.extend(ns_components);
+    }
+
+    user_info.check_privs(&params.local_user, &acl_path, privs, false)?;
+
+    Ok(())
+}
+
+// Fetch the list of namespaces found on target
+async fn fetch_target_namespaces(params: &PushParameters) -> Result<Vec<BackupNamespace>, Error> {
+    let api_path = format!(
+        "api2/json/admin/datastore/{store}/namespace",
+        store = params.target.repo.store(),
+    );
+    let mut result = params.target.client.get(&api_path, None).await?;
+    let namespaces: Vec<NamespaceListItem> = serde_json::from_value(result["data"].take())?;
+    let mut namespaces: Vec<BackupNamespace> = namespaces
+        .into_iter()
+        .map(|namespace| namespace.ns)
+        .collect();
+    namespaces.sort_unstable_by_key(|a| a.name_len());
+
+    Ok(namespaces)
+}
+
+// Remove the provided namespace from the target
+async fn remove_target_namespace(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+) -> Result<BackupGroupDeleteStats, Error> {
+    if namespace.is_root() {
+        bail!("cannot remove root namespace from target");
+    }
+
+    check_ns_remote_datastore_privs(params, namespace, PRIV_REMOTE_DATASTORE_MODIFY)
+        .map_err(|err| format_err!("Pruning remote datastore contents not allowed - {err}"))?;
+
+    let api_path = format!(
+        "api2/json/admin/datastore/{store}/namespace",
+        store = params.target.repo.store(),
+    );
+
+    let target_ns = params.map_to_target(namespace)?;
+    let mut args = serde_json::json!({
+        "ns": target_ns.name(),
+        "delete-groups": true,
+    });
+
+    let api_feature_supported = params.target.api_version.supports_feature("prune-delete-stats");
+    if api_feature_supported {
+        args["error-on-protected"] = serde_json::to_value(false)?;
+    }
+
+    let mut result = params.target.client.delete(&api_path, Some(args)).await?;
+    let data = result["data"].take();
+    let delete_stats: BackupGroupDeleteStats = if api_feature_supported {
+        serde_json::from_value(data)?
+    } else {
+        serde_json::from_value(data).unwrap_or_else(|_| BackupGroupDeleteStats::default())
+    };
+
+    Ok(delete_stats)
+}
+
+// Fetch the list of groups found on target in given namespace
+// Returns sorted list of owned groups and a hashset containing not owned backup groups on target.
+async fn fetch_target_groups(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+) -> Result<(Vec<BackupGroup>, HashSet<BackupGroup>), Error> {
+    let api_path = format!(
+        "api2/json/admin/datastore/{store}/groups",
+        store = params.target.repo.store(),
+    );
+
+    let args = if !namespace.is_root() {
+        let target_ns = params.map_to_target(namespace)?;
+        Some(serde_json::json!({ "ns": target_ns.name() }))
+    } else {
+        None
+    };
+
+    let mut result = params.target.client.get(&api_path, args).await?;
+    let groups: Vec<GroupListItem> = serde_json::from_value(result["data"].take())?;
+
+    let (mut owned, not_owned) = groups.iter().fold(
+        (Vec::new(), HashSet::new()),
+        |(mut owned, mut not_owned), group| {
+            if let Some(ref owner) = group.owner {
+                if params.target.remote_user() == *owner {
+                    owned.push(group.backup.clone());
+                    return (owned, not_owned);
+                }
+            }
+            not_owned.insert(group.backup.clone());
+            (owned, not_owned)
+        },
+    );
+
+    owned.sort_unstable_by(|a, b| {
+        let type_order = a.ty.cmp(&b.ty);
+        if type_order == Ordering::Equal {
+            a.id.cmp(&b.id)
+        } else {
+            type_order
+        }
+    });
+
+    Ok((owned, not_owned))
+}
+
+// Remove the provided backup group in given namespace from the target
+async fn remove_target_group(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    backup_group: &BackupGroup,
+) -> Result<BackupGroupDeleteStats, Error> {
+    check_ns_remote_datastore_privs(params, namespace, PRIV_REMOTE_DATASTORE_PRUNE)
+        .map_err(|err| format_err!("Pruning remote datastore contents not allowed - {err}"))?;
+
+    let api_path = format!(
+        "api2/json/admin/datastore/{store}/groups",
+        store = params.target.repo.store(),
+    );
+
+    let mut args = serde_json::json!({
+        "backup-id": backup_group.id,
+        "backup-type": backup_group.ty,
+    });
+
+    let api_feature_supported = params.target.api_version.supports_feature("prune-delete-stats");
+
+    if api_feature_supported {
+        args["error-on-protected"] = serde_json::to_value(false)?;
+    }
+    if !namespace.is_root() {
+        let target_ns = params.map_to_target(namespace)?;
+        args["ns"] = serde_json::to_value(target_ns.name())?;
+    }
+
+    let mut result = params.target.client.delete(&api_path, Some(args)).await?;
+    let data = result["data"].take();
+    let delete_stats: BackupGroupDeleteStats = if api_feature_supported {
+        serde_json::from_value(data)?
+    } else {
+        serde_json::from_value(data).unwrap_or_else(|_| BackupGroupDeleteStats::default())
+    };
+    Ok(delete_stats)
+}
+
+// Check if the namespace is already present on the target, create it otherwise
+async fn check_or_create_target_namespace(
+    params: &PushParameters,
+    target_namespaces: &[BackupNamespace],
+    namespace: &BackupNamespace,
+) -> Result<bool, Error> {
+    let mut created = false;
+
+    if !namespace.is_root() && !target_namespaces.contains(namespace) {
+        // Namespace not present on target, create namespace.
+        // Sub-namespaces have to be created by creating parent components first.
+
+        check_ns_remote_datastore_privs(params, namespace, PRIV_REMOTE_DATASTORE_MODIFY)
+            .map_err(|err| format_err!("Creating namespace not allowed - {err}"))?;
+
+        let mut parent = BackupNamespace::root();
+        for component in namespace.components() {
+            let current = BackupNamespace::from_parent_ns(&parent, component.to_string())?;
+            // Skip over pre-existing parent namespaces on target
+            if target_namespaces.contains(&current) {
+                parent = current;
+                continue;
+            }
+            let api_path = format!(
+                "api2/json/admin/datastore/{store}/namespace",
+                store = params.target.repo.store(),
+            );
+            let mut args = serde_json::json!({ "name": component.to_string() });
+            if !parent.is_root() {
+                args["parent"] = serde_json::to_value(parent.clone())?;
+            }
+            if let Err(err) = params.target.client.post(&api_path, Some(args)).await {
+                let target_store_and_ns = print_store_and_ns(params.target.repo.store(), &current);
+                bail!("sync into {target_store_and_ns} failed - namespace creation failed: {err}");
+            }
+            created = true;
+            parent = current;
+        }
+    }
+
+    Ok(created)
+}
+
+/// Push contents of source datastore matched by given push parameters to target.
+pub(crate) async fn push_store(mut params: PushParameters) -> Result<SyncStats, Error> {
+    let mut errors = false;
+    if !params.target.api_version.supports_feature("prune-delete-stats") && params.remove_vanished {
+        info!("Older api version on remote detected, delete stats might be incomplete");
+    }
+
+    // Generate list of source namespaces to push to target, limited by max-depth
+    let mut namespaces = params.source.list_namespaces(&mut params.max_depth).await?;
+
+    check_namespace_depth_limit(&params.source.get_ns(), &params.target.ns, &namespaces)?;
+
+    namespaces.sort_unstable_by_key(|a| a.name_len());
+
+    // Fetch all accessible namespaces already present on the target
+    let target_namespaces = fetch_target_namespaces(&params).await?;
+    // Remember synced namespaces, removing non-synced ones when remove vanished flag is set
+    let mut synced_namespaces = HashSet::with_capacity(namespaces.len());
+
+    let (mut groups, mut snapshots) = (0, 0);
+    let mut stats = SyncStats::default();
+    for namespace in namespaces {
+        let source_store_and_ns = print_store_and_ns(params.source.store.name(), &namespace);
+        let target_namespace = params.map_to_target(&namespace)?;
+        let target_store_and_ns = print_store_and_ns(params.target.repo.store(), &target_namespace);
+
+        info!("----");
+        info!("Syncing {source_store_and_ns} into {target_store_and_ns}");
+
+        synced_namespaces.insert(target_namespace.clone());
+
+        match check_or_create_target_namespace(&params, &target_namespaces, &target_namespace).await
+        {
+            Ok(true) => info!("Created namespace {target_namespace}"),
+            Ok(false) => {}
+            Err(err) => {
+                info!("Cannot sync {source_store_and_ns} into {target_store_and_ns} - {err}");
+                errors = true;
+                continue;
+            }
+        }
+
+        match push_namespace(&namespace, &params).await {
+            Ok((sync_progress, sync_stats, sync_errors)) => {
+                errors |= sync_errors;
+                stats.add(sync_stats);
+
+                if params.max_depth != Some(0) {
+                    groups += sync_progress.done_groups;
+                    snapshots += sync_progress.done_snapshots;
+
+                    let ns = if namespace.is_root() {
+                        "root namespace".into()
+                    } else {
+                        format!("namespace {namespace}")
+                    };
+                    info!(
+                        "Finished syncing {ns}, current progress: {groups} groups, {snapshots} snapshots"
+                    );
+                }
+            }
+            Err(err) => {
+                errors = true;
+                info!("Encountered errors while syncing namespace {namespace} - {err}");
+            }
+        }
+    }
+
+    if params.remove_vanished {
+        for target_namespace in target_namespaces {
+            if synced_namespaces.contains(&target_namespace) {
+                continue;
+            }
+            match remove_target_namespace(&params, &target_namespace).await {
+                Ok(delete_stats) => {
+                    stats.add(SyncStats::from(RemovedVanishedStats {
+                        snapshots: delete_stats.removed_snapshots(),
+                        groups: delete_stats.removed_groups(),
+                        namespaces: 1,
+                    }));
+                    if delete_stats.protected_snapshots() > 0 {
+                        warn!(
+                            "kept {protected_count} protected snapshots of namespace '{target_namespace}'",
+                            protected_count = delete_stats.protected_snapshots(),
+                        );
+                        continue;
+                    }
+                }
+                Err(err) => {
+                    warn!("failed to remove vanished namespace {target_namespace} - {err}");
+                    continue;
+                }
+            }
+            info!("removed vanished namespace {target_namespace}");
+        }
+    }
+
+    if errors {
+        bail!("sync failed with some errors.");
+    }
+
+    Ok(stats)
+}
+
+/// Push namespace including all backup groups to target
+///
+/// Iterate over all backup groups in the namespace and push them to the target.
+pub(crate) async fn push_namespace(
+    namespace: &BackupNamespace,
+    params: &PushParameters,
+) -> Result<(StoreProgress, SyncStats, bool), Error> {
+    // Check if user is allowed to perform backups on remote datastore
+    check_ns_remote_datastore_privs(params, namespace, PRIV_REMOTE_DATASTORE_BACKUP)
+        .map_err(|err| format_err!("Pushing to remote not allowed - {err}"))?;
+
+    let mut list: Vec<BackupGroup> = params
+        .source
+        .list_groups(namespace, &params.local_user)
+        .await?;
+
+    list.sort_unstable_by(|a, b| {
+        let type_order = a.ty.cmp(&b.ty);
+        if type_order == Ordering::Equal {
+            a.id.cmp(&b.id)
+        } else {
+            type_order
+        }
+    });
+
+    let total = list.len();
+    let list: Vec<BackupGroup> = list
+        .into_iter()
+        .filter(|group| group.apply_filters(&params.group_filter))
+        .collect();
+
+    info!(
+        "found {filtered} groups to sync (out of {total} total)",
+        filtered = list.len()
+    );
+
+    let mut errors = false;
+    // Remember synced groups, remove others when the remove vanished flag is set
+    let mut synced_groups = HashSet::new();
+    let mut progress = StoreProgress::new(list.len() as u64);
+    let mut stats = SyncStats::default();
+
+    let (owned_target_groups, not_owned_target_groups) =
+        fetch_target_groups(params, namespace).await?;
+
+    for (done, group) in list.into_iter().enumerate() {
+        progress.done_groups = done as u64;
+        progress.done_snapshots = 0;
+        progress.group_snapshots = 0;
+
+        if not_owned_target_groups.contains(&group) {
+            warn!("group '{group}' not owned by remote user on target, skip");
+            continue;
+        }
+        synced_groups.insert(group.clone());
+
+        match push_group(params, namespace, &group, &mut progress).await {
+            Ok(sync_stats) => stats.add(sync_stats),
+            Err(err) => {
+                warn!("sync group '{group}' failed  - {err}");
+                errors = true;
+            }
+        }
+    }
+
+    if params.remove_vanished {
+        // only ever allow to prune owned groups on target
+        for target_group in owned_target_groups {
+            if synced_groups.contains(&target_group) {
+                continue;
+            }
+            if !target_group.apply_filters(&params.group_filter) {
+                continue;
+            }
+
+            info!("delete vanished group '{target_group}'");
+
+            match remove_target_group(params, namespace, &target_group).await {
+                Ok(delete_stats) => {
+                    if delete_stats.protected_snapshots() > 0 {
+                        warn!(
+                            "kept {protected_count} protected snapshots of group '{target_group}'",
+                            protected_count = delete_stats.protected_snapshots(),
+                        );
+                    }
+                    stats.add(SyncStats::from(RemovedVanishedStats {
+                        snapshots: delete_stats.removed_snapshots(),
+                        groups: delete_stats.removed_groups(),
+                        namespaces: 0,
+                    }));
+                }
+                Err(err) => {
+                    warn!("failed to delete vanished group - {err}");
+                    errors = true;
+                    continue;
+                }
+            }
+        }
+    }
+
+    Ok((progress, stats, errors))
+}
+
+async fn fetch_target_snapshots(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    group: &BackupGroup,
+) -> Result<Vec<SnapshotListItem>, Error> {
+    let api_path = format!(
+        "api2/json/admin/datastore/{store}/snapshots",
+        store = params.target.repo.store(),
+    );
+    let mut args = serde_json::to_value(group)?;
+    if !namespace.is_root() {
+        let target_ns = params.map_to_target(namespace)?;
+        args["ns"] = serde_json::to_value(target_ns)?;
+    }
+    let mut result = params.target.client.get(&api_path, Some(args)).await?;
+    let snapshots: Vec<SnapshotListItem> = serde_json::from_value(result["data"].take())?;
+
+    Ok(snapshots)
+}
+
+async fn fetch_previous_backup_time(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    group: &BackupGroup,
+) -> Result<Option<i64>, Error> {
+    let mut snapshots = fetch_target_snapshots(params, namespace, group).await?;
+    snapshots.sort_unstable_by(|a, b| a.backup.time.cmp(&b.backup.time));
+    Ok(snapshots.last().map(|snapshot| snapshot.backup.time))
+}
+
+async fn forget_target_snapshot(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    snapshot: &BackupDir,
+) -> Result<(), Error> {
+    check_ns_remote_datastore_privs(params, namespace, PRIV_REMOTE_DATASTORE_PRUNE)
+        .map_err(|err| format_err!("Pruning remote datastore contents not allowed - {err}"))?;
+
+    let api_path = format!(
+        "api2/json/admin/datastore/{store}/snapshots",
+        store = params.target.repo.store(),
+    );
+    let mut args = serde_json::to_value(snapshot)?;
+    if !namespace.is_root() {
+        let target_ns = params.map_to_target(namespace)?;
+        args["ns"] = serde_json::to_value(target_ns)?;
+    }
+    params.target.client.delete(&api_path, Some(args)).await?;
+
+    Ok(())
+}
+
+/// Push group including all snaphshots to target
+///
+/// Iterate over all snapshots in the group and push them to the target.
+/// The group sync operation consists of the following steps:
+/// - Query snapshots of given group from the source
+/// - Sort snapshots by time
+/// - Apply transfer last cutoff and filters to list
+/// - Iterate the snapshot list and push each snapshot individually
+/// - (Optional): Remove vanished groups if `remove_vanished` flag is set
+pub(crate) async fn push_group(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    group: &BackupGroup,
+    progress: &mut StoreProgress,
+) -> Result<SyncStats, Error> {
+    let mut already_synced_skip_info = SkipInfo::new(SkipReason::AlreadySynced);
+    let mut transfer_last_skip_info = SkipInfo::new(SkipReason::TransferLast);
+
+    let mut snapshots: Vec<BackupDir> = params.source.list_backup_dirs(namespace, group).await?;
+    snapshots.sort_unstable_by(|a, b| a.time.cmp(&b.time));
+
+    let total_snapshots = snapshots.len();
+    let cutoff = params
+        .transfer_last
+        .map(|count| total_snapshots.saturating_sub(count))
+        .unwrap_or_default();
+
+    let last_snapshot_time = fetch_previous_backup_time(params, namespace, group)
+        .await?
+        .unwrap_or(i64::MIN);
+
+    let mut source_snapshots = HashSet::new();
+    let snapshots: Vec<BackupDir> = snapshots
+        .into_iter()
+        .enumerate()
+        .filter(|&(pos, ref snapshot)| {
+            source_snapshots.insert(snapshot.time);
+            if last_snapshot_time > snapshot.time {
+                already_synced_skip_info.update(snapshot.time);
+                return false;
+            } else if already_synced_skip_info.count > 0 {
+                info!("{already_synced_skip_info}");
+                already_synced_skip_info.reset();
+                return true;
+            }
+
+            if pos < cutoff && last_snapshot_time != snapshot.time {
+                transfer_last_skip_info.update(snapshot.time);
+                return false;
+            } else if transfer_last_skip_info.count > 0 {
+                info!("{transfer_last_skip_info}");
+                transfer_last_skip_info.reset();
+            }
+            true
+        })
+        .map(|(_, dir)| dir)
+        .collect();
+
+    progress.group_snapshots = snapshots.len() as u64;
+
+    let target_snapshots = fetch_target_snapshots(params, namespace, group).await?;
+    let target_snapshots: Vec<BackupDir> = target_snapshots
+        .into_iter()
+        .map(|snapshot| snapshot.backup)
+        .collect();
+
+    let mut stats = SyncStats::default();
+    let mut fetch_previous_manifest = !target_snapshots.is_empty();
+    for (pos, source_snapshot) in snapshots.into_iter().enumerate() {
+        if target_snapshots.contains(&source_snapshot) {
+            progress.done_snapshots = pos as u64 + 1;
+            info!("percentage done: {progress}");
+            continue;
+        }
+        let result =
+            push_snapshot(params, namespace, &source_snapshot, fetch_previous_manifest).await;
+        fetch_previous_manifest = true;
+
+        progress.done_snapshots = pos as u64 + 1;
+        info!("percentage done: {progress}");
+
+        // stop on error
+        let sync_stats = result?;
+        stats.add(sync_stats);
+    }
+
+    if params.remove_vanished {
+        let target_snapshots = fetch_target_snapshots(params, namespace, group).await?;
+        for snapshot in target_snapshots {
+            if source_snapshots.contains(&snapshot.backup.time) {
+                continue;
+            }
+            if snapshot.protected {
+                info!(
+                    "don't delete vanished snapshot {name} (protected)",
+                    name = snapshot.backup
+                );
+                continue;
+            }
+            if let Err(err) = forget_target_snapshot(params, namespace, &snapshot.backup).await {
+                info!(
+                    "could not delete vanished snapshot {name} - {err}",
+                    name = snapshot.backup
+                );
+            }
+            info!("delete vanished snapshot {name}", name = snapshot.backup);
+            stats.add(SyncStats::from(RemovedVanishedStats {
+                snapshots: 1,
+                groups: 0,
+                namespaces: 0,
+            }));
+        }
+    }
+
+    Ok(stats)
+}
+
+/// Push snapshot to target
+///
+/// Creates a new snapshot on the target and pushes the content of the source snapshot to the
+/// target by creating a new manifest file and connecting to the remote as backup writer client.
+/// Chunks are written by recreating the index by uploading the chunk stream as read from the
+/// source. Data blobs are uploaded as such.
+pub(crate) async fn push_snapshot(
+    params: &PushParameters,
+    namespace: &BackupNamespace,
+    snapshot: &BackupDir,
+    fetch_previous_manifest: bool,
+) -> Result<SyncStats, Error> {
+    let mut stats = SyncStats::default();
+    let target_ns = params.map_to_target(namespace)?;
+    let backup_dir = params
+        .source
+        .store
+        .backup_dir(namespace.clone(), snapshot.clone())?;
+
+    // Reader locks the snapshot
+    let reader = params.source.reader(namespace, snapshot).await?;
+
+    // Does not lock the manifest, but the reader already assures a locked snapshot
+    let source_manifest = match backup_dir.load_manifest() {
+        Ok((manifest, _raw_size)) => manifest,
+        Err(err) => {
+            // No manifest in snapshot or failed to read, warn and skip
+            log::warn!("failed to load manifest - {err}");
+            return Ok(stats);
+        }
+    };
+
+    // Manifest to be created on target, referencing all the source archives after upload.
+    let mut manifest = BackupManifest::new(snapshot.clone());
+
+    // Writer instance locks the snapshot on the remote side
+    let backup_writer = BackupWriter::start(
+        &params.target.client,
+        None,
+        params.target.repo.store(),
+        &target_ns,
+        snapshot,
+        false,
+        false,
+    )
+    .await?;
+
+    let mut previous_manifest = None;
+    // Use manifest of previous snapshots in group on target for chunk upload deduplication
+    if fetch_previous_manifest {
+        match backup_writer.download_previous_manifest().await {
+            Ok(manifest) => previous_manifest = Some(Arc::new(manifest)),
+            Err(err) => log::info!("Could not download previous manifest - {err}"),
+        }
+    };
+
+    // Dummy upload options: the actual compression and/or encryption already happened while
+    // the chunks were generated during creation of the backup snapshot, therefore pre-existing
+    // chunks (already compressed and/or encrypted) can be pushed to the target.
+    // Further, these steps are skipped in the backup writer upload stream.
+    //
+    // Therefore, these values do not need to fit the values given in the manifest.
+    // The original manifest is uploaded in the end anyways.
+    //
+    // Compression is set to true so that the uploaded manifest will be compressed.
+    // Encrypt is set to assure that above files are not encrypted.
+    let upload_options = UploadOptions {
+        compress: true,
+        encrypt: false,
+        previous_manifest,
+        ..UploadOptions::default()
+    };
+
+    // Avoid double upload penalty by remembering already seen chunks
+    let known_chunks = Arc::new(Mutex::new(HashSet::with_capacity(1024 * 1024)));
+
+    for entry in source_manifest.files() {
+        let mut path = backup_dir.full_path();
+        path.push(&entry.filename);
+        if path.try_exists()? {
+            match ArchiveType::from_path(&entry.filename)? {
+                ArchiveType::Blob => {
+                    let file = std::fs::File::open(path.clone())?;
+                    let backup_stats = backup_writer.upload_blob(file, &entry.filename).await?;
+                    manifest.add_file(
+                        entry.filename.to_string(),
+                        backup_stats.size,
+                        backup_stats.csum,
+                        entry.chunk_crypt_mode(),
+                    )?;
+                    stats.add(SyncStats {
+                        chunk_count: backup_stats.chunk_count as usize,
+                        bytes: backup_stats.size as usize,
+                        elapsed: backup_stats.duration,
+                        removed: None,
+                    });
+                }
+                ArchiveType::DynamicIndex => {
+                    let index = DynamicIndexReader::open(&path)?;
+                    let chunk_reader = reader.chunk_reader(entry.chunk_crypt_mode());
+                    let sync_stats = push_index(
+                        &entry.filename,
+                        index,
+                        chunk_reader,
+                        &backup_writer,
+                        &mut manifest,
+                        entry.chunk_crypt_mode(),
+                        None,
+                        known_chunks.clone(),
+                    )
+                    .await?;
+                    stats.add(sync_stats);
+                }
+                ArchiveType::FixedIndex => {
+                    let index = FixedIndexReader::open(&path)?;
+                    let chunk_reader = reader.chunk_reader(entry.chunk_crypt_mode());
+                    let size = index.index_bytes();
+                    let sync_stats = push_index(
+                        &entry.filename,
+                        index,
+                        chunk_reader,
+                        &backup_writer,
+                        &mut manifest,
+                        entry.chunk_crypt_mode(),
+                        Some(size),
+                        known_chunks.clone(),
+                    )
+                    .await?;
+                    stats.add(sync_stats);
+                }
+            }
+        } else {
+            info!("{path:?} does not exist, skipped.");
+        }
+    }
+
+    // Fetch client log from source and push to target
+    // this has to be handled individually since the log is never part of the manifest
+    let mut client_log_path = backup_dir.full_path();
+    client_log_path.push(CLIENT_LOG_BLOB_NAME);
+    if client_log_path.is_file() {
+        backup_writer
+            .upload_blob_from_file(
+                &client_log_path,
+                CLIENT_LOG_BLOB_NAME,
+                upload_options.clone(),
+            )
+            .await?;
+    }
+    //TODO: only add log line for conditions as described in feedback
+
+    // Rewrite manifest for pushed snapshot, recreating manifest from source on target
+    let manifest_json = serde_json::to_value(source_manifest)?;
+    let manifest_string = serde_json::to_string_pretty(&manifest_json)?;
+    let backup_stats = backup_writer
+        .upload_blob_from_data(
+            manifest_string.into_bytes(),
+            MANIFEST_BLOB_NAME,
+            upload_options,
+        )
+        .await?;
+    backup_writer.finish().await?;
+
+    stats.add(SyncStats {
+        chunk_count: backup_stats.chunk_count as usize,
+        bytes: backup_stats.size as usize,
+        elapsed: backup_stats.duration,
+        removed: None,
+    });
+
+    Ok(stats)
+}
+
+// Read fixed or dynamic index and push to target by uploading via the backup writer instance
+//
+// For fixed indexes, the size must be provided as given by the index reader.
+#[allow(clippy::too_many_arguments)]
+async fn push_index<'a>(
+    filename: &'a str,
+    index: impl IndexFile + Send + 'static,
+    chunk_reader: Arc<dyn AsyncReadChunk>,
+    backup_writer: &BackupWriter,
+    manifest: &mut BackupManifest,
+    crypt_mode: CryptMode,
+    size: Option<u64>,
+    known_chunks: Arc<Mutex<HashSet<[u8; 32]>>>,
+) -> Result<SyncStats, Error> {
+    let (upload_channel_tx, upload_channel_rx) = mpsc::channel(20);
+    let mut chunk_infos =
+        stream::iter(0..index.index_count()).map(move |pos| index.chunk_info(pos).unwrap());
+
+    tokio::spawn(async move {
+        while let Some(chunk_info) = chunk_infos.next().await {
+            // Avoid reading known chunks, as they are not uploaded by the backup writer anyways
+            let needs_upload = {
+                // Need to limit the scope of the lock, otherwise the async block is not `Send`
+                let mut known_chunks = known_chunks.lock().unwrap();
+                // Check if present and insert, chunk will be read and uploaded below if not present
+                known_chunks.insert(chunk_info.digest)
+            };
+
+            let merged_chunk_info = if needs_upload {
+                chunk_reader
+                    .read_raw_chunk(&chunk_info.digest)
+                    .await
+                    .map(|chunk| {
+                        MergedChunkInfo::New(ChunkInfo {
+                            chunk,
+                            digest: chunk_info.digest,
+                            chunk_len: chunk_info.size(),
+                            offset: chunk_info.range.start,
+                        })
+                    })
+            } else {
+                Ok(MergedChunkInfo::Known(vec![(
+                    // Pass size instead of offset, will be replaced with offset by the backup
+                    // writer
+                    chunk_info.size(),
+                    chunk_info.digest,
+                )]))
+            };
+            let _ = upload_channel_tx.send(merged_chunk_info).await;
+        }
+    });
+
+    let merged_chunk_info_stream = ReceiverStream::new(upload_channel_rx).map_err(Error::from);
+
+    let upload_options = UploadOptions {
+        compress: true,
+        encrypt: false,
+        fixed_size: size,
+        ..UploadOptions::default()
+    };
+
+    let upload_stats = backup_writer
+        .upload_index_chunk_info(filename, merged_chunk_info_stream, upload_options)
+        .await?;
+
+    manifest.add_file(
+        filename.to_string(),
+        upload_stats.size,
+        upload_stats.csum,
+        crypt_mode,
+    )?;
+
+    Ok(SyncStats {
+        chunk_count: upload_stats.chunk_count as usize,
+        bytes: upload_stats.size as usize,
+        elapsed: upload_stats.duration,
+        removed: None,
+    })
+}
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 14/29] api types/config: add `sync-push` config type for push sync jobs
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (12 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 13/29] fix #3044: server: implement push support for sync operations Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 15/29] api: push: implement endpoint for sync in push direction Christian Ebner
                   ` (14 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

In order for sync jobs to be either pull or push jobs, allow to
configure the direction of the job.

Adds an additional config type `sync-push` to the sync job config, to
clearly distinguish sync jobs configured in pull and in push
direction and defines and implements the required `SyncDirection` api
type.

This approach was chosen in order to limit possible misconfiguration,
as unintentionally switching the sync direction could potentially
delete still required snapshots.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 pbs-api-types/src/jobs.rs | 38 ++++++++++++++++++++++++++++++++++++++
 pbs-config/src/sync.rs    | 16 +++++++++++++---
 2 files changed, 51 insertions(+), 3 deletions(-)

diff --git a/pbs-api-types/src/jobs.rs b/pbs-api-types/src/jobs.rs
index bf7a6bd5a..e8056beb0 100644
--- a/pbs-api-types/src/jobs.rs
+++ b/pbs-api-types/src/jobs.rs
@@ -498,6 +498,44 @@ pub const TRANSFER_LAST_SCHEMA: Schema =
         .minimum(1)
         .schema();
 
+#[api()]
+#[derive(Copy, Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)]
+#[serde(rename_all = "kebab-case")]
+/// Direction of the sync job, push or pull
+pub enum SyncDirection {
+    /// Sync direction pull
+    #[default]
+    Pull,
+    /// Sync direction push
+    Push,
+}
+
+impl std::fmt::Display for SyncDirection {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        match self {
+            SyncDirection::Pull => f.write_str("pull"),
+            SyncDirection::Push => f.write_str("push"),
+        }
+    }
+}
+
+impl SyncDirection {
+    pub fn as_config_type_str(&self) -> &'static str {
+        match self {
+            SyncDirection::Pull => "sync",
+            SyncDirection::Push => "sync-push",
+        }
+    }
+
+    pub fn from_config_type_str(config_type: &str) -> Result<Self, anyhow::Error> {
+        match config_type {
+            "sync" => Ok(SyncDirection::Pull),
+            "sync-push" => Ok(SyncDirection::Push),
+            _ => bail!("invalid config type for sync job"),
+        }
+    }
+}
+
 #[api(
     properties: {
         id: {
diff --git a/pbs-config/src/sync.rs b/pbs-config/src/sync.rs
index 45453abb1..7fc977e77 100644
--- a/pbs-config/src/sync.rs
+++ b/pbs-config/src/sync.rs
@@ -6,7 +6,7 @@ use anyhow::Error;
 use proxmox_schema::{ApiType, Schema};
 use proxmox_section_config::{SectionConfig, SectionConfigData, SectionConfigPlugin};
 
-use pbs_api_types::{SyncJobConfig, JOB_ID_SCHEMA};
+use pbs_api_types::{SyncDirection, SyncJobConfig, JOB_ID_SCHEMA};
 
 use crate::{open_backup_lockfile, replace_backup_config, BackupLockGuard};
 
@@ -18,9 +18,19 @@ fn init() -> SectionConfig {
         _ => unreachable!(),
     };
 
-    let plugin = SectionConfigPlugin::new("sync".to_string(), Some(String::from("id")), obj_schema);
+    let pull_plugin = SectionConfigPlugin::new(
+        SyncDirection::Pull.as_config_type_str().to_string(),
+        Some(String::from("id")),
+        obj_schema,
+    );
+    let push_plugin = SectionConfigPlugin::new(
+        SyncDirection::Push.as_config_type_str().to_string(),
+        Some(String::from("id")),
+        obj_schema,
+    );
     let mut config = SectionConfig::new(&JOB_ID_SCHEMA);
-    config.register_plugin(plugin);
+    config.register_plugin(pull_plugin);
+    config.register_plugin(push_plugin);
 
     config
 }
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 15/29] api: push: implement endpoint for sync in push direction
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (13 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 14/29] api types/config: add `sync-push` config type for push sync jobs Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 16/29] api: sync: move sync job invocation to server sync module Christian Ebner
                   ` (13 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Expose the sync job in push direction via a dedicated API endpoint,
analogous to the pull direction.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- Avoid double deserialization for backup namespaces
- Drop TryFrom<&SyncJobConfig> for PushParameters impl, as constructing
  them requires an api call to fetch the remote api version now

 src/api2/mod.rs  |   2 +
 src/api2/push.rs | 183 +++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 185 insertions(+)
 create mode 100644 src/api2/push.rs

diff --git a/src/api2/mod.rs b/src/api2/mod.rs
index a83e4c205..03596326b 100644
--- a/src/api2/mod.rs
+++ b/src/api2/mod.rs
@@ -12,6 +12,7 @@ pub mod helpers;
 pub mod node;
 pub mod ping;
 pub mod pull;
+pub mod push;
 pub mod reader;
 pub mod status;
 pub mod tape;
@@ -29,6 +30,7 @@ const SUBDIRS: SubdirMap = &sorted!([
     ("nodes", &node::ROUTER),
     ("ping", &ping::ROUTER),
     ("pull", &pull::ROUTER),
+    ("push", &push::ROUTER),
     ("reader", &reader::ROUTER),
     ("status", &status::ROUTER),
     ("tape", &tape::ROUTER),
diff --git a/src/api2/push.rs b/src/api2/push.rs
new file mode 100644
index 000000000..28f4417d1
--- /dev/null
+++ b/src/api2/push.rs
@@ -0,0 +1,183 @@
+use anyhow::{format_err, Error};
+use futures::{future::FutureExt, select};
+use tracing::info;
+
+use pbs_api_types::{
+    Authid, BackupNamespace, GroupFilter, RateLimitConfig, DATASTORE_SCHEMA,
+    GROUP_FILTER_LIST_SCHEMA, NS_MAX_DEPTH_REDUCED_SCHEMA, PRIV_DATASTORE_READ,
+    PRIV_REMOTE_DATASTORE_BACKUP, PRIV_REMOTE_DATASTORE_PRUNE, REMOTE_ID_SCHEMA,
+    REMOVE_VANISHED_BACKUPS_SCHEMA, TRANSFER_LAST_SCHEMA,
+};
+use proxmox_rest_server::WorkerTask;
+use proxmox_router::{Permission, Router, RpcEnvironment};
+use proxmox_schema::api;
+
+use pbs_config::CachedUserInfo;
+
+use crate::server::push::{push_store, PushParameters};
+
+/// Check if the provided user is allowed to read from the local source and act on the remote
+/// target for pushing content
+pub fn check_push_privs(
+    auth_id: &Authid,
+    store: &str,
+    namespace: &BackupNamespace,
+    remote: &str,
+    remote_store: &str,
+    remote_ns: Option<&BackupNamespace>,
+    delete: bool,
+) -> Result<(), Error> {
+    let user_info = CachedUserInfo::new()?;
+
+    let target_acl_path = match remote_ns {
+        Some(ns) => ns.remote_acl_path(remote, remote_store),
+        None => vec!["remote", remote, remote_store],
+    };
+
+    // Check user is allowed to backup to remote/<remote>/<datastore>/<namespace>
+    user_info.check_privs(
+        auth_id,
+        &target_acl_path,
+        PRIV_REMOTE_DATASTORE_BACKUP,
+        false,
+    )?;
+
+    if delete {
+        // Check user is allowed to prune remote datastore
+        user_info.check_privs(
+            auth_id,
+            &target_acl_path,
+            PRIV_REMOTE_DATASTORE_PRUNE,
+            false,
+        )?;
+    }
+
+    // Check user is allowed to read source datastore
+    user_info.check_privs(
+        auth_id,
+        &namespace.acl_path(store),
+        PRIV_DATASTORE_READ,
+        false,
+    )?;
+
+    Ok(())
+}
+
+#[api(
+    input: {
+        properties: {
+            store: {
+                schema: DATASTORE_SCHEMA,
+            },
+            ns: {
+                type: BackupNamespace,
+                optional: true,
+            },
+            remote: {
+                schema: REMOTE_ID_SCHEMA,
+            },
+            "remote-store": {
+                schema: DATASTORE_SCHEMA,
+            },
+            "remote-ns": {
+                type: BackupNamespace,
+                optional: true,
+            },
+            "remove-vanished": {
+                schema: REMOVE_VANISHED_BACKUPS_SCHEMA,
+                optional: true,
+            },
+            "max-depth": {
+                schema: NS_MAX_DEPTH_REDUCED_SCHEMA,
+                optional: true,
+            },
+            "group-filter": {
+                schema: GROUP_FILTER_LIST_SCHEMA,
+                optional: true,
+            },
+            limit: {
+                type: RateLimitConfig,
+                flatten: true,
+            },
+            "transfer-last": {
+                schema: TRANSFER_LAST_SCHEMA,
+                optional: true,
+            },
+        },
+    },
+    access: {
+        description: r###"The user needs Remote.Backup privilege on '/remote/{remote}/{remote-store}'
+and needs to own the backup group. Datastore.Read is required on '/datastore/{store}'.
+The delete flag additionally requires the Remote.Prune privilege on '/remote/{remote}/{remote-store}'.
+"###,
+        permission: &Permission::Anybody,
+    },
+)]
+/// Push store to other repository
+#[allow(clippy::too_many_arguments)]
+async fn push(
+    store: String,
+    ns: Option<BackupNamespace>,
+    remote: String,
+    remote_store: String,
+    remote_ns: Option<BackupNamespace>,
+    remove_vanished: Option<bool>,
+    max_depth: Option<usize>,
+    group_filter: Option<Vec<GroupFilter>>,
+    limit: RateLimitConfig,
+    transfer_last: Option<usize>,
+    rpcenv: &mut dyn RpcEnvironment,
+) -> Result<String, Error> {
+    let auth_id: Authid = rpcenv.get_auth_id().unwrap().parse()?;
+    let delete = remove_vanished.unwrap_or(false);
+    let ns = ns.unwrap_or_default();
+
+    check_push_privs(
+        &auth_id,
+        &store,
+        &ns,
+        &remote,
+        &remote_store,
+        remote_ns.as_ref(),
+        delete,
+    )?;
+
+    let push_params = PushParameters::new(
+        &store,
+        ns,
+        &remote,
+        &remote_store,
+        remote_ns.unwrap_or_default(),
+        auth_id.clone(),
+        remove_vanished,
+        max_depth,
+        group_filter,
+        limit,
+        transfer_last,
+    )
+    .await?;
+
+    let upid_str = WorkerTask::spawn(
+        "sync",
+        Some(store.clone()),
+        auth_id.to_string(),
+        true,
+        move |worker| async move {
+            info!("push datastore '{store}' to '{remote}/{remote_store}'");
+
+            let push_future = push_store(push_params);
+            (select! {
+                success = push_future.fuse() => success,
+                abort = worker.abort_future().map(|_| Err(format_err!("push aborted"))) => abort,
+            })?;
+
+            info!("push datastore '{store}' end");
+
+            Ok(())
+        },
+    )?;
+
+    Ok(upid_str)
+}
+
+pub const ROUTER: Router = Router::new().post(&API_METHOD_PUSH);
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 16/29] api: sync: move sync job invocation to server sync module
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (14 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 15/29] api: push: implement endpoint for sync in push direction Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 17/29] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job Christian Ebner
                   ` (12 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Moves and refactores the sync_job_do function into the common server
sync module so that it can be reused for both sync directions, pull
and push.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- Adapt to PushParameters requiring api call to fetch remote version on
  instantiation

 src/api2/admin/sync.rs          |  19 ++--
 src/api2/pull.rs                | 108 -----------------------
 src/bin/proxmox-backup-proxy.rs |  15 +++-
 src/server/mod.rs               |   1 +
 src/server/sync.rs              | 150 +++++++++++++++++++++++++++++++-
 5 files changed, 173 insertions(+), 120 deletions(-)

diff --git a/src/api2/admin/sync.rs b/src/api2/admin/sync.rs
index 4e2ba0be8..be324564c 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -10,16 +10,16 @@ use proxmox_router::{
 use proxmox_schema::api;
 use proxmox_sortable_macro::sortable;
 
-use pbs_api_types::{Authid, SyncJobConfig, SyncJobStatus, DATASTORE_SCHEMA, JOB_ID_SCHEMA};
+use pbs_api_types::{
+    Authid, SyncDirection, SyncJobConfig, SyncJobStatus, DATASTORE_SCHEMA, JOB_ID_SCHEMA,
+};
 use pbs_config::sync;
 use pbs_config::CachedUserInfo;
 
 use crate::{
-    api2::{
-        config::sync::{check_sync_job_modify_access, check_sync_job_read_access},
-        pull::do_sync_job,
-    },
+    api2::config::sync::{check_sync_job_modify_access, check_sync_job_read_access},
     server::jobstate::{compute_schedule_status, Job, JobState},
+    server::sync::do_sync_job,
 };
 
 #[api(
@@ -116,7 +116,14 @@ pub fn run_sync_job(
 
     let to_stdout = rpcenv.env_type() == RpcEnvironmentType::CLI;
 
-    let upid_str = do_sync_job(job, sync_job, &auth_id, None, to_stdout)?;
+    let upid_str = do_sync_job(
+        job,
+        sync_job,
+        &auth_id,
+        None,
+        SyncDirection::Pull,
+        to_stdout,
+    )?;
 
     Ok(upid_str)
 }
diff --git a/src/api2/pull.rs b/src/api2/pull.rs
index e733c9839..d039dab59 100644
--- a/src/api2/pull.rs
+++ b/src/api2/pull.rs
@@ -13,10 +13,8 @@ use pbs_api_types::{
     TRANSFER_LAST_SCHEMA,
 };
 use pbs_config::CachedUserInfo;
-use proxmox_human_byte::HumanByte;
 use proxmox_rest_server::WorkerTask;
 
-use crate::server::jobstate::Job;
 use crate::server::pull::{pull_store, PullParameters};
 
 pub fn check_pull_privs(
@@ -93,112 +91,6 @@ impl TryFrom<&SyncJobConfig> for PullParameters {
     }
 }
 
-pub fn do_sync_job(
-    mut job: Job,
-    sync_job: SyncJobConfig,
-    auth_id: &Authid,
-    schedule: Option<String>,
-    to_stdout: bool,
-) -> Result<String, Error> {
-    let job_id = format!(
-        "{}:{}:{}:{}:{}",
-        sync_job.remote.as_deref().unwrap_or("-"),
-        sync_job.remote_store,
-        sync_job.store,
-        sync_job.ns.clone().unwrap_or_default(),
-        job.jobname()
-    );
-    let worker_type = job.jobtype().to_string();
-
-    if sync_job.remote.is_none() && sync_job.store == sync_job.remote_store {
-        bail!("can't sync to same datastore");
-    }
-
-    let upid_str = WorkerTask::spawn(
-        &worker_type,
-        Some(job_id.clone()),
-        auth_id.to_string(),
-        to_stdout,
-        move |worker| async move {
-            job.start(&worker.upid().to_string())?;
-
-            let worker2 = worker.clone();
-            let sync_job2 = sync_job.clone();
-
-            let worker_future = async move {
-                let pull_params = PullParameters::try_from(&sync_job)?;
-
-                info!("Starting datastore sync job '{job_id}'");
-                if let Some(event_str) = schedule {
-                    info!("task triggered by schedule '{event_str}'");
-                }
-
-                info!(
-                    "sync datastore '{}' from '{}{}'",
-                    sync_job.store,
-                    sync_job
-                        .remote
-                        .as_deref()
-                        .map_or(String::new(), |remote| format!("{remote}/")),
-                    sync_job.remote_store,
-                );
-
-                let pull_stats = pull_store(pull_params).await?;
-
-                if pull_stats.bytes != 0 {
-                    let amount = HumanByte::from(pull_stats.bytes);
-                    let rate = HumanByte::new_binary(
-                        pull_stats.bytes as f64 / pull_stats.elapsed.as_secs_f64(),
-                    );
-                    info!(
-                        "Summary: sync job pulled {amount} in {} chunks (average rate: {rate}/s)",
-                        pull_stats.chunk_count,
-                    );
-                } else {
-                    info!("Summary: sync job found no new data to pull");
-                }
-
-                if let Some(removed) = pull_stats.removed {
-                    info!(
-                        "Summary: removed vanished: snapshots: {}, groups: {}, namespaces: {}",
-                        removed.snapshots, removed.groups, removed.namespaces,
-                    );
-                }
-
-                info!("sync job '{}' end", &job_id);
-
-                Ok(())
-            };
-
-            let mut abort_future = worker2
-                .abort_future()
-                .map(|_| Err(format_err!("sync aborted")));
-
-            let result = select! {
-                worker = worker_future.fuse() => worker,
-                abort = abort_future => abort,
-            };
-
-            let status = worker2.create_state(&result);
-
-            match job.finish(status) {
-                Ok(_) => {}
-                Err(err) => {
-                    eprintln!("could not finish job state: {}", err);
-                }
-            }
-
-            if let Err(err) = crate::server::send_sync_status(&sync_job2, &result) {
-                eprintln!("send sync notification failed: {err}");
-            }
-
-            result
-        },
-    )?;
-
-    Ok(upid_str)
-}
-
 #[api(
     input: {
         properties: {
diff --git a/src/bin/proxmox-backup-proxy.rs b/src/bin/proxmox-backup-proxy.rs
index 859f5b0f8..6f19a3fbd 100644
--- a/src/bin/proxmox-backup-proxy.rs
+++ b/src/bin/proxmox-backup-proxy.rs
@@ -40,17 +40,17 @@ use pbs_buildcfg::configdir;
 use proxmox_time::CalendarEvent;
 
 use pbs_api_types::{
-    Authid, DataStoreConfig, Operation, PruneJobConfig, SyncJobConfig, TapeBackupJobConfig,
-    VerificationJobConfig,
+    Authid, DataStoreConfig, Operation, PruneJobConfig, SyncDirection, SyncJobConfig,
+    TapeBackupJobConfig, VerificationJobConfig,
 };
 
 use proxmox_backup::auth_helpers::*;
 use proxmox_backup::server::{self, metric_collection};
 use proxmox_backup::tools::PROXMOX_BACKUP_TCP_KEEPALIVE_TIME;
 
-use proxmox_backup::api2::pull::do_sync_job;
 use proxmox_backup::api2::tape::backup::do_tape_backup_job;
 use proxmox_backup::server::do_prune_job;
+use proxmox_backup::server::do_sync_job;
 use proxmox_backup::server::do_verification_job;
 
 fn main() -> Result<(), Error> {
@@ -611,7 +611,14 @@ async fn schedule_datastore_sync_jobs() {
             };
 
             let auth_id = Authid::root_auth_id().clone();
-            if let Err(err) = do_sync_job(job, job_config, &auth_id, Some(event_str), false) {
+            if let Err(err) = do_sync_job(
+                job,
+                job_config,
+                &auth_id,
+                Some(event_str),
+                SyncDirection::Pull,
+                false,
+            ) {
                 eprintln!("unable to start datastore sync job {job_id} - {err}");
             }
         };
diff --git a/src/server/mod.rs b/src/server/mod.rs
index 7c14ed4b8..b9398d21f 100644
--- a/src/server/mod.rs
+++ b/src/server/mod.rs
@@ -38,6 +38,7 @@ pub mod metric_collection;
 pub(crate) mod pull;
 pub(crate) mod push;
 pub(crate) mod sync;
+pub use sync::do_sync_job;
 
 pub(crate) async fn reload_proxy_certificate() -> Result<(), Error> {
     let proxy_pid = proxmox_rest_server::read_pid(pbs_buildcfg::PROXMOX_BACKUP_PROXY_PID_FN)?;
diff --git a/src/server/sync.rs b/src/server/sync.rs
index bd68dda46..92a3c0933 100644
--- a/src/server/sync.rs
+++ b/src/server/sync.rs
@@ -6,16 +6,19 @@ use std::path::{Path, PathBuf};
 use std::sync::{Arc, Mutex};
 use std::time::Duration;
 
-use anyhow::{bail, format_err, Error};
+use anyhow::{bail, format_err, Context, Error};
+use futures::{future::FutureExt, select};
 use http::StatusCode;
 use serde_json::json;
 use tracing::{info, warn};
 
+use proxmox_human_byte::HumanByte;
+use proxmox_rest_server::WorkerTask;
 use proxmox_router::HttpError;
 
 use pbs_api_types::{
     Authid, BackupDir, BackupGroup, BackupNamespace, CryptMode, GroupListItem, SnapshotListItem,
-    MAX_NAMESPACE_DEPTH, PRIV_DATASTORE_BACKUP, PRIV_DATASTORE_READ,
+    SyncDirection, SyncJobConfig, MAX_NAMESPACE_DEPTH, PRIV_DATASTORE_BACKUP, PRIV_DATASTORE_READ,
 };
 use pbs_client::{BackupReader, BackupRepository, HttpClient, RemoteChunkReader};
 use pbs_datastore::data_blob::DataBlob;
@@ -24,6 +27,9 @@ use pbs_datastore::read_chunk::AsyncReadChunk;
 use pbs_datastore::{DataStore, ListNamespacesRecursive, LocalChunkReader};
 
 use crate::backup::ListAccessibleBackupGroups;
+use crate::server::jobstate::Job;
+use crate::server::pull::{pull_store, PullParameters};
+use crate::server::push::{push_store, PushParameters};
 
 #[derive(Default)]
 pub(crate) struct RemovedVanishedStats {
@@ -568,3 +574,143 @@ pub(crate) fn check_namespace_depth_limit(
     }
     Ok(())
 }
+
+/// Run a sync job in given direction
+pub fn do_sync_job(
+    mut job: Job,
+    sync_job: SyncJobConfig,
+    auth_id: &Authid,
+    schedule: Option<String>,
+    sync_direction: SyncDirection,
+    to_stdout: bool,
+) -> Result<String, Error> {
+    let job_id = format!(
+        "{}:{}:{}:{}:{}",
+        sync_job.remote.as_deref().unwrap_or("-"),
+        sync_job.remote_store,
+        sync_job.store,
+        sync_job.ns.clone().unwrap_or_default(),
+        job.jobname(),
+    );
+    let worker_type = job.jobtype().to_string();
+
+    if sync_job.remote.is_none() && sync_job.store == sync_job.remote_store {
+        bail!("can't sync to same datastore");
+    }
+
+    let upid_str = WorkerTask::spawn(
+        &worker_type,
+        Some(job_id.clone()),
+        auth_id.to_string(),
+        to_stdout,
+        move |worker| async move {
+            job.start(&worker.upid().to_string())?;
+
+            let worker2 = worker.clone();
+            let sync_job2 = sync_job.clone();
+
+            let worker_future = async move {
+                info!("Starting datastore sync job '{job_id}'");
+                if let Some(event_str) = schedule {
+                    info!("task triggered by schedule '{event_str}'");
+                }
+                let sync_stats = match sync_direction {
+                    SyncDirection::Pull => {
+                        info!(
+                            "sync datastore '{}' from '{}{}'",
+                            sync_job.store,
+                            sync_job
+                                .remote
+                                .as_deref()
+                                .map_or(String::new(), |remote| format!("{remote}/")),
+                            sync_job.remote_store,
+                        );
+                        let pull_params = PullParameters::try_from(&sync_job)?;
+                        pull_store(pull_params).await?
+                    }
+                    SyncDirection::Push => {
+                        info!(
+                            "sync datastore '{}' to '{}{}'",
+                            sync_job.store,
+                            sync_job
+                                .remote
+                                .as_deref()
+                                .map_or(String::new(), |remote| format!("{remote}/")),
+                            sync_job.remote_store,
+                        );
+                        let push_params = PushParameters::new(
+                            &sync_job.store,
+                            sync_job.ns.clone().unwrap_or_default(),
+                            sync_job
+                                .remote
+                                .as_deref()
+                                .context("missing required remote")?,
+                            &sync_job.remote_store,
+                            sync_job.remote_ns.clone().unwrap_or_default(),
+                            sync_job
+                                .owner
+                                .as_ref()
+                                .unwrap_or_else(|| Authid::root_auth_id())
+                                .clone(),
+                            sync_job.remove_vanished,
+                            sync_job.max_depth,
+                            sync_job.group_filter.clone(),
+                            sync_job.limit.clone(),
+                            sync_job.transfer_last,
+                        )
+                        .await?;
+                        push_store(push_params).await?
+                    }
+                };
+
+                if sync_stats.bytes != 0 {
+                    let amount = HumanByte::from(sync_stats.bytes);
+                    let rate = HumanByte::new_binary(
+                        sync_stats.bytes as f64 / sync_stats.elapsed.as_secs_f64(),
+                    );
+                    info!(
+                        "Summary: sync job {sync_direction}ed {amount} in {} chunks (average rate: {rate}/s)",
+                        sync_stats.chunk_count,
+                    );
+                } else {
+                    info!("Summary: sync job found no new data to {sync_direction}");
+                }
+
+                if let Some(removed) = sync_stats.removed {
+                    info!(
+                        "Summary: removed vanished: snapshots: {}, groups: {}, namespaces: {}",
+                        removed.snapshots, removed.groups, removed.namespaces,
+                    );
+                }
+
+                info!("sync job '{job_id}' end");
+
+                Ok(())
+            };
+
+            let mut abort_future = worker2
+                .abort_future()
+                .map(|_| Err(format_err!("sync aborted")));
+
+            let result = select! {
+                worker = worker_future.fuse() => worker,
+                abort = abort_future => abort,
+            };
+
+            let status = worker2.create_state(&result);
+
+            match job.finish(status) {
+                Ok(_) => {}
+                Err(err) => eprintln!("could not finish job state: {err}"),
+            }
+
+            if let Err(err) = crate::server::send_sync_status(&sync_job2, &result) {
+                eprintln!("send sync notification failed: {err}");
+            }
+
+            result
+        },
+    )?;
+
+    Ok(upid_str)
+}
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 17/29] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (15 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 16/29] api: sync: move sync job invocation to server sync module Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 18/29] api: config: factor out sync job owner check Christian Ebner
                   ` (11 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Read access to sync jobs is not granted to users not having at least
PRIV_DATASTORE_AUDIT permissions on the datastore. However a user is
able to create or modify such jobs, without having the audit
permission.

Therefore, further restrict the modify check by also including the
audit permissions.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 src/api2/config/sync.rs | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/api2/config/sync.rs b/src/api2/config/sync.rs
index 6fdc69a9e..38325f5b2 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -44,7 +44,7 @@ pub fn check_sync_job_modify_access(
     job: &SyncJobConfig,
 ) -> bool {
     let ns_anchor_privs = user_info.lookup_privs(auth_id, &job.acl_path());
-    if ns_anchor_privs & PRIV_DATASTORE_BACKUP == 0 {
+    if ns_anchor_privs & PRIV_DATASTORE_BACKUP == 0 || ns_anchor_privs & PRIV_DATASTORE_AUDIT == 0 {
         return false;
     }
 
@@ -502,7 +502,7 @@ user: write@pbs
         r###"
 acl:1:/datastore/localstore1:read@pbs,write@pbs:DatastoreAudit
 acl:1:/datastore/localstore1:write@pbs:DatastoreBackup
-acl:1:/datastore/localstore2:write@pbs:DatastorePowerUser
+acl:1:/datastore/localstore2:write@pbs:DatastoreAudit,DatastorePowerUser
 acl:1:/datastore/localstore3:write@pbs:DatastoreAdmin
 acl:1:/remote/remote1:read@pbs,write@pbs:RemoteAudit
 acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 18/29] api: config: factor out sync job owner check
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (16 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 17/29] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 19/29] api: sync jobs: expose optional `sync-direction` parameter Christian Ebner
                   ` (10 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Move the sync job owner check to its own helper function, for it to
be reused for the owner check for sync jobs in push direction.

No functional change intended.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- use pre-existing check_backup_owner helper

 src/api2/config/sync.rs | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)

diff --git a/src/api2/config/sync.rs b/src/api2/config/sync.rs
index 38325f5b2..3963049e9 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -14,6 +14,7 @@ use pbs_api_types::{
 use pbs_config::sync;
 
 use pbs_config::CachedUserInfo;
+use pbs_datastore::check_backup_owner;
 
 pub fn check_sync_job_read_access(
     user_info: &CachedUserInfo,
@@ -34,6 +35,14 @@ pub fn check_sync_job_read_access(
     }
 }
 
+fn is_correct_owner(auth_id: &Authid, job: &SyncJobConfig) -> bool {
+    match job.owner {
+        Some(ref owner) => check_backup_owner(owner, auth_id).is_ok(),
+        // default sync owner
+        None => auth_id == Authid::root_auth_id(),
+    }
+}
+
 /// checks whether user can run the corresponding pull job
 ///
 /// namespace creation/deletion ACL and backup group ownership checks happen in the pull code directly.
@@ -54,17 +63,8 @@ pub fn check_sync_job_modify_access(
         }
     }
 
-    let correct_owner = match job.owner {
-        Some(ref owner) => {
-            owner == auth_id
-                || (owner.is_token() && !auth_id.is_token() && owner.user() == auth_id.user())
-        }
-        // default sync owner
-        None => auth_id == Authid::root_auth_id(),
-    };
-
     // same permission as changing ownership after syncing
-    if !correct_owner && ns_anchor_privs & PRIV_DATASTORE_MODIFY == 0 {
+    if !is_correct_owner(auth_id, job) && ns_anchor_privs & PRIV_DATASTORE_MODIFY == 0 {
         return false;
     }
 
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 19/29] api: sync jobs: expose optional `sync-direction` parameter
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (17 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 18/29] api: config: factor out sync job owner check Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 20/29] api: admin: avoid duplicate name for list sync jobs api method Christian Ebner
                   ` (9 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Exposes and switch the config type for sync job operations based
on the `sync-direction` parameter, exposed on required api endpoints.

If not set, the default config type is `sync` and the default sync
direction is `pull` for full backwards compatibility. Whenever
possible, deterimne the sync direction and config type from the sync
job config directly rather than requiring it as optional api
parameter.

Further, extend read and modify access checks by sync direction to
conditionally check for the required permissions in pull and push
direction.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- Squashed permission check patches into this one, as they make not much
  sense without this
- Only expose optional sync-direction parameter for api endpoints which
  require them, use the job config to determine sync-direction and/or
  config-type otherwise.

 src/api2/admin/sync.rs               |  34 ++--
 src/api2/config/datastore.rs         |  11 +-
 src/api2/config/notifications/mod.rs |  19 +-
 src/api2/config/sync.rs              | 280 ++++++++++++++++++++-------
 src/bin/proxmox-backup-proxy.rs      |  11 +-
 5 files changed, 261 insertions(+), 94 deletions(-)

diff --git a/src/api2/admin/sync.rs b/src/api2/admin/sync.rs
index be324564c..8a242b1c3 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -1,6 +1,7 @@
 //! Datastore Synchronization Job Management
 
 use anyhow::{bail, format_err, Error};
+use serde::Deserialize;
 use serde_json::Value;
 
 use proxmox_router::{
@@ -29,6 +30,10 @@ use crate::{
                 schema: DATASTORE_SCHEMA,
                 optional: true,
             },
+            "sync-direction": {
+                type: SyncDirection,
+                optional: true,
+            },
         },
     },
     returns: {
@@ -44,6 +49,7 @@ use crate::{
 /// List all sync jobs
 pub fn list_sync_jobs(
     store: Option<String>,
+    sync_direction: Option<SyncDirection>,
     _param: Value,
     rpcenv: &mut dyn RpcEnvironment,
 ) -> Result<Vec<SyncJobStatus>, Error> {
@@ -52,8 +58,9 @@ pub fn list_sync_jobs(
 
     let (config, digest) = sync::config()?;
 
+    let sync_direction = sync_direction.unwrap_or_default();
     let job_config_iter = config
-        .convert_to_typed_array("sync")?
+        .convert_to_typed_array(sync_direction.as_config_type_str())?
         .into_iter()
         .filter(|job: &SyncJobConfig| {
             if let Some(store) = &store {
@@ -62,7 +69,9 @@ pub fn list_sync_jobs(
                 true
             }
         })
-        .filter(|job: &SyncJobConfig| check_sync_job_read_access(&user_info, &auth_id, job));
+        .filter(|job: &SyncJobConfig| {
+            check_sync_job_read_access(&user_info, &auth_id, job, sync_direction)
+        });
 
     let mut list = Vec::new();
 
@@ -106,24 +115,23 @@ pub fn run_sync_job(
     let user_info = CachedUserInfo::new()?;
 
     let (config, _digest) = sync::config()?;
-    let sync_job: SyncJobConfig = config.lookup("sync", &id)?;
+    let (config_type, config_section) = config
+        .sections
+        .get(&id)
+        .ok_or_else(|| format_err!("No sync job with id '{id}' found in config"))?;
+
+    let sync_direction = SyncDirection::from_config_type_str(config_type)?;
+    let sync_job = SyncJobConfig::deserialize(config_section)?;
 
-    if !check_sync_job_modify_access(&user_info, &auth_id, &sync_job) {
-        bail!("permission check failed");
+    if !check_sync_job_modify_access(&user_info, &auth_id, &sync_job, sync_direction) {
+        bail!("permission check failed, '{auth_id}' is missing access");
     }
 
     let job = Job::new("syncjob", &id)?;
 
     let to_stdout = rpcenv.env_type() == RpcEnvironmentType::CLI;
 
-    let upid_str = do_sync_job(
-        job,
-        sync_job,
-        &auth_id,
-        None,
-        SyncDirection::Pull,
-        to_stdout,
-    )?;
+    let upid_str = do_sync_job(job, sync_job, &auth_id, None, sync_direction, to_stdout)?;
 
     Ok(upid_str)
 }
diff --git a/src/api2/config/datastore.rs b/src/api2/config/datastore.rs
index ca6edf05a..c151eda10 100644
--- a/src/api2/config/datastore.rs
+++ b/src/api2/config/datastore.rs
@@ -13,8 +13,9 @@ use proxmox_uuid::Uuid;
 
 use pbs_api_types::{
     Authid, DataStoreConfig, DataStoreConfigUpdater, DatastoreNotify, DatastoreTuning, KeepOptions,
-    MaintenanceMode, PruneJobConfig, PruneJobOptions, DATASTORE_SCHEMA, PRIV_DATASTORE_ALLOCATE,
-    PRIV_DATASTORE_AUDIT, PRIV_DATASTORE_MODIFY, PROXMOX_CONFIG_DIGEST_SCHEMA, UPID_SCHEMA,
+    MaintenanceMode, PruneJobConfig, PruneJobOptions, SyncDirection, DATASTORE_SCHEMA,
+    PRIV_DATASTORE_ALLOCATE, PRIV_DATASTORE_AUDIT, PRIV_DATASTORE_MODIFY,
+    PROXMOX_CONFIG_DIGEST_SCHEMA, UPID_SCHEMA,
 };
 use pbs_config::BackupLockGuard;
 use pbs_datastore::chunk_store::ChunkStore;
@@ -498,8 +499,10 @@ pub async fn delete_datastore(
         for job in list_verification_jobs(Some(name.clone()), Value::Null, rpcenv)? {
             delete_verification_job(job.config.id, None, rpcenv)?
         }
-        for job in list_sync_jobs(Some(name.clone()), Value::Null, rpcenv)? {
-            delete_sync_job(job.config.id, None, rpcenv)?
+        for direction in [SyncDirection::Pull, SyncDirection::Push] {
+            for job in list_sync_jobs(Some(name.clone()), Some(direction), Value::Null, rpcenv)? {
+                delete_sync_job(job.config.id, None, rpcenv)?
+            }
         }
         for job in list_prune_jobs(Some(name.clone()), Value::Null, rpcenv)? {
             delete_prune_job(job.config.id, None, rpcenv)?
diff --git a/src/api2/config/notifications/mod.rs b/src/api2/config/notifications/mod.rs
index dfe82ed03..31c4851c1 100644
--- a/src/api2/config/notifications/mod.rs
+++ b/src/api2/config/notifications/mod.rs
@@ -9,7 +9,7 @@ use proxmox_schema::api;
 use proxmox_sortable_macro::sortable;
 
 use crate::api2::admin::datastore::get_datastore_list;
-use pbs_api_types::PRIV_SYS_AUDIT;
+use pbs_api_types::{SyncDirection, PRIV_SYS_AUDIT};
 
 use crate::api2::admin::prune::list_prune_jobs;
 use crate::api2::admin::sync::list_sync_jobs;
@@ -154,13 +154,15 @@ pub fn get_values(
         });
     }
 
-    let sync_jobs = list_sync_jobs(None, param.clone(), rpcenv)?;
-    for job in sync_jobs {
-        values.push(MatchableValue {
-            field: "job-id".into(),
-            value: job.config.id,
-            comment: job.config.comment,
-        });
+    for direction in [SyncDirection::Pull, SyncDirection::Push] {
+        let sync_jobs = list_sync_jobs(None, Some(direction), param.clone(), rpcenv)?;
+        for job in sync_jobs {
+            values.push(MatchableValue {
+                field: "job-id".into(),
+                value: job.config.id,
+                comment: job.config.comment,
+            });
+        }
     }
 
     let verify_jobs = list_verification_jobs(None, param.clone(), rpcenv)?;
@@ -184,6 +186,7 @@ pub fn get_values(
         "package-updates",
         "prune",
         "sync",
+        "sync-push",
         "system-mail",
         "tape-backup",
         "tape-load",
diff --git a/src/api2/config/sync.rs b/src/api2/config/sync.rs
index 3963049e9..2f32aaccb 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -1,6 +1,7 @@
 use ::serde::{Deserialize, Serialize};
 use anyhow::{bail, Error};
 use hex::FromHex;
+use pbs_api_types::SyncDirection;
 use serde_json::Value;
 
 use proxmox_router::{http_bail, Permission, Router, RpcEnvironment};
@@ -8,8 +9,9 @@ use proxmox_schema::{api, param_bail};
 
 use pbs_api_types::{
     Authid, SyncJobConfig, SyncJobConfigUpdater, JOB_ID_SCHEMA, PRIV_DATASTORE_AUDIT,
-    PRIV_DATASTORE_BACKUP, PRIV_DATASTORE_MODIFY, PRIV_DATASTORE_PRUNE, PRIV_REMOTE_AUDIT,
-    PRIV_REMOTE_READ, PROXMOX_CONFIG_DIGEST_SCHEMA,
+    PRIV_DATASTORE_BACKUP, PRIV_DATASTORE_MODIFY, PRIV_DATASTORE_PRUNE, PRIV_DATASTORE_READ,
+    PRIV_REMOTE_AUDIT, PRIV_REMOTE_DATASTORE_BACKUP, PRIV_REMOTE_DATASTORE_MODIFY,
+    PRIV_REMOTE_DATASTORE_PRUNE, PRIV_REMOTE_READ, PROXMOX_CONFIG_DIGEST_SCHEMA,
 };
 use pbs_config::sync;
 
@@ -20,18 +22,35 @@ pub fn check_sync_job_read_access(
     user_info: &CachedUserInfo,
     auth_id: &Authid,
     job: &SyncJobConfig,
+    sync_direction: SyncDirection,
 ) -> bool {
+    // check for audit access on datastore/namespace, applies for pull and push direction
     let ns_anchor_privs = user_info.lookup_privs(auth_id, &job.acl_path());
     if ns_anchor_privs & PRIV_DATASTORE_AUDIT == 0 {
         return false;
     }
 
-    if let Some(remote) = &job.remote {
-        let remote_privs = user_info.lookup_privs(auth_id, &["remote", remote]);
-        remote_privs & PRIV_REMOTE_AUDIT != 0
-    } else {
-        let source_ds_privs = user_info.lookup_privs(auth_id, &["datastore", &job.remote_store]);
-        source_ds_privs & PRIV_DATASTORE_AUDIT != 0
+    match sync_direction {
+        SyncDirection::Pull => {
+            if let Some(remote) = &job.remote {
+                let remote_privs = user_info.lookup_privs(auth_id, &["remote", remote]);
+                remote_privs & PRIV_REMOTE_AUDIT != 0
+            } else {
+                let source_ds_privs =
+                    user_info.lookup_privs(auth_id, &["datastore", &job.remote_store]);
+                source_ds_privs & PRIV_DATASTORE_AUDIT != 0
+            }
+        }
+        SyncDirection::Push => {
+            // check for audit access on remote/datastore/namespace
+            if let Some(target_acl_path) = job.remote_acl_path() {
+                let remote_privs = user_info.lookup_privs(auth_id, &target_acl_path);
+                remote_privs & PRIV_REMOTE_AUDIT != 0
+            } else {
+                // Remote must always be present for sync in push direction, fail otherwise
+                false
+            }
+        }
     }
 }
 
@@ -43,41 +62,93 @@ fn is_correct_owner(auth_id: &Authid, job: &SyncJobConfig) -> bool {
     }
 }
 
-/// checks whether user can run the corresponding pull job
+/// checks whether user can run the corresponding sync job, depending on sync direction
 ///
-/// namespace creation/deletion ACL and backup group ownership checks happen in the pull code directly.
+/// namespace creation/deletion ACL and backup group ownership checks happen in the pull/push code
+/// directly.
 /// remote side checks/filters remote datastore/namespace/group access.
 pub fn check_sync_job_modify_access(
     user_info: &CachedUserInfo,
     auth_id: &Authid,
     job: &SyncJobConfig,
+    sync_direction: SyncDirection,
 ) -> bool {
-    let ns_anchor_privs = user_info.lookup_privs(auth_id, &job.acl_path());
-    if ns_anchor_privs & PRIV_DATASTORE_BACKUP == 0 || ns_anchor_privs & PRIV_DATASTORE_AUDIT == 0 {
-        return false;
-    }
+    match sync_direction {
+        SyncDirection::Pull => {
+            let ns_anchor_privs = user_info.lookup_privs(auth_id, &job.acl_path());
+            if ns_anchor_privs & PRIV_DATASTORE_BACKUP == 0
+                || ns_anchor_privs & PRIV_DATASTORE_AUDIT == 0
+            {
+                return false;
+            }
+
+            if let Some(true) = job.remove_vanished {
+                if ns_anchor_privs & PRIV_DATASTORE_PRUNE == 0 {
+                    return false;
+                }
+            }
 
-    if let Some(true) = job.remove_vanished {
-        if ns_anchor_privs & PRIV_DATASTORE_PRUNE == 0 {
-            return false;
+            // same permission as changing ownership after syncing
+            if !is_correct_owner(auth_id, job) && ns_anchor_privs & PRIV_DATASTORE_MODIFY == 0 {
+                return false;
+            }
+
+            if let Some(remote) = &job.remote {
+                let remote_privs =
+                    user_info.lookup_privs(auth_id, &["remote", remote, &job.remote_store]);
+                return remote_privs & PRIV_REMOTE_READ != 0;
+            }
+            true
         }
-    }
+        SyncDirection::Push => {
+            // Remote must always be present for sync in push direction, fail otherwise
+            let target_privs = if let Some(target_acl_path) = job.remote_acl_path() {
+                user_info.lookup_privs(auth_id, &target_acl_path)
+            } else {
+                return false;
+            };
+
+            // check user is allowed to create backups on remote datastore
+            if target_privs & PRIV_REMOTE_DATASTORE_BACKUP == 0 {
+                return false;
+            }
 
-    // same permission as changing ownership after syncing
-    if !is_correct_owner(auth_id, job) && ns_anchor_privs & PRIV_DATASTORE_MODIFY == 0 {
-        return false;
-    }
+            if let Some(true) = job.remove_vanished {
+                // check user is allowed to prune backup snapshots on remote datastore
+                if target_privs & PRIV_REMOTE_DATASTORE_PRUNE == 0 {
+                    return false;
+                }
+            }
+
+            // check user is not the owner of the sync job, but has remote datastore modify permissions
+            if !is_correct_owner(auth_id, job) && target_privs & PRIV_REMOTE_DATASTORE_MODIFY == 0 {
+                return false;
+            }
+
+            // check user is allowed to read from (local) source datastore/namespace
+            let source_privs = user_info.lookup_privs(auth_id, &job.acl_path());
+            if source_privs & PRIV_DATASTORE_AUDIT == 0 {
+                return false;
+            }
 
-    if let Some(remote) = &job.remote {
-        let remote_privs = user_info.lookup_privs(auth_id, &["remote", remote, &job.remote_store]);
-        return remote_privs & PRIV_REMOTE_READ != 0;
+            // check for either datastore read or datastore backup access
+            // (the later implying read access for owned snapshot groups)
+            if source_privs & PRIV_DATASTORE_READ != 0 {
+                return true;
+            }
+            source_privs & PRIV_DATASTORE_BACKUP != 0
+        }
     }
-    true
 }
 
 #[api(
     input: {
-        properties: {},
+        properties: {
+            "sync-direction": {
+                type: SyncDirection,
+                optional: true,
+            },
+        },
     },
     returns: {
         description: "List configured jobs.",
@@ -92,6 +163,7 @@ pub fn check_sync_job_modify_access(
 /// List all sync jobs
 pub fn list_sync_jobs(
     _param: Value,
+    sync_direction: Option<SyncDirection>,
     rpcenv: &mut dyn RpcEnvironment,
 ) -> Result<Vec<SyncJobConfig>, Error> {
     let auth_id: Authid = rpcenv.get_auth_id().unwrap().parse()?;
@@ -99,13 +171,16 @@ pub fn list_sync_jobs(
 
     let (config, digest) = sync::config()?;
 
-    let list = config.convert_to_typed_array("sync")?;
+    let sync_direction = sync_direction.unwrap_or_default();
+    let list = config.convert_to_typed_array(sync_direction.as_config_type_str())?;
 
     rpcenv["digest"] = hex::encode(digest).into();
 
     let list = list
         .into_iter()
-        .filter(|sync_job| check_sync_job_read_access(&user_info, &auth_id, sync_job))
+        .filter(|sync_job| {
+            check_sync_job_read_access(&user_info, &auth_id, sync_job, sync_direction)
+        })
         .collect();
     Ok(list)
 }
@@ -118,6 +193,10 @@ pub fn list_sync_jobs(
                 type: SyncJobConfig,
                 flatten: true,
             },
+            "sync-direction": {
+                type: SyncDirection,
+                optional: true,
+            },
         },
     },
     access: {
@@ -128,14 +207,16 @@ pub fn list_sync_jobs(
 /// Create a new sync job.
 pub fn create_sync_job(
     config: SyncJobConfig,
+    sync_direction: Option<SyncDirection>,
     rpcenv: &mut dyn RpcEnvironment,
 ) -> Result<(), Error> {
     let auth_id: Authid = rpcenv.get_auth_id().unwrap().parse()?;
     let user_info = CachedUserInfo::new()?;
+    let sync_direction = sync_direction.unwrap_or_default();
 
     let _lock = sync::lock_config()?;
 
-    if !check_sync_job_modify_access(&user_info, &auth_id, &config) {
+    if !check_sync_job_modify_access(&user_info, &auth_id, &config, sync_direction) {
         bail!("permission check failed");
     }
 
@@ -158,7 +239,7 @@ pub fn create_sync_job(
         param_bail!("id", "job '{}' already exists.", config.id);
     }
 
-    section_config.set_data(&config.id, "sync", &config)?;
+    section_config.set_data(&config.id, sync_direction.as_config_type_str(), &config)?;
 
     sync::save_config(&section_config)?;
 
@@ -188,8 +269,17 @@ pub fn read_sync_job(id: String, rpcenv: &mut dyn RpcEnvironment) -> Result<Sync
 
     let (config, digest) = sync::config()?;
 
-    let sync_job = config.lookup("sync", &id)?;
-    if !check_sync_job_read_access(&user_info, &auth_id, &sync_job) {
+    let (sync_job, sync_direction) =
+        if let Some((config_type, config_section)) = config.sections.get(&id) {
+            (
+                SyncJobConfig::deserialize(config_section)?,
+                SyncDirection::from_config_type_str(config_type)?,
+            )
+        } else {
+            http_bail!(NOT_FOUND, "job '{id}' does not exist.")
+        };
+
+    if !check_sync_job_read_access(&user_info, &auth_id, &sync_job, sync_direction) {
         bail!("permission check failed");
     }
 
@@ -284,7 +374,15 @@ pub fn update_sync_job(
         crate::tools::detect_modified_configuration_file(&digest, &expected_digest)?;
     }
 
-    let mut data: SyncJobConfig = config.lookup("sync", &id)?;
+    let (mut data, sync_direction) =
+        if let Some((config_type, config_section)) = config.sections.get(&id) {
+            (
+                SyncJobConfig::deserialize(config_section)?,
+                SyncDirection::from_config_type_str(config_type)?,
+            )
+        } else {
+            http_bail!(NOT_FOUND, "job '{id}' does not exist.")
+        };
 
     if let Some(delete) = delete {
         for delete_prop in delete {
@@ -405,11 +503,11 @@ pub fn update_sync_job(
         }
     }
 
-    if !check_sync_job_modify_access(&user_info, &auth_id, &data) {
+    if !check_sync_job_modify_access(&user_info, &auth_id, &data, sync_direction) {
         bail!("permission check failed");
     }
 
-    config.set_data(&id, "sync", &data)?;
+    config.set_data(&id, sync_direction.as_config_type_str(), &data)?;
 
     sync::save_config(&config)?;
 
@@ -456,17 +554,16 @@ pub fn delete_sync_job(
         crate::tools::detect_modified_configuration_file(&digest, &expected_digest)?;
     }
 
-    match config.lookup("sync", &id) {
-        Ok(job) => {
-            if !check_sync_job_modify_access(&user_info, &auth_id, &job) {
-                bail!("permission check failed");
-            }
-            config.sections.remove(&id);
-        }
-        Err(_) => {
-            http_bail!(NOT_FOUND, "job '{}' does not exist.", id)
+    if let Some((config_type, config_section)) = config.sections.get(&id) {
+        let sync_direction = SyncDirection::from_config_type_str(config_type)?;
+        let job = SyncJobConfig::deserialize(config_section)?;
+        if !check_sync_job_modify_access(&user_info, &auth_id, &job, sync_direction) {
+            bail!("permission check failed");
         }
-    };
+        config.sections.remove(&id);
+    } else {
+        http_bail!(NOT_FOUND, "job '{}' does not exist.", id)
+    }
 
     sync::save_config(&config)?;
 
@@ -536,39 +633,67 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     };
 
     // should work without ACLs
-    assert!(check_sync_job_read_access(&user_info, root_auth_id, &job));
-    assert!(check_sync_job_modify_access(&user_info, root_auth_id, &job));
+    assert!(check_sync_job_read_access(
+        &user_info,
+        root_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
+    assert!(check_sync_job_modify_access(
+        &user_info,
+        root_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
 
     // user without permissions must fail
     assert!(!check_sync_job_read_access(
         &user_info,
         &no_perm_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
     assert!(!check_sync_job_modify_access(
         &user_info,
         &no_perm_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // reading without proper read permissions on either remote or local must fail
-    assert!(!check_sync_job_read_access(&user_info, &read_auth_id, &job));
+    assert!(!check_sync_job_read_access(
+        &user_info,
+        &read_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
 
     // reading without proper read permissions on local end must fail
     job.remote = Some("remote1".to_string());
-    assert!(!check_sync_job_read_access(&user_info, &read_auth_id, &job));
+    assert!(!check_sync_job_read_access(
+        &user_info,
+        &read_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
 
     // reading without proper read permissions on remote end must fail
     job.remote = Some("remote0".to_string());
     job.store = "localstore1".to_string();
-    assert!(!check_sync_job_read_access(&user_info, &read_auth_id, &job));
+    assert!(!check_sync_job_read_access(
+        &user_info,
+        &read_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
 
     // writing without proper write permissions on either end must fail
     job.store = "localstore0".to_string();
     assert!(!check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // writing without proper write permissions on local end must fail
@@ -580,39 +705,54 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     assert!(!check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // reset remote to one where users have access
     job.remote = Some("remote1".to_string());
 
     // user with read permission can only read, but not modify/run
-    assert!(check_sync_job_read_access(&user_info, &read_auth_id, &job));
+    assert!(check_sync_job_read_access(
+        &user_info,
+        &read_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
     job.owner = Some(read_auth_id.clone());
     assert!(!check_sync_job_modify_access(
         &user_info,
         &read_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
     job.owner = None;
     assert!(!check_sync_job_modify_access(
         &user_info,
         &read_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
     job.owner = Some(write_auth_id.clone());
     assert!(!check_sync_job_modify_access(
         &user_info,
         &read_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // user with simple write permission can modify/run
-    assert!(check_sync_job_read_access(&user_info, &write_auth_id, &job));
+    assert!(check_sync_job_read_access(
+        &user_info,
+        &write_auth_id,
+        &job,
+        SyncDirection::Pull,
+    ));
     assert!(check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // but can't modify/run with deletion
@@ -620,7 +760,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     assert!(!check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // unless they have Datastore.Prune as well
@@ -628,7 +769,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     assert!(check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // changing owner is not possible
@@ -636,7 +778,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     assert!(!check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // also not to the default 'root@pam'
@@ -644,7 +787,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     assert!(!check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     // unless they have Datastore.Modify as well
@@ -653,13 +797,15 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
     assert!(check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
     job.owner = None;
     assert!(check_sync_job_modify_access(
         &user_info,
         &write_auth_id,
-        &job
+        &job,
+        SyncDirection::Pull,
     ));
 
     Ok(())
diff --git a/src/bin/proxmox-backup-proxy.rs b/src/bin/proxmox-backup-proxy.rs
index 6f19a3fbd..70283510d 100644
--- a/src/bin/proxmox-backup-proxy.rs
+++ b/src/bin/proxmox-backup-proxy.rs
@@ -589,7 +589,14 @@ async fn schedule_datastore_sync_jobs() {
         Ok((config, _digest)) => config,
     };
 
-    for (job_id, (_, job_config)) in config.sections {
+    for (job_id, (job_type, job_config)) in config.sections {
+        let sync_direction = match SyncDirection::from_config_type_str(&job_type) {
+            Ok(direction) => direction,
+            Err(err) => {
+                eprintln!("unexpected config type in sync job config - {err}");
+                continue;
+            }
+        };
         let job_config: SyncJobConfig = match serde_json::from_value(job_config) {
             Ok(c) => c,
             Err(err) => {
@@ -616,7 +623,7 @@ async fn schedule_datastore_sync_jobs() {
                 job_config,
                 &auth_id,
                 Some(event_str),
-                SyncDirection::Pull,
+                sync_direction,
                 false,
             ) {
                 eprintln!("unable to start datastore sync job {job_id} - {err}");
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 20/29] api: admin: avoid duplicate name for list sync jobs api method
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (18 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 19/29] api: sync jobs: expose optional `sync-direction` parameter Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 21/29] bin: manager: add datastore push cli command Christian Ebner
                   ` (8 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

`list_sync_jobs` exists as api method in `api2::admin::sync` and
`api2::config::sync`.

Rename the admin api endpoint method to `list_config_sync_jobs` in
order to reduce possible confusion when searching/reviewing.

No functional change intended.

Suggested-by: Fabian Grünbichler <f.gruenbichler@proxmox.com>
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- Adapt to modified api methods

 src/api2/admin/sync.rs               | 6 +++---
 src/api2/config/datastore.rs         | 6 ++++--
 src/api2/config/notifications/mod.rs | 4 ++--
 3 files changed, 9 insertions(+), 7 deletions(-)

diff --git a/src/api2/admin/sync.rs b/src/api2/admin/sync.rs
index 8a242b1c3..3a41aa2c7 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -46,8 +46,8 @@ use crate::{
         permission: &Permission::Anybody,
     },
 )]
-/// List all sync jobs
-pub fn list_sync_jobs(
+/// List all configured sync jobs
+pub fn list_config_sync_jobs(
     store: Option<String>,
     sync_direction: Option<SyncDirection>,
     _param: Value,
@@ -144,5 +144,5 @@ const SYNC_INFO_ROUTER: Router = Router::new()
     .subdirs(SYNC_INFO_SUBDIRS);
 
 pub const ROUTER: Router = Router::new()
-    .get(&API_METHOD_LIST_SYNC_JOBS)
+    .get(&API_METHOD_LIST_CONFIG_SYNC_JOBS)
     .match_all("id", &SYNC_INFO_ROUTER);
diff --git a/src/api2/config/datastore.rs b/src/api2/config/datastore.rs
index c151eda10..ec7cc1909 100644
--- a/src/api2/config/datastore.rs
+++ b/src/api2/config/datastore.rs
@@ -21,7 +21,7 @@ use pbs_config::BackupLockGuard;
 use pbs_datastore::chunk_store::ChunkStore;
 
 use crate::api2::admin::{
-    prune::list_prune_jobs, sync::list_sync_jobs, verify::list_verification_jobs,
+    prune::list_prune_jobs, sync::list_config_sync_jobs, verify::list_verification_jobs,
 };
 use crate::api2::config::prune::{delete_prune_job, do_create_prune_job};
 use crate::api2::config::sync::delete_sync_job;
@@ -500,7 +500,9 @@ pub async fn delete_datastore(
             delete_verification_job(job.config.id, None, rpcenv)?
         }
         for direction in [SyncDirection::Pull, SyncDirection::Push] {
-            for job in list_sync_jobs(Some(name.clone()), Some(direction), Value::Null, rpcenv)? {
+            for job in
+                list_config_sync_jobs(Some(name.clone()), Some(direction), Value::Null, rpcenv)?
+            {
                 delete_sync_job(job.config.id, None, rpcenv)?
             }
         }
diff --git a/src/api2/config/notifications/mod.rs b/src/api2/config/notifications/mod.rs
index 31c4851c1..f156c8cfd 100644
--- a/src/api2/config/notifications/mod.rs
+++ b/src/api2/config/notifications/mod.rs
@@ -12,7 +12,7 @@ use crate::api2::admin::datastore::get_datastore_list;
 use pbs_api_types::{SyncDirection, PRIV_SYS_AUDIT};
 
 use crate::api2::admin::prune::list_prune_jobs;
-use crate::api2::admin::sync::list_sync_jobs;
+use crate::api2::admin::sync::list_config_sync_jobs;
 use crate::api2::admin::verify::list_verification_jobs;
 use crate::api2::config::media_pool::list_pools;
 use crate::api2::tape::backup::list_tape_backup_jobs;
@@ -155,7 +155,7 @@ pub fn get_values(
     }
 
     for direction in [SyncDirection::Pull, SyncDirection::Push] {
-        let sync_jobs = list_sync_jobs(None, Some(direction), param.clone(), rpcenv)?;
+        let sync_jobs = list_config_sync_jobs(None, Some(direction), param.clone(), rpcenv)?;
         for job in sync_jobs {
             values.push(MatchableValue {
                 field: "job-id".into(),
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel

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

* [pbs-devel] [PATCH v6 proxmox-backup 21/29] bin: manager: add datastore push cli command
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (19 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 20/29] api: admin: avoid duplicate name for list sync jobs api method Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 22/29] ui: group filter: allow to set namespace for local datastore Christian Ebner
                   ` (7 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Expose the push api endpoint to be callable via the command line
interface.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 src/bin/proxmox-backup-manager.rs | 216 +++++++++++++++++++++++-------
 1 file changed, 169 insertions(+), 47 deletions(-)

diff --git a/src/bin/proxmox-backup-manager.rs b/src/bin/proxmox-backup-manager.rs
index 420e96665..f91d5bf29 100644
--- a/src/bin/proxmox-backup-manager.rs
+++ b/src/bin/proxmox-backup-manager.rs
@@ -12,7 +12,7 @@ use proxmox_sys::fs::CreateOptions;
 
 use pbs_api_types::percent_encoding::percent_encode_component;
 use pbs_api_types::{
-    BackupNamespace, GroupFilter, RateLimitConfig, SyncJobConfig, DATASTORE_SCHEMA,
+    BackupNamespace, GroupFilter, RateLimitConfig, SyncDirection, SyncJobConfig, DATASTORE_SCHEMA,
     GROUP_FILTER_LIST_SCHEMA, IGNORE_VERIFIED_BACKUPS_SCHEMA, NS_MAX_DEPTH_SCHEMA,
     REMOTE_ID_SCHEMA, REMOVE_VANISHED_BACKUPS_SCHEMA, TRANSFER_LAST_SCHEMA, UPID_SCHEMA,
     VERIFICATION_OUTDATED_AFTER_SCHEMA,
@@ -294,6 +294,72 @@ fn task_mgmt_cli() -> CommandLineInterface {
     cmd_def.into()
 }
 
+/// Sync datastore by pulling from or pushing to another repository
+#[allow(clippy::too_many_arguments)]
+async fn sync_datastore(
+    remote: String,
+    remote_store: String,
+    remote_ns: Option<BackupNamespace>,
+    store: String,
+    ns: Option<BackupNamespace>,
+    remove_vanished: Option<bool>,
+    max_depth: Option<usize>,
+    group_filter: Option<Vec<GroupFilter>>,
+    limit: RateLimitConfig,
+    transfer_last: Option<usize>,
+    param: Value,
+    sync_direction: SyncDirection,
+) -> Result<Value, Error> {
+    let output_format = get_output_format(&param);
+
+    let client = connect_to_localhost()?;
+    let mut args = json!({
+        "store": store,
+        "remote": remote,
+        "remote-store": remote_store,
+    });
+
+    if remote_ns.is_some() {
+        args["remote-ns"] = json!(remote_ns);
+    }
+
+    if ns.is_some() {
+        args["ns"] = json!(ns);
+    }
+
+    if max_depth.is_some() {
+        args["max-depth"] = json!(max_depth);
+    }
+
+    if group_filter.is_some() {
+        args["group-filter"] = json!(group_filter);
+    }
+
+    if let Some(remove_vanished) = remove_vanished {
+        args["remove-vanished"] = Value::from(remove_vanished);
+    }
+
+    if transfer_last.is_some() {
+        args["transfer-last"] = json!(transfer_last)
+    }
+
+    let mut limit_json = json!(limit);
+    let limit_map = limit_json
+        .as_object_mut()
+        .ok_or_else(|| format_err!("limit is not an Object"))?;
+
+    args.as_object_mut().unwrap().append(limit_map);
+
+    let result = match sync_direction {
+        SyncDirection::Pull => client.post("api2/json/pull", Some(args)).await?,
+        SyncDirection::Push => client.post("api2/json/push", Some(args)).await?,
+    };
+
+    view_task_result(&client, result, &output_format).await?;
+
+    Ok(Value::Null)
+}
+
 // fixme: avoid API redefinition
 #[api(
    input: {
@@ -342,7 +408,7 @@ fn task_mgmt_cli() -> CommandLineInterface {
         }
    }
 )]
-/// Sync datastore from another repository
+/// Sync datastore by pulling from another repository
 #[allow(clippy::too_many_arguments)]
 async fn pull_datastore(
     remote: String,
@@ -357,52 +423,100 @@ async fn pull_datastore(
     transfer_last: Option<usize>,
     param: Value,
 ) -> Result<Value, Error> {
-    let output_format = get_output_format(&param);
-
-    let client = connect_to_localhost()?;
-
-    let mut args = json!({
-        "store": store,
-        "remote": remote,
-        "remote-store": remote_store,
-    });
-
-    if remote_ns.is_some() {
-        args["remote-ns"] = json!(remote_ns);
-    }
-
-    if ns.is_some() {
-        args["ns"] = json!(ns);
-    }
-
-    if max_depth.is_some() {
-        args["max-depth"] = json!(max_depth);
-    }
-
-    if group_filter.is_some() {
-        args["group-filter"] = json!(group_filter);
-    }
-
-    if let Some(remove_vanished) = remove_vanished {
-        args["remove-vanished"] = Value::from(remove_vanished);
-    }
-
-    if transfer_last.is_some() {
-        args["transfer-last"] = json!(transfer_last)
-    }
-
-    let mut limit_json = json!(limit);
-    let limit_map = limit_json
-        .as_object_mut()
-        .ok_or_else(|| format_err!("limit is not an Object"))?;
-
-    args.as_object_mut().unwrap().append(limit_map);
-
-    let result = client.post("api2/json/pull", Some(args)).await?;
-
-    view_task_result(&client, result, &output_format).await?;
+    sync_datastore(
+        remote,
+        remote_store,
+        remote_ns,
+        store,
+        ns,
+        remove_vanished,
+        max_depth,
+        group_filter,
+        limit,
+        transfer_last,
+        param,
+        SyncDirection::Pull,
+    )
+    .await
+}
 
-    Ok(Value::Null)
+#[api(
+   input: {
+        properties: {
+            "store": {
+                schema: DATASTORE_SCHEMA,
+            },
+            "ns": {
+                type: BackupNamespace,
+                optional: true,
+            },
+            remote: {
+                schema: REMOTE_ID_SCHEMA,
+            },
+            "remote-store": {
+                schema: DATASTORE_SCHEMA,
+            },
+            "remote-ns": {
+                type: BackupNamespace,
+                optional: true,
+            },
+            "remove-vanished": {
+                schema: REMOVE_VANISHED_BACKUPS_SCHEMA,
+                optional: true,
+            },
+            "max-depth": {
+                schema: NS_MAX_DEPTH_SCHEMA,
+                optional: true,
+            },
+            "group-filter": {
+                schema: GROUP_FILTER_LIST_SCHEMA,
+                optional: true,
+            },
+            limit: {
+                type: RateLimitConfig,
+                flatten: true,
+            },
+            "output-format": {
+                schema: OUTPUT_FORMAT,
+                optional: true,
+            },
+            "transfer-last": {
+                schema: TRANSFER_LAST_SCHEMA,
+                optional: true,
+            },
+        }
+   }
+)]
+/// Sync datastore by pushing to another repository
+#[allow(clippy::too_many_arguments)]
+async fn push_datastore(
+    remote: String,
+    remote_store: String,
+    remote_ns: Option<BackupNamespace>,
+    store: String,
+    ns: Option<BackupNamespace>,
+    remove_vanished: Option<bool>,
+    max_depth: Option<usize>,
+    group_filter: Option<Vec<GroupFilter>>,
+    limit: RateLimitConfig,
+    transfer_last: Option<usize>,
+    param: Value,
+) -> Result<Value, Error> {
+    sync_datastore(
+        remote,
+        remote_store,
+        remote_ns,
+        store,
+        ns,
+        remove_vanished,
+        max_depth,
+        group_filter,
+        limit,
+        transfer_last,
+        param,
+        SyncDirection::Push,
+    )
+    .await
 }
 
 #[api(
@@ -528,6 +642,14 @@ async fn run() -> Result<(), Error> {
                 .completion_cb("group-filter", complete_remote_datastore_group_filter)
                 .completion_cb("remote-ns", complete_remote_datastore_namespace),
         )
+        .insert(
+            "push",
+            CliCommand::new(&API_METHOD_PUSH_DATASTORE)
+                .arg_param(&["store", "remote", "remote-store"])
+                .completion_cb("store", pbs_config::datastore::complete_datastore_name)
+                .completion_cb("remote", pbs_config::remote::complete_remote_name)
+                .completion_cb("remote-store", complete_remote_datastore_name),
+        )
         .insert(
             "verify",
             CliCommand::new(&API_METHOD_VERIFY)
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 22/29] ui: group filter: allow to set namespace for local datastore
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (20 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 21/29] bin: manager: add datastore push cli command Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 23/29] ui: sync edit: source group filters based on sync direction Christian Ebner
                   ` (6 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

The namespace has to be set in order to get the correct groups to be
used as group filter options with a local datastore as source,
required for sync jobs in push direction.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- move namespace checks closer to api method url, showing their
  dependence more clearly

 www/form/GroupFilter.js | 21 ++++++++++++++++++---
 1 file changed, 18 insertions(+), 3 deletions(-)

diff --git a/www/form/GroupFilter.js b/www/form/GroupFilter.js
index c9c2d913e..22f889752 100644
--- a/www/form/GroupFilter.js
+++ b/www/form/GroupFilter.js
@@ -252,14 +252,17 @@ Ext.define('PBS.form.GroupFilter', {
 	let url;
 	if (me.remote) {
 	    url = `/api2/json/config/remote/${me.remote}/scan/${me.datastore}/groups`;
+	    if (me.namespace) {
+		url += `?namespace=${me.namespace}`;
+	    }
 	} else if (me.datastore) {
 	    url = `/api2/json/admin/datastore/${me.datastore}/groups`;
+	    if (me.namespace) {
+		url += `?ns=${me.namespace}`;
+	    }
 	} else {
 	    return;
 	}
-	if (me.namespace) {
-	    url += `?namespace=${me.namespace}`;
-	}
 	me.setDsStoreUrl(url);
 	me.dsStore.load({
 	    callback: (records) => {
@@ -279,6 +282,18 @@ Ext.define('PBS.form.GroupFilter', {
 	}
 	me.remote = undefined;
 	me.datastore = datastore;
+	me.namespace = undefined;
+	me.updateGroupSelectors();
+    },
+
+    setLocalNamespace: function(datastore, namespace) {
+	let me = this;
+	if (me.datastore === datastore && me.namespace === namespace) {
+	    return;
+	}
+	me.remote = undefined;
+	me.datastore = datastore;
+	me.namespace = namespace;
 	me.updateGroupSelectors();
     },
 
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 23/29] ui: sync edit: source group filters based on sync direction
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (21 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 22/29] ui: group filter: allow to set namespace for local datastore Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 24/29] ui: add view with separate grids for pull and push sync jobs Christian Ebner
                   ` (5 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Switch to the local datastore, used as sync source for jobs in push
direction, to get the available group filter options.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 www/window/SyncJobEdit.js | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)

diff --git a/www/window/SyncJobEdit.js b/www/window/SyncJobEdit.js
index 6543995e8..9ca79eaa9 100644
--- a/www/window/SyncJobEdit.js
+++ b/www/window/SyncJobEdit.js
@@ -238,7 +238,13 @@ Ext.define('PBS.window.SyncJobEdit', {
 				let remoteNamespaceField = me.up('pbsSyncJobEdit').down('field[name=remote-ns]');
 				remoteNamespaceField.setRemote(remote);
 				remoteNamespaceField.setRemoteStore(value);
-				me.up('tabpanel').down('pbsGroupFilter').setRemoteDatastore(remote, value);
+
+				if (!me.syncDirectionPush) {
+				    me.up('tabpanel').down('pbsGroupFilter').setRemoteDatastore(remote, value);
+				} else {
+				    let localStore = me.up('pbsSyncJobEdit').down('field[name=store]').getValue();
+				    me.up('tabpanel').down('pbsGroupFilter').setLocalDatastore(localStore);
+				}
 			    },
 			},
 		    },
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 24/29] ui: add view with separate grids for pull and push sync jobs
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (22 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 23/29] ui: sync edit: source group filters based on sync direction Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 25/29] ui: sync job: adapt edit window to be used for pull and push Christian Ebner
                   ` (4 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Show sync jobs in pull and in push direction in two separate grids,
visually separating them to limit possible misconfiguration.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- adapted minHeight, both grids are equal

 www/Makefile                   |  1 +
 www/config/SyncPullPushView.js | 61 ++++++++++++++++++++++++++++++++++
 www/config/SyncView.js         | 10 +++++-
 www/datastore/DataStoreList.js |  2 +-
 www/datastore/Panel.js         |  2 +-
 5 files changed, 73 insertions(+), 3 deletions(-)
 create mode 100644 www/config/SyncPullPushView.js

diff --git a/www/Makefile b/www/Makefile
index 609a0ba67..d35e81283 100644
--- a/www/Makefile
+++ b/www/Makefile
@@ -61,6 +61,7 @@ JSSRC=							\
 	config/TrafficControlView.js			\
 	config/ACLView.js				\
 	config/SyncView.js				\
+	config/SyncPullPushView.js			\
 	config/VerifyView.js				\
 	config/PruneView.js				\
 	config/GCView.js				\
diff --git a/www/config/SyncPullPushView.js b/www/config/SyncPullPushView.js
new file mode 100644
index 000000000..3460bc662
--- /dev/null
+++ b/www/config/SyncPullPushView.js
@@ -0,0 +1,61 @@
+Ext.define('PBS.config.SyncPullPush', {
+    extend: 'Ext.panel.Panel',
+    alias: 'widget.pbsSyncJobPullPushView',
+    title: gettext('Sync Jobs'),
+
+    mixins: ['Proxmox.Mixin.CBind'],
+
+    layout: {
+	type: 'vbox',
+	align: 'stretch',
+	multi: true,
+	bodyPadding: 5,
+    },
+    defaults: {
+	collapsible: false,
+	margin: 5,
+    },
+    scrollable: true,
+    items: [
+	{
+	    xtype: 'pbsSyncJobView',
+	    itemId: 'syncJobsPull',
+	    syncDirection: 'pull',
+	    cbind: {
+		datastore: '{datastore}',
+	    },
+	    minHeight: 125, // shows at least one line of content
+	},
+	{
+	    xtype: 'splitter',
+	    performCollapse: false,
+	},
+	{
+	    xtype: 'pbsSyncJobView',
+	    itemId: 'syncJobsPush',
+	    syncDirection: 'push',
+	    cbind: {
+		datastore: '{datastore}',
+	    },
+	    flex: 1,
+	    minHeight: 125, // shows at least one line of content
+	},
+    ],
+    initComponent: function() {
+	let me = this;
+
+	let subPanelIds = me.items.map(el => el.itemId).filter(id => !!id);
+
+	me.callParent();
+
+	for (const itemId of subPanelIds) {
+	    let component = me.getComponent(itemId);
+	    component.relayEvents(me, ['activate', 'deactivate', 'destroy']);
+	}
+    },
+
+    cbindData: function(initialConfig) {
+        let me = this;
+        me.datastore = initialConfig.datastore ? initialConfig.datastore : undefined;
+    },
+});
diff --git a/www/config/SyncView.js b/www/config/SyncView.js
index 4669a23e2..ef17e4315 100644
--- a/www/config/SyncView.js
+++ b/www/config/SyncView.js
@@ -29,7 +29,7 @@ Ext.define('PBS.config.SyncJobView', {
     stateful: true,
     stateId: 'grid-sync-jobs-v1',
 
-    title: gettext('Sync Jobs'),
+    title: gettext('Sync Jobs - Pull Direction'),
 
     controller: {
 	xclass: 'Ext.app.ViewController',
@@ -39,6 +39,7 @@ Ext.define('PBS.config.SyncJobView', {
 	    let view = me.getView();
             Ext.create('PBS.window.SyncJobEdit', {
 		datastore: view.datastore,
+		syncDirection: view.syncDirection,
 		listeners: {
 		    destroy: function() {
 			me.reload();
@@ -117,6 +118,9 @@ Ext.define('PBS.config.SyncJobView', {
 	    if (view.datastore !== undefined) {
 		params.store = view.datastore;
 	    }
+	    if (view.syncDirection !== undefined) {
+		params["sync-direction"] = view.syncDirection;
+	    }
 	    view.getStore().rstore.getProxy().setExtraParams(params);
 	    Proxmox.Utils.monStoreErrors(view, view.getStore().rstore);
 	},
@@ -303,6 +307,10 @@ Ext.define('PBS.config.SyncJobView', {
 	    }
 	}
 
+	if (me.syncDirection === 'push') {
+	    me.title = gettext('Sync Jobs - Push Direction');
+	}
+
 	me.callParent();
     },
 });
diff --git a/www/datastore/DataStoreList.js b/www/datastore/DataStoreList.js
index fc68cfc10..22ef18540 100644
--- a/www/datastore/DataStoreList.js
+++ b/www/datastore/DataStoreList.js
@@ -239,7 +239,7 @@ Ext.define('PBS.datastore.DataStores', {
 	{
 	    iconCls: 'fa fa-refresh',
 	    itemId: 'syncjobs',
-	    xtype: 'pbsSyncJobView',
+	    xtype: 'pbsSyncJobPullPushView',
 	},
 	{
 	    iconCls: 'fa fa-check-circle',
diff --git a/www/datastore/Panel.js b/www/datastore/Panel.js
index ad9fc10fe..e1da7cfac 100644
--- a/www/datastore/Panel.js
+++ b/www/datastore/Panel.js
@@ -68,7 +68,7 @@ Ext.define('PBS.DataStorePanel', {
 	{
 	    iconCls: 'fa fa-refresh',
 	    itemId: 'syncjobs',
-	    xtype: 'pbsSyncJobView',
+	    xtype: 'pbsSyncJobPullPushView',
 	    cbind: {
 		datastore: '{datastore}',
 	    },
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 25/29] ui: sync job: adapt edit window to be used for pull and push
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (23 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 24/29] ui: add view with separate grids for pull and push sync jobs Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 26/29] ui: sync view: set proxy on view instead of model Christian Ebner
                   ` (3 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Switch the subject and labels to be shown based on the direction of
the sync job, and set the `sync-direction` parameter from the
submit values in case of push direction.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- no changes

 www/window/SyncJobEdit.js | 37 ++++++++++++++++++++++++++++++++-----
 1 file changed, 32 insertions(+), 5 deletions(-)

diff --git a/www/window/SyncJobEdit.js b/www/window/SyncJobEdit.js
index 9ca79eaa9..b3aa06057 100644
--- a/www/window/SyncJobEdit.js
+++ b/www/window/SyncJobEdit.js
@@ -9,7 +9,7 @@ Ext.define('PBS.window.SyncJobEdit', {
 
     isAdd: true,
 
-    subject: gettext('Sync Job'),
+    subject: gettext('Sync Job - Pull Direction'),
 
     bodyPadding: 0,
 
@@ -29,6 +29,26 @@ Ext.define('PBS.window.SyncJobEdit', {
 	me.scheduleValue = id ? null : 'hourly';
 	me.authid = id ? null : Proxmox.UserName;
 	me.editDatastore = me.datastore === undefined && me.isCreate;
+
+	if (me.syncDirection === 'push') {
+	    me.subject = gettext('Sync Job - Push Direction');
+	    me.syncDirectionPush = true;
+	    me.syncRemoteLabel = gettext('Target Remote');
+	    me.syncRemoteDatastore = gettext('Target Datastore');
+	    me.syncRemoteNamespace = gettext('Target Namespace');
+	    me.syncLocalOwner = gettext('Local User');
+	    me.extraRequestParams = {
+		"sync-direction": 'push',
+	    };
+	} else {
+	    me.subject = gettext('Sync Job - Pull Direction');
+	    me.syncDirectionPush = false;
+	    me.syncRemoteLabel = gettext('Source Remote');
+	    me.syncRemoteDatastore = gettext('Source Datastore');
+	    me.syncRemoteNamespace = gettext('Source Namespace');
+	    me.syncLocalOwner = gettext('Local Owner');
+	}
+
 	return { };
     },
 
@@ -118,10 +138,10 @@ Ext.define('PBS.window.SyncJobEdit', {
 			},
 		    },
 		    {
-			fieldLabel: gettext('Local Owner'),
 			xtype: 'pbsAuthidSelector',
 			name: 'owner',
 			cbind: {
+			    fieldLabel: '{syncLocalOwner}',
 			    value: '{authid}',
 			    deleteEmpty: '{!isCreate}',
 			},
@@ -151,6 +171,9 @@ Ext.define('PBS.window.SyncJobEdit', {
 			xtype: 'radiogroup',
 			fieldLabel: gettext('Location'),
 			defaultType: 'radiofield',
+			cbind: {
+			    disabled: '{syncDirectionPush}',
+			},
 			items: [
 			    {
 				boxLabel: 'Local',
@@ -201,7 +224,9 @@ Ext.define('PBS.window.SyncJobEdit', {
 			},
 		    },
 		    {
-			fieldLabel: gettext('Source Remote'),
+			cbind: {
+			    fieldLabel: '{syncRemoteLabel}',
+			},
 			xtype: 'pbsRemoteSelector',
 			allowBlank: false,
 			name: 'remote',
@@ -222,13 +247,13 @@ Ext.define('PBS.window.SyncJobEdit', {
 			},
 		    },
 		    {
-			fieldLabel: gettext('Source Datastore'),
 			xtype: 'pbsRemoteStoreSelector',
 			allowBlank: false,
 			autoSelect: false,
 			name: 'remote-store',
 			cbind: {
 			    datastore: '{datastore}',
+			    fieldLabel: '{syncRemoteDatastore}',
 			},
 			listeners: {
 			    change: function(field, value) {
@@ -249,7 +274,9 @@ Ext.define('PBS.window.SyncJobEdit', {
 			},
 		    },
 		    {
-			fieldLabel: gettext('Source Namespace'),
+			cbind: {
+			    fieldLabel: '{syncRemoteNamespace}',
+			},
 			xtype: 'pbsRemoteNamespaceSelector',
 			allowBlank: true,
 			autoSelect: false,
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 26/29] ui: sync view: set proxy on view instead of model
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (24 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 25/29] ui: sync job: adapt edit window to be used for pull and push Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 27/29] api: datastore/namespace: return backup groups delete stats on remove Christian Ebner
                   ` (2 subsequent siblings)
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

In order to load data using the same model from different sources,
set the proxy on the store instead of the model.
This allows to use the view to display sync jobs in either pull or
push direction, by setting the `sync-direction` ont the view.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- drop now unneeded sync direction

 www/config/SyncView.js | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/www/config/SyncView.js b/www/config/SyncView.js
index ef17e4315..30fc5fb54 100644
--- a/www/config/SyncView.js
+++ b/www/config/SyncView.js
@@ -16,10 +16,6 @@ Ext.define('pbs-sync-jobs-status', {
 	'comment',
     ],
     idProperty: 'id',
-    proxy: {
-	type: 'proxmox',
-	url: '/api2/json/admin/sync',
-    },
 });
 
 Ext.define('PBS.config.SyncJobView', {
@@ -142,6 +138,10 @@ Ext.define('PBS.config.SyncJobView', {
 	    storeid: 'pbs-sync-jobs-status',
 	    model: 'pbs-sync-jobs-status',
 	    interval: 5000,
+	    proxy: {
+		type: 'proxmox',
+		url: '/api2/json/admin/sync',
+	    },
 	},
     },
 
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 27/29] api: datastore/namespace: return backup groups delete stats on remove
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (25 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 26/29] ui: sync view: set proxy on view instead of model Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 28/29] api: version: add 'prune-delete-stats' as supported feature Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 29/29] docs: add section for sync jobs in push direction Christian Ebner
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Add and expose the backup group delete statistics by adding the
return type to the corresponding REST API endpoints.

Further, add a `error-on-protected` flag to the api endpoints,
allowing to return without error when set to false. Default remains
enabled.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- rename optional api parameter from `ignore-protected` to `error-on-protected`
  and switch boolean logic

 pbs-datastore/src/datastore.rs | 20 ++++++++++++++------
 src/api2/admin/datastore.rs    | 29 +++++++++++++++++++++--------
 src/api2/admin/namespace.rs    | 31 ++++++++++++++++++++++---------
 src/server/pull.rs             |  6 ++++--
 4 files changed, 61 insertions(+), 25 deletions(-)

diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index c8701d2dd..68c7f2934 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -489,16 +489,22 @@ impl DataStore {
     ///
     /// Does *not* descends into child-namespaces and doesn't remoes the namespace itself either.
     ///
-    /// Returns true if all the groups were removed, and false if some were protected.
-    pub fn remove_namespace_groups(self: &Arc<Self>, ns: &BackupNamespace) -> Result<bool, Error> {
+    /// Returns a tuple with the first item being true if all the groups were removed, and false if some were protected.
+    /// The second item returns the remove statistics.
+    pub fn remove_namespace_groups(
+        self: &Arc<Self>,
+        ns: &BackupNamespace,
+    ) -> Result<(bool, BackupGroupDeleteStats), Error> {
         // FIXME: locking? The single groups/snapshots are already protected, so may not be
         // necessary (depends on what we all allow to do with namespaces)
         log::info!("removing all groups in namespace {}:/{ns}", self.name());
 
         let mut removed_all_groups = true;
+        let mut stats = BackupGroupDeleteStats::default();
 
         for group in self.iter_backup_groups(ns.to_owned())? {
             let delete_stats = group?.destroy()?;
+            stats.add(&delete_stats);
             removed_all_groups = removed_all_groups && delete_stats.all_removed();
         }
 
@@ -515,7 +521,7 @@ impl DataStore {
             }
         }
 
-        Ok(removed_all_groups)
+        Ok((removed_all_groups, stats))
     }
 
     /// Remove a complete backup namespace optionally including all it's, and child namespaces',
@@ -527,13 +533,15 @@ impl DataStore {
         self: &Arc<Self>,
         ns: &BackupNamespace,
         delete_groups: bool,
-    ) -> Result<bool, Error> {
+    ) -> Result<(bool, BackupGroupDeleteStats), Error> {
         let store = self.name();
         let mut removed_all_requested = true;
+        let mut stats = BackupGroupDeleteStats::default();
         if delete_groups {
             log::info!("removing whole namespace recursively below {store}:/{ns}",);
             for ns in self.recursive_iter_backup_ns(ns.to_owned())? {
-                let removed_ns_groups = self.remove_namespace_groups(&ns?)?;
+                let (removed_ns_groups, delete_stats) = self.remove_namespace_groups(&ns?)?;
+                stats.add(&delete_stats);
                 removed_all_requested = removed_all_requested && removed_ns_groups;
             }
         } else {
@@ -574,7 +582,7 @@ impl DataStore {
             }
         }
 
-        Ok(removed_all_requested)
+        Ok((removed_all_requested, stats))
     }
 
     /// Remove a complete backup group including all snapshots.
diff --git a/src/api2/admin/datastore.rs b/src/api2/admin/datastore.rs
index b73ad0ff0..99b579f02 100644
--- a/src/api2/admin/datastore.rs
+++ b/src/api2/admin/datastore.rs
@@ -34,10 +34,10 @@ use pxar::accessor::aio::Accessor;
 use pxar::EntryKind;
 
 use pbs_api_types::{
-    print_ns_and_snapshot, print_store_and_ns, Authid, BackupContent, BackupNamespace, BackupType,
-    Counts, CryptMode, DataStoreConfig, DataStoreListItem, DataStoreStatus,
-    GarbageCollectionJobStatus, GroupListItem, JobScheduleStatus, KeepOptions, Operation,
-    PruneJobOptions, SnapshotListItem, SnapshotVerifyState, BACKUP_ARCHIVE_NAME_SCHEMA,
+    print_ns_and_snapshot, print_store_and_ns, Authid, BackupContent, BackupGroupDeleteStats,
+    BackupNamespace, BackupType, Counts, CryptMode, DataStoreConfig, DataStoreListItem,
+    DataStoreStatus, GarbageCollectionJobStatus, GroupListItem, JobScheduleStatus, KeepOptions,
+    Operation, PruneJobOptions, SnapshotListItem, SnapshotVerifyState, BACKUP_ARCHIVE_NAME_SCHEMA,
     BACKUP_ID_SCHEMA, BACKUP_NAMESPACE_SCHEMA, BACKUP_TIME_SCHEMA, BACKUP_TYPE_SCHEMA,
     DATASTORE_SCHEMA, IGNORE_VERIFIED_BACKUPS_SCHEMA, MAX_NAMESPACE_DEPTH, NS_MAX_DEPTH_SCHEMA,
     PRIV_DATASTORE_AUDIT, PRIV_DATASTORE_BACKUP, PRIV_DATASTORE_MODIFY, PRIV_DATASTORE_PRUNE,
@@ -267,8 +267,17 @@ pub fn list_groups(
                 type: pbs_api_types::BackupGroup,
                 flatten: true,
             },
+            "error-on-protected": {
+                type: bool,
+                optional: true,
+                default: true,
+                description: "Return error when group cannot be deleted because of protected snapshots",
+            }
         },
     },
+    returns: {
+        type: BackupGroupDeleteStats,
+    },
     access: {
         permission: &Permission::Anybody,
         description: "Requires on /datastore/{store}[/{namespace}] either DATASTORE_MODIFY for any \
@@ -279,9 +288,10 @@ pub fn list_groups(
 pub async fn delete_group(
     store: String,
     ns: Option<BackupNamespace>,
+    error_on_protected: bool,
     group: pbs_api_types::BackupGroup,
     rpcenv: &mut dyn RpcEnvironment,
-) -> Result<Value, Error> {
+) -> Result<BackupGroupDeleteStats, Error> {
     let auth_id: Authid = rpcenv.get_auth_id().unwrap().parse()?;
 
     tokio::task::spawn_blocking(move || {
@@ -299,10 +309,13 @@ pub async fn delete_group(
 
         let delete_stats = datastore.remove_backup_group(&ns, &group)?;
         if !delete_stats.all_removed() {
-            bail!("group only partially deleted due to protected snapshots");
+            if error_on_protected {
+                bail!("group only partially deleted due to protected snapshots");
+            } else {
+                warn!("group only partially deleted due to protected snapshots");
+            }
         }
-
-        Ok(Value::Null)
+        Ok(delete_stats)
     })
     .await?
 }
diff --git a/src/api2/admin/namespace.rs b/src/api2/admin/namespace.rs
index 889dc1a3d..e2a5ccd54 100644
--- a/src/api2/admin/namespace.rs
+++ b/src/api2/admin/namespace.rs
@@ -1,13 +1,12 @@
 use anyhow::{bail, Error};
-use serde_json::Value;
 
 use pbs_config::CachedUserInfo;
 use proxmox_router::{http_bail, ApiMethod, Permission, Router, RpcEnvironment};
 use proxmox_schema::*;
 
 use pbs_api_types::{
-    Authid, BackupNamespace, NamespaceListItem, Operation, DATASTORE_SCHEMA, NS_MAX_DEPTH_SCHEMA,
-    PROXMOX_SAFE_ID_FORMAT,
+    Authid, BackupGroupDeleteStats, BackupNamespace, NamespaceListItem, Operation,
+    DATASTORE_SCHEMA, NS_MAX_DEPTH_SCHEMA, PROXMOX_SAFE_ID_FORMAT,
 };
 
 use pbs_datastore::DataStore;
@@ -138,6 +137,12 @@ pub fn list_namespaces(
                 optional: true,
                 default: false,
             },
+            "error-on-protected": {
+                type: bool,
+                optional: true,
+                default: true,
+                description: "Return error when namespace cannot be deleted because of protected snapshots",
+            }
         },
     },
     access: {
@@ -149,24 +154,32 @@ pub fn delete_namespace(
     store: String,
     ns: BackupNamespace,
     delete_groups: bool,
+    error_on_protected: bool,
     _info: &ApiMethod,
     rpcenv: &mut dyn RpcEnvironment,
-) -> Result<Value, Error> {
+) -> Result<BackupGroupDeleteStats, Error> {
     let auth_id: Authid = rpcenv.get_auth_id().unwrap().parse()?;
 
     check_ns_modification_privs(&store, &ns, &auth_id)?;
 
     let datastore = DataStore::lookup_datastore(&store, Some(Operation::Write))?;
 
-    if !datastore.remove_namespace_recursive(&ns, delete_groups)? {
-        if delete_groups {
-            bail!("group only partially deleted due to protected snapshots");
+    let (removed_all, stats) = datastore.remove_namespace_recursive(&ns, delete_groups)?;
+    if !removed_all {
+        let err_msg = if delete_groups {
+            "group only partially deleted due to protected snapshots"
+        } else {
+            "only partially deleted due to existing groups but `delete-groups` not true"
+        };
+
+        if error_on_protected {
+            bail!(err_msg);
         } else {
-            bail!("only partially deleted due to existing groups but `delete-groups` not true ");
+            log::warn!("{err_msg}");
         }
     }
 
-    Ok(Value::Null)
+    Ok(stats)
 }
 
 pub const ROUTER: Router = Router::new()
diff --git a/src/server/pull.rs b/src/server/pull.rs
index 3117f7d2c..d7f5c42ea 100644
--- a/src/server/pull.rs
+++ b/src/server/pull.rs
@@ -645,10 +645,12 @@ fn check_and_remove_ns(params: &PullParameters, local_ns: &BackupNamespace) -> R
     check_ns_modification_privs(params.target.store.name(), local_ns, &params.owner)
         .map_err(|err| format_err!("Removing {local_ns} not allowed - {err}"))?;
 
-    params
+    let (removed_all, _delete_stats) = params
         .target
         .store
-        .remove_namespace_recursive(local_ns, true)
+        .remove_namespace_recursive(local_ns, true)?;
+
+    Ok(removed_all)
 }
 
 fn check_and_remove_vanished_ns(
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 28/29] api: version: add 'prune-delete-stats' as supported feature
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (26 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 27/29] api: datastore/namespace: return backup groups delete stats on remove Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 29/29] docs: add section for sync jobs in push direction Christian Ebner
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Expose the 'prune-delete-stats' as supported feature, in order for
the sync job in pull direction to pass the optional
`error-on-protected=false` flag to the api calls when pruning backup
snapshots, groups or namespaces.
---
changes since version 5:
- not present in previous version

 src/api2/version.rs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/api2/version.rs b/src/api2/version.rs
index a6cec5216..b7ec72c14 100644
--- a/src/api2/version.rs
+++ b/src/api2/version.rs
@@ -8,7 +8,7 @@ use proxmox_schema::api;
 
 use pbs_api_types::ApiVersionInfo;
 
-const FEATURES: &'static [&'static str] = &[];
+const FEATURES: &[&'static str] = &["prune-delete-stats"];
 
 #[api(
     returns: {
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

* [pbs-devel] [PATCH v6 proxmox-backup 29/29] docs: add section for sync jobs in push direction
  2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
                   ` (27 preceding siblings ...)
  2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 28/29] api: version: add 'prune-delete-stats' as supported feature Christian Ebner
@ 2024-10-31 12:15 ` Christian Ebner
  28 siblings, 0 replies; 30+ messages in thread
From: Christian Ebner @ 2024-10-31 12:15 UTC (permalink / raw)
  To: pbs-devel

Documents the caveats of sync jobs in push direction, explicitly
recommending setting up dedicted remotes for these sync jobs.

Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 5:
- not present in previous version

 docs/managing-remotes.rst | 36 ++++++++++++++++++++++++++++++++++++
 1 file changed, 36 insertions(+)

diff --git a/docs/managing-remotes.rst b/docs/managing-remotes.rst
index dd43ccd2b..a3951cbf0 100644
--- a/docs/managing-remotes.rst
+++ b/docs/managing-remotes.rst
@@ -227,3 +227,39 @@ the web interface or using the ``proxmox-backup-manager`` command-line tool:
 .. code-block:: console
 
     # proxmox-backup-manager sync-job update ID --rate-in 20MiB
+
+Sync Direction Push
+^^^^^^^^^^^^^^^^^^^
+
+Sync jobs can be configured for pull or push direction. Sync jobs in push
+direction are not identical in behaviour because of the limited access to the
+target datastore via the remote servers API. Most notably, pushed content will
+always be owned by the user configured in the remote configuration, being
+independent from the local user as configured in the sync job. Latter is used
+exclusively for permission check on the pushing side.
+
+.. note:: It is strongly advised to create a dedicated remote configuration for
+   each individual sync job in push direction, using a dedicated user on the
+   remote. Otherwise, sync jobs pushing to the same target might remove each
+   others snapshots and/or groups, if the remove vanished flag is set or skip
+   snapshots if the backup time is not incremental.
+   This is because the backup groups on the target are owned by the user
+   given in the remote configuration.
+
+The following permissions are required for a sync job in push direction:
+
+#. ``Remote.Audit`` on ``/remote/{remote}`` and ``Remote.DatastoreBackup`` on
+   ``/remote/{remote}/{remote-store}/{remote-ns}`` path or subnamespace.
+#. At least ``Datastore.Read`` on the local source datastore namespace
+   (``/datastore/{store}/{ns}``)
+#. ``Remote.DatastorePrune`` on ``/remote/{remote}/{remote-store}/{remote-ns}``
+   path to remove vanished snapshots and groups. Make sure to use a dedicated
+   remote for each sync job in push direction as noted above.
+#. ``Remote.DatastoreModify`` on ``/remote/{remote}/{remote-store}/{remote-ns}``
+   path to remove vanished namespaces. A remote user with limited access should
+   be used on the remote backup server instance. Consider the implications as
+   noted below.
+
+.. note:: ``Remote.DatastoreModify will allow to remove whole namespaces on the
+   remote target datastore, independent of ownership. Make sure the user as
+   configured in remote.cfg has limited permissions on the remote side.
-- 
2.39.5



_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel


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

end of thread, other threads:[~2024-10-31 12:25 UTC | newest]

Thread overview: 30+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2024-10-31 12:14 [pbs-devel] [PATCH v6 proxmox-backup 00/29] fix #3044: push datastore to remote target Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 01/29] client: backup writer: refactor backup and upload stats counters Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 02/29] client: backup writer: factor out merged chunk stream upload Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 03/29] client: backup writer: allow push uploading index and chunks Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 04/29] config: acl: refactor acl path component check for datastore Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 05/29] config: acl: allow namespace components for remote datastores Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 06/29] api types: add remote acl path method for `BackupNamespace` Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 07/29] api types: implement remote acl path method for sync job Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 08/29] api types: define remote permissions and roles for push sync Christian Ebner
2024-10-31 12:14 ` [pbs-devel] [PATCH v6 proxmox-backup 09/29] datastore: move `BackupGroupDeleteStats` to api types Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 10/29] api types: implement api type for `BackupGroupDeleteStats` Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 11/29] datastore: increment deleted group counter when removing group Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 12/29] api/api-types: refactor api endpoint version, add api types Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 13/29] fix #3044: server: implement push support for sync operations Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 14/29] api types/config: add `sync-push` config type for push sync jobs Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 15/29] api: push: implement endpoint for sync in push direction Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 16/29] api: sync: move sync job invocation to server sync module Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 17/29] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 18/29] api: config: factor out sync job owner check Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 19/29] api: sync jobs: expose optional `sync-direction` parameter Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 20/29] api: admin: avoid duplicate name for list sync jobs api method Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 21/29] bin: manager: add datastore push cli command Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 22/29] ui: group filter: allow to set namespace for local datastore Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 23/29] ui: sync edit: source group filters based on sync direction Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 24/29] ui: add view with separate grids for pull and push sync jobs Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 25/29] ui: sync job: adapt edit window to be used for pull and push Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 26/29] ui: sync view: set proxy on view instead of model Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 27/29] api: datastore/namespace: return backup groups delete stats on remove Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 28/29] api: version: add 'prune-delete-stats' as supported feature Christian Ebner
2024-10-31 12:15 ` [pbs-devel] [PATCH v6 proxmox-backup 29/29] docs: add section for sync jobs in push direction Christian Ebner

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