]> git.ipfire.org Git - thirdparty/kernel/stable.git/commitdiff
btrfs: try to search for data csums in commit root
authorBoris Burkov <boris@bur.io>
Mon, 21 Oct 2024 19:01:53 +0000 (12:01 -0700)
committerDavid Sterba <dsterba@suse.com>
Mon, 22 Sep 2025 08:54:31 +0000 (10:54 +0200)
If you run a workload with:

- a cgroup that does tons of parallel data reading, with a working set
  much larger than its memory limit
- a second cgroup that writes relatively fewer files, with overwrites,
  with no memory limit
(see full code listing at the bottom for a reproducer)

Then what quickly occurs is:

- we have a large number of threads trying to read the csum tree
- we have a decent number of threads deleting csums running delayed refs
- we have a large number of threads in direct reclaim and thus high
  memory pressure

The result of this is that we writeback the csum tree repeatedly mid
transaction, to get back the extent_buffer folios for reclaim. As a
result, we repeatedly COW the csum tree for the delayed refs that are
deleting csums. This means repeatedly write locking the higher levels of
the tree.

As a result of this, we achieve an unpleasant priority inversion. We
have:

- a high degree of contention on the csum root node (and other upper
  nodes) eb rwsem
- a memory starved cgroup doing tons of reclaim on CPU.
- many reader threads in the memory starved cgroup "holding" the sem
  as readers, but not scheduling promptly. i.e., task __state == 0, but
  not running on a cpu.
- btrfs_commit_transaction stuck trying to acquire the sem as a writer.
  (running delayed_refs, deleting csums for unreferenced data extents)

This results in arbitrarily long transactions. This then results in
seriously degraded performance for any cgroup using the filesystem (the
victim cgroup in the script).

It isn't an academic problem, as we see this exact problem in production
at Meta with one cgroup over its memory limit ruining btrfs performance
for the whole system, stalling critical system services that depend on
btrfs syncs.

The underlying scheduling "problem" with global rwsems is sort of thorny
and apparently well known and was discussed at LPC 2024, for example.

As a result, our main lever in the short term is just trying to reduce
contention on our various rwsems with an eye to reducing the frequency
of write locking, to avoid disabling the read lock fast acquisition path.

Luckily, it seems likely that many reads are for old extents written
many transactions ago, and that for those we *can* in fact search the
commit root. The commit_root_sem only gets taken write once, near the
end of transaction commit, no matter how much memory pressure there is,
so we have much less contention between readers and writers.

This change detects when we are trying to read an old extent (according
to extent map generation) and then wires that through bio_ctrl to the
btrfs_bio, which unfortunately isn't allocated yet when we have this
information. When we go to lookup the csums in lookup_bio_sums we can
check this condition on the btrfs_bio and do the commit root lookup
accordingly.

Note that a single bio_ctrl might collect a few extent_maps into a single
bio, so it is important to track a maximum generation across all the
extent_maps used for each bio to make an accurate decision on whether it
is valid to look in the commit root. If any extent_map is updated in the
current generation, we can't use the commit root.

To test and reproduce this issue, I used the following script and
accompanying C program (to avoid bottlenecks in constantly forking
thousands of dd processes):

====== big-read.c ======
  #include <fcntl.h>
  #include <stdio.h>
  #include <stdlib.h>
  #include <sys/mman.h>
  #include <sys/stat.h>
  #include <unistd.h>
  #include <errno.h>

  #define BUF_SZ (128 * (1 << 10UL))

  int read_once(int fd, size_t sz) {
   char buf[BUF_SZ];
   size_t rd = 0;
   int ret = 0;

   while (rd < sz) {
   ret = read(fd, buf, BUF_SZ);
   if (ret < 0) {
   if (errno == EINTR)
   continue;
   fprintf(stderr, "read failed: %d\n", errno);
   return -errno;
   } else if (ret == 0) {
   break;
   } else {
   rd += ret;
   }
   }
   return rd;
  }

  int read_loop(char *fname) {
   int fd;
   struct stat st;
   size_t sz = 0;
   int ret;

   while (1) {
   fd = open(fname, O_RDONLY);
   if (fd == -1) {
   perror("open");
   return 1;
   }
   if (!sz) {
   if (!fstat(fd, &st)) {
   sz = st.st_size;
   } else {
   perror("stat");
   return 1;
   }
   }

                  ret = read_once(fd, sz);
   close(fd);
   }
  }

  int main(int argc, char *argv[]) {
   int fd;
   struct stat st;
   off_t sz;
   char *buf;
   int ret;

   if (argc != 2) {
   fprintf(stderr, "Usage: %s <filename>\n", argv[0]);
   return 1;
   }

   return read_loop(argv[1]);
  }

