public inbox for pbs-devel@lists.proxmox.com
 help / color / mirror / Atom feed
* [pbs-devel] [PATCH proxmox-backup v2 0/3] tape/verify: improve multithreaded
@ 2024-05-07  7:29 Dominik Csapak
  2024-05-07  7:29 ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Dominik Csapak
                   ` (2 more replies)
  0 siblings, 3 replies; 8+ messages in thread
From: Dominik Csapak @ 2024-05-07  7:29 UTC (permalink / raw)
  To: pbs-devel

this is a replacement series for my previous patches[0]

in this version, i reversed the order of patches, so that the tape
improvement comes first. I did this, because for tapes I could measure a
very real and beneficial improvement, while for verification the
situation is more complicated.

There we have currently only the hashing of chunks parallelized, not the
reading of data, but this could also improve throughput (as seen in the
benchmarks of 3/3).

patch 2 and 3 are RFCs because I'm not sure how to proceed with these.
One one hand they can improve the situation (when both are applied) but
makes it worse for other situations.

We could go even further and introduce another
'verification-read-threads' option and have a seperate thread pool for
reading the data and another for verifying them, but I'm not sure if it
would be worth it to spend even more time on it (incl testing, which
takes a lot of time)

changes from v1:
* reordered code from verify,tape to tape,verify
* marked verify patches as rfc
* use a struct for saving the values in the datastore (as thomas
  suggested)
* introduces another verify patch to merge the chunk loading into the
  worker threads

0: https://lists.proxmox.com/pipermail/pbs-devel/2024-April/009162.html

Dominik Csapak (3):
  tape: introduce a tape backup read thread tuning option
  verify: add tuning option for number of threads to use
  verify: move chunk loading into the worker threads

 pbs-api-types/src/datastore.rs              | 15 +++++
 pbs-datastore/src/datastore.rs              | 31 ++++++++++
 src/backup/verify.rs                        | 67 +++++++++------------
 src/tape/pool_writer/new_chunks_iterator.rs | 45 +++++++++-----
 www/Utils.js                                | 10 +++
 www/datastore/OptionView.js                 | 16 +++++
 6 files changed, 130 insertions(+), 54 deletions(-)

-- 
2.39.2



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


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

* [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option
  2024-05-07  7:29 [pbs-devel] [PATCH proxmox-backup v2 0/3] tape/verify: improve multithreaded Dominik Csapak
@ 2024-05-07  7:29 ` Dominik Csapak
  2024-05-07 15:10   ` Max Carrara
  2024-05-08 17:47   ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Max Carrara
  2024-05-07  7:29 ` [pbs-devel] [RFC PATCH proxmox-backup v2 2/3] verify: add tuning option for number of threads to use Dominik Csapak
  2024-05-07  7:29 ` [pbs-devel] [RFC PATCH proxmox-backup v2 3/3] verify: move chunk loading into the worker threads Dominik Csapak
  2 siblings, 2 replies; 8+ messages in thread
From: Dominik Csapak @ 2024-05-07  7:29 UTC (permalink / raw)
  To: pbs-devel

using a single thread for reading is not optimal in some cases, e.g.
when the underlying storage can handle more reads in parallel than with
a single thread.

We use the ParallelHandler to handle the actual reads. Make the
sync_channel buffer size depending on the number of threads so we have
space for two chunks per thread.

How this impacts the backup speed largely depends on the underlying
storage and how the backup is laid out on it.

I benchmarked the following setups:

* Setup A: relatively spread out backup on a virtualized pbs on single HDDs
* Setup B: mostly sequential chunks on a virtualized pbs on single HDDs
* Setup C: backup on virtualized pbs on a fast NVME
* Setup D: backup on bare metal pbs with ZFS in a RAID10 with 6 HDDs
  and 2 fast special devices in a mirror

(values are reported in MB/s as seen in the task log, caches were
cleared between runs, backups were bigger than the memory available)

setup  1 thread  2 threads  4 threads  8 threads
A      55        70         80         95
B      110       89         100        108
C      294       294        294        294
D      118       180        300        300

So there are cases where multiple read threads speed up the tape backup
(dramatically). On the other hand there are situations where reading
from a single thread is actually faster, probably because we can read
from the HDD sequentially.

I used a new default value of '4' here since that gave good performance
on all setups (not necessarily the best) and we also have a default
value of '4' for verification threads.

We use a struct here for the datastore since we want to introduce other
thread tuning options too.

Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>
---
 pbs-api-types/src/datastore.rs              |  8 ++++
 pbs-datastore/src/datastore.rs              | 26 ++++++++++++
 src/tape/pool_writer/new_chunks_iterator.rs | 45 +++++++++++++--------
 www/Utils.js                                |  5 +++
 www/datastore/OptionView.js                 |  8 ++++
 5 files changed, 76 insertions(+), 16 deletions(-)

diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
index 31767417a..1dae3867f 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -209,6 +209,11 @@ pub enum DatastoreFSyncLevel {
             type: ChunkOrder,
             optional: true,
         },
+        "tape-backup-read-threads": {
+            type: usize,
+            optional: true,
+            minimum: 1,
+        },
     },
 )]
 #[derive(Serialize, Deserialize, Default)]
@@ -220,6 +225,9 @@ pub struct DatastoreTuning {
     pub chunk_order: Option<ChunkOrder>,
     #[serde(skip_serializing_if = "Option::is_none")]
     pub sync_level: Option<DatastoreFSyncLevel>,
+    #[serde(skip_serializing_if = "Option::is_none")]
+    /// Configures how many threads to use to read from the datastore while backing up to tape.
+    pub tape_backup_read_threads: Option<usize>,
 }
 
 pub const DATASTORE_TUNING_STRING_SCHEMA: Schema = StringSchema::new("Datastore tuning options")
diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index f95da7615..73a1cfa39 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -49,6 +49,19 @@ pub fn check_backup_owner(owner: &Authid, auth_id: &Authid) -> Result<(), Error>
     Ok(())
 }
 
+/// Contains the configuration of how many threads to use in various situations
+pub struct ThreadConfiguration {
+    pub tape_backup_read_threads: usize,
+}
+
+impl Default for ThreadConfiguration {
+    fn default() -> Self {
+        Self {
+            tape_backup_read_threads: 4,
+        }
+    }
+}
+
 /// Datastore Management
 ///
 /// A Datastore can store severals backups, and provides the
@@ -61,6 +74,7 @@ pub struct DataStoreImpl {
     chunk_order: ChunkOrder,
     last_digest: Option<[u8; 32]>,
     sync_level: DatastoreFSyncLevel,
+    thread_config: ThreadConfiguration,
 }
 
 impl DataStoreImpl {
@@ -75,6 +89,7 @@ impl DataStoreImpl {
             chunk_order: Default::default(),
             last_digest: None,
             sync_level: Default::default(),
+            thread_config: Default::default(),
         })
     }
 }
@@ -305,6 +320,11 @@ impl DataStore {
                 .parse_property_string(config.tuning.as_deref().unwrap_or(""))?,
         )?;
 
+        let mut thread_config = ThreadConfiguration::default();
+        if let Some(value) = tuning.tape_backup_read_threads {
+            thread_config.tape_backup_read_threads = value;
+        }
+
         Ok(DataStoreImpl {
             chunk_store,
             gc_mutex: Mutex::new(()),
@@ -313,6 +333,7 @@ impl DataStore {
             chunk_order: tuning.chunk_order.unwrap_or_default(),
             last_digest,
             sync_level: tuning.sync_level.unwrap_or_default(),
+            thread_config,
         })
     }
 
@@ -1377,6 +1398,11 @@ impl DataStore {
         Ok(())
     }
 
+    /// returns the datatstore thread configuration
+    pub fn get_thread_configuration(&self) -> &ThreadConfiguration {
+        &self.inner.thread_config
+    }
+
     /// Destroy a datastore. This requires that there are no active operations on the datastore.
     ///
     /// This is a synchronous operation and should be run in a worker-thread.
diff --git a/src/tape/pool_writer/new_chunks_iterator.rs b/src/tape/pool_writer/new_chunks_iterator.rs
index 1454b33d2..a2a8091f6 100644
--- a/src/tape/pool_writer/new_chunks_iterator.rs
+++ b/src/tape/pool_writer/new_chunks_iterator.rs
@@ -6,8 +6,9 @@ use anyhow::{format_err, Error};
 use pbs_datastore::{DataBlob, DataStore, SnapshotReader};
 
 use crate::tape::CatalogSet;
+use crate::tools::parallel_handler::ParallelHandler;
 
-/// Chunk iterator which use a separate thread to read chunks
+/// Chunk iterator which uses separate threads to read chunks
 ///
 /// The iterator skips duplicate chunks and chunks already in the
 /// catalog.
@@ -25,7 +26,11 @@ impl NewChunksIterator {
         snapshot_reader: Arc<Mutex<SnapshotReader>>,
         catalog_set: Arc<Mutex<CatalogSet>>,
     ) -> Result<(std::thread::JoinHandle<()>, Self), Error> {
-        let (tx, rx) = std::sync::mpsc::sync_channel(3);
+        let read_threads = datastore
+            .get_thread_configuration()
+            .tape_backup_read_threads;
+
+        let (tx, rx) = std::sync::mpsc::sync_channel(read_threads * 2);
 
         let reader_thread = std::thread::spawn(move || {
             let snapshot_reader = snapshot_reader.lock().unwrap();
@@ -35,36 +40,44 @@ impl NewChunksIterator {
             let datastore_name = snapshot_reader.datastore_name().to_string();
 
             let result: Result<(), Error> = proxmox_lang::try_block!({
-                let mut chunk_iter = snapshot_reader.chunk_iterator(move |digest| {
+                let chunk_iter = snapshot_reader.chunk_iterator(move |digest| {
                     catalog_set
                         .lock()
                         .unwrap()
                         .contains_chunk(&datastore_name, digest)
                 })?;
 
-                loop {
-                    let digest = match chunk_iter.next() {
-                        None => {
-                            let _ = tx.send(Ok(None)); // ignore send error
-                            break;
+                let reader_pool =
+                    ParallelHandler::new("tape backup chunk reader pool", read_threads, {
+                        let tx = tx.clone();
+                        move |digest| {
+                            let blob = datastore.load_chunk(&digest)?;
+                            //println!("LOAD CHUNK {}", hex::encode(&digest));
+
+                            tx.send(Ok(Some((digest, blob)))).map_err(|err| {
+                                format_err!("error sending result from reader thread: {err}")
+                            })?;
+
+                            Ok(())
                         }
-                        Some(digest) => digest?,
-                    };
+                    });
+
+                for digest in chunk_iter {
+                    let digest = digest?;
 
                     if chunk_index.contains(&digest) {
                         continue;
                     }
 
-                    let blob = datastore.load_chunk(&digest)?;
-                    //println!("LOAD CHUNK {}", hex::encode(&digest));
-                    if let Err(err) = tx.send(Ok(Some((digest, blob)))) {
-                        eprintln!("could not send chunk to reader thread: {err}");
-                        break;
-                    }
+                    reader_pool.send(digest)?;
 
                     chunk_index.insert(digest);
                 }
 
+                reader_pool.complete()?;
+
+                let _ = tx.send(Ok(None)); // ignore send error
+
                 Ok(())
             });
             if let Err(err) = result {
diff --git a/www/Utils.js b/www/Utils.js
index 1d7351a32..4d224cd4a 100644
--- a/www/Utils.js
+++ b/www/Utils.js
@@ -790,6 +790,11 @@ Ext.define('PBS.Utils', {
 	sync = PBS.Utils.tuningOptions['sync-level'][sync ?? '__default__'];
 	options.push(`${gettext('Sync Level')}: ${sync}`);
 
+	let tapeBackupRT = tuning['tape-backup-read-threads'];
+	delete tuning['tape-backup-read-threads'];
+	tapeBackupRT ??= Proxmox.Utils.defaultText + ` (4)`;
+	options.push(`${gettext('Tape Backup Read Threads')}: ${tapeBackupRT}`);
+
 	for (const [k, v] of Object.entries(tuning)) {
 	    options.push(`${k}: ${v}`);
 	}
diff --git a/www/datastore/OptionView.js b/www/datastore/OptionView.js
index e1f38af6f..cfbb89151 100644
--- a/www/datastore/OptionView.js
+++ b/www/datastore/OptionView.js
@@ -271,6 +271,14 @@ Ext.define('PBS.Datastore.Options', {
 			    deleteEmpty: true,
 			    value: '__default__',
 			},
+			{
+			    xtype: 'proxmoxintegerfield',
+			    name: 'tape-backup-read-threads',
+			    fieldLabel: gettext('Tape Backup Read Threads'),
+			    min: 1,
+			    emptyText: '4',
+			    deleteEmpty: true,
+			},
 		    ],
 		},
 	    },
-- 
2.39.2



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


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

* [pbs-devel] [RFC PATCH proxmox-backup v2 2/3] verify: add tuning option for number of threads to use
  2024-05-07  7:29 [pbs-devel] [PATCH proxmox-backup v2 0/3] tape/verify: improve multithreaded Dominik Csapak
  2024-05-07  7:29 ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Dominik Csapak
@ 2024-05-07  7:29 ` Dominik Csapak
  2024-05-07  7:29 ` [pbs-devel] [RFC PATCH proxmox-backup v2 3/3] verify: move chunk loading into the worker threads Dominik Csapak
  2 siblings, 0 replies; 8+ messages in thread
From: Dominik Csapak @ 2024-05-07  7:29 UTC (permalink / raw)
  To: pbs-devel

a similar argument as for tape backups, but limited to the hashing part
of verification.

The ParallelHandler we use here only parallelizes the actual
verification, not the reading of chunks.

So in case the CPU is much slower than reading from storage, this could
help increase throughput.

In my benchmarks, the only difference i saw was when the cpu was limited
and I reduced the number of threads. So i guess this knob could help in
situations where the single-threaded read performance of the storage
outperforms the 4 cpu threads in hashing.

Because of that, I left the default at '4' threads like before.

Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>
---
 pbs-api-types/src/datastore.rs | 7 +++++++
 pbs-datastore/src/datastore.rs | 5 +++++
 src/backup/verify.rs           | 4 +++-
 www/Utils.js                   | 5 +++++
 www/datastore/OptionView.js    | 8 ++++++++
 5 files changed, 28 insertions(+), 1 deletion(-)

diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
index 1dae3867f..3fb9ff766 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -214,6 +214,11 @@ pub enum DatastoreFSyncLevel {
             optional: true,
             minimum: 1,
         },
+        "verification-threads": {
+            type: usize,
+            optional: true,
+            minimum: 1,
+        },
     },
 )]
 #[derive(Serialize, Deserialize, Default)]
@@ -228,6 +233,8 @@ pub struct DatastoreTuning {
     #[serde(skip_serializing_if = "Option::is_none")]
     /// Configures how many threads to use to read from the datastore while backing up to tape.
     pub tape_backup_read_threads: Option<usize>,
+    /// Configures how many threads to use for hashing on verify.
+    pub verification_threads: Option<usize>,
 }
 
 pub const DATASTORE_TUNING_STRING_SCHEMA: Schema = StringSchema::new("Datastore tuning options")
diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index 73a1cfa39..b6552d92c 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -52,12 +52,14 @@ pub fn check_backup_owner(owner: &Authid, auth_id: &Authid) -> Result<(), Error>
 /// Contains the configuration of how many threads to use in various situations
 pub struct ThreadConfiguration {
     pub tape_backup_read_threads: usize,
+    pub verification_threads: usize,
 }
 
 impl Default for ThreadConfiguration {
     fn default() -> Self {
         Self {
             tape_backup_read_threads: 4,
+            verification_threads: 4,
         }
     }
 }
@@ -324,6 +326,9 @@ impl DataStore {
         if let Some(value) = tuning.tape_backup_read_threads {
             thread_config.tape_backup_read_threads = value;
         }
+        if let Some(value) = tuning.verification_threads {
+            thread_config.verification_threads = value;
+        }
 
         Ok(DataStoreImpl {
             chunk_store,
diff --git a/src/backup/verify.rs b/src/backup/verify.rs
index c972e5328..c9fbb2e33 100644
--- a/src/backup/verify.rs
+++ b/src/backup/verify.rs
@@ -123,9 +123,11 @@ fn verify_index_chunks(
     let verified_chunks2 = Arc::clone(&verify_worker.verified_chunks);
     let errors2 = Arc::clone(&errors);
 
+    let thread_count = datastore2.get_thread_configuration().verification_threads;
+
     let decoder_pool = ParallelHandler::new(
         "verify chunk decoder",
-        4,
+        thread_count,
         move |(chunk, digest, size): (DataBlob, [u8; 32], u64)| {
             let chunk_crypt_mode = match chunk.crypt_mode() {
                 Err(err) => {
diff --git a/www/Utils.js b/www/Utils.js
index 4d224cd4a..a94bdf86e 100644
--- a/www/Utils.js
+++ b/www/Utils.js
@@ -795,6 +795,11 @@ Ext.define('PBS.Utils', {
 	tapeBackupRT ??= Proxmox.Utils.defaultText + ` (4)`;
 	options.push(`${gettext('Tape Backup Read Threads')}: ${tapeBackupRT}`);
 
+	let verificationThreads = tuning['verification-threads'];
+	delete tuning['verification-threads'];
+	verificationThreads ??= Proxmox.Utils.defaultText + ` (4)`;
+	options.push(`${gettext('Verification Threads')}: ${verificationThreads}`);
+
 	for (const [k, v] of Object.entries(tuning)) {
 	    options.push(`${k}: ${v}`);
 	}
diff --git a/www/datastore/OptionView.js b/www/datastore/OptionView.js
index cfbb89151..c28294bfd 100644
--- a/www/datastore/OptionView.js
+++ b/www/datastore/OptionView.js
@@ -279,6 +279,14 @@ Ext.define('PBS.Datastore.Options', {
 			    emptyText: '4',
 			    deleteEmpty: true,
 			},
+			{
+			    xtype: 'proxmoxintegerfield',
+			    name: 'verification-threads',
+			    fieldLabel: gettext('Verification Threads'),
+			    min: 1,
+			    emptyText: '4',
+			    deleteEmpty: true,
+			},
 		    ],
 		},
 	    },
-- 
2.39.2



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


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

* [pbs-devel] [RFC PATCH proxmox-backup v2 3/3] verify: move chunk loading into the worker threads
  2024-05-07  7:29 [pbs-devel] [PATCH proxmox-backup v2 0/3] tape/verify: improve multithreaded Dominik Csapak
  2024-05-07  7:29 ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Dominik Csapak
  2024-05-07  7:29 ` [pbs-devel] [RFC PATCH proxmox-backup v2 2/3] verify: add tuning option for number of threads to use Dominik Csapak
