all lists on lists.proxmox.com
 help / color / mirror / Atom feed
* [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path
@ 2025-11-11 12:29 Samuel Rufinatscha
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
                   ` (4 more replies)
  0 siblings, 5 replies; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-11 12:29 UTC (permalink / raw)
  To: pbs-devel

Hi,

this series reduces CPU time in datastore lookups by avoiding repeated
datastore.cfg reads/parses in both `lookup_datastore()` and
`DataStore::Drop`. It also adds a TTL so manual config edits are
noticed without reintroducing hashing on every request.

While investigating #6049 [1], cargo-flamegraph [2] showed hotspots during
repeated `/status` calls in `lookup_datastore()` and in `Drop`,
dominated by `pbs_config::datastore::config()` (config parse).

The parsing cost itself should eventually be investigated in a future
effort. Furthermore, cargo-flamegraph showed that when using a
token-based auth method to access the API, a significant amount of time
is spent in validation on every request, likely related to bcrypt.
Also this should be eventually revisited in a future effort.

## Approach

[PATCH 1/3] Fast path for datastore lookups
Use the shared-memory `ConfigVersionCache` generation for `datastore.cfg`.
Tag each cached `DataStoreImpl` with the last seen generation; when it
matches, reuse the cached instance. Fall back to the existing slow path
on mismatch or when the cache is unavailable.

[PATCH 2/3] Fast path for `Drop`
Reuse the maintenance mode eviction decision captured at lookup time,
removing the config reload from `Drop`.

[PATCH 3/3] TTL to catch manual edits
If a cached entry is older than `DATASTORE_CONFIG_CACHE_TTL_SECS`
(default 60s), the next lookup refreshes it via the slow path. This
detects manual file edits without hashing on every request.

## Results

End-to-end `/status?verbose=0` (1000 stores, 5 req/store, parallel=16):

  Metric                 Baseline   [1/3]    [2/3]
  ------------------------------------------------
  Total time             13s        11s      10s
  Throughput (all)       384.62     454.55   500.00
  Cold RPS (round #1)    76.92      90.91    100.00
  Warm RPS (2..N)        307.69     363.64   400.00

Patch 1 improves overall throughput by ~18% (−15% total time). Patch 2
adds ~10% on top. Patch 3 is a robustness feature; a 0.1 s probe shows
periodic latency spikes at TTL expiry and flat latencies otherwise.

## Reproduction steps

VM: 4 vCPU, ~8 GiB RAM, VirtIO-SCSI; disks:
  - scsi0 32G (OS)
  - scsi1 1000G (datastores)

Install PBS from ISO on the VM.

Set up ZFS on /dev/sdb (adjust if different):

    zpool create -f -o ashift=12 pbsbench /dev/sdb
    zfs set mountpoint=/pbsbench pbsbench
    zfs create pbsbench/pbs-bench

Raise file-descriptor limit:

    sudo systemctl edit proxmox-backup-proxy.service

Add the following lines:

    [Service]
    LimitNOFILE=1048576

Reload systemd and restart the proxy:

    sudo systemctl daemon-reload
    sudo systemctl restart proxmox-backup-proxy.service

Verify the limit:

    systemctl show proxmox-backup-proxy.service | grep LimitNOFILE

Create 1000 ZFS-backed datastores (as used in #6049 [1]):

    seq -w 001 1000 | xargs -n1 -P1 bash -c '
      id=$0
      name="ds${id}"
      dataset="pbsbench/pbs-bench/${name}"
      path="/pbsbench/pbs-bench/${name}"
      zfs create -o mountpoint="$path" "$dataset"
      proxmox-backup-manager datastore create "$name" "$path" \
        --comment "ZFS dataset-based datastore"
    '

Build PBS from this series, then run the server under manually
under flamegraph:

    systemctl stop proxmox-backup-proxy
    cargo flamegraph --release --bin proxmox-backup-proxy

Benchmark script (`bench.sh`) used for the numbers above:

    #!/usr/bin/env bash
    set -euo pipefail

    # --- Config ---------------------------------------------------------------
    HOST='https://localhost:8007'
    USER='root@pam'
    PASS="$(cat passfile)"

    DATASTORE_PATH="/pbsbench/pbs-bench"
    MAX_STORES=1000      # how many stores to include
    PARALLEL=16         # concurrent workers
    REPEAT=5            # requests per store (1 cold + REPEAT-1 warm)

    PRINT_FIRST=false   # true => log first request's HTTP code per store

    # --- Helpers --------------------------------------------------------------
    fmt_rps () {
      local n="$1" t="$2"
      awk -v n="$n" -v t="$t" 'BEGIN { if (t > 0) printf("%.2f\n", n/t); else print "0.00" }'
    }

    # --- Login ---------------------------------------------------------------
    auth=$(curl -ks -X POST "$HOST/api2/json/access/ticket" \
      -d "username=$USER" -d "password=$PASS")
    ticket=$(echo "$auth" | jq -r '.data.ticket')

    if [[ -z "${ticket:-}" || "$ticket" == "null" ]]; then
      echo "[ERROR] Login failed (no ticket)"
      exit 1
    fi

    # --- Collect stores (deterministic order) --------------------------------
    mapfile -t STORES < <(
      find "$DATASTORE_PATH" -mindepth 1 -maxdepth 1 -type d -printf '%f\n' \
      | sort | head -n "$MAX_STORES"
    )

    USED_STORES=${#STORES[@]}
    if (( USED_STORES == 0 )); then
      echo "[ERROR] No datastore dirs under $DATASTORE_PATH"
      exit 1
    fi

    echo "[INFO] Running with stores=$USED_STORES, repeat=$REPEAT, parallel=$PARALLEL"

    # --- Temp counters --------------------------------------------------------
    SUCCESS_ALL="$(mktemp)"
    FAIL_ALL="$(mktemp)"
    COLD_OK="$(mktemp)"
    WARM_OK="$(mktemp)"
    trap 'rm -f "$SUCCESS_ALL" "$FAIL_ALL" "$COLD_OK" "$WARM_OK"' EXIT

    export HOST ticket REPEAT SUCCESS_ALL FAIL_ALL COLD_OK WARM_OK PRINT_FIRST

    SECONDS=0

    # --- Fire requests --------------------------------------------------------
    printf "%s\n" "${STORES[@]}" \
    | xargs -P"$PARALLEL" -I{} bash -c '
      store="$1"
      url="$HOST/api2/json/admin/datastore/$store/status?verbose=0"

      for ((i=1;i<=REPEAT;i++)); do
        code=$(curl -ks -o /dev/null -w "%{http_code}" -b "PBSAuthCookie=$ticket" "$url" || echo 000)

        if [[ "$code" == "200" ]]; then
          echo 1 >> "$SUCCESS_ALL"
          if (( i == 1 )); then
            echo 1 >> "$COLD_OK"
          else
            echo 1 >> "$WARM_OK"
          fi
          if [[ "$PRINT_FIRST" == "true" && $i -eq 1 ]]; then
            ts=$(date +%H:%M:%S)
            echo "[$ts] $store #$i HTTP:200"
          fi
        else
          echo 1 >> "$FAIL_ALL"
          if [[ "$PRINT_FIRST" == "true" && $i -eq 1 ]]; then
            ts=$(date +%H:%M:%S)
            echo "[$ts] $store #$i HTTP:$code (FAIL)"
          fi
        fi
      done
    ' _ {}

    # --- Summary --------------------------------------------------------------
    elapsed=$SECONDS
    ok=$(wc -l < "$SUCCESS_ALL" 2>/dev/null || echo 0)
    fail=$(wc -l < "$FAIL_ALL" 2>/dev/null || echo 0)
    cold_ok=$(wc -l < "$COLD_OK" 2>/dev/null || echo 0)
    warm_ok=$(wc -l < "$WARM_OK" 2>/dev/null || echo 0)

    expected=$(( USED_STORES * REPEAT ))
    total=$(( ok + fail ))

    rps_all=$(fmt_rps "$ok" "$elapsed")
    rps_cold=$(fmt_rps "$cold_ok" "$elapsed")
    rps_warm=$(fmt_rps "$warm_ok" "$elapsed")

    echo "===== Summary ====="
    echo "Stores used:         $USED_STORES"
    echo "Expected requests:   $expected"
    echo "Executed requests:   $total"
    echo "OK (HTTP 200):       $ok"
    echo "Failed:              $fail"
    printf "Total time:          %dm %ds\n" $((elapsed/60)) $((elapsed%60))
    echo "Throughput all RPS:  $rps_all"
    echo "Cold RPS (round #1): $rps_cold"
    echo "Warm RPS (#2..N):    $rps_warm"

## Maintainer notes

- No dependency bumps, no API changes, no breaking changes in this
  series.

## Patch summary

[PATCH 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups
[PATCH 2/3] partial fix #6049: datastore: use config fast-path in Drop
[PATCH 3/3] datastore: add TTL fallback to catch manual config edits

Thanks for reviewing!

[1] Bugzilla #6049: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
[2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph

Samuel Rufinatscha (3):
  partial fix #6049: datastore: impl ConfigVersionCache fast path for
    lookups
  partial fix #6049: datastore: use config fast-path in Drop
  datastore: add TTL fallback to catch manual config edits

 pbs-config/src/config_version_cache.rs |  10 ++-
 pbs-datastore/src/datastore.rs         | 119 ++++++++++++++++++-------
 2 files changed, 96 insertions(+), 33 deletions(-)

-- 
2.47.3



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

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

* [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups
  2025-11-11 12:29 [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Samuel Rufinatscha
@ 2025-11-11 12:29 ` Samuel Rufinatscha
  2025-11-12 13:24   ` Fabian Grünbichler
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-11 12:29 UTC (permalink / raw)
  To: pbs-devel

Repeated /status requests caused lookup_datastore() to re-read and
parse datastore.cfg on every call. The issue was mentioned in report
#6049 [1]. cargo-flamegraph [2] confirmed that the hot path is
dominated by pbs_config::datastore::config() (config parsing).

This patch adds a fast path to lookup_datastore() using the shared-
memory ConfigVersionCache generation counter for datastore.cfg. It
stores the last seen generation number alongside the cached
DataStoreImpl and, when a subsequent lookup sees the same generation,
we reuse the cached instance without re-reading the on-disk config. If
the generation differs (or the cache is unavailable), it falls back to
the existing slow path with no behavioral changes.

Behavioral notes

- The generation is bumped via the existing save_config() path, so
  API-driven config changes are detected immediately.
- Manual edits to datastore.cfg are not detected by this commit; a TTL
  guard is introduced in a later patch to cover that case.
- DataStore::drop still performs a config read on the common path; this
  is addressed in the next patch.

Testing

cargo-flamegraph confirms the config-parse hotspot in
lookup_datastore() disappears from the hot path.

Additionally, an end-to-end benchmark was performed using the
`/status?verbose=0` API with 1000 datastores, 5 requests per store,
and 16-way parallelism:

| Metric                   | Before | After  | Δ (abs) | Δ (%)   |
|--------------------------|:------:|:------:|:-------:|:-------:|
| Total time               | 13s    | 11s    | −2s     | −15.38% |
| Throughput (all rounds)  | 384.62 | 454.55 | +69.93  | +18.18% |
| Cold RPS (round #1)      | 76.92  | 90.91  | +13.99  | +18.19% |
| Warm RPS (rounds 2..N)   | 307.69 | 363.64 | +55.95  | +18.18% |

Throughput improved by ~18% overall, with total time reduced by ~15%.
Warm lookups now reuse cached datastore instances and skip redundant
config parsing entirely. The first-access round also shows a similar
improvement, likely due to reduced contention and I/O when many stores
are accessed in parallel.

Note: A second hotspot remains in Drop where a config read occurs; that
is addressed in the next commit.

Links

[1] Bugzilla: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
[2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph

Fixes: #6049
Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
---
 pbs-config/src/config_version_cache.rs | 10 +++-
 pbs-datastore/src/datastore.rs         | 77 +++++++++++++++++---------
 2 files changed, 59 insertions(+), 28 deletions(-)

diff --git a/pbs-config/src/config_version_cache.rs b/pbs-config/src/config_version_cache.rs
index e8fb994f..b875f7e0 100644
--- a/pbs-config/src/config_version_cache.rs
+++ b/pbs-config/src/config_version_cache.rs
@@ -26,7 +26,6 @@ struct ConfigVersionCacheDataInner {
     // Traffic control (traffic-control.cfg) generation/version.
     traffic_control_generation: AtomicUsize,
     // datastore (datastore.cfg) generation/version
-    // FIXME: remove with PBS 3.0
     datastore_generation: AtomicUsize,
     // Add further atomics here
 }
@@ -145,8 +144,15 @@ impl ConfigVersionCache {
             .fetch_add(1, Ordering::AcqRel);
     }
 
+    /// Returns the datastore generation number.
+    pub fn datastore_generation(&self) -> usize {
+        self.shmem
+            .data()
+            .datastore_generation
+            .load(Ordering::Acquire)
+    }
+
     /// Increase the datastore generation number.
-    // FIXME: remove with PBS 3.0 or make actually useful again in datastore lookup
     pub fn increase_datastore_generation(&self) -> usize {
         self.shmem
             .data()
diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index 70af94d8..18eebb58 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -32,7 +32,7 @@ use pbs_api_types::{
     MaintenanceType, Operation, UPID,
 };
 use pbs_config::s3::S3_CFG_TYPE_ID;
-use pbs_config::BackupLockGuard;
+use pbs_config::{BackupLockGuard, ConfigVersionCache};
 
 use crate::backup_info::{
     BackupDir, BackupGroup, BackupInfo, OLD_LOCKING, PROTECTED_MARKER_FILENAME,
@@ -140,10 +140,10 @@ pub struct DataStoreImpl {
     last_gc_status: Mutex<GarbageCollectionStatus>,
     verify_new: bool,
     chunk_order: ChunkOrder,
-    last_digest: Option<[u8; 32]>,
     sync_level: DatastoreFSyncLevel,
     backend_config: DatastoreBackendConfig,
     lru_store_caching: Option<LocalDatastoreLruCache>,
+    cached_config_tag: Option<CachedDatastoreConfigTag>,
 }
 
 impl DataStoreImpl {
@@ -156,10 +156,10 @@ impl DataStoreImpl {
             last_gc_status: Mutex::new(GarbageCollectionStatus::default()),
             verify_new: false,
             chunk_order: Default::default(),
-            last_digest: None,
             sync_level: Default::default(),
             backend_config: Default::default(),
             lru_store_caching: None,
+            cached_config_tag: None,
         })
     }
 }
@@ -224,6 +224,15 @@ pub enum DatastoreBackend {
     S3(Arc<S3Client>),
 }
 
+/// Used to determine whether a cached datastore instance is still valid
+/// or needs to be reloaded after a config change.
+struct CachedDatastoreConfigTag {
+    /// Maintenance mode at the time the lookup hint was captured, if any.
+    last_maintenance_mode: Option<MaintenanceMode>,
+    /// Datastore generation number from `ConfigVersionCache`; `None` when the cache wasn't available.
+    last_generation: Option<usize>,
+}
+
 impl DataStore {
     // This one just panics on everything
     #[doc(hidden)]
@@ -299,13 +308,40 @@ impl DataStore {
         // we use it to decide whether it is okay to delete the datastore.
         let _config_lock = pbs_config::datastore::lock_config()?;
 
-        // we could use the ConfigVersionCache's generation for staleness detection, but  we load
-        // the config anyway -> just use digest, additional benefit: manual changes get detected
-        let (config, digest) = pbs_config::datastore::config()?;
+        // Get the current datastore.cfg generation number
+        let gen_num = ConfigVersionCache::new()
+            .ok()
+            .map(|c| c.datastore_generation());
+
+        // Fast-path: if we have a cached entry created under the same datastore.cfg generation number, reuse it.
+        if let (Some(gen_num), Some(ds)) =
+            (gen_num, DATASTORE_MAP.lock().unwrap().get(name).cloned())
+        {
+            if let Some(cached_tag) = &ds.cached_config_tag {
+                if cached_tag.last_generation == Some(gen_num) {
+                    if let Some(mm) = &cached_tag.last_maintenance_mode {
+                        if let Err(error) = mm.check(operation) {
+                            bail!("datastore '{name}' is unavailable: {error}");
+                        }
+                    }
+                    if let Some(operation) = operation {
+                        update_active_operations(name, operation, 1)?;
+                    }
+                    return Ok(Arc::new(Self {
+                        inner: ds,
+                        operation,
+                    }));
+                }
+            }
+        }
+
+        // Slow path: (re)load config
+        let (config, _digest) = pbs_config::datastore::config()?;
         let config: DataStoreConfig = config.lookup("datastore", name)?;
 
-        if let Some(maintenance_mode) = config.get_maintenance_mode() {
-            if let Err(error) = maintenance_mode.check(operation) {
+        let maintenance_mode = config.get_maintenance_mode();
+        if let Some(mm) = &maintenance_mode {
+            if let Err(error) = mm.check(operation) {
                 bail!("datastore '{name}' is unavailable: {error}");
             }
         }
@@ -321,16 +357,6 @@ impl DataStore {
 
         // reuse chunk store so that we keep using the same process locker instance!
         let chunk_store = if let Some(datastore) = &entry {
-            let last_digest = datastore.last_digest.as_ref();
-            if let Some(true) = last_digest.map(|last_digest| last_digest == &digest) {
-                if let Some(operation) = operation {
-                    update_active_operations(name, operation, 1)?;
-                }
-                return Ok(Arc::new(Self {
-                    inner: Arc::clone(datastore),
-                    operation,
-                }));
-            }
             Arc::clone(&datastore.chunk_store)
         } else {
             let tuning: DatastoreTuning = serde_json::from_value(
@@ -344,7 +370,11 @@ impl DataStore {
             )?)
         };
 
-        let datastore = DataStore::with_store_and_config(chunk_store, config, Some(digest))?;
+        let mut datastore = DataStore::with_store_and_config(chunk_store, config)?;
+        datastore.cached_config_tag = Some(CachedDatastoreConfigTag {
+            last_maintenance_mode: maintenance_mode,
+            last_generation: gen_num,
+        });
 
         let datastore = Arc::new(datastore);
         datastore_cache.insert(name.to_string(), datastore.clone());
@@ -430,11 +460,7 @@ impl DataStore {
             config.absolute_path(),
             tuning.sync_level.unwrap_or_default(),
         )?;
-        let inner = Arc::new(Self::with_store_and_config(
-            Arc::new(chunk_store),
-            config,
-            None,
-        )?);
+        let inner = Arc::new(Self::with_store_and_config(Arc::new(chunk_store), config)?);
 
         if let Some(operation) = operation {
             update_active_operations(&name, operation, 1)?;
@@ -446,7 +472,6 @@ impl DataStore {
     fn with_store_and_config(
         chunk_store: Arc<ChunkStore>,
         config: DataStoreConfig,
-        last_digest: Option<[u8; 32]>,
     ) -> Result<DataStoreImpl, Error> {
         let mut gc_status_path = chunk_store.base_path();
         gc_status_path.push(".gc-status");
@@ -506,10 +531,10 @@ impl DataStore {
             last_gc_status: Mutex::new(gc_status),
             verify_new: config.verify_new.unwrap_or(false),
             chunk_order: tuning.chunk_order.unwrap_or_default(),
-            last_digest,
             sync_level: tuning.sync_level.unwrap_or_default(),
             backend_config,
             lru_store_caching,
+            cached_config_tag: None,
         })
     }
 
-- 
2.47.3



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

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

* [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop
  2025-11-11 12:29 [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Samuel Rufinatscha
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
@ 2025-11-11 12:29 ` Samuel Rufinatscha
  2025-11-12 11:24   ` Fabian Grünbichler
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 3/3] datastore: add TTL fallback to catch manual config edits Samuel Rufinatscha
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-11 12:29 UTC (permalink / raw)
  To: pbs-devel

The Drop impl of DataStore re-read datastore.cfg to decide whether
the entry should be evicted from the in-process cache (based on
maintenance mode’s clear_from_cache). During the investigation of
issue #6049 [1], a flamegraph [2] showed that the config reload in Drop
accounted for a measurable share of CPU time under load.

This patch makes Drop O(1) on the fast path by reusing the maintenance-
mode decision captured at lookup time and stored with the cached
datastore entry. When the last reference goes away we:
- decrement active-operation counters, and
- evict only if the cached decision mandates eviction.

If the cache tag is absent or not fresh, a subsequent slow-path lookup
will be performed.

Testing

Compared flamegraphs before and after: prior to this change
(on top of patch 1), stacks originating from Drop included
pbs_config::datastore::config(). After the change, those vanish from
the drop path.

An end-to-end benchmark using `/status?verbose=0` with 1000 datastores,
5 requests per store, and 16-way parallelism shows a further
improvement:

| Metric                  | After commit 1 | After commit 2 | Δ (abs) | Δ (%)   |
|-------------------------|:--------------:|:--------------:|:-------:|:-------:|
| Total time              | 11s            | 10s            | −1s     | −9.09%  |
| Throughput (all rounds) | 454.55         | 500.00         | +45.45  | +10.00% |
| Cold RPS (round #1)     | 90.91          | 100.00         | +9.09   | +10.00% |
| Warm RPS (rounds 2..N)  | 363.64         | 400.00         | +36.36  | +10.00% |

Optimizing Drop improves overall throughput by ~10%. The gain appears
in both cold and warm rounds, and the flamegraph confirms the config
reload no longer sits on the hot path.

Links

[1] Bugzilla: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
[2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph

Fixes: #6049
Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
---
 pbs-datastore/src/datastore.rs | 31 +++++++++++++++++++++++++++----
 1 file changed, 27 insertions(+), 4 deletions(-)

diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index 18eebb58..da80416a 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -200,15 +200,38 @@ impl Drop for DataStore {
             // remove datastore from cache iff
             //  - last task finished, and
             //  - datastore is in a maintenance mode that mandates it
-            let remove_from_cache = last_task
-                && pbs_config::datastore::config()
+
+            // first check: check if last task finished
+            if !last_task {
+                return;
+            }
+
+            let cached_tag = self.inner.cached_config_tag.as_ref();
+            let last_gen_num = cached_tag.and_then(|c| c.last_generation);
+            let gen_num = ConfigVersionCache::new()
+                .ok()
+                .map(|c| c.datastore_generation());
+
+            let cache_is_fresh = match (last_gen_num, gen_num) {
+                (Some(a), Some(b)) => a == b,
+                _ => false,
+            };
+
+            let mm_mandate = if cache_is_fresh {
+                cached_tag
+                    .and_then(|c| c.last_maintenance_mode.as_ref())
+                    .is_some_and(|m| m.clear_from_cache())
+            } else {
+                pbs_config::datastore::config()
                     .and_then(|(s, _)| s.lookup::<DataStoreConfig>("datastore", self.name()))
                     .is_ok_and(|c| {
                         c.get_maintenance_mode()
                             .is_some_and(|m| m.clear_from_cache())
-                    });
+                    })
+            };
 
-            if remove_from_cache {
+            // second check: check maintenance mode mandate
+            if mm_mandate {
                 DATASTORE_MAP.lock().unwrap().remove(self.name());
             }
         }
-- 
2.47.3



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

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

* [pbs-devel] [PATCH proxmox-backup 3/3] datastore: add TTL fallback to catch manual config edits
  2025-11-11 12:29 [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Samuel Rufinatscha
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
@ 2025-11-11 12:29 ` Samuel Rufinatscha
  2025-11-12 11:27 ` [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Fabian Grünbichler
  2025-11-14 15:08 ` [pbs-devel] superseded: " Samuel Rufinatscha
  4 siblings, 0 replies; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-11 12:29 UTC (permalink / raw)
  To: pbs-devel

The lookup fast path reacts to API-driven config changes because
save_config() bumps the generation. Manual edits of datastore.cfg do
not bump the counter. To keep the system robust against such edits
without reintroducing config reading and hashing on the hot path, this
patch adds a TTL to the cache entry.

If the datastore’s cached tag is older than
DATASTORE_CONFIG_CACHE_TTL_SECS (set to 60s), the next lookup takes
the slow path (re-read/parse) and refreshes the cached entry. Within
the TTL window, unchanged generations still use the fast path.

Note: Manual edits may remain unseen until the TTL elapses or any API
config write occurs.

Testing

With the TTL enabled, flamegraphs for hot status requests remain flat. A
0.1 second interval test confirmed periodic latency spikes at TTL expiry.

Maintainer notes

No dependency bumps or breaking changes.

Links

[1] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph

Refs: #6049
Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
---
 pbs-datastore/src/datastore.rs | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)

diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index da80416a..5eaae49b 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -22,7 +22,7 @@ use proxmox_sys::error::SysError;
 use proxmox_sys::fs::{file_read_optional_string, replace_file, CreateOptions};
 use proxmox_sys::linux::procfs::MountInfo;
 use proxmox_sys::process_locker::{ProcessLockExclusiveGuard, ProcessLockSharedGuard};
-use proxmox_time::TimeSpan;
+use proxmox_time::{epoch_i64, TimeSpan};
 use proxmox_worker_task::WorkerTaskContext;
 
 use pbs_api_types::{
@@ -56,6 +56,9 @@ pub const GROUP_OWNER_FILE_NAME: &str = "owner";
 /// Filename for in-use marker stored on S3 object store backend
 pub const S3_DATASTORE_IN_USE_MARKER: &str = ".in-use";
 const NAMESPACE_MARKER_FILENAME: &str = ".namespace";
+/// Max age in seconds to reuse the datastore lookup fast path
+/// before forcing a slow-path config read.
+const DATASTORE_CONFIG_CACHE_TTL_SECS: i64 = 60;
 
 /// checks if auth_id is owner, or, if owner is a token, if
 /// auth_id is the user of the token
@@ -254,6 +257,8 @@ struct CachedDatastoreConfigTag {
     last_maintenance_mode: Option<MaintenanceMode>,
     /// Datastore generation number from `ConfigVersionCache`; `None` when the cache wasn't available.
     last_generation: Option<usize>,
+    /// Epoch seconds when this lookup hint was created.
+    last_update: i64,
 }
 
 impl DataStore {
@@ -335,13 +340,16 @@ impl DataStore {
         let gen_num = ConfigVersionCache::new()
             .ok()
             .map(|c| c.datastore_generation());
+        let now = epoch_i64();
 
         // Fast-path: if we have a cached entry created under the same datastore.cfg generation number, reuse it.
         if let (Some(gen_num), Some(ds)) =
             (gen_num, DATASTORE_MAP.lock().unwrap().get(name).cloned())
         {
             if let Some(cached_tag) = &ds.cached_config_tag {
-                if cached_tag.last_generation == Some(gen_num) {
+                if cached_tag.last_generation == Some(gen_num)
+                    && (now - cached_tag.last_update) < DATASTORE_CONFIG_CACHE_TTL_SECS
+                {
                     if let Some(mm) = &cached_tag.last_maintenance_mode {
                         if let Err(error) = mm.check(operation) {
                             bail!("datastore '{name}' is unavailable: {error}");
@@ -397,6 +405,7 @@ impl DataStore {
         datastore.cached_config_tag = Some(CachedDatastoreConfigTag {
             last_maintenance_mode: maintenance_mode,
             last_generation: gen_num,
+            last_update: now,
         });
 
         let datastore = Arc::new(datastore);
-- 
2.47.3



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

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

* Re: [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
@ 2025-11-12 11:24   ` Fabian Grünbichler
  2025-11-12 15:20     ` Samuel Rufinatscha
  0 siblings, 1 reply; 11+ messages in thread
From: Fabian Grünbichler @ 2025-11-12 11:24 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion

On November 11, 2025 1:29 pm, Samuel Rufinatscha wrote:
> The Drop impl of DataStore re-read datastore.cfg to decide whether
> the entry should be evicted from the in-process cache (based on
> maintenance mode’s clear_from_cache). During the investigation of
> issue #6049 [1], a flamegraph [2] showed that the config reload in Drop
> accounted for a measurable share of CPU time under load.
> 
> This patch makes Drop O(1) on the fast path by reusing the maintenance-

I am not sure what the O(1) is refering to? This patch implements a
faster cache lookup in front of the (slow) config parsing variant, but
that doesn't really align well with what the "Big O" notation tries to
express ;)

The parsing below still scales with the number of datastores in the
config, after all. It can just be skipped sometimes :)

> mode decision captured at lookup time and stored with the cached
> datastore entry. When the last reference goes away we:
> - decrement active-operation counters, and
> - evict only if the cached decision mandates eviction.
> 
> If the cache tag is absent or not fresh, a subsequent slow-path lookup
> will be performed.
> 
> Testing
> 
> Compared flamegraphs before and after: prior to this change
> (on top of patch 1), stacks originating from Drop included
> pbs_config::datastore::config(). After the change, those vanish from
> the drop path.
> 
> An end-to-end benchmark using `/status?verbose=0` with 1000 datastores,
> 5 requests per store, and 16-way parallelism shows a further
> improvement:
> 
> | Metric                  | After commit 1 | After commit 2 | Δ (abs) | Δ (%)   |
> |-------------------------|:--------------:|:--------------:|:-------:|:-------:|
> | Total time              | 11s            | 10s            | −1s     | −9.09%  |
> | Throughput (all rounds) | 454.55         | 500.00         | +45.45  | +10.00% |
> | Cold RPS (round #1)     | 90.91          | 100.00         | +9.09   | +10.00% |
> | Warm RPS (rounds 2..N)  | 363.64         | 400.00         | +36.36  | +10.00% |
> 
> Optimizing Drop improves overall throughput by ~10%. The gain appears
> in both cold and warm rounds, and the flamegraph confirms the config
> reload no longer sits on the hot path.
> 
> Links
> 
> [1] Bugzilla: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
> [2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph
> 
> Fixes: #6049
> Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
> ---
>  pbs-datastore/src/datastore.rs | 31 +++++++++++++++++++++++++++----
>  1 file changed, 27 insertions(+), 4 deletions(-)
> 
> diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
> index 18eebb58..da80416a 100644
> --- a/pbs-datastore/src/datastore.rs
> +++ b/pbs-datastore/src/datastore.rs
> @@ -200,15 +200,38 @@ impl Drop for DataStore {
>              // remove datastore from cache iff
>              //  - last task finished, and
>              //  - datastore is in a maintenance mode that mandates it
> -            let remove_from_cache = last_task
> -                && pbs_config::datastore::config()
> +
> +            // first check: check if last task finished
> +            if !last_task {
> +                return;
> +            }
> +
> +            let cached_tag = self.inner.cached_config_tag.as_ref();
> +            let last_gen_num = cached_tag.and_then(|c| c.last_generation);
> +            let gen_num = ConfigVersionCache::new()
> +                .ok()
> +                .map(|c| c.datastore_generation());
> +
> +            let cache_is_fresh = match (last_gen_num, gen_num) {
> +                (Some(a), Some(b)) => a == b,
> +                _ => false,
> +            };

this is just last_gen_num == gen_num and checking that either is Some.
if we make the tag always contain a generation instead of an option, we
can simplify this code ;)

> +
> +            let mm_mandate = if cache_is_fresh {
> +                cached_tag
> +                    .and_then(|c| c.last_maintenance_mode.as_ref())
> +                    .is_some_and(|m| m.clear_from_cache())
> +            } else {
> +                pbs_config::datastore::config()
>                      .and_then(|(s, _)| s.lookup::<DataStoreConfig>("datastore", self.name()))
>                      .is_ok_and(|c| {
>                          c.get_maintenance_mode()
>                              .is_some_and(|m| m.clear_from_cache())
> -                    });
> +                    })
> +            };
>  
> -            if remove_from_cache {
> +            // second check: check maintenance mode mandate
> +            if mm_mandate {
>                  DATASTORE_MAP.lock().unwrap().remove(self.name());
>              }
>          }
> -- 
> 2.47.3
> 
> 
> 
> _______________________________________________
> pbs-devel mailing list
> pbs-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
> 


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

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

* Re: [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path
  2025-11-11 12:29 [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Samuel Rufinatscha
                   ` (2 preceding siblings ...)
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 3/3] datastore: add TTL fallback to catch manual config edits Samuel Rufinatscha
@ 2025-11-12 11:27 ` Fabian Grünbichler
  2025-11-12 17:27   ` Samuel Rufinatscha
  2025-11-14 15:08 ` [pbs-devel] superseded: " Samuel Rufinatscha
  4 siblings, 1 reply; 11+ messages in thread
From: Fabian Grünbichler @ 2025-11-12 11:27 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion

On November 11, 2025 1:29 pm, Samuel Rufinatscha wrote:
> Hi,
> 
> this series reduces CPU time in datastore lookups by avoiding repeated
> datastore.cfg reads/parses in both `lookup_datastore()` and
> `DataStore::Drop`. It also adds a TTL so manual config edits are
> noticed without reintroducing hashing on every request.
> 
> While investigating #6049 [1], cargo-flamegraph [2] showed hotspots during
> repeated `/status` calls in `lookup_datastore()` and in `Drop`,
> dominated by `pbs_config::datastore::config()` (config parse).
> 
> The parsing cost itself should eventually be investigated in a future
> effort. Furthermore, cargo-flamegraph showed that when using a
> token-based auth method to access the API, a significant amount of time
> is spent in validation on every request, likely related to bcrypt.
> Also this should be eventually revisited in a future effort.

please file a bug for the token part, if there isn't already one!

thanks for diving into this, it already looks promising, even though the
effect on more "normal" systems with more reasonable numbers of
datastores and clients will be less pronounced ;)

the big TL;DR would be that we trade faster datastore lookups (which
happen quite frequently, in particular if there are many datastores with
clients checking their status) against slightly delayed reload of the
configuration in case of manual, behind-our-backs edits, with one
particular corner case that is slightly problematic, but also a bit
contrived:
- datastore is looked up
- config is edited (manually) to set maintenance mode to one that
  requires removing from the datastore map once the last task exits
- last task drops the datastore struct
- no regular edits happened in the meantime
- the Drop handler doesn't know it needs to remove the datastore from
  the map
- open FD is held by proxy, datastore fails to be unmounted/..

we could solve this issue by not only bumping the generation on save,
but also when we reload the config (in particular if we cache the whole
config!). that would make the Drop handler efficient enough for idle
systems that have mostly lookups but no long running tasks. as soon as a
datastore has long running tasks, the last such task will likely exit
long after the TTL for its config lookup has expired, so will need to do
a refresh - although that refresh could again be from the global cache,
instead of from disk? still wouldn't close the window entirely, but make
it pretty unlikely to be hit 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] 11+ messages in thread

* Re: [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups
  2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
@ 2025-11-12 13:24   ` Fabian Grünbichler
  2025-11-13 12:59     ` Samuel Rufinatscha
  0 siblings, 1 reply; 11+ messages in thread
From: Fabian Grünbichler @ 2025-11-12 13:24 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion

On November 11, 2025 1:29 pm, Samuel Rufinatscha wrote:
> Repeated /status requests caused lookup_datastore() to re-read and
> parse datastore.cfg on every call. The issue was mentioned in report
> #6049 [1]. cargo-flamegraph [2] confirmed that the hot path is
> dominated by pbs_config::datastore::config() (config parsing).
> 
> This patch adds a fast path to lookup_datastore() using the shared-
> memory ConfigVersionCache generation counter for datastore.cfg. It
> stores the last seen generation number alongside the cached
> DataStoreImpl and, when a subsequent lookup sees the same generation,
> we reuse the cached instance without re-reading the on-disk config. If
> the generation differs (or the cache is unavailable), it falls back to
> the existing slow path with no behavioral changes.
> 
> Behavioral notes
> 
> - The generation is bumped via the existing save_config() path, so
>   API-driven config changes are detected immediately.
> - Manual edits to datastore.cfg are not detected by this commit; a TTL
>   guard is introduced in a later patch to cover that case.
> - DataStore::drop still performs a config read on the common path; this
>   is addressed in the next patch.
> 
> Testing
> 
> cargo-flamegraph confirms the config-parse hotspot in
> lookup_datastore() disappears from the hot path.
> 
> Additionally, an end-to-end benchmark was performed using the
> `/status?verbose=0` API with 1000 datastores, 5 requests per store,
> and 16-way parallelism:
> 
> | Metric                   | Before | After  | Δ (abs) | Δ (%)   |
> |--------------------------|:------:|:------:|:-------:|:-------:|
> | Total time               | 13s    | 11s    | −2s     | −15.38% |
> | Throughput (all rounds)  | 384.62 | 454.55 | +69.93  | +18.18% |
> | Cold RPS (round #1)      | 76.92  | 90.91  | +13.99  | +18.19% |
> | Warm RPS (rounds 2..N)   | 307.69 | 363.64 | +55.95  | +18.18% |
> 
> Throughput improved by ~18% overall, with total time reduced by ~15%.
> Warm lookups now reuse cached datastore instances and skip redundant
> config parsing entirely. The first-access round also shows a similar
> improvement, likely due to reduced contention and I/O when many stores
> are accessed in parallel.
> 
> Note: A second hotspot remains in Drop where a config read occurs; that
> is addressed in the next commit.

it would be interesting to also have numbers for just lookup+Drop,
without all the HTTP and TLS overhead to really isolate it. that should
also make it easier to reliably benchmark with something like hyperfine
;)

for my simple config (5 datastores) single-threaded lookup+drop of a
single datastore 100k times gives around 1.31 speedup for the whole
series. the slightly modified version from below (which basically runs
the most expensive part of Drop only once) for the same test setup still
gives a speedup of 1.17

re-running the same benchmark with a config with 1k datastores, querying
M datastores N times gives the following speedups:

M = 1, N = 1000: 15.6x faster
M = 10, N = 100: 14.5x
M = 100, N = 10: 8.8x
M = 1000, N = 1: 1.8x (so this is basically showing the speedup of the
improved Drop handling alone!)

and then a slightly modified version, that keeps the DataStore instance
around until all N lookups for that datastore are done, then dropping
them in bulk (which mimics having lots of lookups while a task is
running, making the Drop handler only do the expensive part every once
in a while when the last task for a given datastore exits):

use anyhow::Error;

use pbs_api_types::Operation;
use pbs_datastore::DataStore;

fn main() -> Result<(), Error> {
    let mut args = std::env::args();
    args.next();

    let datastores = if let Some(n) = args.next() {
        n.parse::<usize>()?
    } else {
        1000
    };

    let iterations = if let Some(n) = args.next() {
        n.parse::<usize>()?
    } else {
        1000
    };

    for d in 1..=datastores {
        let name = format!("pbs_bench_{d}");
        let mut stores = Vec::with_capacity(iterations);
        for i in 1..=iterations {
            stores.push(DataStore::lookup_datastore(&name, Some(Operation::Write))?);
        }
    }

    Ok(())
}

M = 1, N = 1000: 8.5x faster
M = 10, N = 100: 7.9x
M = 100, N = 10: 5.2x
M = 1000, N = 1: 1.9x

looking at the flamegraph of this isolated benchmark it's now obvious
that the remaining overhead is lockfiles and tracking the operations
(both in lookup and when dropping)..

side-note: I actually found a bug in our operations tracking while
benchmarking this series that gave me wildly different numbers because
the "drop last task" part never got executed as a result:

https://lore.proxmox.com/pbs-devel/20251112131525.645971-1-f.gruenbichler@proxmox.com/T/#u

> 
> Links
> 
> [1] Bugzilla: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
> [2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph
> 
> Fixes: #6049
> Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
> ---
>  pbs-config/src/config_version_cache.rs | 10 +++-
>  pbs-datastore/src/datastore.rs         | 77 +++++++++++++++++---------
>  2 files changed, 59 insertions(+), 28 deletions(-)
> 
> diff --git a/pbs-config/src/config_version_cache.rs b/pbs-config/src/config_version_cache.rs
> index e8fb994f..b875f7e0 100644
> --- a/pbs-config/src/config_version_cache.rs
> +++ b/pbs-config/src/config_version_cache.rs
> @@ -26,7 +26,6 @@ struct ConfigVersionCacheDataInner {
>      // Traffic control (traffic-control.cfg) generation/version.
>      traffic_control_generation: AtomicUsize,
>      // datastore (datastore.cfg) generation/version
> -    // FIXME: remove with PBS 3.0
>      datastore_generation: AtomicUsize,
>      // Add further atomics here
>  }
> @@ -145,8 +144,15 @@ impl ConfigVersionCache {
>              .fetch_add(1, Ordering::AcqRel);
>      }
>  
> +    /// Returns the datastore generation number.
> +    pub fn datastore_generation(&self) -> usize {
> +        self.shmem
> +            .data()
> +            .datastore_generation
> +            .load(Ordering::Acquire)
> +    }
> +
>      /// Increase the datastore generation number.
> -    // FIXME: remove with PBS 3.0 or make actually useful again in datastore lookup
>      pub fn increase_datastore_generation(&self) -> usize {
>          self.shmem
>              .data()

this part could be split out into its own patch

> diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
> index 70af94d8..18eebb58 100644
> --- a/pbs-datastore/src/datastore.rs
> +++ b/pbs-datastore/src/datastore.rs
> @@ -32,7 +32,7 @@ use pbs_api_types::{
>      MaintenanceType, Operation, UPID,
>  };
>  use pbs_config::s3::S3_CFG_TYPE_ID;
> -use pbs_config::BackupLockGuard;
> +use pbs_config::{BackupLockGuard, ConfigVersionCache};
>  
>  use crate::backup_info::{
>      BackupDir, BackupGroup, BackupInfo, OLD_LOCKING, PROTECTED_MARKER_FILENAME,
> @@ -140,10 +140,10 @@ pub struct DataStoreImpl {
>      last_gc_status: Mutex<GarbageCollectionStatus>,
>      verify_new: bool,
>      chunk_order: ChunkOrder,
> -    last_digest: Option<[u8; 32]>,
>      sync_level: DatastoreFSyncLevel,
>      backend_config: DatastoreBackendConfig,
>      lru_store_caching: Option<LocalDatastoreLruCache>,
> +    cached_config_tag: Option<CachedDatastoreConfigTag>,
>  }
>  
>  impl DataStoreImpl {
> @@ -156,10 +156,10 @@ impl DataStoreImpl {
>              last_gc_status: Mutex::new(GarbageCollectionStatus::default()),
>              verify_new: false,
>              chunk_order: Default::default(),
> -            last_digest: None,
>              sync_level: Default::default(),
>              backend_config: Default::default(),
>              lru_store_caching: None,
> +            cached_config_tag: None,
>          })
>      }
>  }
> @@ -224,6 +224,15 @@ pub enum DatastoreBackend {
>      S3(Arc<S3Client>),
>  }
>  
> +/// Used to determine whether a cached datastore instance is still valid
> +/// or needs to be reloaded after a config change.
> +struct CachedDatastoreConfigTag {
> +    /// Maintenance mode at the time the lookup hint was captured, if any.
> +    last_maintenance_mode: Option<MaintenanceMode>,
> +    /// Datastore generation number from `ConfigVersionCache`; `None` when the cache wasn't available.
> +    last_generation: Option<usize>,

if the whole tag is an option, do we really need to make the generation
an option as well?

> +}
> +
>  impl DataStore {
>      // This one just panics on everything
>      #[doc(hidden)]
> @@ -299,13 +308,40 @@ impl DataStore {
>          // we use it to decide whether it is okay to delete the datastore.
>          let _config_lock = pbs_config::datastore::lock_config()?;
>  
> -        // we could use the ConfigVersionCache's generation for staleness detection, but  we load
> -        // the config anyway -> just use digest, additional benefit: manual changes get detected
> -        let (config, digest) = pbs_config::datastore::config()?;
> +        // Get the current datastore.cfg generation number
> +        let gen_num = ConfigVersionCache::new()
> +            .ok()
> +            .map(|c| c.datastore_generation());
> +
> +        // Fast-path: if we have a cached entry created under the same datastore.cfg generation number, reuse it.
> +        if let (Some(gen_num), Some(ds)) =
> +            (gen_num, DATASTORE_MAP.lock().unwrap().get(name).cloned())
> +        {
> +            if let Some(cached_tag) = &ds.cached_config_tag {
> +                if cached_tag.last_generation == Some(gen_num) {
> +                    if let Some(mm) = &cached_tag.last_maintenance_mode {
> +                        if let Err(error) = mm.check(operation) {
> +                            bail!("datastore '{name}' is unavailable: {error}");
> +                        }
> +                    }
> +                    if let Some(operation) = operation {
> +                        update_active_operations(name, operation, 1)?;
> +                    }
> +                    return Ok(Arc::new(Self {
> +                        inner: ds,
> +                        operation,
> +                    }));
> +                }
> +            }
> +        }
> +
> +        // Slow path: (re)load config
> +        let (config, _digest) = pbs_config::datastore::config()?;
>          let config: DataStoreConfig = config.lookup("datastore", name)?;
>  
> -        if let Some(maintenance_mode) = config.get_maintenance_mode() {
> -            if let Err(error) = maintenance_mode.check(operation) {
> +        let maintenance_mode = config.get_maintenance_mode();
> +        if let Some(mm) = &maintenance_mode {
> +            if let Err(error) = mm.check(operation) {
>                  bail!("datastore '{name}' is unavailable: {error}");
>              }
>          }

after this here we have a check for the mount status in the old hot
path, that is missing in the new hot path. the mount status can change
even if the config doesn't, so we should probably add this back to the
hot path and re-run the numbers?

that check again needs more parts of the config, so maybe we could
explore caching the full config here? e.g., add a new static
Mutex<Option<(DataStoreConfig, usize)>> (extended by the timestamp in
the last patch) and adapt the patches here to use it? depending on
whether we make the cached config available outside of lookup_datastore,
we could then even not add the maintenance mode to the cache tag, and
just store the generation number there, and retrieve the maintenance
mode from the cached config in the Drop implementation..

there's a little more duplication here, e.g. we lock the map and check
for an entry in both the fast and slow paths, we could do it once up
front (nobody can change the map while we have it locked anyway), the
checks are written twice and could probably be extracted into a helper
so that future similar checks are added to both paths and not to only
one by accident, ..

> @@ -321,16 +357,6 @@ impl DataStore {
>  
>          // reuse chunk store so that we keep using the same process locker instance!
>          let chunk_store = if let Some(datastore) = &entry {
> -            let last_digest = datastore.last_digest.as_ref();
> -            if let Some(true) = last_digest.map(|last_digest| last_digest == &digest) {
> -                if let Some(operation) = operation {
> -                    update_active_operations(name, operation, 1)?;
> -                }
> -                return Ok(Arc::new(Self {
> -                    inner: Arc::clone(datastore),
> -                    operation,
> -                }));
> -            }
>              Arc::clone(&datastore.chunk_store)
>          } else {
>              let tuning: DatastoreTuning = serde_json::from_value(
> @@ -344,7 +370,11 @@ impl DataStore {
>              )?)
>          };
>  
> -        let datastore = DataStore::with_store_and_config(chunk_store, config, Some(digest))?;
> +        let mut datastore = DataStore::with_store_and_config(chunk_store, config)?;
> +        datastore.cached_config_tag = Some(CachedDatastoreConfigTag {
> +            last_maintenance_mode: maintenance_mode,
> +            last_generation: gen_num,
> +        });

this part should be in with_store_and_config, which should get the
last_generation (and later last_update) as parameter(s), just like it
had the digest before this patch..

>  
>          let datastore = Arc::new(datastore);
>          datastore_cache.insert(name.to_string(), datastore.clone());
> @@ -430,11 +460,7 @@ impl DataStore {
>              config.absolute_path(),
>              tuning.sync_level.unwrap_or_default(),
>          )?;
> -        let inner = Arc::new(Self::with_store_and_config(
> -            Arc::new(chunk_store),
> -            config,
> -            None,
> -        )?);
> +        let inner = Arc::new(Self::with_store_and_config(Arc::new(chunk_store), config)?);
>  
>          if let Some(operation) = operation {
>              update_active_operations(&name, operation, 1)?;
> @@ -446,7 +472,6 @@ impl DataStore {
>      fn with_store_and_config(
>          chunk_store: Arc<ChunkStore>,
>          config: DataStoreConfig,
> -        last_digest: Option<[u8; 32]>,
>      ) -> Result<DataStoreImpl, Error> {
>          let mut gc_status_path = chunk_store.base_path();
>          gc_status_path.push(".gc-status");
> @@ -506,10 +531,10 @@ impl DataStore {
>              last_gc_status: Mutex::new(gc_status),
>              verify_new: config.verify_new.unwrap_or(false),
>              chunk_order: tuning.chunk_order.unwrap_or_default(),
> -            last_digest,
>              sync_level: tuning.sync_level.unwrap_or_default(),
>              backend_config,
>              lru_store_caching,
> +            cached_config_tag: None,
>          })
>      }
>  
> -- 
> 2.47.3
> 
> 
> 
> _______________________________________________
> pbs-devel mailing list
> pbs-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
> 


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

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

* Re: [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop
  2025-11-12 11:24   ` Fabian Grünbichler
@ 2025-11-12 15:20     ` Samuel Rufinatscha
  0 siblings, 0 replies; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-12 15:20 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion, Fabian Grünbichler

On 11/12/25 12:25 PM, Fabian Grünbichler wrote:
> On November 11, 2025 1:29 pm, Samuel Rufinatscha wrote:
>> The Drop impl of DataStore re-read datastore.cfg to decide whether
>> the entry should be evicted from the in-process cache (based on
>> maintenance mode’s clear_from_cache). During the investigation of
>> issue #6049 [1], a flamegraph [2] showed that the config reload in Drop
>> accounted for a measurable share of CPU time under load.
>>
>> This patch makes Drop O(1) on the fast path by reusing the maintenance-
> 
> I am not sure what the O(1) is refering to? This patch implements a
> faster cache lookup in front of the (slow) config parsing variant, but
> that doesn't really align well with what the "Big O" notation tries to
> express ;)
> 
> The parsing below still scales with the number of datastores in the
> config, after all. It can just be skipped sometimes :)
>

Good point — the O(1) reference is a rather misleading. I’ll rephrase it 
in v2 :)

>> mode decision captured at lookup time and stored with the cached
>> datastore entry. When the last reference goes away we:
>> - decrement active-operation counters, and
>> - evict only if the cached decision mandates eviction.
>>
>> If the cache tag is absent or not fresh, a subsequent slow-path lookup
>> will be performed.
>>
>> Testing
>>
>> Compared flamegraphs before and after: prior to this change
>> (on top of patch 1), stacks originating from Drop included
>> pbs_config::datastore::config(). After the change, those vanish from
>> the drop path.
>>
>> An end-to-end benchmark using `/status?verbose=0` with 1000 datastores,
>> 5 requests per store, and 16-way parallelism shows a further
>> improvement:
>>
>> | Metric                  | After commit 1 | After commit 2 | Δ (abs) | Δ (%)   |
>> |-------------------------|:--------------:|:--------------:|:-------:|:-------:|
>> | Total time              | 11s            | 10s            | −1s     | −9.09%  |
>> | Throughput (all rounds) | 454.55         | 500.00         | +45.45  | +10.00% |
>> | Cold RPS (round #1)     | 90.91          | 100.00         | +9.09   | +10.00% |
>> | Warm RPS (rounds 2..N)  | 363.64         | 400.00         | +36.36  | +10.00% |
>>
>> Optimizing Drop improves overall throughput by ~10%. The gain appears
>> in both cold and warm rounds, and the flamegraph confirms the config
>> reload no longer sits on the hot path.
>>
>> Links
>>
>> [1] Bugzilla: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
>> [2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph
>>
>> Fixes: #6049
>> Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
>> ---
>>   pbs-datastore/src/datastore.rs | 31 +++++++++++++++++++++++++++----
>>   1 file changed, 27 insertions(+), 4 deletions(-)
>>
>> diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
>> index 18eebb58..da80416a 100644
>> --- a/pbs-datastore/src/datastore.rs
>> +++ b/pbs-datastore/src/datastore.rs
>> @@ -200,15 +200,38 @@ impl Drop for DataStore {
>>               // remove datastore from cache iff
>>               //  - last task finished, and
>>               //  - datastore is in a maintenance mode that mandates it
>> -            let remove_from_cache = last_task
>> -                && pbs_config::datastore::config()
>> +
>> +            // first check: check if last task finished
>> +            if !last_task {
>> +                return;
>> +            }
>> +
>> +            let cached_tag = self.inner.cached_config_tag.as_ref();
>> +            let last_gen_num = cached_tag.and_then(|c| c.last_generation);
>> +            let gen_num = ConfigVersionCache::new()
>> +                .ok()
>> +                .map(|c| c.datastore_generation());
>> +
>> +            let cache_is_fresh = match (last_gen_num, gen_num) {
>> +                (Some(a), Some(b)) => a == b,
>> +                _ => false,
>> +            };
> 
> this is just last_gen_num == gen_num and checking that either is Some.
> if we make the tag always contain a generation instead of an option, we
> can simplify this code ;)
>

Good point, will adjust this. I think we could keep 
`ConfigVersionCache::new().ok()` and create the optional cache tag only 
if the generation number is `Some`. This way, the lookup would still be 
able to perform a slow path read if the cache isn’t available for any 
reason.

>> +
>> +            let mm_mandate = if cache_is_fresh {
>> +                cached_tag
>> +                    .and_then(|c| c.last_maintenance_mode.as_ref())
>> +                    .is_some_and(|m| m.clear_from_cache())
>> +            } else {
>> +                pbs_config::datastore::config()
>>                       .and_then(|(s, _)| s.lookup::<DataStoreConfig>("datastore", self.name()))
>>                       .is_ok_and(|c| {
>>                           c.get_maintenance_mode()
>>                               .is_some_and(|m| m.clear_from_cache())
>> -                    });
>> +                    })
>> +            };
>>   
>> -            if remove_from_cache {
>> +            // second check: check maintenance mode mandate
>> +            if mm_mandate {
>>                   DATASTORE_MAP.lock().unwrap().remove(self.name());
>>               }
>>           }
>> -- 
>> 2.47.3
>>
>>
>>
>> _______________________________________________
>> pbs-devel mailing list
>> pbs-devel@lists.proxmox.com
>> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
>>
> 
> 
> _______________________________________________
> pbs-devel mailing list
> pbs-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel



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

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

* Re: [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path
  2025-11-12 11:27 ` [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Fabian Grünbichler
@ 2025-11-12 17:27   ` Samuel Rufinatscha
  0 siblings, 0 replies; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-12 17:27 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion, Fabian Grünbichler

On 11/12/25 12:27 PM, Fabian Grünbichler wrote:
> On November 11, 2025 1:29 pm, Samuel Rufinatscha wrote:
>> Hi,
>>
>> this series reduces CPU time in datastore lookups by avoiding repeated
>> datastore.cfg reads/parses in both `lookup_datastore()` and
>> `DataStore::Drop`. It also adds a TTL so manual config edits are
>> noticed without reintroducing hashing on every request.
>>
>> While investigating #6049 [1], cargo-flamegraph [2] showed hotspots during
>> repeated `/status` calls in `lookup_datastore()` and in `Drop`,
>> dominated by `pbs_config::datastore::config()` (config parse).
>>
>> The parsing cost itself should eventually be investigated in a future
>> effort. Furthermore, cargo-flamegraph showed that when using a
>> token-based auth method to access the API, a significant amount of time
>> is spent in validation on every request, likely related to bcrypt.
>> Also this should be eventually revisited in a future effort.
> 
> please file a bug for the token part, if there isn't already one!
> 

Thanks for the in-depth review Fabian! I created a bug report for the 
token part and added the relevant flamegraph - this should help narrow 
down the issue: https://bugzilla.proxmox.com/show_bug.cgi?id=7017

> thanks for diving into this, it already looks promising, even though the
> effect on more "normal" systems with more reasonable numbers of
> datastores and clients will be less pronounced ;)
> 
> the big TL;DR would be that we trade faster datastore lookups (which
> happen quite frequently, in particular if there are many datastores with
> clients checking their status) against slightly delayed reload of the
> configuration in case of manual, behind-our-backs edits, with one
> particular corner case that is slightly problematic, but also a bit
> contrived:
> - datastore is looked up
> - config is edited (manually) to set maintenance mode to one that
>    requires removing from the datastore map once the last task exits
> - last task drops the datastore struct
> - no regular edits happened in the meantime
> - the Drop handler doesn't know it needs to remove the datastore from
>    the map
> - open FD is held by proxy, datastore fails to be unmounted/..
> 
> we could solve this issue by not only bumping the generation on save,
> but also when we reload the config (in particular if we cache the whole
> config!). that would make the Drop handler efficient enough for idle
> systems that have mostly lookups but no long running tasks. as soon as a
> datastore has long running tasks, the last such task will likely exit
> long after the TTL for its config lookup has expired, so will need to do
> a refresh - although that refresh could again be from the global cache,
> instead of from disk? still wouldn't close the window entirely, but make
> it pretty unlikely to be hit in practice..
> 
> 
> _______________________________________________
> pbs-devel mailing list
> pbs-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
> 
> 

Good idea! I will add the bump in the lookup_datastore() slow path 
directly after (config, digest) is read and increment the generation if 
the digest changed but generation hasn’t - this should also help avoid 
unnecessary cache invalidations.

In Drop we then either check if the shared gen differs from the cached 
tag gen or the tag is TTL expired, otherwise use the cached decision.





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

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

* Re: [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups
  2025-11-12 13:24   ` Fabian Grünbichler
@ 2025-11-13 12:59     ` Samuel Rufinatscha
  0 siblings, 0 replies; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-13 12:59 UTC (permalink / raw)
  To: Proxmox Backup Server development discussion, Fabian Grünbichler

On 11/12/25 2:24 PM, Fabian Grünbichler wrote:
> On November 11, 2025 1:29 pm, Samuel Rufinatscha wrote:
>> Repeated /status requests caused lookup_datastore() to re-read and
>> parse datastore.cfg on every call. The issue was mentioned in report
>> #6049 [1]. cargo-flamegraph [2] confirmed that the hot path is
>> dominated by pbs_config::datastore::config() (config parsing).
>>
>> This patch adds a fast path to lookup_datastore() using the shared-
>> memory ConfigVersionCache generation counter for datastore.cfg. It
>> stores the last seen generation number alongside the cached
>> DataStoreImpl and, when a subsequent lookup sees the same generation,
>> we reuse the cached instance without re-reading the on-disk config. If
>> the generation differs (or the cache is unavailable), it falls back to
>> the existing slow path with no behavioral changes.
>>
>> Behavioral notes
>>
>> - The generation is bumped via the existing save_config() path, so
>>    API-driven config changes are detected immediately.
>> - Manual edits to datastore.cfg are not detected by this commit; a TTL
>>    guard is introduced in a later patch to cover that case.
>> - DataStore::drop still performs a config read on the common path; this
>>    is addressed in the next patch.
>>
>> Testing
>>
>> cargo-flamegraph confirms the config-parse hotspot in
>> lookup_datastore() disappears from the hot path.
>>
>> Additionally, an end-to-end benchmark was performed using the
>> `/status?verbose=0` API with 1000 datastores, 5 requests per store,
>> and 16-way parallelism:
>>
>> | Metric                   | Before | After  | Δ (abs) | Δ (%)   |
>> |--------------------------|:------:|:------:|:-------:|:-------:|
>> | Total time               | 13s    | 11s    | −2s     | −15.38% |
>> | Throughput (all rounds)  | 384.62 | 454.55 | +69.93  | +18.18% |
>> | Cold RPS (round #1)      | 76.92  | 90.91  | +13.99  | +18.19% |
>> | Warm RPS (rounds 2..N)   | 307.69 | 363.64 | +55.95  | +18.18% |
>>
>> Throughput improved by ~18% overall, with total time reduced by ~15%.
>> Warm lookups now reuse cached datastore instances and skip redundant
>> config parsing entirely. The first-access round also shows a similar
>> improvement, likely due to reduced contention and I/O when many stores
>> are accessed in parallel.
>>
>> Note: A second hotspot remains in Drop where a config read occurs; that
>> is addressed in the next commit.
> 
> it would be interesting to also have numbers for just lookup+Drop,
> without all the HTTP and TLS overhead to really isolate it. that should
> also make it easier to reliably benchmark with something like hyperfine
> ;)
> 

Good point, I will isolate the numbers - seems like the TLS overhead is 
quite huge. Thanks for the hyperfine reference!

> for my simple config (5 datastores) single-threaded lookup+drop of a
> single datastore 100k times gives around 1.31 speedup for the whole
> series. the slightly modified version from below (which basically runs
> the most expensive part of Drop only once) for the same test setup still
> gives a speedup of 1.17
> 

So the lookup optimization dominates the speedup if we hold longer to 
the datastores, great to see.

> re-running the same benchmark with a config with 1k datastores, querying> M datastores N times gives the following speedups:
> 
> M = 1, N = 1000: 15.6x faster
> M = 10, N = 100: 14.5x
> M = 100, N = 10: 8.8x
> M = 1000, N = 1: 1.8x (so this is basically showing the speedup of the
> improved Drop handling alone!)
> 
> and then a slightly modified version, that keeps the DataStore instance
> around until all N lookups for that datastore are done, then dropping
> them in bulk (which mimics having lots of lookups while a task is
> running, making the Drop handler only do the expensive part every once
> in a while when the last task for a given datastore exits):
> 
> use anyhow::Error;
> 
> use pbs_api_types::Operation;
> use pbs_datastore::DataStore;
> 
> fn main() -> Result<(), Error> {
>      let mut args = std::env::args();
>      args.next();
> 
>      let datastores = if let Some(n) = args.next() {
>          n.parse::<usize>()?
>      } else {
>          1000
>      };
> 
>      let iterations = if let Some(n) = args.next() {
>          n.parse::<usize>()?
>      } else {
>          1000
>      };
> 
>      for d in 1..=datastores {
>          let name = format!("pbs_bench_{d}");
>          let mut stores = Vec::with_capacity(iterations);
>          for i in 1..=iterations {
>              stores.push(DataStore::lookup_datastore(&name, Some(Operation::Write))?);
>          }
>      }
> 
>      Ok(())
> }
> 
> M = 1, N = 1000: 8.5x faster
> M = 10, N = 100: 7.9x
> M = 100, N = 10: 5.2x
> M = 1000, N = 1: 1.9x
> 
> looking at the flamegraph of this isolated benchmark it's now obvious
> that the remaining overhead is lockfiles and tracking the operations
> (both in lookup and when dropping)..
> 

Thanks a lot also for setting up the test environment and providing your 
numbers, which are helpful to compare against!

Regarding the overhead in lockfiles and tracking operations, it is a 
good confirmation that everything else on the hot path is optimized - I 
think the locks and tracking operations could maybe be revisited in a 
future effort!

> side-note: I actually found a bug in our operations tracking while
> benchmarking this series that gave me wildly different numbers because
> the "drop last task" part never got executed as a result:
> 
> https://lore.proxmox.com/pbs-devel/20251112131525.645971-1-f.gruenbichler@proxmox.com/T/#u
> 

Nice catch!

>>
>> Links
>>
>> [1] Bugzilla: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
>> [2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph
>>
>> Fixes: #6049
>> Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
>> ---
>>   pbs-config/src/config_version_cache.rs | 10 +++-
>>   pbs-datastore/src/datastore.rs         | 77 +++++++++++++++++---------
>>   2 files changed, 59 insertions(+), 28 deletions(-)
>>
>> diff --git a/pbs-config/src/config_version_cache.rs b/pbs-config/src/config_version_cache.rs
>> index e8fb994f..b875f7e0 100644
>> --- a/pbs-config/src/config_version_cache.rs
>> +++ b/pbs-config/src/config_version_cache.rs
>> @@ -26,7 +26,6 @@ struct ConfigVersionCacheDataInner {
>>       // Traffic control (traffic-control.cfg) generation/version.
>>       traffic_control_generation: AtomicUsize,
>>       // datastore (datastore.cfg) generation/version
>> -    // FIXME: remove with PBS 3.0
>>       datastore_generation: AtomicUsize,
>>       // Add further atomics here
>>   }
>> @@ -145,8 +144,15 @@ impl ConfigVersionCache {
>>               .fetch_add(1, Ordering::AcqRel);
>>       }
>>   
>> +    /// Returns the datastore generation number.
>> +    pub fn datastore_generation(&self) -> usize {
>> +        self.shmem
>> +            .data()
>> +            .datastore_generation
>> +            .load(Ordering::Acquire)
>> +    }
>> +
>>       /// Increase the datastore generation number.
>> -    // FIXME: remove with PBS 3.0 or make actually useful again in datastore lookup
>>       pub fn increase_datastore_generation(&self) -> usize {
>>           self.shmem
>>               .data()
> 
> this part could be split out into its own patch
> 

Will factor this out into a separate patch.

>> diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
>> index 70af94d8..18eebb58 100644
>> --- a/pbs-datastore/src/datastore.rs
>> +++ b/pbs-datastore/src/datastore.rs
>> @@ -32,7 +32,7 @@ use pbs_api_types::{
>>       MaintenanceType, Operation, UPID,
>>   };
>>   use pbs_config::s3::S3_CFG_TYPE_ID;
>> -use pbs_config::BackupLockGuard;
>> +use pbs_config::{BackupLockGuard, ConfigVersionCache};
>>   
>>   use crate::backup_info::{
>>       BackupDir, BackupGroup, BackupInfo, OLD_LOCKING, PROTECTED_MARKER_FILENAME,
>> @@ -140,10 +140,10 @@ pub struct DataStoreImpl {
>>       last_gc_status: Mutex<GarbageCollectionStatus>,
>>       verify_new: bool,
>>       chunk_order: ChunkOrder,
>> -    last_digest: Option<[u8; 32]>,
>>       sync_level: DatastoreFSyncLevel,
>>       backend_config: DatastoreBackendConfig,
>>       lru_store_caching: Option<LocalDatastoreLruCache>,
>> +    cached_config_tag: Option<CachedDatastoreConfigTag>,
>>   }
>>   
>>   impl DataStoreImpl {
>> @@ -156,10 +156,10 @@ impl DataStoreImpl {
>>               last_gc_status: Mutex::new(GarbageCollectionStatus::default()),
>>               verify_new: false,
>>               chunk_order: Default::default(),
>> -            last_digest: None,
>>               sync_level: Default::default(),
>>               backend_config: Default::default(),
>>               lru_store_caching: None,
>> +            cached_config_tag: None,
>>           })
>>       }
>>   }
>> @@ -224,6 +224,15 @@ pub enum DatastoreBackend {
>>       S3(Arc<S3Client>),
>>   }
>>   
>> +/// Used to determine whether a cached datastore instance is still valid
>> +/// or needs to be reloaded after a config change.
>> +struct CachedDatastoreConfigTag {
>> +    /// Maintenance mode at the time the lookup hint was captured, if any.
>> +    last_maintenance_mode: Option<MaintenanceMode>,
>> +    /// Datastore generation number from `ConfigVersionCache`; `None` when the cache wasn't available.
>> +    last_generation: Option<usize>,
> 
> if the whole tag is an option, do we really need to make the generation
> an option as well?
> 

Good point, keeping the tag optional only is enough and will simplify 
generation checks.

>> +}
>> +
>>   impl DataStore {
>>       // This one just panics on everything
>>       #[doc(hidden)]
>> @@ -299,13 +308,40 @@ impl DataStore {
>>           // we use it to decide whether it is okay to delete the datastore.
>>           let _config_lock = pbs_config::datastore::lock_config()?;
>>   
>> -        // we could use the ConfigVersionCache's generation for staleness detection, but  we load
>> -        // the config anyway -> just use digest, additional benefit: manual changes get detected
>> -        let (config, digest) = pbs_config::datastore::config()?;
>> +        // Get the current datastore.cfg generation number
>> +        let gen_num = ConfigVersionCache::new()
>> +            .ok()
>> +            .map(|c| c.datastore_generation());
>> +
>> +        // Fast-path: if we have a cached entry created under the same datastore.cfg generation number, reuse it.
>> +        if let (Some(gen_num), Some(ds)) =
>> +            (gen_num, DATASTORE_MAP.lock().unwrap().get(name).cloned())
>> +        {
>> +            if let Some(cached_tag) = &ds.cached_config_tag {
>> +                if cached_tag.last_generation == Some(gen_num) {
>> +                    if let Some(mm) = &cached_tag.last_maintenance_mode {
>> +                        if let Err(error) = mm.check(operation) {
>> +                            bail!("datastore '{name}' is unavailable: {error}");
>> +                        }
>> +                    }
>> +                    if let Some(operation) = operation {
>> +                        update_active_operations(name, operation, 1)?;
>> +                    }
>> +                    return Ok(Arc::new(Self {
>> +                        inner: ds,
>> +                        operation,
>> +                    }));
>> +                }
>> +            }
>> +        }
>> +
>> +        // Slow path: (re)load config
>> +        let (config, _digest) = pbs_config::datastore::config()?;
>>           let config: DataStoreConfig = config.lookup("datastore", name)?;
>>   
>> -        if let Some(maintenance_mode) = config.get_maintenance_mode() {
>> -            if let Err(error) = maintenance_mode.check(operation) {
>> +        let maintenance_mode = config.get_maintenance_mode();
>> +        if let Some(mm) = &maintenance_mode {
>> +            if let Err(error) = mm.check(operation) {
>>                   bail!("datastore '{name}' is unavailable: {error}");
>>               }
>>           }
> 
> after this here we have a check for the mount status in the old hot
> path, that is missing in the new hot path. the mount status can change
> even if the config doesn't, so we should probably add this back to the
> hot path and re-run the numbers?
> 
> that check again needs more parts of the config, so maybe we could
> explore caching the full config here? e.g., add a new static
> Mutex<Option<(DataStoreConfig, usize)>> (extended by the timestamp in
> the last patch) and adapt the patches here to use it? depending on
> whether we make the cached config available outside of lookup_datastore,
> we could then even not add the maintenance mode to the cache tag, and
> just store the generation number there, and retrieve the maintenance
> mode from the cached config in the Drop implementation..
> 

For the mount check we will need device_uuid and datastore mount dir 
path which we could add to the cached entry, however I think I would 
also rather explore caching of the full global config here. This should 
result in further performance gains - this would/should allow for 
eventual fast-lookups to any other datastore. Will try to integrate it 
in v2.

> there's a little more duplication here, e.g. we lock the map and check
> for an entry in both the fast and slow paths, we could do it once up
> front (nobody can change the map while we have it locked anyway), the
> checks are written twice and could probably be extracted into a helper
> so that future similar checks are added to both paths and not to only
> one by accident, ..
> 

Good point, I will factor it out (e.g. small closure).

>> @@ -321,16 +357,6 @@ impl DataStore {
>>   
>>           // reuse chunk store so that we keep using the same process locker instance!
>>           let chunk_store = if let Some(datastore) = &entry {
>> -            let last_digest = datastore.last_digest.as_ref();
>> -            if let Some(true) = last_digest.map(|last_digest| last_digest == &digest) {
>> -                if let Some(operation) = operation {
>> -                    update_active_operations(name, operation, 1)?;
>> -                }
>> -                return Ok(Arc::new(Self {
>> -                    inner: Arc::clone(datastore),
>> -                    operation,
>> -                }));
>> -            }
>>               Arc::clone(&datastore.chunk_store)
>>           } else {
>>               let tuning: DatastoreTuning = serde_json::from_value(
>> @@ -344,7 +370,11 @@ impl DataStore {
>>               )?)
>>           };
>>   
>> -        let datastore = DataStore::with_store_and_config(chunk_store, config, Some(digest))?;
>> +        let mut datastore = DataStore::with_store_and_config(chunk_store, config)?;
>> +        datastore.cached_config_tag = Some(CachedDatastoreConfigTag {
>> +            last_maintenance_mode: maintenance_mode,
>> +            last_generation: gen_num,
>> +        });
> 
> this part should be in with_store_and_config, which should get the
> last_generation (and later last_update) as parameter(s), just like it
> had the digest before this patch..
> 

Agree, I will move this part.

>>   
>>           let datastore = Arc::new(datastore);
>>           datastore_cache.insert(name.to_string(), datastore.clone());
>> @@ -430,11 +460,7 @@ impl DataStore {
>>               config.absolute_path(),
>>               tuning.sync_level.unwrap_or_default(),
>>           )?;
>> -        let inner = Arc::new(Self::with_store_and_config(
>> -            Arc::new(chunk_store),
>> -            config,
>> -            None,
>> -        )?);
>> +        let inner = Arc::new(Self::with_store_and_config(Arc::new(chunk_store), config)?);
>>   
>>           if let Some(operation) = operation {
>>               update_active_operations(&name, operation, 1)?;
>> @@ -446,7 +472,6 @@ impl DataStore {
>>       fn with_store_and_config(
>>           chunk_store: Arc<ChunkStore>,
>>           config: DataStoreConfig,
>> -        last_digest: Option<[u8; 32]>,
>>       ) -> Result<DataStoreImpl, Error> {
>>           let mut gc_status_path = chunk_store.base_path();
>>           gc_status_path.push(".gc-status");
>> @@ -506,10 +531,10 @@ impl DataStore {
>>               last_gc_status: Mutex::new(gc_status),
>>               verify_new: config.verify_new.unwrap_or(false),
>>               chunk_order: tuning.chunk_order.unwrap_or_default(),
>> -            last_digest,
>>               sync_level: tuning.sync_level.unwrap_or_default(),
>>               backend_config,
>>               lru_store_caching,
>> +            cached_config_tag: None,
>>           })
>>       }
>>   
>> -- 
>> 2.47.3
>>
>>
>>
>> _______________________________________________
>> pbs-devel mailing list
>> pbs-devel@lists.proxmox.com
>> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel
>>
> 
> 
> _______________________________________________
> pbs-devel mailing list
> pbs-devel@lists.proxmox.com
> https://lists.proxmox.com/cgi-bin/mailman/listinfo/pbs-devel



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

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

* [pbs-devel] superseded: [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path
  2025-11-11 12:29 [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Samuel Rufinatscha
                   ` (3 preceding siblings ...)
  2025-11-12 11:27 ` [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Fabian Grünbichler
@ 2025-11-14 15:08 ` Samuel Rufinatscha
  4 siblings, 0 replies; 11+ messages in thread
From: Samuel Rufinatscha @ 2025-11-14 15:08 UTC (permalink / raw)
  To: pbs-devel

https://lore.proxmox.com/pbs-devel/20251114150544.224839-1-s.rufinatscha@proxmox.com/T/#t

On 11/11/25 1:29 PM, Samuel Rufinatscha wrote:
> Hi,
> 
> this series reduces CPU time in datastore lookups by avoiding repeated
> datastore.cfg reads/parses in both `lookup_datastore()` and
> `DataStore::Drop`. It also adds a TTL so manual config edits are
> noticed without reintroducing hashing on every request.
> 
> While investigating #6049 [1], cargo-flamegraph [2] showed hotspots during
> repeated `/status` calls in `lookup_datastore()` and in `Drop`,
> dominated by `pbs_config::datastore::config()` (config parse).
> 
> The parsing cost itself should eventually be investigated in a future
> effort. Furthermore, cargo-flamegraph showed that when using a
> token-based auth method to access the API, a significant amount of time
> is spent in validation on every request, likely related to bcrypt.
> Also this should be eventually revisited in a future effort.
> 
> ## Approach
> 
> [PATCH 1/3] Fast path for datastore lookups
> Use the shared-memory `ConfigVersionCache` generation for `datastore.cfg`.
> Tag each cached `DataStoreImpl` with the last seen generation; when it
> matches, reuse the cached instance. Fall back to the existing slow path
> on mismatch or when the cache is unavailable.
> 
> [PATCH 2/3] Fast path for `Drop`
> Reuse the maintenance mode eviction decision captured at lookup time,
> removing the config reload from `Drop`.
> 
> [PATCH 3/3] TTL to catch manual edits
> If a cached entry is older than `DATASTORE_CONFIG_CACHE_TTL_SECS`
> (default 60s), the next lookup refreshes it via the slow path. This
> detects manual file edits without hashing on every request.
> 
> ## Results
> 
> End-to-end `/status?verbose=0` (1000 stores, 5 req/store, parallel=16):
> 
>    Metric                 Baseline   [1/3]    [2/3]
>    ------------------------------------------------
>    Total time             13s        11s      10s
>    Throughput (all)       384.62     454.55   500.00
>    Cold RPS (round #1)    76.92      90.91    100.00
>    Warm RPS (2..N)        307.69     363.64   400.00
> 
> Patch 1 improves overall throughput by ~18% (−15% total time). Patch 2
> adds ~10% on top. Patch 3 is a robustness feature; a 0.1 s probe shows
> periodic latency spikes at TTL expiry and flat latencies otherwise.
> 
> ## Reproduction steps
> 
> VM: 4 vCPU, ~8 GiB RAM, VirtIO-SCSI; disks:
>    - scsi0 32G (OS)
>    - scsi1 1000G (datastores)
> 
> Install PBS from ISO on the VM.
> 
> Set up ZFS on /dev/sdb (adjust if different):
> 
>      zpool create -f -o ashift=12 pbsbench /dev/sdb
>      zfs set mountpoint=/pbsbench pbsbench
>      zfs create pbsbench/pbs-bench
> 
> Raise file-descriptor limit:
> 
>      sudo systemctl edit proxmox-backup-proxy.service
> 
> Add the following lines:
> 
>      [Service]
>      LimitNOFILE=1048576
> 
> Reload systemd and restart the proxy:
> 
>      sudo systemctl daemon-reload
>      sudo systemctl restart proxmox-backup-proxy.service
> 
> Verify the limit:
> 
>      systemctl show proxmox-backup-proxy.service | grep LimitNOFILE
> 
> Create 1000 ZFS-backed datastores (as used in #6049 [1]):
> 
>      seq -w 001 1000 | xargs -n1 -P1 bash -c '
>        id=$0
>        name="ds${id}"
>        dataset="pbsbench/pbs-bench/${name}"
>        path="/pbsbench/pbs-bench/${name}"
>        zfs create -o mountpoint="$path" "$dataset"
>        proxmox-backup-manager datastore create "$name" "$path" \
>          --comment "ZFS dataset-based datastore"
>      '
> 
> Build PBS from this series, then run the server under manually
> under flamegraph:
> 
>      systemctl stop proxmox-backup-proxy
>      cargo flamegraph --release --bin proxmox-backup-proxy
> 
> Benchmark script (`bench.sh`) used for the numbers above:
> 
>      #!/usr/bin/env bash
>      set -euo pipefail
> 
>      # --- Config ---------------------------------------------------------------
>      HOST='https://localhost:8007'
>      USER='root@pam'
>      PASS="$(cat passfile)"
> 
>      DATASTORE_PATH="/pbsbench/pbs-bench"
>      MAX_STORES=1000      # how many stores to include
>      PARALLEL=16         # concurrent workers
>      REPEAT=5            # requests per store (1 cold + REPEAT-1 warm)
> 
>      PRINT_FIRST=false   # true => log first request's HTTP code per store
> 
>      # --- Helpers --------------------------------------------------------------
>      fmt_rps () {
>        local n="$1" t="$2"
>        awk -v n="$n" -v t="$t" 'BEGIN { if (t > 0) printf("%.2f\n", n/t); else print "0.00" }'
>      }
> 
>      # --- Login ---------------------------------------------------------------
>      auth=$(curl -ks -X POST "$HOST/api2/json/access/ticket" \
>        -d "username=$USER" -d "password=$PASS")
>      ticket=$(echo "$auth" | jq -r '.data.ticket')
> 
>      if [[ -z "${ticket:-}" || "$ticket" == "null" ]]; then
>        echo "[ERROR] Login failed (no ticket)"
>        exit 1
>      fi
> 
>      # --- Collect stores (deterministic order) --------------------------------
>      mapfile -t STORES < <(
>        find "$DATASTORE_PATH" -mindepth 1 -maxdepth 1 -type d -printf '%f\n' \
>        | sort | head -n "$MAX_STORES"
>      )
> 
>      USED_STORES=${#STORES[@]}
>      if (( USED_STORES == 0 )); then
>        echo "[ERROR] No datastore dirs under $DATASTORE_PATH"
>        exit 1
>      fi
> 
>      echo "[INFO] Running with stores=$USED_STORES, repeat=$REPEAT, parallel=$PARALLEL"
> 
>      # --- Temp counters --------------------------------------------------------
>      SUCCESS_ALL="$(mktemp)"
>      FAIL_ALL="$(mktemp)"
>      COLD_OK="$(mktemp)"
>      WARM_OK="$(mktemp)"
>      trap 'rm -f "$SUCCESS_ALL" "$FAIL_ALL" "$COLD_OK" "$WARM_OK"' EXIT
> 
>      export HOST ticket REPEAT SUCCESS_ALL FAIL_ALL COLD_OK WARM_OK PRINT_FIRST
> 
>      SECONDS=0
> 
>      # --- Fire requests --------------------------------------------------------
>      printf "%s\n" "${STORES[@]}" \
>      | xargs -P"$PARALLEL" -I{} bash -c '
>        store="$1"
>        url="$HOST/api2/json/admin/datastore/$store/status?verbose=0"
> 
>        for ((i=1;i<=REPEAT;i++)); do
>          code=$(curl -ks -o /dev/null -w "%{http_code}" -b "PBSAuthCookie=$ticket" "$url" || echo 000)
> 
>          if [[ "$code" == "200" ]]; then
>            echo 1 >> "$SUCCESS_ALL"
>            if (( i == 1 )); then
>              echo 1 >> "$COLD_OK"
>            else
>              echo 1 >> "$WARM_OK"
>            fi
>            if [[ "$PRINT_FIRST" == "true" && $i -eq 1 ]]; then
>              ts=$(date +%H:%M:%S)
>              echo "[$ts] $store #$i HTTP:200"
>            fi
>          else
>            echo 1 >> "$FAIL_ALL"
>            if [[ "$PRINT_FIRST" == "true" && $i -eq 1 ]]; then
>              ts=$(date +%H:%M:%S)
>              echo "[$ts] $store #$i HTTP:$code (FAIL)"
>            fi
>          fi
>        done
>      ' _ {}
> 
>      # --- Summary --------------------------------------------------------------
>      elapsed=$SECONDS
>      ok=$(wc -l < "$SUCCESS_ALL" 2>/dev/null || echo 0)
>      fail=$(wc -l < "$FAIL_ALL" 2>/dev/null || echo 0)
>      cold_ok=$(wc -l < "$COLD_OK" 2>/dev/null || echo 0)
>      warm_ok=$(wc -l < "$WARM_OK" 2>/dev/null || echo 0)
> 
>      expected=$(( USED_STORES * REPEAT ))
>      total=$(( ok + fail ))
> 
>      rps_all=$(fmt_rps "$ok" "$elapsed")
>      rps_cold=$(fmt_rps "$cold_ok" "$elapsed")
>      rps_warm=$(fmt_rps "$warm_ok" "$elapsed")
> 
>      echo "===== Summary ====="
>      echo "Stores used:         $USED_STORES"
>      echo "Expected requests:   $expected"
>      echo "Executed requests:   $total"
>      echo "OK (HTTP 200):       $ok"
>      echo "Failed:              $fail"
>      printf "Total time:          %dm %ds\n" $((elapsed/60)) $((elapsed%60))
>      echo "Throughput all RPS:  $rps_all"
>      echo "Cold RPS (round #1): $rps_cold"
>      echo "Warm RPS (#2..N):    $rps_warm"
> 
> ## Maintainer notes
> 
> - No dependency bumps, no API changes, no breaking changes in this
>    series.
> 
> ## Patch summary
> 
> [PATCH 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups
> [PATCH 2/3] partial fix #6049: datastore: use config fast-path in Drop
> [PATCH 3/3] datastore: add TTL fallback to catch manual config edits
> 
> Thanks for reviewing!
> 
> [1] Bugzilla #6049: https://bugzilla.proxmox.com/show_bug.cgi?id=6049
> [2] cargo-flamegraph: https://github.com/flamegraph-rs/flamegraph
> 
> Samuel Rufinatscha (3):
>    partial fix #6049: datastore: impl ConfigVersionCache fast path for
>      lookups
>    partial fix #6049: datastore: use config fast-path in Drop
>    datastore: add TTL fallback to catch manual config edits
> 
>   pbs-config/src/config_version_cache.rs |  10 ++-
>   pbs-datastore/src/datastore.rs         | 119 ++++++++++++++++++-------
>   2 files changed, 96 insertions(+), 33 deletions(-)
> 



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

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

end of thread, other threads:[~2025-11-14 15:08 UTC | newest]

Thread overview: 11+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2025-11-11 12:29 [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Samuel Rufinatscha
2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 1/3] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
2025-11-12 13:24   ` Fabian Grünbichler
2025-11-13 12:59     ` Samuel Rufinatscha
2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 2/3] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
2025-11-12 11:24   ` Fabian Grünbichler
2025-11-12 15:20     ` Samuel Rufinatscha
2025-11-11 12:29 ` [pbs-devel] [PATCH proxmox-backup 3/3] datastore: add TTL fallback to catch manual config edits Samuel Rufinatscha
2025-11-12 11:27 ` [pbs-devel] [PATCH proxmox-backup 0/3] datastore: remove config reload on hot path Fabian Grünbichler
2025-11-12 17:27   ` Samuel Rufinatscha
2025-11-14 15:08 ` [pbs-devel] superseded: " Samuel Rufinatscha

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.
Service provided by Proxmox Server Solutions GmbH | Privacy | Legal