====== repro.sh ======
  #!/usr/bin/env bash

  SCRIPT=$(readlink -f "$0")
  DIR=$(dirname "$SCRIPT")

  dev=$1
  mnt=$2
  shift
  shift

  CG_ROOT=/sys/fs/cgroup
  BAD_CG=$CG_ROOT/bad-nbr
  GOOD_CG=$CG_ROOT/good-nbr
  NR_BIGGOS=1
  NR_LITTLE=10
  NR_VICTIMS=32
  NR_VILLAINS=512

  START_SEC=$(date +%s)

  _elapsed() {
   echo "elapsed: $(($(date +%s) - $START_SEC))"
  }

  _stats() {
   local sysfs=/sys/fs/btrfs/$(findmnt -no UUID $dev)

   echo "================"
   date
   _elapsed
   cat $sysfs/commit_stats
   cat $BAD_CG/memory.pressure
  }

  _setup_cgs() {
   echo "+memory +cpuset" > $CG_ROOT/cgroup.subtree_control
   mkdir -p $GOOD_CG
   mkdir -p $BAD_CG
   echo max > $BAD_CG/memory.max
   # memory.high much less than the working set will cause heavy reclaim
   echo $((1 << 30)) > $BAD_CG/memory.high

   # victims get a subset of villain CPUs
   echo 0 > $GOOD_CG/cpuset.cpus
   echo 0,1,2,3 > $BAD_CG/cpuset.cpus
  }

  _kill_cg() {
   local cg=$1
   local attempts=0
   echo "kill cgroup $cg"
   [ -f $cg/cgroup.procs ] || return
   while true; do
   attempts=$((attempts + 1))
   echo 1 > $cg/cgroup.kill
   sleep 1
   procs=$(wc -l $cg/cgroup.procs | cut -d' ' -f1)
   [ $procs -eq 0 ] && break
   done
   rmdir $cg
   echo "killed cgroup $cg in $attempts attempts"
  }

  _biggo_vol() {
   echo $mnt/biggo_vol.$1
  }

  _biggo_file() {
   echo $(_biggo_vol $1)/biggo
  }

  _subvoled_biggos() {
   total_sz=$((10 << 30))
   per_sz=$((total_sz / $NR_VILLAINS))
   dd_count=$((per_sz >> 20))
   echo "create $NR_VILLAINS subvols with a file of size $per_sz bytes for a total of $total_sz bytes."
   for i in $(seq $NR_VILLAINS)
   do
   btrfs subvol create $(_biggo_vol $i) &>/dev/null
   dd if=/dev/zero of=$(_biggo_file $i) bs=1M count=$dd_count &>/dev/null
   done
   echo "done creating subvols."
  }

  _setup() {
   [ -f .done ] && rm .done
   findmnt -n $dev && exit 1
        if [ -f .re-mkfs ]; then
mkfs.btrfs -f -m single -d single $dev >/dev/null || exit 2
else
echo "touch .re-mkfs to populate the test fs"
fi

   mount -o noatime $dev $mnt || exit 3
   [ -f .re-mkfs ] && _subvoled_biggos
   _setup_cgs
  }

  _my_cleanup() {
   echo "CLEANUP!"
   _kill_cg $BAD_CG
   _kill_cg $GOOD_CG
   sleep 1
   umount $mnt
  }

  _bad_exit() {
   _err "Unexpected Exit! $?"
   _stats
   exit $?
  }

  trap _my_cleanup EXIT
  trap _bad_exit INT TERM

  _setup

  # Use a lot of page cache reading the big file
  _villain() {
   local i=$1
   echo $BASHPID > $BAD_CG/cgroup.procs
   $DIR/big-read $(_biggo_file $i)
  }

  # Hit del_csum a lot by overwriting lots of small new files
  _victim() {
   echo $BASHPID > $GOOD_CG/cgroup.procs
   i=0;
   while (true)
   do
   local tmp=$mnt/tmp.$i

   dd if=/dev/zero of=$tmp bs=4k count=2 >/dev/null 2>&1
   i=$((i+1))
   [ $i -eq $NR_LITTLE ] && i=0
   done
  }

  _one_sync() {
   echo "sync..."
   before=$(date +%s)
   sync
   after=$(date +%s)
   echo "sync done in $((after - before))s"
   _stats
  }

  # sync in a loop
  _sync() {
   echo "start sync loop"
   syncs=0
   echo $BASHPID > $GOOD_CG/cgroup.procs
   while true
   do
   [ -f .done ] && break
   _one_sync
   syncs=$((syncs + 1))
   [ -f .done ] && break
   sleep 10
   done
   if [ $syncs -eq 0 ]; then
   echo "do at least one sync!"
   _one_sync
   fi
   echo "sync loop done."
  }

  _sleep() {
   local time=${1-60}
   local now=$(date +%s)
   local end=$((now + time))
   while [ $now -lt $end ];
   do
   echo "SLEEP: $((end - now))s left. Sleep 10."
   sleep 10
   now=$(date +%s)
   done
  }

  echo "start $NR_VILLAINS villains"
  for i in $(seq $NR_VILLAINS)
  do
   _villain $i &
   disown # get rid of annoying log on kill (done via cgroup anyway)
  done

  echo "start $NR_VICTIMS victims"
  for i in $(seq $NR_VICTIMS)
  do
   _victim &
   disown
  done

  _sync &
  SYNC_PID=$!

  _sleep $1
  _elapsed
  touch .done
  wait $SYNC_PID

  echo "OK"
  exit 0

