public inbox for pbs-devel@lists.proxmox.com
 help / color / mirror / Atom feed
* [pbs-devel] [PATCH proxmox-backup v2 0/4] datastore: remove config reload on hot path
@ 2025-11-14 15:05 Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 1/4] partial fix #6049: config: enable config version cache for datastore Samuel Rufinatscha
                   ` (3 more replies)
  0 siblings, 4 replies; 5+ messages in thread
From: Samuel Rufinatscha @ 2025-11-14 15:05 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 request [3].

## Approach

[PATCH 1/4] Extend ConfigVersionCache for datastore generation
  Expose a dedicated datastore generation counter and an increment
  helper so callers can cheaply track datastore.cfg versions.

[PATCH 2/4] Fast path for datastore lookups
  Cache the parsed datastore.cfg keyed by the shared datastore
  generation. lookup_datastore() reuses both the cached config and an
  existing DataStoreImpl when the generation matches, and falls back
  to the old slow path otherwise.

[PATCH 3/4] Fast path for Drop
  Make DataStore::Drop use the cached config if possible instead of
  rereading datastore.cfg from disk.

[PATCH 4/4] TTL to catch manual edits
  Add a small TTL around the cached config and bump the datastore
  generation whenever the config is reloaded. This catches manual
  edits to datastore.cfg without reintroducing hashing or
  config parsing on every request.

## Benchmark results

All the following benchmarks are based on top of
https://lore.proxmox.com/pbs-devel/20251112131525.645971-1-f.gruenbichler@proxmox.com/T/#u

### End-to-end

Testing `/status?verbose=0` end-to-end with 1000 stores, 5 req/store
and parallel=16 before/after the series:

    Metric                 Before     After
    ----------------------------------------
    Total time             12s        9s
    Throughput (all)       416.67     555.56
    Cold RPS (round #1)    83.33      111.11
    Warm RPS (#2..N)       333.33     444.44

Running under flamegraph [2], TLS appears to consume a significant
amount of CPU time and blur the results. Still, a ~33% higher overall
throughput and ~25% less end-to-end time for this workload.

### Isolated benchmarks (hyperfine)

In addition to the end-to-end tests, I measured two standalone benchmarks
with hyperfine, each using a config with 1000
datastores.  `M` is the number of distinct datastores looked up and
`N` is the number of lookups per datastore.

Drop-direct variant:

Drops the `DataStore` after every lookup, so the `Drop` path runs on
every iteration:

    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!("ds{:04}", d);

            for i in 1..=iterations {
                DataStore::lookup_datastore(&name, Some(Operation::Write))?;
            }
        }

        Ok(())
    }

    +----+------+-----------+-----------+---------+
    | M  | N    | Baseline  | Patched   | Speedup |
    +----+------+-----------+-----------+---------+
    | 1  | 1000 | 1.670 s   | 34.3 ms   | 48.7x   |
    | 10 | 100  | 1.672 s   | 34.5 ms   | 48.4x   |
    | 100| 10   | 1.679 s   | 35.1 ms   | 47.8x   |
    |1000| 1    | 1.787 s   | 38.2 ms   | 46.8x   |
    +----+------+-----------+-----------+---------+

Bulk-drop variant:

Keeps the `DataStore` instances alive for
all `N` lookups of a given datastore and then drops them in bulk,
mimicking a task that performs many lookups while it is running and
only triggers the expensive `Drop` logic when the last user 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!("ds{:04}", d);

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

        Ok(())
    }

    +------+------+---------------+--------------+---------+
    | M    |  N   | Baseline mean | Patched mean | Speedup |
    +------+------+---------------+--------------+---------+
    | 1    | 1000 |   884.0 ms    |   33.9 ms    |  26.1x  |
    | 10   | 100  |   881.8 ms    |   35.3 ms    |  25.0x  |
    | 100  | 10   |   969.3 ms    |   35.9 ms    |  27.0x  |
    | 1000 | 1    |  1827.0 ms    |   40.7 ms    |  44.9x  |
    +------+------+---------------+--------------+---------+

Both variants show that the combination of the cached config lookups
and the cheaper `Drop` handling reduces the hot-path cost from ~1.7 s
per run to a few tens of milliseconds in these benchmarks.

## 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

## Other resources:

### E2E benchmark script:

    #!/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 and no breaking changes.

## Patch summary

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

Thanks,
Samuel

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

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

 pbs-config/src/config_version_cache.rs |  10 +-
 pbs-datastore/Cargo.toml               |   1 +
 pbs-datastore/src/datastore.rs         | 187 +++++++++++++++++++------
 3 files changed, 152 insertions(+), 46 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] 5+ messages in thread

* [pbs-devel] [PATCH proxmox-backup v2 1/4] partial fix #6049: config: enable config version cache for datastore
  2025-11-14 15:05 [pbs-devel] [PATCH proxmox-backup v2 0/4] datastore: remove config reload on hot path Samuel Rufinatscha
@ 2025-11-14 15:05 ` Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 2/4] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
                   ` (2 subsequent siblings)
  3 siblings, 0 replies; 5+ messages in thread
From: Samuel Rufinatscha @ 2025-11-14 15:05 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).

To solve the issue, this patch prepares the config version cache,
so that datastore config caching can be built on top of it.
This patch specifically:
(1) implements increment function in order to invalidate generations
(2) removes obsolete comments

Links

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

Signed-off-by: Samuel Rufinatscha <s.rufinatscha@proxmox.com>
---
 pbs-config/src/config_version_cache.rs | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 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()
-- 
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] 5+ messages in thread

* [pbs-devel] [PATCH proxmox-backup v2 2/4] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups
  2025-11-14 15:05 [pbs-devel] [PATCH proxmox-backup v2 0/4] datastore: remove config reload on hot path Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 1/4] partial fix #6049: config: enable config version cache for datastore Samuel Rufinatscha
@ 2025-11-14 15:05 ` Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 3/4] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 4/4] partial fix #6049: datastore: add TTL fallback to catch manual config edits Samuel Rufinatscha
  3 siblings, 0 replies; 5+ messages in thread