@ 2024-05-07  7:29 ` Dominik Csapak
  2 siblings, 0 replies; 8+ messages in thread
From: Dominik Csapak @ 2024-05-07  7:29 UTC (permalink / raw)
  To: pbs-devel

so that the chunk loading can also be done in parallel.

This can has a very big impact on how fast verification is, depending on
the underlying storage.

I measured on the following setups:

* Setup A: spread out backup on virtualized PBS on single HDD
* Setup B: backup with mostly sequential chunks on virtualized PBS on HDDs
* Setup C: backup on virtualized PBS on a fast NVME

(value is in MiB/s read speed from the task log, caches were cleared
between runs, cpu was mostly idle for the HDD tests)

setup  baseline(current code)  1 thread  2 threads  4 threads  8 threads
A      89                      75        73         79         85
B      67                      56        61         67         72
C      1133                    616       1133       1850       2558

This data shows that on spinning disks, having a single read thread
that continuously reads, can be better than simply having multiple
threads that read from disk.

On fast disks though, reading in parallel makes it much faster, even
with the same number of threads.

Since the results are so varied across backups/storages/etc. I opted
to not automatically calculate the default (e.g. by disk type/cpu
cores/etc.)

Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>
---
As mentioned in the cover letter, we could go even further and have
a seperate reading thread pool (thats configurable) so we could
have the default values we have now but users can still tune it to
their liking.

 pbs-api-types/src/datastore.rs |  2 +-
 src/backup/verify.rs           | 65 ++++++++++++++--------------------
 2 files changed, 28 insertions(+), 39 deletions(-)

diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
index 3fb9ff766..6278a6d99 100644
--- a/pbs-api-types/src/datastore.rs
+++ b/pbs-api-types/src/datastore.rs
@@ -233,7 +233,7 @@ pub struct DatastoreTuning {
     #[serde(skip_serializing_if = "Option::is_none")]
     /// Configures how many threads to use to read from the datastore while backing up to tape.
     pub tape_backup_read_threads: Option<usize>,
-    /// Configures how many threads to use for hashing on verify.
+    /// Configures how many threads to use for reading and hashing on verify.
     pub verification_threads: Option<usize>,
 }
 
diff --git a/src/backup/verify.rs b/src/backup/verify.rs
index c9fbb2e33..e2789a865 100644
--- a/src/backup/verify.rs
+++ b/src/backup/verify.rs
@@ -1,6 +1,6 @@
 use nix::dir::Dir;
 use std::collections::HashSet;
-use std::sync::atomic::{AtomicUsize, Ordering};
+use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
 use std::sync::{Arc, Mutex};
 use std::time::Instant;
 
@@ -15,7 +15,7 @@ use pbs_api_types::{
 use pbs_datastore::backup_info::{BackupDir, BackupGroup, BackupInfo};
 use pbs_datastore::index::IndexFile;
 use pbs_datastore::manifest::{archive_type, ArchiveType, BackupManifest, FileInfo};
-use pbs_datastore::{DataBlob, DataStore, StoreProgress};
+use pbs_datastore::{DataStore, StoreProgress};
 use proxmox_sys::fs::lock_dir_noblock_shared;
 
 use crate::tools::parallel_handler::ParallelHandler;
@@ -114,8 +114,8 @@ fn verify_index_chunks(
 
     let start_time = Instant::now();
 
-    let mut read_bytes = 0;
-    let mut decoded_bytes = 0;
+    let read_bytes = Arc::new(AtomicU64::new(0));
+    let decoded_bytes = Arc::new(AtomicU64::new(0));
 
     let worker2 = Arc::clone(&verify_worker.worker);
     let datastore2 = Arc::clone(&verify_worker.datastore);
@@ -125,10 +125,23 @@ fn verify_index_chunks(
 
     let thread_count = datastore2.get_thread_configuration().verification_threads;
 
-    let decoder_pool = ParallelHandler::new(
-        "verify chunk decoder",
-        thread_count,
-        move |(chunk, digest, size): (DataBlob, [u8; 32], u64)| {
+    let decoder_pool = ParallelHandler::new("verify chunk decoder", thread_count, {
+        let read_bytes = read_bytes.clone();
+        let decoded_bytes = decoded_bytes.clone();
+        move |(digest, size): ([u8; 32], u64)| {
+            let chunk = match datastore2.load_chunk(&digest) {
+                Err(err) => {
+                    corrupt_chunks2.lock().unwrap().insert(digest);
+                    task_log!(worker2, "can't verify chunk, load failed - {}", err);
+                    errors2.fetch_add(1, Ordering::SeqCst);
+                    rename_corrupted_chunk(datastore2.clone(), &digest, &worker2);
+                    return Ok(());
+                }
+                Ok(chunk) => {
+                    read_bytes.fetch_add(chunk.raw_size(), Ordering::SeqCst);
+                    chunk
+                }
+            };
             let chunk_crypt_mode = match chunk.crypt_mode() {
                 Err(err) => {
                     corrupt_chunks2.lock().unwrap().insert(digest);
@@ -157,10 +170,11 @@ fn verify_index_chunks(
             } else {
                 verified_chunks2.lock().unwrap().insert(digest);
             }
+            decoded_bytes.fetch_add(size, Ordering::SeqCst);
 
             Ok(())
-        },
-    );
+        }
+    });
 
     let skip_chunk = |digest: &[u8; 32]| -> bool {
         if verify_worker
@@ -213,40 +227,15 @@ fn verify_index_chunks(
             continue; // already verified or marked corrupt
         }
 
-        match verify_worker.datastore.load_chunk(&info.digest) {
-            Err(err) => {
-                verify_worker
-                    .corrupt_chunks
-                    .lock()
-                    .unwrap()
-                    .insert(info.digest);
-                task_log!(
-                    verify_worker.worker,
-                    "can't verify chunk, load failed - {}",
-                    err
-                );
-                errors.fetch_add(1, Ordering::SeqCst);
-                rename_corrupted_chunk(
-                    verify_worker.datastore.clone(),
-                    &info.digest,
-                    &verify_worker.worker,
-                );
-            }
-            Ok(chunk) => {
-                let size = info.size();
-                read_bytes += chunk.raw_size();
-                decoder_pool.send((chunk, info.digest, size))?;
-                decoded_bytes += size;
-            }
-        }
+        decoder_pool.send((info.digest, info.size()))?;
     }
 
     decoder_pool.complete()?;
 
     let elapsed = start_time.elapsed().as_secs_f64();
 
-    let read_bytes_mib = (read_bytes as f64) / (1024.0 * 1024.0);
-    let decoded_bytes_mib = (decoded_bytes as f64) / (1024.0 * 1024.0);
+    let read_bytes_mib = (read_bytes.load(Ordering::SeqCst) as f64) / (1024.0 * 1024.0);
+    let decoded_bytes_mib = (decoded_bytes.load(Ordering::SeqCst) as f64) / (1024.0 * 1024.0);
 
     let read_speed = read_bytes_mib / elapsed;
     let decode_speed = decoded_bytes_mib / elapsed;
-- 
2.39.2



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


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

* Re: [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option
  2024-05-07  7:29 ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Dominik Csapak
@ 2024-05-07 15:10   ` Max Carrara
  2024-05-08  6:56     ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning opti Dominik Csapak
  2024-05-08 17:47   ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Max Carrara
  1 sibling, 1 reply; 8+ messages in thread