Without this patch, that reproducer:

- Ran for 6+ minutes instead of 60s
- Hung hundreds of threads in D state on the csum reader lock
- Got a commit stuck for 3 minutes

sync done in 388s
================
Wed Jul  9 09:52:31 PM UTC 2025
elapsed: 420
commits 2
cur_commit_ms 0
last_commit_ms 159446
max_commit_ms 159446
total_commit_ms 160058
some avg10=99.03 avg60=98.97 avg300=75.43 total=418033386
full avg10=82.79 avg60=80.52 avg300=59.45 total=324995274

419 hits state R, D comms big-read
                 btrfs_tree_read_lock_nested
                 btrfs_read_lock_root_node
                 btrfs_search_slot
                 btrfs_lookup_csum
                 btrfs_lookup_bio_sums
                 btrfs_submit_bbio

1 hits state D comms btrfs-transacti
                 btrfs_tree_lock_nested
                 btrfs_lock_root_node
                 btrfs_search_slot
                 btrfs_del_csums
                 __btrfs_run_delayed_refs
                 btrfs_run_delayed_refs

With the patch, the reproducer exits naturally, in 65s, completing a
pretty decent 4 commits, despite heavy memory pressure. Occasionally you
can still trigger a rather long commit (couple seconds) but never one
that is minutes long.

sync done in 3s
================
elapsed: 65
commits 4
cur_commit_ms 0
last_commit_ms 485
max_commit_ms 689
total_commit_ms 2453
some avg10=98.28 avg60=64.54 avg300=19.39 total=64849893
full avg10=74.43 avg60=48.50 avg300=14.53 total=48665168

some random rwalker samples showed the most common stack in reclaim,
rather than the csum tree:
145 hits state R comms bash, sleep, dd, shuf
                 shrink_folio_list
                 shrink_lruvec
                 shrink_node
                 do_try_to_free_pages
                 try_to_free_mem_cgroup_pages
                 reclaim_high

Link: https://lpc.events/event/18/contributions/1883/
Reviewed-by: Filipe Manana <fdmanana@suse.com>
Signed-off-by: Boris Burkov <boris@bur.io>
Signed-off-by: David Sterba <dsterba@suse.com>
fs/btrfs/bio.c
fs/btrfs/bio.h
fs/btrfs/compression.c
fs/btrfs/extent_io.c
fs/btrfs/file-item.c

index 50b5fc1c06d7cc5d067e23211a0f3e32b751fa54..ea7f7a17a3d5bbf709da7dd57060c5a103eac3b4 100644 (file)
@@ -93,6 +93,7 @@ static struct btrfs_bio *btrfs_split_bio(struct btrfs_fs_info *fs_info,
                refcount_inc(&orig_bbio->ordered->refs);
                bbio->ordered = orig_bbio->ordered;
        }
+       bbio->csum_search_commit_root = orig_bbio->csum_search_commit_root;
        atomic_inc(&orig_bbio->pending_ios);
        return bbio;
 }