From: Samuel Rufinatscha @ 2025-11-14 15:05 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 implements caching of the global datastore.cfg using the
generation numbers from the shared config version cache. It caches the
datastore.cfg along with the generation number and, when a subsequent
lookup sees the same generation, it reuses the cached config without
re-reading it from disk. 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; a TTL
  guard is introduced in a dedicated patch in this series.
- DataStore::drop still performs a config read on the common path,
  this is covered in a dedicated patch in this series.

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/Cargo.toml       |   1 +
 pbs-datastore/src/datastore.rs | 120 +++++++++++++++++++++++----------
 2 files changed, 87 insertions(+), 34 deletions(-)

diff --git a/pbs-datastore/Cargo.toml b/pbs-datastore/Cargo.toml
index 8ce930a9..42f49a7b 100644
--- a/pbs-datastore/Cargo.toml
+++ b/pbs-datastore/Cargo.toml
@@ -40,6 +40,7 @@ proxmox-io.workspace = true
 proxmox-lang.workspace=true
 proxmox-s3-client = { workspace = true, features = [ "impl" ] }
 proxmox-schema = { workspace = true, features = [ "api-macro" ] }
+proxmox-section-config.workspace = true
 proxmox-serde = { workspace = true, features = [ "serde_json" ] }
 proxmox-sys.workspace = true
 proxmox-systemd.workspace = true
diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index 031fa958..e7748872 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -32,7 +32,8 @@ 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 proxmox_section_config::SectionConfigData;
 
 use crate::backup_info::{
     BackupDir, BackupGroup, BackupInfo, OLD_LOCKING, PROTECTED_MARKER_FILENAME,
@@ -46,6 +47,17 @@ use crate::s3::S3_CONTENT_PREFIX;
 use crate::task_tracking::{self, update_active_operations};
 use crate::{DataBlob, LocalDatastoreLruCache};
 
+// Cache for fully parsed datastore.cfg
+struct DatastoreConfigCache {
+    // Parsed datastore.cfg file
+    config: Arc<SectionConfigData>,
+    // Generation number from ConfigVersionCache
+    last_generation: usize,
+}
+
+static DATASTORE_CONFIG_CACHE: LazyLock<Mutex<Option<DatastoreConfigCache>>> =
+    LazyLock::new(|| Mutex::new(None));
+
 static DATASTORE_MAP: LazyLock<Mutex<HashMap<String, Arc<DataStoreImpl>>>> =
     LazyLock::new(|| Mutex::new(HashMap::new()));
 
@@ -140,10 +152,12 @@ 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>,
+    /// Datastore generation number from `ConfigVersionCache` at creation time, used to
+    /// validate reuse of this cached `DataStoreImpl`.
+    config_generation: Option<usize>,
 }
 
 impl DataStoreImpl {
@@ -156,10 +170,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,
+            config_generation: None,
         })
     }
 }