From: Max Carrara @ 2024-05-07 15:10 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion

On Tue May 7, 2024 at 9:29 AM CEST, Dominik Csapak wrote:
> using a single thread for reading is not optimal in some cases, e.g.
> when the underlying storage can handle more reads in parallel than with
> a single thread.
>

Got only a few tiny notes that are otherwise unremarkable:

Would prefer to reword this as:

Using a single thread for reading is not optimal in some cases, e.g.
when the underlying storage can handle reads from multiple threads in
parallel.

> We use the ParallelHandler to handle the actual reads. Make the
> sync_channel buffer size depending on the number of threads so we have
> space for two chunks per thread.

Would s/depending/depend/ for the above paragraph.

>
> How this impacts the backup speed largely depends on the underlying
> storage and how the backup is laid out on it.
>
> I benchmarked the following setups:
>
> * Setup A: relatively spread out backup on a virtualized pbs on single HDDs
> * Setup B: mostly sequential chunks on a virtualized pbs on single HDDs
> * Setup C: backup on virtualized pbs on a fast NVME
> * Setup D: backup on bare metal pbs with ZFS in a RAID10 with 6 HDDs
>   and 2 fast special devices in a mirror
>
> (values are reported in MB/s as seen in the task log, caches were
> cleared between runs, backups were bigger than the memory available)
>
> setup  1 thread  2 threads  4 threads  8 threads
> A      55        70         80         95
> B      110       89         100        108
> C      294       294        294        294
> D      118       180        300        300
>
> So there are cases where multiple read threads speed up the tape backup
> (dramatically). On the other hand there are situations where reading
> from a single thread is actually faster, probably because we can read
> from the HDD sequentially.
>
> I used a new default value of '4' here since that gave good performance
> on all setups (not necessarily the best) and we also have a default
> value of '4' for verification threads.
>
> We use a struct here for the datastore since we want to introduce other
> thread tuning options too.
>
> Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>