index dc2eb43b70970b110c47697f462b81013de5beca..00883aea55d70fb3b3e1983b1bec2d7a08f32d05 100644 (file)
@@ -82,6 +82,8 @@ struct btrfs_bio {
        /* Save the first error status of split bio. */
        blk_status_t status;
 
+       /* Use the commit root to look up csums (data read bio only). */
+       bool csum_search_commit_root;
        /*
         * This member must come last, bio_alloc_bioset will allocate enough
         * bytes for entire btrfs_bio but relies on bio being last.
index 35e3071cec0636f6c0b1ebc47295e997908da73d..06e119ee2649242ac12c0db49b69801ca96a106a 100644 (file)
@@ -602,6 +602,7 @@ void btrfs_submit_compressed_read(struct btrfs_bio *bbio)
        cb->compressed_len = compressed_len;
        cb->compress_type = btrfs_extent_map_compression(em);
        cb->orig_bbio = bbio;
+       cb->bbio.csum_search_commit_root = bbio->csum_search_commit_root;
 
        btrfs_free_extent_map(em);
 
index b21cb72835ccf46c4122f0e501d088578641be0a..7ab45b2346216e3117a54bb548ea6f525b538b77 100644 (file)
@@ -101,6 +101,26 @@ struct btrfs_bio_ctrl {
        enum btrfs_compression_type compress_type;
        u32 len_to_oe_boundary;
        blk_opf_t opf;
+       /*
+        * For data read bios, we attempt to optimize csum lookups if the extent
+        * generation is older than the current one. To make this possible, we
+        * need to track the maximum generation of an extent in a bio_ctrl to
+        * make the decision when submitting the bio.
+        *
+        * The pattern between do_readpage(), submit_one_bio() and
+        * submit_extent_folio() is quite subtle, so tracking this is tricky.
+        *
+        * As we process extent E, we might submit a bio with existing built up
+        * extents before adding E to a new bio, or we might just add E to the
+        * bio. As a result, E's generation could apply to the current bio or
+        * to the next one, so we need to be careful to update the bio_ctrl's
+        * generation with E's only when we are sure E is added to bio_ctrl->bbio
+        * in submit_extent_folio().
+        *
+        * See the comment in btrfs_lookup_bio_sums() for more detail on the
+        * need for this optimization.
+        */
+       u64 generation;
        btrfs_bio_end_io_t end_io_func;
        struct writeback_control *wbc;
 
@@ -131,6 +151,26 @@ struct btrfs_bio_ctrl {
        u64 last_em_start;
 };
 
+/*
+ * Helper to set the csum search commit root option for a bio_ctrl's bbio
+ * before submitting the bio.
+ *
+ * Only for use by submit_one_bio().
+ */
+static void bio_set_csum_search_commit_root(struct btrfs_bio_ctrl *bio_ctrl)
+{
+       struct btrfs_bio *bbio = bio_ctrl->bbio;
+
+       ASSERT(bbio);
+
+       if (!(btrfs_op(&bbio->bio) == BTRFS_MAP_READ && is_data_inode(bbio->inode)))
+               return;
+
+       bio_ctrl->bbio->csum_search_commit_root =
+               (bio_ctrl->generation &&
+                bio_ctrl->generation < btrfs_get_fs_generation(bbio->inode->root->fs_info));
+}
+
 static void submit_one_bio(struct btrfs_bio_ctrl *bio_ctrl)
 {
        struct btrfs_bio *bbio = bio_ctrl->bbio;
@@ -141,6 +181,8 @@ static void submit_one_bio(struct btrfs_bio_ctrl *bio_ctrl)
        /* Caller should ensure the bio has at least some range added */
        ASSERT(bbio->bio.bi_iter.bi_size);
 
+       bio_set_csum_search_commit_root(bio_ctrl);
+
        if (btrfs_op(&bbio->bio) == BTRFS_MAP_READ &&
            bio_ctrl->compress_type != BTRFS_COMPRESS_NONE)
                btrfs_submit_compressed_read(bbio);
@@ -149,6 +191,12 @@ static void submit_one_bio(struct btrfs_bio_ctrl *bio_ctrl)
 
        /* The bbio is owned by the end_io handler now */
        bio_ctrl->bbio = NULL;
+       /*
+        * We used the generation to decide whether to lookup csums in the
+        * commit_root or not when we called bio_set_csum_search_commit_root()
+        * above. Now, reset the generation for the next bio.
+        */
+       bio_ctrl->generation = 0;
 }
 
 /*
@@ -719,6 +767,8 @@ static void alloc_new_bio(struct btrfs_inode *inode,
  * @size:      portion of page that we want to write to
  * @pg_offset: offset of the new bio or to check whether we are adding
  *              a contiguous page to the previous one
+ * @read_em_generation: generation of the extent_map we are submitting
+ *                     (only used for read)
  *
  * The will either add the page into the existing @bio_ctrl->bbio, or allocate a
  * new one in @bio_ctrl->bbio.
@@ -727,7 +777,8 @@ static void alloc_new_bio(struct btrfs_inode *inode,
  */
 static void submit_extent_folio(struct btrfs_bio_ctrl *bio_ctrl,
                               u64 disk_bytenr, struct folio *folio,
-                              size_t size, unsigned long pg_offset)
+                              size_t size, unsigned long pg_offset,
+                              u64 read_em_generation)
 {
        struct btrfs_inode *inode = folio_to_inode(folio);
        loff_t file_offset = folio_pos(folio) + pg_offset;
@@ -758,6 +809,11 @@ static void submit_extent_folio(struct btrfs_bio_ctrl *bio_ctrl,
                        submit_one_bio(bio_ctrl);
                        continue;
                }
+               /*
+                * Now that the folio is definitely added to the bio, include its
+                * generation in the max generation calculation.
+                */
+               bio_ctrl->generation = max(bio_ctrl->generation, read_em_generation);
                bio_ctrl->next_file_offset += len;
 
                if (bio_ctrl->wbc)
@@ -960,6 +1016,7 @@ static int btrfs_do_readpage(struct folio *folio, struct extent_map **em_cached,
                bool force_bio_submit = false;
                u64 disk_bytenr;
                u64 block_start;
+               u64 em_gen;
 
                ASSERT(IS_ALIGNED(cur, fs_info->sectorsize));
                if (cur >= last_byte) {
@@ -1043,6 +1100,7 @@ static int btrfs_do_readpage(struct folio *folio, struct extent_map **em_cached,
 
                bio_ctrl->last_em_start = em->start;
 
+               em_gen = em->generation;
                btrfs_free_extent_map(em);
                em = NULL;
 
@@ -1066,7 +1124,7 @@ static int btrfs_do_readpage(struct folio *folio, struct extent_map **em_cached,
                if (force_bio_submit)
                        submit_one_bio(bio_ctrl);
                submit_extent_folio(bio_ctrl, disk_bytenr, folio, blocksize,
-                                   pg_offset);
+                                   pg_offset, em_gen);
        }
        return 0;
 }
@@ -1600,7 +1658,7 @@ static int submit_one_sector(struct btrfs_inode *inode,
        ASSERT(folio_test_writeback(folio));
 
        submit_extent_folio(bio_ctrl, disk_bytenr, folio,
-                           sectorsize, filepos - folio_pos(folio));
+                           sectorsize, filepos - folio_pos(folio), 0);
        return 0;
 }
 
index c09fbc257634aba66eb4fb06bf06ba350a7c4693..4dd3d8a02519eccabe9da8d303a2e9193a56bd2d 100644 (file)
@@ -397,6 +397,36 @@ int btrfs_lookup_bio_sums(struct btrfs_bio *bbio)
                path->skip_locking = 1;
        }
 
+       /*
+        * If we are searching for a csum of an extent from a past
+        * transaction, we can search in the commit root and reduce
+        * lock contention on the csum tree extent buffers.
+        *
+        * This is important because that lock is an rwsem which gets
+        * pretty heavy write load under memory pressure and sustained
+        * csum overwrites, unlike the commit_root_sem. (Memory pressure
+        * makes us writeback the nodes multiple times per transaction,
+        * which makes us cow them each time, taking the write lock.)
+        *
+        * Due to how rwsem is implemented, there is a possible
+        * priority inversion where the readers holding the lock don't
+        * get scheduled (say they're in a cgroup stuck in heavy reclaim)
+        * which then blocks writers, including transaction commit. By
+        * using a semaphore with fewer writers (only a commit switching
+        * the roots), we make this issue less likely.
+        *
+        * Note that we don't rely on btrfs_search_slot to lock the
+        * commit root csum. We call search_slot multiple times, which would
+        * create a potential race where a commit comes in between searches
+        * while we are not holding the commit_root_sem, and we get csums
+        * from across transactions.
+        */
+       if (bbio->csum_search_commit_root) {
+               path->search_commit_root = 1;
+               path->skip_locking = 1;
+               down_read(&fs_info->commit_root_sem);
+       }
+
        while (bio_offset < orig_len) {
                int count;
                u64 cur_disk_bytenr = orig_disk_bytenr + bio_offset;
@@ -442,6 +472,8 @@ int btrfs_lookup_bio_sums(struct btrfs_bio *bbio)
                bio_offset += count * sectorsize;
        }
 
+       if (bbio->csum_search_commit_root)
+               up_read(&fs_info->commit_root_sem);
        return ret;
 }