@@ -254,6 +268,37 @@ impl DatastoreBackend {
     }
 }
 
+/// Return the cached datastore SectionConfig and its generation.
+fn datastore_section_config_cached() -> Result<(Arc<SectionConfigData>, Option<usize>), Error> {
+    let gen = ConfigVersionCache::new()
+        .ok()
+        .map(|c| c.datastore_generation());
+
+    let mut guard = DATASTORE_CONFIG_CACHE.lock().unwrap();
+
+    // Fast path: re-use cached datastore.cfg
+    if let (Some(gen), Some(cache)) = (gen, guard.as_ref()) {
+        if cache.last_generation == gen {
+            return Ok((cache.config.clone(), Some(gen)));
+        }
+    }
+
+    // Slow path: re-read datastore.cfg
+    let (config_raw, _digest) = pbs_config::datastore::config()?;
+    let config = Arc::new(config_raw);
+
+    if let Some(gen_val) = gen {
+        *guard = Some(DatastoreConfigCache {
+            config: config.clone(),
+            last_generation: gen_val,
+        });
+    } else {
+        *guard = None;
+    }
+
+    Ok((config, gen))
+}
+
 impl DataStore {
     // This one just panics on everything
     #[doc(hidden)]
@@ -325,56 +370,63 @@ impl DataStore {
         name: &str,
         operation: Option<Operation>,
     ) -> Result<Arc<DataStore>, Error> {
-        // Avoid TOCTOU between checking maintenance mode and updating active operation counter, as
-        // we use it to decide whether it is okay to delete the datastore.
+        // Avoid TOCTOU between checking maintenance mode and updating active operations.
         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()?;
-        let config: DataStoreConfig = config.lookup("datastore", name)?;
+        // Get the current datastore.cfg generation number and cached config
+        let (section_config, gen_num) = datastore_section_config_cached()?;
+
+        let datastore_cfg: DataStoreConfig = section_config.lookup("datastore", name)?;
+        let maintenance_mode = datastore_cfg.get_maintenance_mode();
+        let mount_status = get_datastore_mount_status(&datastore_cfg);
 
-        if let Some(maintenance_mode) = config.get_maintenance_mode() {
-            if let Err(error) = maintenance_mode.check(operation) {
+        if let Some(mm) = &maintenance_mode {
+            if let Err(error) = mm.check(operation.clone()) {
                 bail!("datastore '{name}' is unavailable: {error}");
             }
         }
 
-        if get_datastore_mount_status(&config) == Some(false) {
-            let mut datastore_cache = DATASTORE_MAP.lock().unwrap();
-            datastore_cache.remove(&config.name);
-            bail!("datastore '{}' is not mounted", config.name);
+        let mut datastore_cache = DATASTORE_MAP.lock().unwrap();
+
+        if mount_status == Some(false) {
+            datastore_cache.remove(&datastore_cfg.name);
+            bail!("datastore '{}' is not mounted", datastore_cfg.name);
         }
 
-        let mut datastore_cache = DATASTORE_MAP.lock().unwrap();
-        let entry = datastore_cache.get(name);
-
-        // 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)?;
+        // Re-use DataStoreImpl
+        if let Some(existing) = datastore_cache.get(name).cloned() {
+            if let (Some(last_generation), Some(gen_num)) = (existing.config_generation, gen_num) {
+                if last_generation == gen_num {
+                    if let Some(op) = operation {
+                        update_active_operations(name, op, 1)?;
+                    }
+
+                    return Ok(Arc::new(Self {
+                        inner: existing,
+                        operation,
+                    }));
                 }
-                return Ok(Arc::new(Self {
-                    inner: Arc::clone(datastore),
-                    operation,
-                }));
             }
-            Arc::clone(&datastore.chunk_store)
+        }
+
+        // (Re)build DataStoreImpl
+
+        // Reuse chunk store so that we keep using the same process locker instance!
+        let chunk_store = if let Some(existing) = datastore_cache.get(name) {
+            Arc::clone(&existing.chunk_store)
         } else {
             let tuning: DatastoreTuning = serde_json::from_value(
                 DatastoreTuning::API_SCHEMA
-                    .parse_property_string(config.tuning.as_deref().unwrap_or(""))?,
+                    .parse_property_string(datastore_cfg.tuning.as_deref().unwrap_or(""))?,
             )?;
             Arc::new(ChunkStore::open(
                 name,
-                config.absolute_path(),
+                datastore_cfg.absolute_path(),
                 tuning.sync_level.unwrap_or_default(),
             )?)
         };
 
-        let datastore = DataStore::with_store_and_config(chunk_store, config, Some(digest))?;
+        let datastore = DataStore::with_store_and_config(chunk_store, datastore_cfg, gen_num)?;
 
         let datastore = Arc::new(datastore);
         datastore_cache.insert(name.to_string(), datastore.clone());
@@ -476,7 +528,7 @@ impl DataStore {
     fn with_store_and_config(
         chunk_store: Arc<ChunkStore>,
         config: DataStoreConfig,
-        last_digest: Option<[u8; 32]>,
+        generation: Option<usize>,
     ) -> Result<DataStoreImpl, Error> {
         let mut gc_status_path = chunk_store.base_path();
         gc_status_path.push(".gc-status");
@@ -536,10 +588,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,
+            config_generation: generation,
         })
     }
 
-- 
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] 5+ messages in thread

* [pbs-devel] [PATCH proxmox-backup v2 3/4] partial fix #6049: datastore: use config fast-path in Drop
  2025-11-14 15:05 [pbs-devel] [PATCH proxmox-backup v2 0/4] datastore: remove config reload on hot path Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 1/4] partial fix #6049: config: enable config version cache for datastore Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 2/4] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
@ 2025-11-14 15:05 ` Samuel Rufinatscha
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 4/4] partial fix #6049: datastore: add TTL fallback to catch manual config edits Samuel Rufinatscha
  3 siblings, 0 replies; 5+ messages in thread
From: Samuel Rufinatscha @ 2025-11-14 15:05 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 adds the datastore config fast path to the Drop impl to
eventually avoid an expensive config reload from disk to capture
the maintenance mandate.

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 | 43 +++++++++++++++++++++++++++-------
 1 file changed, 34 insertions(+), 9 deletions(-)

diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index e7748872..0fabf592 100644
--- a/pbs-datastore/src/datastore.rs
+++ b/pbs-datastore/src/datastore.rs
@@ -214,15 +214,40 @@ 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()
-                    .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 {
+
+            // first check: check if last task finished
+            if !last_task {
+                return;
+            }
+
+            let (section_config, _gen) = match datastore_section_config_cached() {
+                Ok(v) => v,
+                Err(err) => {
+                    log::error!(
+                        "failed to load datastore config in Drop for {} - {err}",
+                        self.name()
+                    );
+                    return;
+                }
+            };
+
+            let datastore_cfg: DataStoreConfig =
+                match section_config.lookup("datastore", self.name()) {
+                    Ok(cfg) => cfg,
+                    Err(err) => {
+                        log::error!(
+                            "failed to look up datastore '{}' in Drop - {err}",
+                            self.name()
+                        );
+                        return;
+                    }
+                };
+
+            // second check: check maintenance mode mandate
+            if datastore_cfg
+                .get_maintenance_mode()
+                .is_some_and(|m| m.clear_from_cache())
+            {
                 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] 5+ messages in thread

* [pbs-devel] [PATCH proxmox-backup v2 4/4] partial fix #6049: datastore: add TTL fallback to catch manual config edits
  2025-11-14 15:05 [pbs-devel] [PATCH proxmox-backup v2 0/4] datastore: remove config reload on hot path Samuel Rufinatscha
                   ` (2 preceding siblings ...)
  2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 3/4] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
@ 2025-11-14 15:05 ` Samuel Rufinatscha
  3 siblings, 0 replies; 5+ messages in thread
From: Samuel Rufinatscha @ 2025-11-14 15:05 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 cached config is older than
DATASTORE_CONFIG_CACHE_TTL_SECS (set to 60s), the next lookup takes
the slow path and refreshes the cached entry. Within
the TTL window, unchanged generations still use the fast path.

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 | 46 +++++++++++++++++++++++++---------
 1 file changed, 34 insertions(+), 12 deletions(-)

diff --git a/pbs-datastore/src/datastore.rs b/pbs-datastore/src/datastore.rs
index 0fabf592..7a18435c 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::{
@@ -53,6 +53,8 @@ struct DatastoreConfigCache {
     config: Arc<SectionConfigData>,
     // Generation number from ConfigVersionCache
     last_generation: usize,
+    // Last update time (epoch seconds)
+    last_update: i64,
 }
 
 static DATASTORE_CONFIG_CACHE: LazyLock<Mutex<Option<DatastoreConfigCache>>> =
@@ -61,6 +63,8 @@ static DATASTORE_CONFIG_CACHE: LazyLock<Mutex<Option<DatastoreConfigCache>>> =
 static DATASTORE_MAP: LazyLock<Mutex<HashMap<String, Arc<DataStoreImpl>>>> =
     LazyLock::new(|| Mutex::new(HashMap::new()));
 
+/// Max age in seconds to reuse the cached datastore config.
+const DATASTORE_CONFIG_CACHE_TTL_SECS: i64 = 60;
 /// Filename to store backup group notes
 pub const GROUP_NOTES_FILE_NAME: &str = "notes";
 /// Filename to store backup group owner
@@ -295,16 +299,22 @@ impl DatastoreBackend {
 
 /// Return the cached datastore SectionConfig and its generation.
 fn datastore_section_config_cached() -> Result<(Arc<SectionConfigData>, Option<usize>), Error> {
-    let gen = ConfigVersionCache::new()
-        .ok()
-        .map(|c| c.datastore_generation());
+    let now = epoch_i64();
+    let version_cache = ConfigVersionCache::new().ok();
+    let current_gen = version_cache.as_ref().map(|c| c.datastore_generation());
 
     let mut guard = DATASTORE_CONFIG_CACHE.lock().unwrap();
 
-    // Fast path: re-use cached datastore.cfg
-    if let (Some(gen), Some(cache)) = (gen, guard.as_ref()) {
-        if cache.last_generation == gen {
-            return Ok((cache.config.clone(), Some(gen)));
+    // Fast path: re-use cached datastore.cfg if cache is available, generation matches and TTL not expired
+    if let (Some(current_gen), Some(config_cache)) = (current_gen, guard.as_ref()) {
+        let gen_matches = config_cache.last_generation == current_gen;
+        let ttl_ok = (now - config_cache.last_update) < DATASTORE_CONFIG_CACHE_TTL_SECS;
+
+        if gen_matches && ttl_ok {
+            return Ok((
+                config_cache.config.clone(),
+                Some(config_cache.last_generation),
+            ));
         }
     }
 
@@ -312,16 +322,28 @@ fn datastore_section_config_cached() -> Result<(Arc<SectionConfigData>, Option<u
     let (config_raw, _digest) = pbs_config::datastore::config()?;
     let config = Arc::new(config_raw);
 
-    if let Some(gen_val) = gen {
+    // Update cache
+    let new_gen = if let Some(handle) = version_cache {
+        // Bump datastore generation whenever we reload the config.
+        // This ensures that Drop handlers will detect that a newer config exists
+        // and will not rely on a stale cached entry for maintenance mandate.
+        let prev_gen = handle.increase_datastore_generation();
+        let new_gen = prev_gen + 1;
+
         *guard = Some(DatastoreConfigCache {
             config: config.clone(),
-            last_generation: gen_val,
+            last_generation: new_gen,
+            last_update: now,
         });
+
+        Some(new_gen)
     } else {
+        // if the cache was not available, use again the slow path next time
         *guard = None;
-    }
+        None
+    };
 
-    Ok((config, gen))
+    Ok((config, new_gen))
 }
 
 impl 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] 5+ messages in thread

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

Thread overview: 5+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2025-11-14 15:05 [pbs-devel] [PATCH proxmox-backup v2 0/4] datastore: remove config reload on hot path Samuel Rufinatscha
2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 1/4] partial fix #6049: config: enable config version cache for datastore Samuel Rufinatscha
2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 2/4] partial fix #6049: datastore: impl ConfigVersionCache fast path for lookups Samuel Rufinatscha
2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 3/4] partial fix #6049: datastore: use config fast-path in Drop Samuel Rufinatscha
2025-11-14 15:05 ` [pbs-devel] [PATCH proxmox-backup v2 4/4] partial fix #6049: datastore: add TTL fallback to catch manual config edits Samuel Rufinatscha

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