The patch is pretty straightforward; there's one question inline, but
otherwise this looks fine to me. Solid work!

Wasn't able to test this yet as setting up a VTL on Bookworm failed for
me, unfortunately. Will try to test this tomorrow if possible.

For now:

Reviewed-by: Max Carrara <m.carrara@proxmox.com>

> ---
>  pbs-api-types/src/datastore.rs              |  8 ++++
>  pbs-datastore/src/datastore.rs              | 26 ++++++++++++
>  src/tape/pool_writer/new_chunks_iterator.rs | 45 +++++++++++++--------
>  www/Utils.js                                |  5 +++
>  www/datastore/OptionView.js                 |  8 ++++
>  5 files changed, 76 insertions(+), 16 deletions(-)
>
> diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
> index 31767417a..1dae3867f 100644
> --- a/pbs-api-types/src/datastore.rs
> +++ b/pbs-api-types/src/datastore.rs
> @@ -209,6 +209,11 @@ pub enum DatastoreFSyncLevel {
>              type: ChunkOrder,
>              optional: true,
>          },
> +        "tape-backup-read-threads": {
> +            type: usize,
> +            optional: true,
> +            minimum: 1,
> +        },
>      },
>  )]
>  #[derive(Serialize, Deserialize, Default)]
> @@ -220,6 +225,9 @@ pub struct DatastoreTuning {
>      pub chunk_order: Option<ChunkOrder>,
>      #[serde(skip_serializing_if = "Option::is_none")]
>      pub sync_level: Option<DatastoreFSyncLevel>,
> +    #[serde(skip_serializing_if = "Option::is_none")]
> +    /// Configures how many threads to use to read from the datastore while backing up to tape.
> +    pub tape_backup_read_threads: Option<usize>,
>  }
>  
>  pub const DATASTORE_TUNING_STRING_SCHEMA: Schema = StringSchema::new("Datastore tuning options")
> diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
> index f95da7615..73a1cfa39 100644
> --- a/pbs-datastore/src/datastore.rs
> +++ b/pbs-datastore/src/datastore.rs
> @@ -49,6 +49,19 @@ pub fn check_backup_owner(owner: &Authid, auth_id: &Authid) -> Result<(), Error>
>      Ok(())
>  }
>  
> +/// Contains the configuration of how many threads to use in various situations
> +pub struct ThreadConfiguration {
> +    pub tape_backup_read_threads: usize,
> +}
> +
> +impl Default for ThreadConfiguration {
> +    fn default() -> Self {
> +        Self {
> +            tape_backup_read_threads: 4,
> +        }
> +    }
> +}
> +
>  /// Datastore Management
>  ///
>  /// A Datastore can store severals backups, and provides the
> @@ -61,6 +74,7 @@ pub struct DataStoreImpl {
>      chunk_order: ChunkOrder,
>      last_digest: Option<[u8; 32]>,
>      sync_level: DatastoreFSyncLevel,
> +    thread_config: ThreadConfiguration,
>  }
>  
>  impl DataStoreImpl {
> @@ -75,6 +89,7 @@ impl DataStoreImpl {
>              chunk_order: Default::default(),
>              last_digest: None,
>              sync_level: Default::default(),
> +            thread_config: Default::default(),
>          })
>      }
>  }
> @@ -305,6 +320,11 @@ impl DataStore {
>                  .parse_property_string(config.tuning.as_deref().unwrap_or(""))?,
>          )?;
>  
> +        let mut thread_config = ThreadConfiguration::default();
> +        if let Some(value) = tuning.tape_backup_read_threads {
> +            thread_config.tape_backup_read_threads = value;
> +        }
> +
>          Ok(DataStoreImpl {
>              chunk_store,
>              gc_mutex: Mutex::new(()),
> @@ -313,6 +333,7 @@ impl DataStore {
>              chunk_order: tuning.chunk_order.unwrap_or_default(),
>              last_digest,
>              sync_level: tuning.sync_level.unwrap_or_default(),
> +            thread_config,
>          })
>      }
>  
> @@ -1377,6 +1398,11 @@ impl DataStore {
>          Ok(())
>      }
>  
> +    /// returns the datatstore thread configuration
> +    pub fn get_thread_configuration(&self) -> &ThreadConfiguration {
> +        &self.inner.thread_config
> +    }
> +
>      /// Destroy a datastore. This requires that there are no active operations on the datastore.
>      ///
>      /// This is a synchronous operation and should be run in a worker-thread.
> diff --git a/src/tape/pool_writer/new_chunks_iterator.rs b/src/tape/pool_writer/new_chunks_iterator.rs
> index 1454b33d2..a2a8091f6 100644
> --- a/src/tape/pool_writer/new_chunks_iterator.rs
> +++ b/src/tape/pool_writer/new_chunks_iterator.rs
> @@ -6,8 +6,9 @@ use anyhow::{format_err, Error};
>  use pbs_datastore::{DataBlob, DataStore, SnapshotReader};
>  
>  use crate::tape::CatalogSet;
> +use crate::tools::parallel_handler::ParallelHandler;
>  
> -/// Chunk iterator which use a separate thread to read chunks
> +/// Chunk iterator which uses separate threads to read chunks
>  ///
>  /// The iterator skips duplicate chunks and chunks already in the
>  /// catalog.
> @@ -25,7 +26,11 @@ impl NewChunksIterator {
>          snapshot_reader: Arc<Mutex<SnapshotReader>>,
>          catalog_set: Arc<Mutex<CatalogSet>>,
>      ) -> Result<(std::thread::JoinHandle<()>, Self), Error> {
> -        let (tx, rx) = std::sync::mpsc::sync_channel(3);
> +        let read_threads = datastore
> +            .get_thread_configuration()
> +            .tape_backup_read_threads;
> +
> +        let (tx, rx) = std::sync::mpsc::sync_channel(read_threads * 2);

Is there any reason you're using `* 2` here? For example, is the
throughput unaffected if you use a larger value, like `* 8`?

If the constant has an effect like that it should IMO be documented, but
if not, then it can just stay like it is.

>  
>          let reader_thread = std::thread::spawn(move || {
>              let snapshot_reader = snapshot_reader.lock().unwrap();
> @@ -35,36 +40,44 @@ impl NewChunksIterator {
>              let datastore_name = snapshot_reader.datastore_name().to_string();
>  
>              let result: Result<(), Error> = proxmox_lang::try_block!({
> -                let mut chunk_iter = snapshot_reader.chunk_iterator(move |digest| {
> +                let chunk_iter = snapshot_reader.chunk_iterator(move |digest| {
>                      catalog_set
>                          .lock()
>                          .unwrap()
>                          .contains_chunk(&datastore_name, digest)
>                  })?;
>  
> -                loop {
> -                    let digest = match chunk_iter.next() {
> -                        None => {
> -                            let _ = tx.send(Ok(None)); // ignore send error
> -                            break;
> +                let reader_pool =
> +                    ParallelHandler::new("tape backup chunk reader pool", read_threads, {
> +                        let tx = tx.clone();
> +                        move |digest| {
> +                            let blob = datastore.load_chunk(&digest)?;
> +                            //println!("LOAD CHUNK {}", hex::encode(&digest));
> +
> +                            tx.send(Ok(Some((digest, blob)))).map_err(|err| {
> +                                format_err!("error sending result from reader thread: {err}")
> +                            })?;
> +
> +                            Ok(())
>                          }
> -                        Some(digest) => digest?,
> -                    };
> +                    });
> +
> +                for digest in chunk_iter {
> +                    let digest = digest?;
>  
>                      if chunk_index.contains(&digest) {
>                          continue;
>                      }
>  
> -                    let blob = datastore.load_chunk(&digest)?;
> -                    //println!("LOAD CHUNK {}", hex::encode(&digest));
> -                    if let Err(err) = tx.send(Ok(Some((digest, blob)))) {
> -                        eprintln!("could not send chunk to reader thread: {err}");
> -                        break;
> -                    }
> +                    reader_pool.send(digest)?;
>  
>                      chunk_index.insert(digest);
>                  }
>  
> +                reader_pool.complete()?;
> +
> +                let _ = tx.send(Ok(None)); // ignore send error
> +
>                  Ok(())
>              });
>              if let Err(err) = result {
> diff --git a/www/Utils.js b/www/Utils.js
> index 1d7351a32..4d224cd4a 100644
> --- a/www/Utils.js
> +++ b/www/Utils.js
> @@ -790,6 +790,11 @@ Ext.define('PBS.Utils', {
>  	sync = PBS.Utils.tuningOptions['sync-level'][sync ?? '__default__'];
>  	options.push(`${gettext('Sync Level')}: ${sync}`);
>  
> +	let tapeBackupRT = tuning['tape-backup-read-threads'];
> +	delete tuning['tape-backup-read-threads'];
> +	tapeBackupRT ??= Proxmox.Utils.defaultText + ` (4)`;
> +	options.push(`${gettext('Tape Backup Read Threads')}: ${tapeBackupRT}`);
> +
>  	for (const [k, v] of Object.entries(tuning)) {
>  	    options.push(`${k}: ${v}`);
>  	}
> diff --git a/www/datastore/OptionView.js b/www/datastore/OptionView.js
> index e1f38af6f..cfbb89151 100644
> --- a/www/datastore/OptionView.js
> +++ b/www/datastore/OptionView.js
> @@ -271,6 +271,14 @@ Ext.define('PBS.Datastore.Options', {
>  			    deleteEmpty: true,
>  			    value: '__default__',
>  			},
> +			{
> +			    xtype: 'proxmoxintegerfield',
> +			    name: 'tape-backup-read-threads',
> +			    fieldLabel: gettext('Tape Backup Read Threads'),
> +			    min: 1,
> +			    emptyText: '4',
> +			    deleteEmpty: true,
> +			},
>  		    ],
>  		},
>  	    },



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


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

* Re: [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning opti
  2024-05-07 15:10   ` Max Carrara
@ 2024-05-08  6:56     ` Dominik Csapak
  2024-05-08 13:37       ` Max Carrara
  0 siblings, 1 reply; 8+ messages in thread
From: Dominik Csapak @ 2024-05-08  6:56 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion, Max Carrara

On 5/7/24 17:10, Max Carrara wrote:
> On Tue May 7, 2024 at 9:29 AM CEST, Dominik Csapak wrote:
>> using a single thread for reading is not optimal in some cases, e.g.
>> when the underlying storage can handle more reads in parallel than with
>> a single thread.
>>
> 
> Got only a few tiny notes that are otherwise unremarkable:
> 
> Would prefer to reword this as:
> 
> Using a single thread for reading is not optimal in some cases, e.g.
> when the underlying storage can handle reads from multiple threads in
> parallel.
> 
>> We use the ParallelHandler to handle the actual reads. Make the
>> sync_channel buffer size depending on the number of threads so we have
>> space for two chunks per thread.
> 
> Would s/depending/depend/ for the above paragraph.
> 

ok to both comments, will update in a v3 (if necessary)
or if preferred the one committing could fixup the commit message when applying

>>
>> How this impacts the backup speed largely depends on the underlying
>> storage and how the backup is laid out on it.
>>
>> I benchmarked the following setups:
>>
>> * Setup A: relatively spread out backup on a virtualized pbs on single HDDs
>> * Setup B: mostly sequential chunks on a virtualized pbs on single HDDs
>> * Setup C: backup on virtualized pbs on a fast NVME
>> * Setup D: backup on bare metal pbs with ZFS in a RAID10 with 6 HDDs
>>    and 2 fast special devices in a mirror
>>
>> (values are reported in MB/s as seen in the task log, caches were
>> cleared between runs, backups were bigger than the memory available)
>>
>> setup  1 thread  2 threads  4 threads  8 threads
>> A      55        70         80         95
>> B      110       89         100        108
>> C      294       294        294        294
>> D      118       180        300        300
>>
>> So there are cases where multiple read threads speed up the tape backup
>> (dramatically). On the other hand there are situations where reading
>> from a single thread is actually faster, probably because we can read
>> from the HDD sequentially.
>>
>> I used a new default value of '4' here since that gave good performance
>> on all setups (not necessarily the best) and we also have a default
>> value of '4' for verification threads.
>>
>> We use a struct here for the datastore since we want to introduce other
>> thread tuning options too.
>>
>> Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>
> 
> The patch is pretty straightforward; there's one question inline, but
> otherwise this looks fine to me. Solid work!
> 
> Wasn't able to test this yet as setting up a VTL on Bookworm failed for
> me, unfortunately. Will try to test this tomorrow if possible.

it's probably obvious, but just to be sure:

make sure the target storage for the vtl is faster than your
source storage, otherwise you'll always be limited by that...

> 
> For now:
> 
> Reviewed-by: Max Carrara <m.carrara@proxmox.com>
> 
[snip]
> diff --git a/src/tape/pool_writer/new_chunks_iterator.rs b/src/tape/pool_writer/new_chunks_iterator.rs
>> index 1454b33d2..a2a8091f6 100644
>> --- a/src/tape/pool_writer/new_chunks_iterator.rs
>> +++ b/src/tape/pool_writer/new_chunks_iterator.rs
>> @@ -6,8 +6,9 @@ use anyhow::{format_err, Error};
>>   use pbs_datastore::{DataBlob, DataStore, SnapshotReader};
>>   
>>   use crate::tape::CatalogSet;
>> +use crate::tools::parallel_handler::ParallelHandler;
>>   
>> -/// Chunk iterator which use a separate thread to read chunks
>> +/// Chunk iterator which uses separate threads to read chunks
>>   ///
>>   /// The iterator skips duplicate chunks and chunks already in the
>>   /// catalog.
>> @@ -25,7 +26,11 @@ impl NewChunksIterator {
>>           snapshot_reader: Arc<Mutex<SnapshotReader>>,
>>           catalog_set: Arc<Mutex<CatalogSet>>,
>>       ) -> Result<(std::thread::JoinHandle<()>, Self), Error> {
>> -        let (tx, rx) = std::sync::mpsc::sync_channel(3);
>> +        let read_threads = datastore
>> +            .get_thread_configuration()
>> +            .tape_backup_read_threads;
>> +
>> +        let (tx, rx) = std::sync::mpsc::sync_channel(read_threads * 2);
> 
> Is there any reason you're using `* 2` here? For example, is the
> throughput unaffected if you use a larger value, like `* 8`?
> 
> If the constant has an effect like that it should IMO be documented, but
> if not, then it can just stay like it is.

i did not benchmark that much here (would have increased my benchmark time since
that's another dimension in the testing with many possible values)

but my though process was this:

if the source storage is much slower than what we can write to tape, that
buffer will be empty most of the time

if the source storage is much faster than what we can write to tape, it
will be full most of the time

in both cases the buffer size won't matter much (as long as there is a bit of
buffer)

the sweet spot is when the storage is about as fast as the tape writes
and in that case you want to be able to have just a bit more buffer
than threads reading from the storage so there is always a read
going on (so the buffer will never empty, since that slows
down the tape write)

so 1*number of threads seemed to low, but i think anything bigger
than 2* is just a waste of memory

does that make sense?

if we really want, i can make additional benchmarks but my guess
is that it won't make much difference in practice


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


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

* Re: [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning opti
  2024-05-08  6:56     ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning opti Dominik Csapak
@ 2024-05-08 13:37       ` Max Carrara
  0 siblings, 0 replies; 8+ messages in thread
From: Max Carrara @ 2024-05-08 13:37 UTC (permalink / raw)
  To: Dominik Csapak, Proxmox Backup Server development discussion

On Wed May 8, 2024 at 8:56 AM CEST, Dominik Csapak wrote:
> On 5/7/24 17:10, Max Carrara wrote:
> > On Tue May 7, 2024 at 9:29 AM CEST, Dominik Csapak wrote:
> >> using a single thread for reading is not optimal in some cases, e.g.
> >> when the underlying storage can handle more reads in parallel than with
> >> a single thread.
> >>
> > 
> > Got only a few tiny notes that are otherwise unremarkable:
> > 
> > Would prefer to reword this as:
> > 
> > Using a single thread for reading is not optimal in some cases, e.g.
> > when the underlying storage can handle reads from multiple threads in
> > parallel.
> > 
> >> We use the ParallelHandler to handle the actual reads. Make the
> >> sync_channel buffer size depending on the number of threads so we have
> >> space for two chunks per thread.
> > 
> > Would s/depending/depend/ for the above paragraph.
> > 
>
> ok to both comments, will update in a v3 (if necessary)
> or if preferred the one committing could fixup the commit message when applying
>
> >>
> >> How this impacts the backup speed largely depends on the underlying
> >> storage and how the backup is laid out on it.
> >>
> >> I benchmarked the following setups:
> >>
> >> * Setup A: relatively spread out backup on a virtualized pbs on single HDDs
> >> * Setup B: mostly sequential chunks on a virtualized pbs on single HDDs
> >> * Setup C: backup on virtualized pbs on a fast NVME
> >> * Setup D: backup on bare metal pbs with ZFS in a RAID10 with 6 HDDs
> >>    and 2 fast special devices in a mirror
> >>
> >> (values are reported in MB/s as seen in the task log, caches were
> >> cleared between runs, backups were bigger than the memory available)
> >>
> >> setup  1 thread  2 threads  4 threads  8 threads
> >> A      55        70         80         95
> >> B      110       89         100        108
> >> C      294       294        294        294
> >> D      118       180        300        300
> >>
> >> So there are cases where multiple read threads speed up the tape backup
> >> (dramatically). On the other hand there are situations where reading
> >> from a single thread is actually faster, probably because we can read
> >> from the HDD sequentially.
> >>
> >> I used a new default value of '4' here since that gave good performance
> >> on all setups (not necessarily the best) and we also have a default
> >> value of '4' for verification threads.
> >>
> >> We use a struct here for the datastore since we want to introduce other
> >> thread tuning options too.
> >>
> >> Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>
> > 
> > The patch is pretty straightforward; there's one question inline, but
> > otherwise this looks fine to me. Solid work!
> > 
> > Wasn't able to test this yet as setting up a VTL on Bookworm failed for
> > me, unfortunately. Will try to test this tomorrow if possible.
>
> it's probably obvious, but just to be sure:
>
> make sure the target storage for the vtl is faster than your
> source storage, otherwise you'll always be limited by that...

Thanks for the tip! Will consider it. Am in the process of spinning up
another VM.

>
> > 
> > For now:
> > 
> > Reviewed-by: Max Carrara <m.carrara@proxmox.com>
> > 
> [snip]
> > diff --git a/src/tape/pool_writer/new_chunks_iterator.rs b/src/tape/pool_writer/new_chunks_iterator.rs
> >> index 1454b33d2..a2a8091f6 100644
> >> --- a/src/tape/pool_writer/new_chunks_iterator.rs
> >> +++ b/src/tape/pool_writer/new_chunks_iterator.rs
> >> @@ -6,8 +6,9 @@ use anyhow::{format_err, Error};
> >>   use pbs_datastore::{DataBlob, DataStore, SnapshotReader};
> >>   
> >>   use crate::tape::CatalogSet;
> >> +use crate::tools::parallel_handler::ParallelHandler;
> >>   
> >> -/// Chunk iterator which use a separate thread to read chunks
> >> +/// Chunk iterator which uses separate threads to read chunks
> >>   ///
> >>   /// The iterator skips duplicate chunks and chunks already in the
> >>   /// catalog.
> >> @@ -25,7 +26,11 @@ impl NewChunksIterator {
> >>           snapshot_reader: Arc<Mutex<SnapshotReader>>,
> >>           catalog_set: Arc<Mutex<CatalogSet>>,
> >>       ) -> Result<(std::thread::JoinHandle<()>, Self), Error> {
> >> -        let (tx, rx) = std::sync::mpsc::sync_channel(3);
> >> +        let read_threads = datastore
> >> +            .get_thread_configuration()
> >> +            .tape_backup_read_threads;
> >> +
> >> +        let (tx, rx) = std::sync::mpsc::sync_channel(read_threads * 2);
> > 
> > Is there any reason you're using `* 2` here? For example, is the
> > throughput unaffected if you use a larger value, like `* 8`?
> > 
> > If the constant has an effect like that it should IMO be documented, but
> > if not, then it can just stay like it is.
>
> i did not benchmark that much here (would have increased my benchmark time since
> that's another dimension in the testing with many possible values)
>
> but my though process was this:
>
> if the source storage is much slower than what we can write to tape, that
> buffer will be empty most of the time
>
> if the source storage is much faster than what we can write to tape, it
> will be full most of the time
>
> in both cases the buffer size won't matter much (as long as there is a bit of
> buffer)
>
> the sweet spot is when the storage is about as fast as the tape writes
> and in that case you want to be able to have just a bit more buffer
> than threads reading from the storage so there is always a read
> going on (so the buffer will never empty, since that slows
> down the tape write)
>
> so 1*number of threads seemed to low, but i think anything bigger
> than 2* is just a waste of memory
>
> does that make sense?
>
> if we really want, i can make additional benchmarks but my guess
> is that it won't make much difference in practice

Okay that makes a lot of sense, that has cleared everything up for me.
Thanks!

I don't think this needs to be changed at all, then. LGTM



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


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

* Re: [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option
  2024-05-07  7:29 ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Dominik Csapak
  2024-05-07 15:10   ` Max Carrara
@ 2024-05-08 17:47   ` Max Carrara
  1 sibling, 0 replies; 8+ messages in thread
From: Max Carrara @ 2024-05-08 17:47 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion

Finally managed to test this; will summarize all my comments in this
reply to make it easier to keep track.

There are only a few minor rewords in the commit message that I suggest
(see my comments inline); otherwise, this patch LGTM. Good work!

Testing
-------

* Set up a fresh Debian Bookworm VM and configured a VTL as per our
  docs [0]
  * The docs *mostly* work for Bookworm btw, just had to manually import
    a bunch of device definitions from text files from the Quadstor docs
    in the UI [1]. (Thanks for the help, Sterzy!)
* Eventually managed to add the VTL to my virtualized PBS
* Configured everything on PBS (changers, drives, media pools, etc.)
  * Had to get a little more comfortable with it first, because I hadn't
    had the time to play around with tapes yet.
* Ran a backup without the patch first as a comparison
* Applied the patches & rebuilt PBS
* Ran some backups again, didn't really notice any difference in terms
  of speed
  * That might just be because of my configuration or perhaps I missed a
    step (tape is more... elaborate than I thought); once I'm back from
    my holidays, we can have a look at that if you want
  * Then again, most backups ran just fine, so see this as a passed
    smoke test

Note: *Sometimes* the backup would sporadically fail with this message:

TASK ERROR: write filemark  failed - scsi command failed: transport error

I'm assuming that this is probably also related to my configuration;
after starting a failed backup job again, it just magically worked.

To conclude, I'm not sure if the above testing provides any additional
insights, but the patch at the very least doesn't negatively impact
anything on my side, at least nothing that I've noticed.

I will probably test this some more once I'm back in the office, just to
really get a good grasp of everything regarding tapes.


Review
------

As mentioned in my initial response, the code's absolutely fine and the
changes are easy to follow. See the two comments regarding the commit
message inline; the suggested changes there could IMO just be made when
applying the patch.

Nice work!

Reviewed-by: Max Carrara <m.carrara@proxmox.com>
Tested-by: Max Carrara <m.carrara@proxmox.com>

[0]: https://pbs.proxmox.com/wiki/index.php/Installing_a_Virtual_Tape_Library
[1]: https://www.quadstor.com/vtlsupport/configuring-device-definitions.html

On Tue May 7, 2024 at 9:29 AM CEST, Dominik Csapak wrote:
> using a single thread for reading is not optimal in some cases, e.g.
> when the underlying storage can handle more reads in parallel than with
> a single thread.

Would prefer to reword this as:

Using a single thread for reading is not optimal in some cases, e.g.
when the underlying storage can handle reads from multiple threads in
parallel.

>
> We use the ParallelHandler to handle the actual reads. Make the
> sync_channel buffer size depending on the number of threads so we have
> space for two chunks per thread.

Would s/depending/depend/ for the above paragraph.

>
> How this impacts the backup speed largely depends on the underlying
> storage and how the backup is laid out on it.
>
> I benchmarked the following setups:
>
> * Setup A: relatively spread out backup on a virtualized pbs on single HDDs
> * Setup B: mostly sequential chunks on a virtualized pbs on single HDDs
> * Setup C: backup on virtualized pbs on a fast NVME
> * Setup D: backup on bare metal pbs with ZFS in a RAID10 with 6 HDDs
>   and 2 fast special devices in a mirror
>
> (values are reported in MB/s as seen in the task log, caches were
> cleared between runs, backups were bigger than the memory available)
>
> setup  1 thread  2 threads  4 threads  8 threads
> A      55        70         80         95
> B      110       89         100        108
> C      294       294        294        294
> D      118       180        300        300
>
> So there are cases where multiple read threads speed up the tape backup
> (dramatically). On the other hand there are situations where reading
> from a single thread is actually faster, probably because we can read
> from the HDD sequentially.
>
> I used a new default value of '4' here since that gave good performance
> on all setups (not necessarily the best) and we also have a default
> value of '4' for verification threads.
>
> We use a struct here for the datastore since we want to introduce other
> thread tuning options too.
>
> Signed-off-by: Dominik Csapak <d.csapak@proxmox.com>
> ---
>  pbs-api-types/src/datastore.rs              |  8 ++++
>  pbs-datastore/src/datastore.rs              | 26 ++++++++++++
>  src/tape/pool_writer/new_chunks_iterator.rs | 45 +++++++++++++--------
>  www/Utils.js                                |  5 +++
>  www/datastore/OptionView.js                 |  8 ++++
>  5 files changed, 76 insertions(+), 16 deletions(-)
>
> diff --git a/pbs-api-types/src/datastore.rs b/pbs-api-types/src/datastore.rs
> index 31767417a..1dae3867f 100644
> --- a/pbs-api-types/src/datastore.rs
> +++ b/pbs-api-types/src/datastore.rs
> @@ -209,6 +209,11 @@ pub enum DatastoreFSyncLevel {
>              type: ChunkOrder,
>              optional: true,
>          },
> +        "tape-backup-read-threads": {
> +            type: usize,
> +            optional: true,
> +            minimum: 1,
> +        },
>      },
>  )]
>  #[derive(Serialize, Deserialize, Default)]
> @@ -220,6 +225,9 @@ pub struct DatastoreTuning {
>      pub chunk_order: Option<ChunkOrder>,
>      #[serde(skip_serializing_if = "Option::is_none")]
>      pub sync_level: Option<DatastoreFSyncLevel>,
> +    #[serde(skip_serializing_if = "Option::is_none")]
> +    /// Configures how many threads to use to read from the datastore while backing up to tape.
> +    pub tape_backup_read_threads: Option<usize>,
>  }
>  
>  pub const DATASTORE_TUNING_STRING_SCHEMA: Schema = StringSchema::new("Datastore tuning options")
> diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
> index f95da7615..73a1cfa39 100644
> --- a/pbs-datastore/src/datastore.rs
> +++ b/pbs-datastore/src/datastore.rs
> @@ -49,6 +49,19 @@ pub fn check_backup_owner(owner: &Authid, auth_id: &Authid) -> Result<(), Error>
>      Ok(())
>  }
>  
> +/// Contains the configuration of how many threads to use in various situations
> +pub struct ThreadConfiguration {
> +    pub tape_backup_read_threads: usize,
> +}
> +
> +impl Default for ThreadConfiguration {
> +    fn default() -> Self {
> +        Self {
> +            tape_backup_read_threads: 4,
> +        }
> +    }
> +}
> +
>  /// Datastore Management
>  ///
>  /// A Datastore can store severals backups, and provides the
> @@ -61,6 +74,7 @@ pub struct DataStoreImpl {
>      chunk_order: ChunkOrder,
>      last_digest: Option<[u8; 32]>,
>      sync_level: DatastoreFSyncLevel,
> +    thread_config: ThreadConfiguration,
>  }
>  
>  impl DataStoreImpl {
> @@ -75,6 +89,7 @@ impl DataStoreImpl {
>              chunk_order: Default::default(),
>              last_digest: None,
>              sync_level: Default::default(),
> +            thread_config: Default::default(),
>          })
>      }
>  }
> @@ -305,6 +320,11 @@ impl DataStore {
>                  .parse_property_string(config.tuning.as_deref().unwrap_or(""))?,
>          )?;
>  
> +        let mut thread_config = ThreadConfiguration::default();
> +        if let Some(value) = tuning.tape_backup_read_threads {
> +            thread_config.tape_backup_read_threads = value;
> +        }
> +
>          Ok(DataStoreImpl {
>              chunk_store,
>              gc_mutex: Mutex::new(()),
> @@ -313,6 +333,7 @@ impl DataStore {
>              chunk_order: tuning.chunk_order.unwrap_or_default(),
>              last_digest,
>              sync_level: tuning.sync_level.unwrap_or_default(),
> +            thread_config,
>          })
>      }
>  
> @@ -1377,6 +1398,11 @@ impl DataStore {
>          Ok(())
>      }
>  
> +    /// returns the datatstore thread configuration
> +    pub fn get_thread_configuration(&self) -> &ThreadConfiguration {
> +        &self.inner.thread_config
> +    }
> +
>      /// Destroy a datastore. This requires that there are no active operations on the datastore.
>      ///
>      /// This is a synchronous operation and should be run in a worker-thread.
> diff --git a/src/tape/pool_writer/new_chunks_iterator.rs b/src/tape/pool_writer/new_chunks_iterator.rs
> index 1454b33d2..a2a8091f6 100644
> --- a/src/tape/pool_writer/new_chunks_iterator.rs
> +++ b/src/tape/pool_writer/new_chunks_iterator.rs
> @@ -6,8 +6,9 @@ use anyhow::{format_err, Error};
>  use pbs_datastore::{DataBlob, DataStore, SnapshotReader};
>  
>  use crate::tape::CatalogSet;
> +use crate::tools::parallel_handler::ParallelHandler;
>  
> -/// Chunk iterator which use a separate thread to read chunks
> +/// Chunk iterator which uses separate threads to read chunks
>  ///
>  /// The iterator skips duplicate chunks and chunks already in the
>  /// catalog.
> @@ -25,7 +26,11 @@ impl NewChunksIterator {
>          snapshot_reader: Arc<Mutex<SnapshotReader>>,
>          catalog_set: Arc<Mutex<CatalogSet>>,
>      ) -> Result<(std::thread::JoinHandle<()>, Self), Error> {
> -        let (tx, rx) = std::sync::mpsc::sync_channel(3);
> +        let read_threads = datastore
> +            .get_thread_configuration()
> +            .tape_backup_read_threads;
> +
> +        let (tx, rx) = std::sync::mpsc::sync_channel(read_threads * 2);
>  
>          let reader_thread = std::thread::spawn(move || {
>              let snapshot_reader = snapshot_reader.lock().unwrap();
> @@ -35,36 +40,44 @@ impl NewChunksIterator {
>              let datastore_name = snapshot_reader.datastore_name().to_string();
>  
>              let result: Result<(), Error> = proxmox_lang::try_block!({
> -                let mut chunk_iter = snapshot_reader.chunk_iterator(move |digest| {
> +                let chunk_iter = snapshot_reader.chunk_iterator(move |digest| {
>                      catalog_set
>                          .lock()
>                          .unwrap()
>                          .contains_chunk(&datastore_name, digest)
>                  })?;
>  
> -                loop {
> -                    let digest = match chunk_iter.next() {
> -                        None => {
> -                            let _ = tx.send(Ok(None)); // ignore send error
> -                            break;
> +                let reader_pool =
> +                    ParallelHandler::new("tape backup chunk reader pool", read_threads, {
> +                        let tx = tx.clone();
> +                        move |digest| {
> +                            let blob = datastore.load_chunk(&digest)?;
> +                            //println!("LOAD CHUNK {}", hex::encode(&digest));
> +
> +                            tx.send(Ok(Some((digest, blob)))).map_err(|err| {
> +                                format_err!("error sending result from reader thread: {err}")
> +                            })?;
> +
> +                            Ok(())
>                          }
> -                        Some(digest) => digest?,
> -                    };
> +                    });
> +
> +                for digest in chunk_iter {
> +                    let digest = digest?;
>  
>                      if chunk_index.contains(&digest) {
>                          continue;
>                      }
>  
> -                    let blob = datastore.load_chunk(&digest)?;
> -                    //println!("LOAD CHUNK {}", hex::encode(&digest));
> -                    if let Err(err) = tx.send(Ok(Some((digest, blob)))) {
> -                        eprintln!("could not send chunk to reader thread: {err}");
> -                        break;
> -                    }
> +                    reader_pool.send(digest)?;
>  
>                      chunk_index.insert(digest);
>                  }
>  
> +                reader_pool.complete()?;
> +
> +                let _ = tx.send(Ok(None)); // ignore send error
> +
>                  Ok(())
>              });
>              if let Err(err) = result {
> diff --git a/www/Utils.js b/www/Utils.js
> index 1d7351a32..4d224cd4a 100644
> --- a/www/Utils.js
> +++ b/www/Utils.js
> @@ -790,6 +790,11 @@ Ext.define('PBS.Utils', {
>  	sync = PBS.Utils.tuningOptions['sync-level'][sync ?? '__default__'];
>  	options.push(`${gettext('Sync Level')}: ${sync}`);
>  
> +	let tapeBackupRT = tuning['tape-backup-read-threads'];
> +	delete tuning['tape-backup-read-threads'];
> +	tapeBackupRT ??= Proxmox.Utils.defaultText + ` (4)`;
> +	options.push(`${gettext('Tape Backup Read Threads')}: ${tapeBackupRT}`);
> +
>  	for (const [k, v] of Object.entries(tuning)) {
>  	    options.push(`${k}: ${v}`);
>  	}
> diff --git a/www/datastore/OptionView.js b/www/datastore/OptionView.js
> index e1f38af6f..cfbb89151 100644
> --- a/www/datastore/OptionView.js
> +++ b/www/datastore/OptionView.js
> @@ -271,6 +271,14 @@ Ext.define('PBS.Datastore.Options', {
>  			    deleteEmpty: true,
>  			    value: '__default__',
>  			},
> +			{
> +			    xtype: 'proxmoxintegerfield',
> +			    name: 'tape-backup-read-threads',
> +			    fieldLabel: gettext('Tape Backup Read Threads'),
> +			    min: 1,
> +			    emptyText: '4',
> +			    deleteEmpty: true,
> +			},
>  		    ],
>  		},
>  	    },



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


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

end of thread, other threads:[~2024-05-08 17:48 UTC | newest]

Thread overview: 8+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2024-05-07  7:29 [pbs-devel] [PATCH proxmox-backup v2 0/3] tape/verify: improve multithreaded Dominik Csapak
2024-05-07  7:29 ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Dominik Csapak
2024-05-07 15:10   ` Max Carrara
2024-05-08  6:56     ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning opti Dominik Csapak
2024-05-08 13:37       ` Max Carrara
2024-05-08 17:47   ` [pbs-devel] [PATCH proxmox-backup v2 1/3] tape: introduce a tape backup read thread tuning option Max Carrara
2024-05-07  7:29 ` [pbs-devel] [RFC PATCH proxmox-backup v2 2/3] verify: add tuning option for number of threads to use Dominik Csapak
2024-05-07  7:29 ` [pbs-devel] [RFC PATCH proxmox-backup v2 3/3] verify: move chunk loading into the worker threads Dominik Csapak

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