* [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target
@ 2024-10-17 13:26 Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 01/31] client: backup writer: refactor backup and upload stats counters Christian Ebner
` (32 more replies)
0 siblings, 33 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 for further feedback to the previous version of the
patches, especially regarding users and ACLs.
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 (31):
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: implement remote acl path method for sync job
api types: define remote permissions and roles for push sync
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: sync jobs: expose optional `sync-direction` parameter
api: admin: avoid duplicate name for list sync jobs api method
api: config: Require PRIV_DATASTORE_AUDIT to modify sync job
api: config: factor out sync job owner check
api: config: extend read access check by sync direction
api: config: extend modify access check by sync direction
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: pass sync-direction to allow removing push jobs
ui: sync view: do not use data model proxy for store
ui: sync view: set sync direction when invoking run task via api
datastore: move `BackupGroupDeleteStats` to api types
api types: implement api type for `BackupGroupDeleteStats`
api/api-types: refactor api endpoint version, add api types
datastore: increment deleted group counter when removing group
api: datastore/namespace: return backup groups delete stats on remove
server: sync job: use delete stats provided by the api
pbs-api-types/src/acl.rs | 25 +
pbs-api-types/src/datastore.rs | 64 ++
pbs-api-types/src/jobs.rs | 51 ++
pbs-api-types/src/lib.rs | 3 +
pbs-api-types/src/version.rs | 98 +++
pbs-client/src/backup_stats.rs | 130 ++++
pbs-client/src/backup_writer.rs | 242 ++++---
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 | 44 +-
src/api2/config/datastore.rs | 15 +-
src/api2/config/notifications/mod.rs | 21 +-
src/api2/config/sync.rs | 285 ++++++--
src/api2/mod.rs | 2 +
src/api2/pull.rs | 108 ---
src/api2/push.rs | 222 +++++++
src/api2/version.rs | 39 +-
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 | 948 +++++++++++++++++++++++++++
src/server/sync.rs | 128 +++-
www/Makefile | 1 +
www/config/SyncPullPushView.js | 60 ++
www/config/SyncView.js | 47 +-
www/datastore/DataStoreList.js | 2 +-
www/datastore/Panel.js | 2 +-
www/form/GroupFilter.js | 18 +-
www/window/SyncJobEdit.js | 45 +-
35 files changed, 2552 insertions(+), 448 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 01/31] client: backup writer: refactor backup and upload stats counters
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 02/31] client: backup writer: factor out merged chunk stream upload Christian Ebner
` (31 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- not present in previous version
pbs-client/src/backup_stats.rs | 130 ++++++++++++++++++++++++++++++++
pbs-client/src/backup_writer.rs | 104 ++++++++-----------------
pbs-client/src/lib.rs | 3 +
3 files changed, 165 insertions(+), 72 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..7aa618667
--- /dev/null
+++ b/pbs-client/src/backup_stats.rs
@@ -0,0 +1,130 @@
+//! 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;
+
+/// 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)),
+ }
+ }
+
+ /// Increment total chunk counter by `count`, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_total_chunks(&mut self, count: usize) -> usize {
+ self.total_chunk_count.fetch_add(count, Ordering::SeqCst)
+ }
+
+ /// Increment known chunk counter by `count`, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_known_chunks(&mut self, count: usize) -> usize {
+ self.known_chunk_count.fetch_add(count, Ordering::SeqCst)
+ }
+
+ /// Increment injected chunk counter by `count`, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_injected_chunks(&mut self, count: usize) -> usize {
+ self.injected_chunk_count.fetch_add(count, Ordering::SeqCst)
+ }
+
+ /// Increment stream length counter by given size, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_total_stream_len(&mut self, size: usize) -> usize {
+ self.total_stream_len.fetch_add(size, Ordering::SeqCst)
+ }
+
+ /// Increment reused length counter by given size, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_reused_stream_len(&mut self, size: usize) -> usize {
+ self.reused_stream_len.fetch_add(size, Ordering::SeqCst)
+ }
+
+ /// Increment compressed length counter by given size, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_compressed_stream_len(&mut self, size: u64) -> u64 {
+ self.compressed_stream_len.fetch_add(size, Ordering::SeqCst)
+ }
+
+ /// Increment stream length counter by given size, returns previous value
+ #[inline(always)]
+ pub(crate) fn inc_injected_stream_len(&mut self, size: usize) -> usize {
+ self.injected_stream_len.fetch_add(size, Ordering::SeqCst)
+ }
+
+ /// Return a Arc clone to the total stream length counter
+ #[inline(always)]
+ pub(crate) fn total_stream_len_counter(&self) -> Arc<AtomicUsize> {
+ self.total_stream_len.clone()
+ }
+
+ /// 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 d63c09b5a..5ccfcc9b3 100644
--- a/pbs-client/src/backup_writer.rs
+++ b/pbs-client/src/backup_writer.rs
@@ -1,7 +1,7 @@
use std::collections::HashSet;
use std::future::Future;
-use std::sync::atomic::{AtomicU64, 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};
@@ -22,6 +22,7 @@ use pbs_tools::crypt_config::CryptConfig;
use proxmox_human_byte::HumanByte;
+use super::backup_stats::{BackupStats, UploadCounters, UploadStats};
use super::inject_reused_chunks::{InjectChunks, InjectReusedChunks, InjectedChunksInfo};
use super::merge_known_chunks::{MergeKnownChunks, MergedChunkInfo};
@@ -39,11 +40,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 {
@@ -53,18 +49,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],
-}
-
type UploadQueueSender = mpsc::Sender<(MergedChunkInfo, Option<h2::client::ResponseFuture>)>;
type UploadResultReceiver = oneshot::Receiver<Result<(), Error>>;
@@ -188,6 +172,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)?;
@@ -205,7 +190,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(
@@ -214,6 +204,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"),
@@ -237,7 +228,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>>(
@@ -413,10 +409,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() -> (
@@ -637,21 +630,8 @@ impl BackupWriter {
compress: bool,
injections: Option<std::sync::mpsc::Receiver<InjectChunks>>,
) -> 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 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 mut counters = UploadCounters::new();
+ let counters_readonly = counters.clone();
let append_chunk_path = format!("{}_index", prefix);
let upload_chunk_path = format!("{}_chunk", prefix);
@@ -666,22 +646,21 @@ impl BackupWriter {
let index_csum_2 = index_csum.clone();
stream
- .inject_reused_chunks(injections, stream_len.clone())
+ .inject_reused_chunks(injections, counters.total_stream_len_counter())
.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);
+ counters.inc_total_chunks(count);
+ counters.inc_injected_chunks(count);
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.inc_total_stream_len(chunk.size() as usize) as u64;
+ counters.inc_reused_stream_len(chunk.size() as usize);
+ counters.inc_injected_stream_len(chunk.size() as usize);
let digest = chunk.digest();
known.push((offset, digest));
let end_offset = offset + chunk.size();
@@ -694,8 +673,8 @@ 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;
+ counters.inc_total_chunks(1);
+ let offset = counters.inc_total_stream_len(chunk_len) as u64;
let mut chunk_builder = DataChunkBuilder::new(data.as_ref()).compress(compress);
@@ -718,14 +697,14 @@ impl BackupWriter {
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);
+ counters.inc_known_chunks(1);
+ counters.inc_reused_stream_len(chunk_len);
future::ok(MergedChunkInfo::Known(vec![(offset, *digest)]))
} else {
- let compressed_stream_len2 = compressed_stream_len.clone();
+ let mut counters = counters.clone();
known_chunks.insert(*digest);
future::ready(chunk_builder.build().map(move |(chunk, digest)| {
- compressed_stream_len2.fetch_add(chunk.raw_size(), Ordering::SeqCst);
+ counters.inc_compressed_stream_len(chunk.raw_size());
MergedChunkInfo::New(ChunkInfo {
chunk,
digest,
@@ -794,29 +773,10 @@ 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();
- 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/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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 02/31] client: backup writer: factor out merged chunk stream upload
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 01/31] client: backup writer: refactor backup and upload stats counters Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 03/31] client: backup writer: allow push uploading index and chunks Christian Ebner
` (30 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- adapted to refactored upload stat counters
pbs-client/src/backup_writer.rs | 43 +++++++++++++++++++++------------
1 file changed, 28 insertions(+), 15 deletions(-)
diff --git a/pbs-client/src/backup_writer.rs b/pbs-client/src/backup_writer.rs
index 5ccfcc9b3..27d1c73b1 100644
--- a/pbs-client/src/backup_writer.rs
+++ b/pbs-client/src/backup_writer.rs
@@ -6,6 +6,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};
@@ -633,19 +634,12 @@ impl BackupWriter {
let mut counters = UploadCounters::new();
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);
-
- 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();
- stream
+ let stream = stream
.inject_reused_chunks(injections, counters.total_stream_len_counter())
.and_then(move |chunk_info| match chunk_info {
InjectedChunksInfo::Known(chunks) => {
@@ -715,7 +709,28 @@ impl BackupWriter {
}
}
})
- .merge_known_chunks()
+ .merge_known_chunks();
+
+ Self::upload_merged_chunk_stream(h2, wid, prefix, stream, index_csum_2, counters_readonly)
+ }
+
+ fn upload_merged_chunk_stream(
+ h2: H2Client,
+ wid: u64,
+ 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 (upload_queue, upload_result) =
+ Self::append_chunk_queue(h2.clone(), wid, append_chunk_path);
+
+ let start_time = std::time::Instant::now();
+
+ stream
.try_for_each(move |merged_chunk_info| {
let upload_queue = upload_queue.clone();
@@ -725,10 +740,8 @@ impl BackupWriter {
let digest_str = hex::encode(digest);
log::trace!(
- "upload new chunk {} ({} bytes, offset {})",
- digest_str,
- chunk_info.chunk_len,
- offset
+ "upload new chunk {digest_str} ({chunk_len} bytes, offset {offset})",
+ chunk_len = chunk_info.chunk_len,
);
let chunk_data = chunk_info.chunk.into_inner();
@@ -773,10 +786,10 @@ 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();
- 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 03/31] client: backup writer: allow push uploading index and chunks
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 01/31] client: backup writer: refactor backup and upload stats counters Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 02/31] client: backup writer: factor out merged chunk stream upload Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 04/31] config: acl: refactor acl path component check for datastore Christian Ebner
` (29 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- known chunks are now handled by the caller (so it can be avoided to
read them)
- adapt for new upload stat counters
pbs-client/src/backup_writer.rs | 95 +++++++++++++++++++++++++++++++++
pbs-client/src/lib.rs | 1 +
2 files changed, 96 insertions(+)
diff --git a/pbs-client/src/backup_writer.rs b/pbs-client/src/backup_writer.rs
index 27d1c73b1..573f48f54 100644
--- a/pbs-client/src/backup_writer.rs
+++ b/pbs-client/src/backup_writer.rs
@@ -259,6 +259,101 @@ 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) => {
+ counters.inc_total_chunks(1);
+ let chunk_len = chunk_info.chunk_len;
+ let offset = counters.inc_total_stream_len(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(&chunk_info.digest);
+ }
+ MergedChunkInfo::Known(ref mut known_chunk_list) => {
+ for (chunk_len, digest) in known_chunk_list {
+ counters.inc_total_chunks(1);
+ counters.inc_known_chunks(1);
+ counters.inc_reused_stream_len(*chunk_len as usize);
+ let offset = counters.inc_total_stream_len(*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,
+ 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 04/31] config: acl: refactor acl path component check for datastore
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (2 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 03/31] client: backup writer: allow push uploading index and chunks Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 05/31] config: acl: allow namespace components for remote datastores Christian Ebner
` (28 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- add optimized component len check
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 05/31] config: acl: allow namespace components for remote datastores
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (3 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 04/31] config: acl: refactor acl path component check for datastore Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 06/31] api types: implement remote acl path method for sync job Christian Ebner
` (27 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- Optimize component len check
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 06/31] api types: implement remote acl path method for sync job
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (4 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 05/31] config: acl: allow namespace components for remote datastores Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 07/31] api types: define remote permissions and roles for push sync Christian Ebner
` (26 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- not present in previous version
pbs-api-types/src/jobs.rs | 13 +++++++++++++
1 file changed, 13 insertions(+)
diff --git a/pbs-api-types/src/jobs.rs b/pbs-api-types/src/jobs.rs
index 868702bc0..3971dd49f 100644
--- a/pbs-api-types/src/jobs.rs
+++ b/pbs-api-types/src/jobs.rs
@@ -594,6 +594,19 @@ impl SyncJobConfig {
None => vec!["datastore", &self.store],
}
}
+
+ pub fn remote_acl_path(&self) -> Option<Vec<&str>> {
+ let remote = self.remote.as_ref()?;
+ let mut remote_acl_path = vec!["remote", remote, &self.remote_store];
+
+ if let Some(namespace) = self.remote_ns.as_ref() {
+ if !namespace.is_root() {
+ let comp: Vec<&str> = namespace.components().collect();
+ remote_acl_path.extend(comp);
+ }
+ }
+ Some(remote_acl_path)
+ }
}
#[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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 07/31] api types: define remote permissions and roles for push sync
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (5 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 06/31] api types: implement remote acl path method for sync job Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 08/31] fix #3044: server: implement push support for sync operations Christian Ebner
` (25 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- adapt to reworked priv check, drop Remote.DatastoreModify role
pbs-api-types/src/acl.rs | 25 +++++++++++++++++++++++++
1 file changed, 25 insertions(+)
diff --git a/pbs-api-types/src/acl.rs b/pbs-api-types/src/acl.rs
index a8ae57a9d..86560f7f6 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,21 @@ 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_READ
+ | PRIV_REMOTE_DATASTORE_MODIFY
+ | PRIV_REMOTE_DATASTORE_BACKUP;
+
+#[rustfmt::skip]
+#[allow(clippy::identity_op)]
+/// Remote.DatastorePrune can prune snapshots, groups and namespaces on the remote.
+pub const ROLE_REMOTE_DATASTORE_PRUNE: u64 = 0
+ | PRIV_REMOTE_DATASTORE_PRUNE;
+
#[rustfmt::skip]
#[allow(clippy::identity_op)]
/// Tape.Audit can audit the tape backup configuration and media content
@@ -225,6 +246,10 @@ 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,
/// 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 08/31] fix #3044: server: implement push support for sync operations
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (6 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 07/31] api types: define remote permissions and roles for push sync Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 09/31] api types/config: add `sync-push` config type for push sync jobs Christian Ebner
` (24 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- Avoid reading known chunks, only re-index based on digest
- Avoid tempfile for manifest, upload source manifest directly
- Add map_to_target helper for source to target namespace mapping
- Only try creating non pre-existing namespace components on target
- Drop `job_user`, privs are now all checked for `local_user`
- Removing vanished namespaces now requires PRIV_REMOTE_DATASTORE_MODIFY
src/server/mod.rs | 1 +
src/server/push.rs | 910 +++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 911 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..bf6045214
--- /dev/null
+++ b/src/server/push.rs
@@ -0,0 +1,910 @@
+//! 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;
+
+use pbs_api_types::{
+ print_store_and_ns, Authid, BackupDir, BackupGroup, 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 {
+ // Name of the remote as found in remote.cfg
+ remote: String,
+ // Target repository on remote
+ repo: BackupRepository,
+ // Target namespace on remote
+ ns: BackupNamespace,
+ // Http client to connect to remote
+ client: HttpClient,
+}
+
+/// 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) 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 target = PushTarget {
+ remote: remote_id.to_string(),
+ repo,
+ ns: remote_ns,
+ client,
+ };
+ 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", ¶ms.target.remote, 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(¶ms.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<(), 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 args = serde_json::json!({
+ "ns": target_ns.name(),
+ "delete-groups": true,
+ });
+
+ params.target.client.delete(&api_path, Some(args)).await?;
+
+ Ok(())
+}
+
+// Fetch the list of groups found on target in given namespace
+async fn fetch_target_groups(
+ params: &PushParameters,
+ namespace: &BackupNamespace,
+) -> Result<Vec<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 groups: Vec<BackupGroup> = groups.into_iter().map(|group| group.backup).collect();
+
+ groups.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(groups)
+}
+
+// Remove the provided backup group in given namespace from the target
+async fn remove_target_group(
+ params: &PushParameters,
+ namespace: &BackupNamespace,
+ backup_group: &BackupGroup,
+) -> 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}/groups",
+ store = params.target.repo.store(),
+ );
+
+ let mut args = serde_json::json!({
+ "backup-id": backup_group.id,
+ "backup-type": backup_group.ty,
+ });
+ if !namespace.is_root() {
+ let target_ns = params.map_to_target(namespace)?;
+ args["ns"] = serde_json::to_value(target_ns.name())?;
+ }
+
+ params.target.client.delete(&api_path, Some(args)).await?;
+
+ Ok(())
+}
+
+// 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(¤t) {
+ 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(), ¤t);
+ 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;
+
+ // 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(¶ms.source.get_ns(), ¶ms.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(¶ms).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(¶ms, &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, ¶ms).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;
+ }
+ if let Err(err) = remove_target_namespace(¶ms, &target_namespace).await {
+ info!("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, ¶ms.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(¶ms.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();
+
+ for (done, group) in list.into_iter().enumerate() {
+ progress.done_groups = done as u64;
+ progress.done_snapshots = 0;
+ progress.group_snapshots = 0;
+ synced_groups.insert(group.clone());
+
+ match push_group(params, namespace, &group, &mut progress).await {
+ Ok(sync_stats) => stats.add(sync_stats),
+ Err(err) => {
+ info!("sync group '{group}' failed - {err}");
+ errors = true;
+ }
+ }
+ }
+
+ if params.remove_vanished {
+ let target_groups = fetch_target_groups(params, namespace).await?;
+ for target_group in target_groups {
+ if synced_groups.contains(&target_group) {
+ continue;
+ }
+ if !target_group.apply_filters(¶ms.group_filter) {
+ continue;
+ }
+
+ info!("delete vanished group '{target_group}'");
+
+ let count_before = match fetch_target_groups(params, namespace).await {
+ Ok(snapshots) => snapshots.len(),
+ Err(_err) => 0, // ignore errors
+ };
+
+ if let Err(err) = remove_target_group(params, namespace, &target_group).await {
+ info!("{err}");
+ errors = true;
+ continue;
+ }
+
+ let mut count_after = match fetch_target_groups(params, namespace).await {
+ Ok(snapshots) => snapshots.len(),
+ Err(_err) => 0, // ignore errors
+ };
+
+ let deleted_groups = if count_after > 0 {
+ info!("kept some protected snapshots of group '{target_group}'");
+ 0
+ } else {
+ 1
+ };
+
+ if count_after > count_before {
+ count_after = count_before;
+ }
+
+ stats.add(SyncStats::from(RemovedVanishedStats {
+ snapshots: count_before - count_after,
+ groups: deleted_groups,
+ namespaces: 0,
+ }));
+ }
+ }
+
+ 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(
+ ¶ms.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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 09/31] api types/config: add `sync-push` config type for push sync jobs
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (7 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 08/31] fix #3044: server: implement push support for sync operations Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 10/31] api: push: implement endpoint for sync in push direction Christian Ebner
` (23 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- Rework and reduce code by using `api` macro
- use `*_config_type_str` helpers
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 3971dd49f..d0cf2c377 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 10/31] api: push: implement endpoint for sync in push direction
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (8 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 09/31] api types/config: add `sync-push` config type for push sync jobs Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 11/31] api: sync: move sync job invocation to server sync module Christian Ebner
` (22 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- include namespace in remote acl path
- Allow to run with PRIV_REMOTE_DATASTORE_BACKUP
src/api2/mod.rs | 2 +
src/api2/push.rs | 222 +++++++++++++++++++++++++++++++++++++++++++++++
2 files changed, 224 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..ead2bf2e0
--- /dev/null
+++ b/src/api2/push.rs
@@i -0,0 +1,222 @@
+use anyhow::{format_err, Context, Error};
+use futures::{future::FutureExt, select};
+use tracing::info;
+
+use pbs_api_types::{
+ Authid, BackupNamespace, GroupFilter, RateLimitConfig, SyncJobConfig, 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: Option<&str>,
+ remote: &str,
+ remote_store: &str,
+ remote_ns: Option<&str>,
+ delete: bool,
+) -> Result<(), Error> {
+ let user_info = CachedUserInfo::new()?;
+
+ let target_acl_path = match remote_ns {
+ Some(ns) => vec!["remote", remote, remote_store, ns],
+ 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,
+ )?;
+ }
+
+ let local_acl_path = match namespace {
+ Some(ns) => vec!["datastore", store, ns],
+ None => vec!["datastore", store],
+ };
+
+ // Check user is allowed to read source datastore
+ user_info.check_privs(auth_id, &local_acl_path, PRIV_DATASTORE_READ, false)?;
+
+ Ok(())
+}
+
+impl TryFrom<&SyncJobConfig> for PushParameters {
+ type Error = Error;
+
+ fn try_from(sync_job: &SyncJobConfig) -> Result<Self, Self::Error> {
+ 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,
+ )
+ }
+}
+
+#[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();
+ let source_namespace = if !ns.is_root() {
+ Some(ns.to_string())
+ } else {
+ None
+ };
+
+ let remote_ns = remote_ns.unwrap_or_default();
+ let remote_namespace = if !remote_ns.is_root() {
+ Some(ns.to_string())
+ } else {
+ None
+ };
+
+ check_push_privs(
+ &auth_id,
+ &store,
+ source_namespace.as_deref(),
+ &remote,
+ &remote_store,
+ remote_namespace.as_deref(),
+ delete,
+ )?;
+
+ let push_params = PushParameters::new(
+ &store,
+ ns,
+ &remote,
+ &remote_store,
+ remote_ns,
+ auth_id.clone(),
+ remove_vanished,
+ max_depth,
+ group_filter,
+ limit,
+ transfer_last,
+ )?;
+
+ 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 11/31] api: sync: move sync job invocation to server sync module
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (9 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 10/31] api: push: implement endpoint for sync in push direction Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 12/31] api: sync jobs: expose optional `sync-direction` parameter Christian Ebner
` (21 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- Drop not needed `job_user`, priv checks handled by `local_user`
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 | 128 +++++++++++++++++++++++++++++++-
5 files changed, 152 insertions(+), 119 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..d0df7df6e 100644
--- a/src/server/sync.rs
+++ b/src/server/sync.rs
@@ -7,15 +7,18 @@ use std::sync::{Arc, Mutex};
use std::time::Duration;
use anyhow::{bail, format_err, 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,123 @@ 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::try_from(&sync_job)?;
+ 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 12/31] api: sync jobs: expose optional `sync-direction` parameter
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (10 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 11/31] api: sync: move sync job invocation to server sync module Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 13/31] api: admin: avoid duplicate name for list sync jobs api method Christian Ebner
` (20 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 UTC (permalink / raw)
To: pbs-devel
Exposes and switch the config type for sync job operations based
on the `sync-direction` parameter. If not set, the default config
type is `sync` and the default sync direction is `pull` for full
backwards compatibility.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- Use `SyncDirection` api type directly
- Deduplicate code by iterating over sync direction enum variants
src/api2/admin/sync.rs | 27 +++++++++------
src/api2/config/datastore.rs | 11 +++---
src/api2/config/notifications/mod.rs | 19 ++++++-----
src/api2/config/sync.rs | 51 +++++++++++++++++++++++-----
src/bin/proxmox-backup-proxy.rs | 11 ++++--
5 files changed, 86 insertions(+), 33 deletions(-)
diff --git a/src/api2/admin/sync.rs b/src/api2/admin/sync.rs
index be324564c..c6b309859 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -29,6 +29,10 @@ use crate::{
schema: DATASTORE_SCHEMA,
optional: true,
},
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
},
},
returns: {
@@ -44,6 +48,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 +57,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 {
@@ -88,7 +94,11 @@ pub fn list_sync_jobs(
properties: {
id: {
schema: JOB_ID_SCHEMA,
- }
+ },
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
}
},
access: {
@@ -99,6 +109,7 @@ pub fn list_sync_jobs(
/// Runs the sync jobs manually.
pub fn run_sync_job(
id: String,
+ sync_direction: Option<SyncDirection>,
_info: &ApiMethod,
rpcenv: &mut dyn RpcEnvironment,
) -> Result<String, Error> {
@@ -106,7 +117,8 @@ pub fn run_sync_job(
let user_info = CachedUserInfo::new()?;
let (config, _digest) = sync::config()?;
- let sync_job: SyncJobConfig = config.lookup("sync", &id)?;
+ let sync_direction = sync_direction.unwrap_or_default();
+ let sync_job: SyncJobConfig = config.lookup(sync_direction.as_config_type_str(), &id)?;
if !check_sync_job_modify_access(&user_info, &auth_id, &sync_job) {
bail!("permission check failed");
@@ -116,14 +128,7 @@ 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,
- 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..e2d876555 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, Some(direction), 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 6fdc69a9e..b78267025 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};
@@ -77,7 +78,12 @@ pub fn check_sync_job_modify_access(
#[api(
input: {
- properties: {},
+ properties: {
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
+ },
},
returns: {
description: "List configured jobs.",
@@ -92,6 +98,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,7 +106,8 @@ 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();
@@ -118,6 +126,10 @@ pub fn list_sync_jobs(
type: SyncJobConfig,
flatten: true,
},
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
},
},
access: {
@@ -128,6 +140,7 @@ 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()?;
@@ -158,7 +171,8 @@ pub fn create_sync_job(
param_bail!("id", "job '{}' already exists.", config.id);
}
- section_config.set_data(&config.id, "sync", &config)?;
+ let sync_direction = sync_direction.unwrap_or_default();
+ section_config.set_data(&config.id, sync_direction.as_config_type_str(), &config)?;
sync::save_config(§ion_config)?;
@@ -173,6 +187,10 @@ pub fn create_sync_job(
id: {
schema: JOB_ID_SCHEMA,
},
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
},
},
returns: { type: SyncJobConfig },
@@ -182,13 +200,18 @@ pub fn create_sync_job(
},
)]
/// Read a sync job configuration.
-pub fn read_sync_job(id: String, rpcenv: &mut dyn RpcEnvironment) -> Result<SyncJobConfig, Error> {
+pub fn read_sync_job(
+ id: String,
+ sync_direction: Option<SyncDirection>,
+ rpcenv: &mut dyn RpcEnvironment,
+) -> Result<SyncJobConfig, Error> {
let auth_id: Authid = rpcenv.get_auth_id().unwrap().parse()?;
let user_info = CachedUserInfo::new()?;
let (config, digest) = sync::config()?;
- let sync_job = config.lookup("sync", &id)?;
+ let sync_direction = sync_direction.unwrap_or_default();
+ let sync_job = config.lookup(sync_direction.as_config_type_str(), &id)?;
if !check_sync_job_read_access(&user_info, &auth_id, &sync_job) {
bail!("permission check failed");
}
@@ -252,6 +275,10 @@ pub enum DeletableProperty {
type: DeletableProperty,
}
},
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
digest: {
optional: true,
schema: PROXMOX_CONFIG_DIGEST_SCHEMA,
@@ -269,6 +296,7 @@ pub fn update_sync_job(
id: String,
update: SyncJobConfigUpdater,
delete: Option<Vec<DeletableProperty>>,
+ sync_direction: Option<SyncDirection>,
digest: Option<String>,
rpcenv: &mut dyn RpcEnvironment,
) -> Result<(), Error> {
@@ -284,7 +312,8 @@ pub fn update_sync_job(
crate::tools::detect_modified_configuration_file(&digest, &expected_digest)?;
}
- let mut data: SyncJobConfig = config.lookup("sync", &id)?;
+ let sync_direction = sync_direction.unwrap_or_default();
+ let mut data: SyncJobConfig = config.lookup(sync_direction.as_config_type_str(), &id)?;
if let Some(delete) = delete {
for delete_prop in delete {
@@ -409,7 +438,7 @@ pub fn update_sync_job(
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)?;
@@ -427,6 +456,10 @@ pub fn update_sync_job(
id: {
schema: JOB_ID_SCHEMA,
},
+ "sync-direction": {
+ type: SyncDirection,
+ optional: true,
+ },
digest: {
optional: true,
schema: PROXMOX_CONFIG_DIGEST_SCHEMA,
@@ -441,6 +474,7 @@ pub fn update_sync_job(
/// Remove a sync job configuration
pub fn delete_sync_job(
id: String,
+ sync_direction: Option<SyncDirection>,
digest: Option<String>,
rpcenv: &mut dyn RpcEnvironment,
) -> Result<(), Error> {
@@ -456,7 +490,8 @@ pub fn delete_sync_job(
crate::tools::detect_modified_configuration_file(&digest, &expected_digest)?;
}
- match config.lookup("sync", &id) {
+ let sync_direction = sync_direction.unwrap_or_default();
+ match config.lookup(sync_direction.as_config_type_str(), &id) {
Ok(job) => {
if !check_sync_job_modify_access(&user_info, &auth_id, &job) {
bail!("permission check failed");
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 13/31] api: admin: avoid duplicate name for list sync jobs api method
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (11 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 12/31] api: sync jobs: expose optional `sync-direction` parameter Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 14/31] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job Christian Ebner
` (19 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- not present in previous version
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 c6b309859..1afe9fec2 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -45,8 +45,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,
@@ -141,5 +141,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 e2d876555..eb5a84ca0 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, Some(direction), 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 14/31] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (12 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 13/31] api: admin: avoid duplicate name for list sync jobs api method Christian Ebner
@ 2024-10-17 13:26 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 15/31] api: config: factor out sync job owner check Christian Ebner
` (18 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:26 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 3:
- not present in previous version
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 b78267025..ad6ba0c85 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -45,7 +45,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;
}
@@ -537,7 +537,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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 15/31] api: config: factor out sync job owner check
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (13 preceding siblings ...)
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 14/31] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 16/31] api: config: extend read access check by sync direction Christian Ebner
` (17 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- not present in previous version
src/api2/config/sync.rs | 22 ++++++++++++----------
1 file changed, 12 insertions(+), 10 deletions(-)
diff --git a/src/api2/config/sync.rs b/src/api2/config/sync.rs
index ad6ba0c85..aed46aeb0 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -35,6 +35,17 @@ pub fn check_sync_job_read_access(
}
}
+fn is_correct_owner(auth_id: &Authid, job: &SyncJobConfig) -> bool {
+ 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(),
+ }
+}
+
/// checks whether user can run the corresponding pull job
///
/// namespace creation/deletion ACL and backup group ownership checks happen in the pull code directly.
@@ -55,17 +66,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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 16/31] api: config: extend read access check by sync direction
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (14 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 15/31] api: config: factor out sync job owner check Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 17/31] api: config: extend modify " Christian Ebner
` (16 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 UTC (permalink / raw)
To: pbs-devel
Add the sync direction as additional parameter for the priv helper to
check for the required permissions in pull and push direction.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- not present in previous version
src/api2/admin/sync.rs | 4 ++-
src/api2/config/sync.rs | 80 +++++++++++++++++++++++++++++++++--------
2 files changed, 68 insertions(+), 16 deletions(-)
diff --git a/src/api2/admin/sync.rs b/src/api2/admin/sync.rs
index 1afe9fec2..7a4e38942 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -68,7 +68,9 @@ pub fn list_config_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();
diff --git a/src/api2/config/sync.rs b/src/api2/config/sync.rs
index aed46aeb0..e0d96afe5 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -20,18 +20,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
+ }
+ }
}
}
@@ -115,7 +132,9 @@ pub fn list_sync_jobs(
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)
}
@@ -214,7 +233,7 @@ pub fn read_sync_job(
let sync_direction = sync_direction.unwrap_or_default();
let sync_job = config.lookup(sync_direction.as_config_type_str(), &id)?;
- if !check_sync_job_read_access(&user_info, &auth_id, &sync_job) {
+ if !check_sync_job_read_access(&user_info, &auth_id, &sync_job, sync_direction) {
bail!("permission check failed");
}
@@ -573,14 +592,20 @@ 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_read_access(
+ &user_info,
+ root_auth_id,
+ &job,
+ SyncDirection::Pull,
+ ));
assert!(check_sync_job_modify_access(&user_info, root_auth_id, &job));
// 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,
@@ -589,16 +614,31 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
));
// 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();
@@ -624,7 +664,12 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
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,
@@ -645,7 +690,12 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
));
// 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,
--
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 17/31] api: config: extend modify access check by sync direction
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (15 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 16/31] api: config: extend read access check by sync direction Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 18/31] bin: manager: add datastore push cli command Christian Ebner
` (15 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 UTC (permalink / raw)
To: pbs-devel
Add the sync direction as additional parameter for the priv helper to
check for the required permissions in pull and push direction.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- not present in previous version
src/api2/admin/sync.rs | 4 +-
src/api2/config/sync.rs | 136 +++++++++++++++++++++++++++++-----------
2 files changed, 100 insertions(+), 40 deletions(-)
diff --git a/src/api2/admin/sync.rs b/src/api2/admin/sync.rs
index 7a4e38942..f2c0f0e85 100644
--- a/src/api2/admin/sync.rs
+++ b/src/api2/admin/sync.rs
@@ -122,8 +122,8 @@ pub fn run_sync_job(
let sync_direction = sync_direction.unwrap_or_default();
let sync_job: SyncJobConfig = config.lookup(sync_direction.as_config_type_str(), &id)?;
- 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)?;
diff --git a/src/api2/config/sync.rs b/src/api2/config/sync.rs
index e0d96afe5..cffcf429f 100644
--- a/src/api2/config/sync.rs
+++ b/src/api2/config/sync.rs
@@ -9,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;
@@ -63,36 +64,77 @@ 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;
+ }
+ }
+
+ // 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 {
- if ns_anchor_privs & PRIV_DATASTORE_PRUNE == 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;
+ };
- // same permission as changing ownership after syncing
- if !is_correct_owner(auth_id, job) && ns_anchor_privs & PRIV_DATASTORE_MODIFY == 0 {
- return false;
- }
+ // check user is allowed to create backups on remote datastore
+ if target_privs & PRIV_REMOTE_DATASTORE_BACKUP == 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;
+ 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;
+ }
+ source_privs & PRIV_DATASTORE_READ != 0
+ }
}
- true
}
#[api(
@@ -166,10 +208,11 @@ pub fn create_sync_job(
) -> 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");
}
@@ -192,7 +235,6 @@ pub fn create_sync_job(
param_bail!("id", "job '{}' already exists.", config.id);
}
- let sync_direction = sync_direction.unwrap_or_default();
section_config.set_data(&config.id, sync_direction.as_config_type_str(), &config)?;
sync::save_config(§ion_config)?;
@@ -455,7 +497,7 @@ 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");
}
@@ -514,7 +556,7 @@ pub fn delete_sync_job(
let sync_direction = sync_direction.unwrap_or_default();
match config.lookup(sync_direction.as_config_type_str(), &id) {
Ok(job) => {
- if !check_sync_job_modify_access(&user_info, &auth_id, &job) {
+ if !check_sync_job_modify_access(&user_info, &auth_id, &job, sync_direction) {
bail!("permission check failed");
}
config.sections.remove(&id);
@@ -598,7 +640,12 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
&job,
SyncDirection::Pull,
));
- assert!(check_sync_job_modify_access(&user_info, root_auth_id, &job));
+ 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(
@@ -610,7 +657,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
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
@@ -645,7 +693,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
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
@@ -657,7 +706,8 @@ 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
@@ -674,19 +724,22 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
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
@@ -699,7 +752,8 @@ acl:1:/remote/remote1/remotestore1:write@pbs:RemoteSyncOperator
assert!(check_sync_job_modify_access(
&user_info,
&write_auth_id,
- &job
+ &job,
+ SyncDirection::Pull,
));
// but can't modify/run with deletion
@@ -707,7 +761,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
@@ -715,7 +770,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
@@ -723,7 +779,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'
@@ -731,7 +788,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
@@ -740,13 +798,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(())
--
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 18/31] bin: manager: add datastore push cli command
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (16 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 17/31] api: config: extend modify " Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 19/31] ui: group filter: allow to set namespace for local datastore Christian Ebner
` (14 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- 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(¶m);
+
+ 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(¶m);
-
- 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 19/31] ui: group filter: allow to set namespace for local datastore
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (17 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 18/31] bin: manager: add datastore push cli command Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 20/31] ui: sync edit: source group filters based on sync direction Christian Ebner
` (13 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- no changes
www/form/GroupFilter.js | 18 +++++++++++++++++-
1 file changed, 17 insertions(+), 1 deletion(-)
diff --git a/www/form/GroupFilter.js b/www/form/GroupFilter.js
index c9c2d913e..7275b00ed 100644
--- a/www/form/GroupFilter.js
+++ b/www/form/GroupFilter.js
@@ -258,7 +258,11 @@ Ext.define('PBS.form.GroupFilter', {
return;
}
if (me.namespace) {
- url += `?namespace=${me.namespace}`;
+ if (me.remote) {
+ url += `?namespace=${me.namespace}`;
+ } else {
+ url += `?ns=${me.namespace}`;
+ }
}
me.setDsStoreUrl(url);
me.dsStore.load({
@@ -279,6 +283,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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 20/31] ui: sync edit: source group filters based on sync direction
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (18 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 19/31] ui: group filter: allow to set namespace for local datastore Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 21/31] ui: add view with separate grids for pull and push sync jobs Christian Ebner
` (12 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 21/31] ui: add view with separate grids for pull and push sync jobs
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (19 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 20/31] ui: sync edit: source group filters based on sync direction Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 22/31] ui: sync job: adapt edit window to be used for pull and push Christian Ebner
` (11 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- no changes
www/Makefile | 1 +
www/config/SyncPullPushView.js | 60 ++++++++++++++++++++++++++++++++++
www/config/SyncView.js | 11 ++++++-
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..1588207c9
--- /dev/null
+++ b/www/config/SyncPullPushView.js
@@ -0,0 +1,60 @@
+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,
+ },
+ defaults: {
+ collapsible: false,
+ margin: '7 10 3 10',
+ },
+ 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: 160, // 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..68a147615 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();
@@ -55,6 +56,7 @@ Ext.define('PBS.config.SyncJobView', {
Ext.create('PBS.window.SyncJobEdit', {
datastore: view.datastore,
+ syncDirection: view.syncDirection,
id: selection[0].data.id,
listeners: {
destroy: function() {
@@ -117,6 +119,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 +308,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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 22/31] ui: sync job: adapt edit window to be used for pull and push
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (20 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 21/31] ui: add view with separate grids for pull and push sync jobs Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 23/31] ui: sync: pass sync-direction to allow removing push jobs Christian Ebner
` (10 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 23/31] ui: sync: pass sync-direction to allow removing push jobs
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (21 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 22/31] ui: sync job: adapt edit window to be used for pull and push Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 24/31] ui: sync view: do not use data model proxy for store Christian Ebner
` (9 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 UTC (permalink / raw)
To: pbs-devel
Without the `sync-direction` parameter set, the job will not be
found in the config, because the `sync` config type is used instead
of the correct `sync-push` for sync jobs in push direction.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- no changes
www/config/SyncView.js | 22 +++++++++++++++++++++-
1 file changed, 21 insertions(+), 1 deletion(-)
diff --git a/www/config/SyncView.js b/www/config/SyncView.js
index 68a147615..981b9b251 100644
--- a/www/config/SyncView.js
+++ b/www/config/SyncView.js
@@ -104,6 +104,26 @@ Ext.define('PBS.config.SyncJobView', {
});
},
+ removeSyncJob: function(btn, event, rec) {
+ let me = this;
+ let view = me.getView();
+ let params = {};
+ if (view.syncDirection !== undefined) {
+ params["sync-direction"] = view.syncDirection;
+ }
+ Proxmox.Utils.API2Request({
+ url: '/config/sync/' + rec.getId(),
+ method: 'DELETE',
+ params: params,
+ callback: function(options, success, response) {
+ Ext.callback(me.callback, me.scope, [options, success, response, 0, me]);
+ },
+ failure: function(response, opt) {
+ Ext.Msg.alert(gettext('Error'), response.htmlStatus);
+ },
+ });
+ },
+
render_optional_owner: function(value, metadata, record) {
if (!value) return '-';
return Ext.String.htmlEncode(value);
@@ -161,7 +181,7 @@ Ext.define('PBS.config.SyncJobView', {
},
{
xtype: 'proxmoxStdRemoveButton',
- baseurl: '/config/sync/',
+ handler: 'removeSyncJob',
confirmMsg: gettext('Remove entry?'),
callback: 'reload',
},
--
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 24/31] ui: sync view: do not use data model proxy for store
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (22 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 23/31] ui: sync: pass sync-direction to allow removing push jobs Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 25/31] ui: sync view: set sync direction when invoking run task via api Christian Ebner
` (8 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 additional `sync-direction` parameter
to the proxy's api calls.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- no changes
www/config/SyncView.js | 9 ++++-----
1 file changed, 4 insertions(+), 5 deletions(-)
diff --git a/www/config/SyncView.js b/www/config/SyncView.js
index 981b9b251..39e464fc5 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', {
@@ -160,9 +156,12 @@ Ext.define('PBS.config.SyncJobView', {
sorters: 'id',
rstore: {
type: 'update',
- 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 25/31] ui: sync view: set sync direction when invoking run task via api
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (23 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 24/31] ui: sync view: do not use data model proxy for store Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 26/31] datastore: move `BackupGroupDeleteStats` to api types Christian Ebner
` (7 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 UTC (permalink / raw)
To: pbs-devel
Set the correct sync direction for the task to be executed.
Otherwise the task whit the correct id cannot be found as the config
lookup requires the correct config type to be set, which is done
based on the sync direction.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- no changes
www/config/SyncView.js | 5 +++++
1 file changed, 5 insertions(+)
diff --git a/www/config/SyncView.js b/www/config/SyncView.js
index 39e464fc5..bf0c7e8c0 100644
--- a/www/config/SyncView.js
+++ b/www/config/SyncView.js
@@ -83,9 +83,14 @@ Ext.define('PBS.config.SyncJobView', {
if (selection.length < 1) return;
let id = selection[0].data.id;
+ let params = {};
+ if (view.syncDirection !== undefined) {
+ params["sync-direction"] = view.syncDirection;
+ }
Proxmox.Utils.API2Request({
method: 'POST',
url: `/admin/sync/${id}/run`,
+ params: params,
success: function(response, opt) {
Ext.create('Proxmox.window.TaskViewer', {
upid: response.result.data,
--
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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 26/31] datastore: move `BackupGroupDeleteStats` to api types
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (24 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 25/31] ui: sync view: set sync direction when invoking run task via api Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 27/31] api types: implement api type for `BackupGroupDeleteStats` Christian Ebner
` (6 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- 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 31767417a..c148d5dca 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -1569,3 +1569,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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 27/31] api types: implement api type for `BackupGroupDeleteStats`
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (25 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 26/31] datastore: move `BackupGroupDeleteStats` to api types Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 28/31] api/api-types: refactor api endpoint version, add api types Christian Ebner
` (5 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- 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 c148d5dca..a32a326be 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -1570,8 +1570,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
@@ -1583,6 +1603,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
}
@@ -1591,6 +1615,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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 28/31] api/api-types: refactor api endpoint version, add api types
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (26 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 27/31] api types: implement api type for `BackupGroupDeleteStats` Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 29/31] datastore: increment deleted group counter when removing group Christian Ebner
` (4 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 UTC (permalink / raw)
To: pbs-devel
Add a dedicated api type for the `version` api endpoint and the type
to allow for version comparisons. Will be used for version
comparisons in order to detect older hosts with slightly different
api parameters.
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 3:
- not present in previous version
pbs-api-types/src/lib.rs | 3 ++
pbs-api-types/src/version.rs | 98 ++++++++++++++++++++++++++++++++++++
src/api2/version.rs | 39 +++++++-------
3 files changed, 123 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 635292a54..e0abfd963 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..9811b2c21
--- /dev/null
+++ b/pbs-api-types/src/version.rs
@@ -0,0 +1,98 @@
+//! Defines the types for the api version info endpoint
+use std::cmp::Ordering;
+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,
+ }
+ }
+)]
+#[derive(serde::Deserialize, serde::Serialize)]
+pub struct ApiVersionInfo {
+ pub version: String,
+ pub release: String,
+ pub repoid: String,
+}
+
+pub type ApiVersionMajor = u64;
+pub type ApiVersionMinor = u64;
+pub type ApiVersionRelease = u64;
+
+pub struct ApiVersion {
+ major: ApiVersionMajor,
+ minor: ApiVersionMinor,
+ release: ApiVersionRelease,
+}
+
+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,
+ })
+ }
+}
+
+impl ApiVersion {
+ pub fn new(major: ApiVersionMajor, minor: ApiVersionMinor, release: ApiVersionRelease) -> Self {
+ Self {
+ major,
+ minor,
+ release,
+ }
+ }
+
+ pub fn is_min_required(&self, version: ApiVersion) -> bool {
+ match (
+ version.major.cmp(&self.major),
+ version.minor.cmp(&self.minor),
+ version.release.cmp(&self.release),
+ ) {
+ (Ordering::Less, _, _) => true,
+ (Ordering::Greater, _, _) => false,
+ (Ordering::Equal, Ordering::Less, _) => true,
+ (Ordering::Equal, Ordering::Greater, _) => false,
+ (Ordering::Equal, Ordering::Equal, Ordering::Less) => true,
+ (Ordering::Equal, Ordering::Equal, Ordering::Equal) => true,
+ (Ordering::Equal, Ordering::Equal, Ordering::Greater) => false,
+ }
+ }
+}
diff --git a/src/api2/version.rs b/src/api2/version.rs
index 0e91688b5..4d104f2d6 100644
--- a/src/api2/version.rs
+++ b/src/api2/version.rs
@@ -1,27 +1,32 @@
//! 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;
+
+#[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(),
+ })
}
-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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 29/31] datastore: increment deleted group counter when removing group
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (27 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 28/31] api/api-types: refactor api endpoint version, add api types Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 30/31] api: datastore/namespace: return backup groups delete stats on remove Christian Ebner
` (3 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 3:
- 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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 30/31] api: datastore/namespace: return backup groups delete stats on remove
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (28 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 29/31] datastore: increment deleted group counter when removing group Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 31/31] server: sync job: use delete stats provided by the api Christian Ebner
` (2 subsequent siblings)
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 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 `ignore-protected` flag to the api endpoints, allowing
to return without error when set.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- make new endpoint failure behaviour opt-in
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 7660dd7f6..81f81b2b3 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,
},
+ "ignore-protected": {
+ type: bool,
+ optional: true,
+ default: false,
+ description: "No 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>,
+ ignore_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 ignore_protected {
+ warn!("group only partially deleted due to protected snapshots");
+ } else {
+ bail!("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..8f99eb401 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,
},
+ "ignore-protected": {
+ type: bool,
+ optional: true,
+ default: false,
+ description: "No 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,
+ ignore_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 ignore_protected {
+ log::warn!("{err_msg}");
} else {
- bail!("only partially deleted due to existing groups but `delete-groups` not true ");
+ bail!(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, ¶ms.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] 34+ messages in thread
* [pbs-devel] [PATCH v4 proxmox 31/31] server: sync job: use delete stats provided by the api
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (29 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 30/31] api: datastore/namespace: return backup groups delete stats on remove Christian Ebner
@ 2024-10-17 13:27 ` Christian Ebner
2024-10-18 6:55 ` [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
2024-10-18 8:44 ` Christian Ebner
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-17 13:27 UTC (permalink / raw)
To: pbs-devel
Use the API exposed additional delete statistics to generate the
task log output for sync jobs in push direction instead of fetching the
contents before and after deleting.
Detect older api versions of the target server for fallback.
Signed-off-by: Christian Ebner <c.ebner@proxmox.com>
---
changes since version 3:
- fetch target api version to remain backwards compatible
src/server/push.rs | 126 +++++++++++++++++++++++++++++----------------
1 file changed, 82 insertions(+), 44 deletions(-)
diff --git a/src/server/push.rs b/src/server/push.rs
index bf6045214..a915b6964 100644
--- a/src/server/push.rs
+++ b/src/server/push.rs
@@ -8,11 +8,12 @@ use anyhow::{bail, format_err, Error};
use futures::stream::{self, StreamExt, TryStreamExt};
use tokio::sync::mpsc;
use tokio_stream::wrappers::ReceiverStream;
-use tracing::info;
+use tracing::{info, warn};
use pbs_api_types::{
- print_store_and_ns, Authid, BackupDir, BackupGroup, BackupNamespace, CryptMode, GroupFilter,
- GroupListItem, NamespaceListItem, Operation, RateLimitConfig, Remote, SnapshotListItem,
+ 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};
@@ -41,6 +42,8 @@ pub(crate) struct PushTarget {
ns: BackupNamespace,
// Http client to connect to remote
client: HttpClient,
+ // Api version reported by the target
+ api_version: Option<ApiVersion>,
}
/// Parameters for a push operation
@@ -106,6 +109,7 @@ impl PushParameters {
repo,
ns: remote_ns,
client,
+ api_version: None,
};
let group_filter = group_filter.unwrap_or_default();
@@ -124,6 +128,17 @@ impl PushParameters {
fn map_to_target(&self, namespace: &BackupNamespace) -> Result<BackupNamespace, Error> {
namespace.map_prefix(&self.source.ns, &self.target.ns)
}
+
+ // Fetch and set the api version info for the target
+ pub(crate) async fn fetch_target_api_version_info(&mut self) -> Result<(), Error> {
+ let api_path = "api2/json/version";
+ let mut result = self.target.client.get(api_path, None).await?;
+ let data = result["data"].take();
+ let version_info: ApiVersionInfo = serde_json::from_value(data)?;
+ self.target.api_version = Some(ApiVersion::try_from(version_info)?);
+
+ Ok(())
+ }
}
// Check if the job user given in the push parameters has the provided privs on the remote
@@ -167,7 +182,7 @@ async fn fetch_target_namespaces(params: &PushParameters) -> Result<Vec<BackupNa
async fn remove_target_namespace(
params: &PushParameters,
namespace: &BackupNamespace,
-) -> Result<(), Error> {
+) -> Result<BackupGroupDeleteStats, Error> {
if namespace.is_root() {
bail!("cannot remove root namespace from target");
}
@@ -181,14 +196,23 @@ async fn remove_target_namespace(
);
let target_ns = params.map_to_target(namespace)?;
- let args = serde_json::json!({
+ let mut args = serde_json::json!({
"ns": target_ns.name(),
"delete-groups": true,
});
- params.target.client.delete(&api_path, Some(args)).await?;
+ if let Some(version) = ¶ms.target.api_version {
+ if version.is_min_required(ApiVersion::new(3, 2, 8)) {
+ args["ignore-protected"] = serde_json::to_value(true)?;
+ }
+ }
- Ok(())
+ let mut result = params.target.client.delete(&api_path, Some(args)).await?;
+ let data = result["data"].take();
+ let delete_stats: BackupGroupDeleteStats =
+ serde_json::from_value(data).unwrap_or_else(|_| BackupGroupDeleteStats::default());
+
+ Ok(delete_stats)
}
// Fetch the list of groups found on target in given namespace
@@ -229,7 +253,7 @@ async fn remove_target_group(
params: &PushParameters,
namespace: &BackupNamespace,
backup_group: &BackupGroup,
-) -> Result<(), Error> {
+) -> 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}"))?;
@@ -242,14 +266,23 @@ async fn remove_target_group(
"backup-id": backup_group.id,
"backup-type": backup_group.ty,
});
+
+ if let Some(version) = ¶ms.target.api_version {
+ if version.is_min_required(ApiVersion::new(3, 2, 8)) {
+ args["ignore-protected"] = serde_json::to_value(true)?;
+ }
+ }
if !namespace.is_root() {
let target_ns = params.map_to_target(namespace)?;
args["ns"] = serde_json::to_value(target_ns.name())?;
}
- params.target.client.delete(&api_path, Some(args)).await?;
+ let mut result = params.target.client.delete(&api_path, Some(args)).await?;
+ let data = result["data"].take();
+ let delete_stats: BackupGroupDeleteStats =
+ serde_json::from_value(data).unwrap_or_else(|_| BackupGroupDeleteStats::default());
- Ok(())
+ Ok(delete_stats)
}
// Check if the namespace is already present on the target, create it otherwise
@@ -298,6 +331,7 @@ async fn check_or_create_target_namespace(
/// 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;
+ params.fetch_target_api_version_info().await?;
// 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?;
@@ -365,9 +399,25 @@ pub(crate) async fn push_store(mut params: PushParameters) -> Result<SyncStats,
if synced_namespaces.contains(&target_namespace) {
continue;
}
- if let Err(err) = remove_target_namespace(¶ms, &target_namespace).await {
- info!("failed to remove vanished namespace {target_namespace} - {err}");
- continue;
+ match remove_target_namespace(¶ms, &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}");
}
@@ -449,38 +499,26 @@ pub(crate) async fn push_namespace(
info!("delete vanished group '{target_group}'");
- let count_before = match fetch_target_groups(params, namespace).await {
- Ok(snapshots) => snapshots.len(),
- Err(_err) => 0, // ignore errors
- };
-
- if let Err(err) = remove_target_group(params, namespace, &target_group).await {
- info!("{err}");
- errors = true;
- continue;
- }
-
- let mut count_after = match fetch_target_groups(params, namespace).await {
- Ok(snapshots) => snapshots.len(),
- Err(_err) => 0, // ignore errors
- };
-
- let deleted_groups = if count_after > 0 {
- info!("kept some protected snapshots of group '{target_group}'");
- 0
- } else {
- 1
- };
-
- if count_after > count_before {
- count_after = count_before;
+ 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) => {
+ info!("failed to delete vanished group - {err}");
+ errors = true;
+ continue;
+ }
}
-
- stats.add(SyncStats::from(RemovedVanishedStats {
- snapshots: count_before - count_after,
- groups: deleted_groups,
- namespaces: 0,
- }));
}
}
--
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] 34+ messages in thread
* Re: [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (30 preceding siblings ...)
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 31/31] server: sync job: use delete stats provided by the api Christian Ebner
@ 2024-10-18 6:55 ` Christian Ebner
2024-10-18 8:44 ` Christian Ebner
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-18 6:55 UTC (permalink / raw)
To: pbs-devel
This patch series does not apply on master anymore, will rebase and send
a new version of the patches soon.
Also, I just noticed that the repository in the subject prefix was
wrong, these patches are of course intended for `proxmox-backup`.
Please disregard this version.
_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
^ permalink raw reply [flat|nested] 34+ messages in thread
* Re: [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
` (31 preceding siblings ...)
2024-10-18 6:55 ` [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
@ 2024-10-18 8:44 ` Christian Ebner
32 siblings, 0 replies; 34+ messages in thread
From: Christian Ebner @ 2024-10-18 8:44 UTC (permalink / raw)
To: pbs-devel
superseded-by version 5:
https://lore.proxmox.com/pbs-devel/20241018084242.144010-1-c.ebner@proxmox.com/
_______________________________________________
pbs-devel mailing list
pbs-devel@lists.proxmox.com
https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
^ permalink raw reply [flat|nested] 34+ messages in thread
end of thread, other threads:[~2024-10-18 8:43 UTC | newest]
Thread overview: 34+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2024-10-17 13:26 [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 01/31] client: backup writer: refactor backup and upload stats counters Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 02/31] client: backup writer: factor out merged chunk stream upload Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 03/31] client: backup writer: allow push uploading index and chunks Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 04/31] config: acl: refactor acl path component check for datastore Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 05/31] config: acl: allow namespace components for remote datastores Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 06/31] api types: implement remote acl path method for sync job Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 07/31] api types: define remote permissions and roles for push sync Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 08/31] fix #3044: server: implement push support for sync operations Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 09/31] api types/config: add `sync-push` config type for push sync jobs Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 10/31] api: push: implement endpoint for sync in push direction Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 11/31] api: sync: move sync job invocation to server sync module Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 12/31] api: sync jobs: expose optional `sync-direction` parameter Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 13/31] api: admin: avoid duplicate name for list sync jobs api method Christian Ebner
2024-10-17 13:26 ` [pbs-devel] [PATCH v4 proxmox 14/31] api: config: Require PRIV_DATASTORE_AUDIT to modify sync job Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 15/31] api: config: factor out sync job owner check Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 16/31] api: config: extend read access check by sync direction Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 17/31] api: config: extend modify " Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 18/31] bin: manager: add datastore push cli command Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 19/31] ui: group filter: allow to set namespace for local datastore Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 20/31] ui: sync edit: source group filters based on sync direction Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 21/31] ui: add view with separate grids for pull and push sync jobs Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 22/31] ui: sync job: adapt edit window to be used for pull and push Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 23/31] ui: sync: pass sync-direction to allow removing push jobs Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 24/31] ui: sync view: do not use data model proxy for store Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 25/31] ui: sync view: set sync direction when invoking run task via api Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 26/31] datastore: move `BackupGroupDeleteStats` to api types Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 27/31] api types: implement api type for `BackupGroupDeleteStats` Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 28/31] api/api-types: refactor api endpoint version, add api types Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 29/31] datastore: increment deleted group counter when removing group Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 30/31] api: datastore/namespace: return backup groups delete stats on remove Christian Ebner
2024-10-17 13:27 ` [pbs-devel] [PATCH v4 proxmox 31/31] server: sync job: use delete stats provided by the api Christian Ebner
2024-10-18 6:55 ` [pbs-devel] [PATCH v4 proxmox 00/31] fix #3044: push datastore to remote target Christian Ebner
2024-10-18 8:44 ` Christian Ebner
This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox