linux-btrfs.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/3] btrfs: use btrfs_path::reada to replace the
@ 2021-12-13 13:10 Qu Wenruo
  2021-12-13 13:10 ` [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity() Qu Wenruo
                   ` (3 more replies)
  0 siblings, 4 replies; 7+ messages in thread
From: Qu Wenruo @ 2021-12-13 13:10 UTC (permalink / raw)
  To: linux-btrfs

[PROBLEMS]
The metadata readahead code is introduced in 2011 (surprisingly, the
commit message even contains changelog), but now there is only one user
for it, and even for the only one user, the readahead mechanism can't
provide all it needs:

- No support for commit tree readahead
  Only support readahead on current tree.

- Bad layer separation
  To manage on-fly bios, btrfs_reada_add() mechanism internally manages
  a kinda complex zone system, and it's bind to per-device.

  This is against the common layer separation, as metadata should all be
  in btrfs logical address space, should not be bound to device physical
  layer.

  In fact, this is the cause of all recent reada related bugs.

- No caller for asynchronous metadata readahead
  Even btrfs_reada_add() is designed to be fully asynchronous, scrub
  only calls it in a synchronous way (call btrfs_reada_add() and
  immediately call btrfs_reada_wait()).
  Thus rendering a lot of code unnecessary.

[ALTERNATIVE]
On the other hand, we have btrfs_path::reada mechanism, which is already
used by tons of btrfs sub-systems like send.

[MODIFICATION]
This patch will use btrfs_path::reada to replace btrfs_reada_add()
mechanism.

[BENCHMARK]
The conclusion looks like this:

For the worst case (no dirty metadata, slow HDD), there will be around 5%
performance drop for scrub.
For other cases (even SATA SSD), there is no distinguishable performance
difference.

The number is reported scrub speed, in MiB/s.
The resolution is limited by the reported duration, which only has a
resolution of 1 second.

	Old		New		Diff
SSD	455.3		466.332		+2.42%
HDD	103.927 	98.012		-5.69%


[BENCHMARK DETAILS]
Both tests are done in the same host and VM, the VM has one dedicated
SSD and one dedicated HDD attached to it (virtio driver though)

Host:
CPU:	5900X
RAM:	DDR4 3200MT, no ECC

	During the benchmark, there is no other active other than light
	DE operations.

VM:
vCPU:	16x
RAM:	4G

	The VM kernel doesn't have any heavy debug options to screw up
	the benchmark result.

Test drives:
SSD:	500G SATA SSD
	Crucial CT500MX500SSD1
	(With quite some wear, as it's my main test drive for fstests)

HDD:	2T 5400rpm SATA HDD (device-managed SMR)
	WDC WD20SPZX-22UA7T0
	(Very new, invested just for this benchmark)

Filesystem contents:
For filesystems on SSD and HDD, they are generated using the following
fio job file:

  [scrub-populate]
  directory=/mnt/btrfs
  nrfiles=16384
  openfiles=16
  filesize=2k-512k
  readwrite=randwrite
  ioengine=libaio
  fallocate=none
  numjobs=4
  
Then randomly remove 4096 files (1/16th of total files) to create enough
gaps in extent tree.

Finally run scrub on each filesystem 5 times, with cycle mount and
module reload between each run.

Full result can be fetched here:
https://docs.google.com/spreadsheets/d/1cwUAlbKPfp4baKrS92debsCt6Ejqvxr_Ylspj_SDFT0/edit?usp=sharing

[CHANGELOG]
RFC->v1:
- Add benchmark result
- Add extent tree readahead using btrfs_path::reada

Qu Wenruo (3):
  btrfs: remove the unnecessary path parameter for scrub_raid56_parity()
  btrfs: remove reada mechanism
  btrfs: use btrfs_path::reada for scrub extent tree readahead

 fs/btrfs/Makefile      |    2 +-
 fs/btrfs/ctree.h       |   25 -
 fs/btrfs/dev-replace.c |    5 -
 fs/btrfs/disk-io.c     |   20 +-
 fs/btrfs/extent_io.c   |    3 -
 fs/btrfs/reada.c       | 1086 ----------------------------------------
 fs/btrfs/scrub.c       |   65 +--
 fs/btrfs/super.c       |    1 -
 fs/btrfs/volumes.c     |    7 -
 fs/btrfs/volumes.h     |    7 -
 10 files changed, 18 insertions(+), 1203 deletions(-)
 delete mode 100644 fs/btrfs/reada.c

-- 
2.34.1


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

* [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity()
  2021-12-13 13:10 [PATCH 0/3] btrfs: use btrfs_path::reada to replace the Qu Wenruo
@ 2021-12-13 13:10 ` Qu Wenruo
  2021-12-14 12:44   ` Qu Wenruo
  2021-12-13 13:10 ` [PATCH 2/3] btrfs: remove reada mechanism Qu Wenruo
                   ` (2 subsequent siblings)
  3 siblings, 1 reply; 7+ messages in thread
From: Qu Wenruo @ 2021-12-13 13:10 UTC (permalink / raw)
  To: linux-btrfs

In function scrub_stripe() we allocated two btrfs_path, one @path for
extent tree search and another @ppath for full stripe extent tree search
for RAID56.

This is totally uncessary, as the @ppath usage is completely inside
scrub_raid56_parity(), thus we can move the path allocation into
scrub_raid56_parity() completely.

Signed-off-by: Qu Wenruo <wqu@suse.com>
---
 fs/btrfs/scrub.c | 28 ++++++++++++++--------------
 1 file changed, 14 insertions(+), 14 deletions(-)

diff --git a/fs/btrfs/scrub.c b/fs/btrfs/scrub.c
index 15a123e67108..a3bd3dfd5e8b 100644
--- a/fs/btrfs/scrub.c
+++ b/fs/btrfs/scrub.c
@@ -2892,7 +2892,6 @@ static void scrub_parity_put(struct scrub_parity *sparity)
 static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
 						  struct map_lookup *map,
 						  struct btrfs_device *sdev,
-						  struct btrfs_path *path,
 						  u64 logic_start,
 						  u64 logic_end)
 {
@@ -2901,6 +2900,7 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
 	struct btrfs_root *csum_root;
 	struct btrfs_extent_item *extent;
 	struct btrfs_io_context *bioc = NULL;
+	struct btrfs_path *path;
 	u64 flags;
 	int ret;
 	int slot;
@@ -2919,6 +2919,14 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
 	int extent_mirror_num;
 	int stop_loop = 0;
 
+	path = btrfs_alloc_path();
+	if (!path) {
+		spin_lock(&sctx->stat_lock);
+		sctx->stat.malloc_errors++;
+		spin_unlock(&sctx->stat_lock);
+		return -ENOMEM;
+	}
+
 	ASSERT(map->stripe_len <= U32_MAX);
 	nsectors = map->stripe_len >> fs_info->sectorsize_bits;
 	bitmap_len = scrub_calc_parity_bitmap_len(nsectors);
@@ -2928,6 +2936,7 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
 		spin_lock(&sctx->stat_lock);
 		sctx->stat.malloc_errors++;
 		spin_unlock(&sctx->stat_lock);
+		btrfs_free_path(path);
 		return -ENOMEM;
 	}
 
@@ -3117,7 +3126,7 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
 	scrub_wr_submit(sctx);
 	mutex_unlock(&sctx->wr_lock);
 
-	btrfs_release_path(path);
+	btrfs_free_path(path);
 	return ret < 0 ? ret : 0;
 }
 
@@ -3167,7 +3176,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 					   int num, u64 base, u64 length,
 					   struct btrfs_block_group *cache)
 {
-	struct btrfs_path *path, *ppath;
+	struct btrfs_path *path;
 	struct btrfs_fs_info *fs_info = sctx->fs_info;
 	struct btrfs_root *root;
 	struct btrfs_root *csum_root;
@@ -3229,12 +3238,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 	if (!path)
 		return -ENOMEM;
 
-	ppath = btrfs_alloc_path();
-	if (!ppath) {
-		btrfs_free_path(path);
-		return -ENOMEM;
-	}
-
 	/*
 	 * work on commit root. The related disk blocks are static as
 	 * long as COW is applied. This means, it is save to rewrite
@@ -3243,8 +3246,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 	path->search_commit_root = 1;
 	path->skip_locking = 1;
 
-	ppath->search_commit_root = 1;
-	ppath->skip_locking = 1;
 	/*
 	 * trigger the readahead for extent tree csum tree and wait for
 	 * completion. During readahead, the scrub is officially paused
@@ -3347,7 +3348,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 				stripe_logical += base;
 				stripe_end = stripe_logical + increment;
 				ret = scrub_raid56_parity(sctx, map, scrub_dev,
-							  ppath, stripe_logical,
+							  stripe_logical,
 							  stripe_end);
 				if (ret)
 					goto out;
@@ -3518,7 +3519,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 						stripe_end = stripe_logical +
 								increment;
 						ret = scrub_raid56_parity(sctx,
-							map, scrub_dev, ppath,
+							map, scrub_dev,
 							stripe_logical,
 							stripe_end);
 						if (ret)
@@ -3565,7 +3566,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 
 	blk_finish_plug(&plug);
 	btrfs_free_path(path);
-	btrfs_free_path(ppath);
 
 	if (sctx->is_dev_replace && ret >= 0) {
 		int ret2;
-- 
2.34.1


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

* [PATCH 2/3] btrfs: remove reada mechanism
  2021-12-13 13:10 [PATCH 0/3] btrfs: use btrfs_path::reada to replace the Qu Wenruo
  2021-12-13 13:10 ` [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity() Qu Wenruo
@ 2021-12-13 13:10 ` Qu Wenruo
  2021-12-13 13:10 ` [PATCH 3/3] btrfs: use btrfs_path::reada for scrub extent tree readahead Qu Wenruo
  2021-12-14 12:41 ` [PATCH 0/3] btrfs: use btrfs_path::reada to replace the David Sterba
  3 siblings, 0 replies; 7+ messages in thread
From: Qu Wenruo @ 2021-12-13 13:10 UTC (permalink / raw)
  To: linux-btrfs

Currently there is only one user for btrfs metadata readahead, and
that's scrub.

But even for the single user, it's not providing the correct
functionality it needs, as scrub needs reada for commit root, which
current readahead can't provide. (Although it's pretty easy to add such
feature).

Despite this, there are some extra problems related to metadata
readahead:

- Duplicated feature with btrfs_path::reada

- Partly duplicated feature of btrfs_fs_info::buffer_radix
  Btrfs already caches its metadata in buffer_radix, while readahead
  tries to read the tree block no matter if it's already cached.

- Poor layer separation
  To my surprise, metadata readahead works kinda at device level.
  This is definitely not the correct layer it should be, since metadata
  is at btrfs logical address space, it should not bother device at all.

  This brings extra chance for bugs to sneak in, while brings
  unnecessary complexity.

- Dead codes
  In the very beginning of scrub.c we have #undef DEBUG, rendering all
  the debug related code useless and unable to test.

Thus here I purpose to remove the metadata readahead mechanism
completely.

Signed-off-by: Qu Wenruo <wqu@suse.com>
---
 fs/btrfs/Makefile      |    2 +-
 fs/btrfs/ctree.h       |   25 -
 fs/btrfs/dev-replace.c |    5 -
 fs/btrfs/disk-io.c     |   20 +-
 fs/btrfs/extent_io.c   |    3 -
 fs/btrfs/reada.c       | 1086 ----------------------------------------
 fs/btrfs/scrub.c       |   36 --
 fs/btrfs/super.c       |    1 -
 fs/btrfs/volumes.c     |    7 -
 fs/btrfs/volumes.h     |    7 -
 10 files changed, 3 insertions(+), 1189 deletions(-)
 delete mode 100644 fs/btrfs/reada.c

diff --git a/fs/btrfs/Makefile b/fs/btrfs/Makefile
index 3dcf9bcc2326..4188ba3fd8c3 100644
--- a/fs/btrfs/Makefile
+++ b/fs/btrfs/Makefile
@@ -27,7 +27,7 @@ btrfs-y += super.o ctree.o extent-tree.o print-tree.o root-tree.o dir-item.o \
 	   extent_io.o volumes.o async-thread.o ioctl.o locking.o orphan.o \
 	   export.o tree-log.o free-space-cache.o zlib.o lzo.o zstd.o \
 	   compression.o delayed-ref.o relocation.o delayed-inode.o scrub.o \
-	   reada.o backref.o ulist.o qgroup.o send.o dev-replace.o raid56.o \
+	   backref.o ulist.o qgroup.o send.o dev-replace.o raid56.o \
 	   uuid-tree.o props.o free-space-tree.o tree-checker.o space-info.o \
 	   block-rsv.o delalloc-space.o block-group.o discard.o reflink.o \
 	   subpage.o tree-mod-log.o
diff --git a/fs/btrfs/ctree.h b/fs/btrfs/ctree.h
index dfee4b403da1..eeda7b687fca 100644
--- a/fs/btrfs/ctree.h
+++ b/fs/btrfs/ctree.h
@@ -821,7 +821,6 @@ struct btrfs_fs_info {
 	struct btrfs_workqueue *endio_write_workers;
 	struct btrfs_workqueue *endio_freespace_worker;
 	struct btrfs_workqueue *caching_workers;
-	struct btrfs_workqueue *readahead_workers;
 
 	/*
 	 * fixup workers take dirty pages that didn't properly go through
@@ -958,13 +957,6 @@ struct btrfs_fs_info {
 
 	struct btrfs_delayed_root *delayed_root;
 
-	/* readahead tree */
-	spinlock_t reada_lock;
-	struct radix_tree_root reada_tree;
-
-	/* readahead works cnt */
-	atomic_t reada_works_cnt;
-
 	/* Extent buffer radix tree */
 	spinlock_t buffer_lock;
 	/* Entries are eb->start / sectorsize */
@@ -3836,23 +3828,6 @@ static inline void btrfs_bio_counter_dec(struct btrfs_fs_info *fs_info)
 	btrfs_bio_counter_sub(fs_info, 1);
 }
 
-/* reada.c */
-struct reada_control {
-	struct btrfs_fs_info	*fs_info;		/* tree to prefetch */
-	struct btrfs_key	key_start;
-	struct btrfs_key	key_end;	/* exclusive */
-	atomic_t		elems;
-	struct kref		refcnt;
-	wait_queue_head_t	wait;
-};
-struct reada_control *btrfs_reada_add(struct btrfs_root *root,
-			      struct btrfs_key *start, struct btrfs_key *end);
-int btrfs_reada_wait(void *handle);
-void btrfs_reada_detach(void *handle);
-int btree_readahead_hook(struct extent_buffer *eb, int err);
-void btrfs_reada_remove_dev(struct btrfs_device *dev);
-void btrfs_reada_undo_remove_dev(struct btrfs_device *dev);
-
 static inline int is_fstree(u64 rootid)
 {
 	if (rootid == BTRFS_FS_TREE_OBJECTID ||
diff --git a/fs/btrfs/dev-replace.c b/fs/btrfs/dev-replace.c
index 66fa61cb3f23..62b9651ea662 100644
--- a/fs/btrfs/dev-replace.c
+++ b/fs/btrfs/dev-replace.c
@@ -906,9 +906,6 @@ static int btrfs_dev_replace_finishing(struct btrfs_fs_info *fs_info,
 	}
 	btrfs_wait_ordered_roots(fs_info, U64_MAX, 0, (u64)-1);
 
-	if (!scrub_ret)
-		btrfs_reada_remove_dev(src_device);
-
 	/*
 	 * We have to use this loop approach because at this point src_device
 	 * has to be available for transaction commit to complete, yet new
@@ -917,7 +914,6 @@ static int btrfs_dev_replace_finishing(struct btrfs_fs_info *fs_info,
 	while (1) {
 		trans = btrfs_start_transaction(root, 0);
 		if (IS_ERR(trans)) {
-			btrfs_reada_undo_remove_dev(src_device);
 			mutex_unlock(&dev_replace->lock_finishing_cancel_unmount);
 			return PTR_ERR(trans);
 		}
@@ -968,7 +964,6 @@ static int btrfs_dev_replace_finishing(struct btrfs_fs_info *fs_info,
 		up_write(&dev_replace->rwsem);
 		mutex_unlock(&fs_info->chunk_mutex);
 		mutex_unlock(&fs_info->fs_devices->device_list_mutex);
-		btrfs_reada_undo_remove_dev(src_device);
 		btrfs_rm_dev_replace_blocked(fs_info);
 		if (tgt_device)
 			btrfs_destroy_dev_replace_tgtdev(tgt_device);
diff --git a/fs/btrfs/disk-io.c b/fs/btrfs/disk-io.c
index 5c598e124c25..8d7438aecdd0 100644
--- a/fs/btrfs/disk-io.c
+++ b/fs/btrfs/disk-io.c
@@ -665,9 +665,6 @@ static int validate_subpage_buffer(struct page *page, u64 start, u64 end,
 	if (ret < 0)
 		goto err;
 
-	if (test_and_clear_bit(EXTENT_BUFFER_READAHEAD, &eb->bflags))
-		btree_readahead_hook(eb, ret);
-
 	set_extent_buffer_uptodate(eb);
 
 	free_extent_buffer(eb);
@@ -715,10 +712,6 @@ int btrfs_validate_metadata_buffer(struct btrfs_bio *bbio,
 	}
 	ret = validate_extent_buffer(eb);
 err:
-	if (reads_done &&
-	    test_and_clear_bit(EXTENT_BUFFER_READAHEAD, &eb->bflags))
-		btree_readahead_hook(eb, ret);
-
 	if (ret) {
 		/*
 		 * our io error hook is going to dec the io pages
@@ -2224,7 +2217,6 @@ static void btrfs_stop_all_workers(struct btrfs_fs_info *fs_info)
 	btrfs_destroy_workqueue(fs_info->endio_freespace_worker);
 	btrfs_destroy_workqueue(fs_info->delayed_workers);
 	btrfs_destroy_workqueue(fs_info->caching_workers);
-	btrfs_destroy_workqueue(fs_info->readahead_workers);
 	btrfs_destroy_workqueue(fs_info->flush_workers);
 	btrfs_destroy_workqueue(fs_info->qgroup_rescan_workers);
 	if (fs_info->discard_ctl.discard_workers)
@@ -2437,9 +2429,6 @@ static int btrfs_init_workqueues(struct btrfs_fs_info *fs_info)
 	fs_info->delayed_workers =
 		btrfs_alloc_workqueue(fs_info, "delayed-meta", flags,
 				      max_active, 0);
-	fs_info->readahead_workers =
-		btrfs_alloc_workqueue(fs_info, "readahead", flags,
-				      max_active, 2);
 	fs_info->qgroup_rescan_workers =
 		btrfs_alloc_workqueue(fs_info, "qgroup-rescan", flags, 1, 0);
 	fs_info->discard_ctl.discard_workers =
@@ -2451,9 +2440,8 @@ static int btrfs_init_workqueues(struct btrfs_fs_info *fs_info)
 	      fs_info->endio_meta_write_workers &&
 	      fs_info->endio_write_workers && fs_info->endio_raid56_workers &&
 	      fs_info->endio_freespace_worker && fs_info->rmw_workers &&
-	      fs_info->caching_workers && fs_info->readahead_workers &&
-	      fs_info->fixup_workers && fs_info->delayed_workers &&
-	      fs_info->qgroup_rescan_workers &&
+	      fs_info->caching_workers && fs_info->fixup_workers &&
+	      fs_info->delayed_workers && fs_info->qgroup_rescan_workers &&
 	      fs_info->discard_ctl.discard_workers)) {
 		return -ENOMEM;
 	}
@@ -3083,7 +3071,6 @@ void btrfs_init_fs_info(struct btrfs_fs_info *fs_info)
 
 	atomic_set(&fs_info->async_delalloc_pages, 0);
 	atomic_set(&fs_info->defrag_running, 0);
-	atomic_set(&fs_info->reada_works_cnt, 0);
 	atomic_set(&fs_info->nr_delayed_iputs, 0);
 	atomic64_set(&fs_info->tree_mod_seq, 0);
 	fs_info->global_root_tree = RB_ROOT;
@@ -3094,9 +3081,6 @@ void btrfs_init_fs_info(struct btrfs_fs_info *fs_info)
 	fs_info->tree_mod_log = RB_ROOT;
 	fs_info->commit_interval = BTRFS_DEFAULT_COMMIT_INTERVAL;
 	fs_info->avg_delayed_ref_runtime = NSEC_PER_SEC >> 6; /* div by 64 */
-	/* readahead state */
-	INIT_RADIX_TREE(&fs_info->reada_tree, GFP_NOFS & ~__GFP_DIRECT_RECLAIM);
-	spin_lock_init(&fs_info->reada_lock);
 	btrfs_init_ref_verify(fs_info);
 
 	fs_info->thread_pool_size = min_t(unsigned long,
diff --git a/fs/btrfs/extent_io.c b/fs/btrfs/extent_io.c
index 1a67f4b3986b..c9f1f5f3846b 100644
--- a/fs/btrfs/extent_io.c
+++ b/fs/btrfs/extent_io.c
@@ -3087,9 +3087,6 @@ static void end_bio_extent_readpage(struct bio *bio)
 			set_bit(EXTENT_BUFFER_READ_ERR, &eb->bflags);
 			eb->read_mirror = mirror;
 			atomic_dec(&eb->io_pages);
-			if (test_and_clear_bit(EXTENT_BUFFER_READAHEAD,
-					       &eb->bflags))
-				btree_readahead_hook(eb, -EIO);
 		}
 readpage_ok:
 		if (likely(uptodate)) {
diff --git a/fs/btrfs/reada.c b/fs/btrfs/reada.c
deleted file mode 100644
index eb96fdc3be25..000000000000
--- a/fs/btrfs/reada.c
+++ /dev/null
@@ -1,1086 +0,0 @@
-// SPDX-License-Identifier: GPL-2.0
-/*
- * Copyright (C) 2011 STRATO.  All rights reserved.
- */
-
-#include <linux/sched.h>
-#include <linux/pagemap.h>
-#include <linux/writeback.h>
-#include <linux/blkdev.h>
-#include <linux/slab.h>
-#include <linux/workqueue.h>
-#include "ctree.h"
-#include "volumes.h"
-#include "disk-io.h"
-#include "transaction.h"
-#include "dev-replace.h"
-#include "block-group.h"
-
-#undef DEBUG
-
-/*
- * This is the implementation for the generic read ahead framework.
- *
- * To trigger a readahead, btrfs_reada_add must be called. It will start
- * a read ahead for the given range [start, end) on tree root. The returned
- * handle can either be used to wait on the readahead to finish
- * (btrfs_reada_wait), or to send it to the background (btrfs_reada_detach).
- *
- * The read ahead works as follows:
- * On btrfs_reada_add, the root of the tree is inserted into a radix_tree.
- * reada_start_machine will then search for extents to prefetch and trigger
- * some reads. When a read finishes for a node, all contained node/leaf
- * pointers that lie in the given range will also be enqueued. The reads will
- * be triggered in sequential order, thus giving a big win over a naive
- * enumeration. It will also make use of multi-device layouts. Each disk
- * will have its on read pointer and all disks will by utilized in parallel.
- * Also will no two disks read both sides of a mirror simultaneously, as this
- * would waste seeking capacity. Instead both disks will read different parts
- * of the filesystem.
- * Any number of readaheads can be started in parallel. The read order will be
- * determined globally, i.e. 2 parallel readaheads will normally finish faster
- * than the 2 started one after another.
- */
-
-#define MAX_IN_FLIGHT 6
-
-struct reada_extctl {
-	struct list_head	list;
-	struct reada_control	*rc;
-	u64			generation;
-};
-
-struct reada_extent {
-	u64			logical;
-	u64			owner_root;
-	struct btrfs_key	top;
-	struct list_head	extctl;
-	int 			refcnt;
-	spinlock_t		lock;
-	struct reada_zone	*zones[BTRFS_MAX_MIRRORS];
-	int			nzones;
-	int			scheduled;
-	int			level;
-};
-
-struct reada_zone {
-	u64			start;
-	u64			end;
-	u64			elems;
-	struct list_head	list;
-	spinlock_t		lock;
-	int			locked;
-	struct btrfs_device	*device;
-	struct btrfs_device	*devs[BTRFS_MAX_MIRRORS]; /* full list, incl
-							   * self */
-	int			ndevs;
-	struct kref		refcnt;
-};
-
-struct reada_machine_work {
-	struct btrfs_work	work;
-	struct btrfs_fs_info	*fs_info;
-};
-
-static void reada_extent_put(struct btrfs_fs_info *, struct reada_extent *);
-static void reada_control_release(struct kref *kref);
-static void reada_zone_release(struct kref *kref);
-static void reada_start_machine(struct btrfs_fs_info *fs_info);
-static void __reada_start_machine(struct btrfs_fs_info *fs_info);
-
-static int reada_add_block(struct reada_control *rc, u64 logical,
-			   struct btrfs_key *top, u64 owner_root,
-			   u64 generation, int level);
-
-/* recurses */
-/* in case of err, eb might be NULL */
-static void __readahead_hook(struct btrfs_fs_info *fs_info,
-			     struct reada_extent *re, struct extent_buffer *eb,
-			     int err)
-{
-	int nritems;
-	int i;
-	u64 bytenr;
-	u64 generation;
-	struct list_head list;
-
-	spin_lock(&re->lock);
-	/*
-	 * just take the full list from the extent. afterwards we
-	 * don't need the lock anymore
-	 */
-	list_replace_init(&re->extctl, &list);
-	re->scheduled = 0;
-	spin_unlock(&re->lock);
-
-	/*
-	 * this is the error case, the extent buffer has not been
-	 * read correctly. We won't access anything from it and
-	 * just cleanup our data structures. Effectively this will
-	 * cut the branch below this node from read ahead.
-	 */
-	if (err)
-		goto cleanup;
-
-	/*
-	 * FIXME: currently we just set nritems to 0 if this is a leaf,
-	 * effectively ignoring the content. In a next step we could
-	 * trigger more readahead depending from the content, e.g.
-	 * fetch the checksums for the extents in the leaf.
-	 */
-	if (!btrfs_header_level(eb))
-		goto cleanup;
-
-	nritems = btrfs_header_nritems(eb);
-	generation = btrfs_header_generation(eb);
-	for (i = 0; i < nritems; i++) {
-		struct reada_extctl *rec;
-		u64 n_gen;
-		struct btrfs_key key;
-		struct btrfs_key next_key;
-
-		btrfs_node_key_to_cpu(eb, &key, i);
-		if (i + 1 < nritems)
-			btrfs_node_key_to_cpu(eb, &next_key, i + 1);
-		else
-			next_key = re->top;
-		bytenr = btrfs_node_blockptr(eb, i);
-		n_gen = btrfs_node_ptr_generation(eb, i);
-
-		list_for_each_entry(rec, &list, list) {
-			struct reada_control *rc = rec->rc;
-
-			/*
-			 * if the generation doesn't match, just ignore this
-			 * extctl. This will probably cut off a branch from
-			 * prefetch. Alternatively one could start a new (sub-)
-			 * prefetch for this branch, starting again from root.
-			 * FIXME: move the generation check out of this loop
-			 */
-#ifdef DEBUG
-			if (rec->generation != generation) {
-				btrfs_debug(fs_info,
-					    "generation mismatch for (%llu,%d,%llu) %llu != %llu",
-					    key.objectid, key.type, key.offset,
-					    rec->generation, generation);
-			}
-#endif
-			if (rec->generation == generation &&
-			    btrfs_comp_cpu_keys(&key, &rc->key_end) < 0 &&
-			    btrfs_comp_cpu_keys(&next_key, &rc->key_start) > 0)
-				reada_add_block(rc, bytenr, &next_key,
-						btrfs_header_owner(eb), n_gen,
-						btrfs_header_level(eb) - 1);
-		}
-	}
-
-cleanup:
-	/*
-	 * free extctl records
-	 */
-	while (!list_empty(&list)) {
-		struct reada_control *rc;
-		struct reada_extctl *rec;
-
-		rec = list_first_entry(&list, struct reada_extctl, list);
-		list_del(&rec->list);
-		rc = rec->rc;
-		kfree(rec);
-
-		kref_get(&rc->refcnt);
-		if (atomic_dec_and_test(&rc->elems)) {
-			kref_put(&rc->refcnt, reada_control_release);
-			wake_up(&rc->wait);
-		}
-		kref_put(&rc->refcnt, reada_control_release);
-
-		reada_extent_put(fs_info, re);	/* one ref for each entry */
-	}
-
-	return;
-}
-
-int btree_readahead_hook(struct extent_buffer *eb, int err)
-{
-	struct btrfs_fs_info *fs_info = eb->fs_info;
-	int ret = 0;
-	struct reada_extent *re;
-
-	/* find extent */
-	spin_lock(&fs_info->reada_lock);
-	re = radix_tree_lookup(&fs_info->reada_tree,
-			       eb->start >> fs_info->sectorsize_bits);
-	if (re)
-		re->refcnt++;
-	spin_unlock(&fs_info->reada_lock);
-	if (!re) {
-		ret = -1;
-		goto start_machine;
-	}
-
-	__readahead_hook(fs_info, re, eb, err);
-	reada_extent_put(fs_info, re);	/* our ref */
-
-start_machine:
-	reada_start_machine(fs_info);
-	return ret;
-}
-
-static struct reada_zone *reada_find_zone(struct btrfs_device *dev, u64 logical,
-					  struct btrfs_io_context *bioc)
-{
-	struct btrfs_fs_info *fs_info = dev->fs_info;
-	int ret;
-	struct reada_zone *zone;
-	struct btrfs_block_group *cache = NULL;
-	u64 start;
-	u64 end;
-	int i;
-
-	zone = NULL;
-	spin_lock(&fs_info->reada_lock);
-	ret = radix_tree_gang_lookup(&dev->reada_zones, (void **)&zone,
-				     logical >> fs_info->sectorsize_bits, 1);
-	if (ret == 1 && logical >= zone->start && logical <= zone->end) {
-		kref_get(&zone->refcnt);
-		spin_unlock(&fs_info->reada_lock);
-		return zone;
-	}
-
-	spin_unlock(&fs_info->reada_lock);
-
-	cache = btrfs_lookup_block_group(fs_info, logical);
-	if (!cache)
-		return NULL;
-
-	start = cache->start;
-	end = start + cache->length - 1;
-	btrfs_put_block_group(cache);
-
-	zone = kzalloc(sizeof(*zone), GFP_KERNEL);
-	if (!zone)
-		return NULL;
-
-	ret = radix_tree_preload(GFP_KERNEL);
-	if (ret) {
-		kfree(zone);
-		return NULL;
-	}
-
-	zone->start = start;
-	zone->end = end;
-	INIT_LIST_HEAD(&zone->list);
-	spin_lock_init(&zone->lock);
-	zone->locked = 0;
-	kref_init(&zone->refcnt);
-	zone->elems = 0;
-	zone->device = dev; /* our device always sits at index 0 */
-	for (i = 0; i < bioc->num_stripes; ++i) {
-		/* bounds have already been checked */
-		zone->devs[i] = bioc->stripes[i].dev;
-	}
-	zone->ndevs = bioc->num_stripes;
-
-	spin_lock(&fs_info->reada_lock);
-	ret = radix_tree_insert(&dev->reada_zones,
-			(unsigned long)(zone->end >> fs_info->sectorsize_bits),
-			zone);
-
-	if (ret == -EEXIST) {
-		kfree(zone);
-		ret = radix_tree_gang_lookup(&dev->reada_zones, (void **)&zone,
-					logical >> fs_info->sectorsize_bits, 1);
-		if (ret == 1 && logical >= zone->start && logical <= zone->end)
-			kref_get(&zone->refcnt);
-		else
-			zone = NULL;
-	}
-	spin_unlock(&fs_info->reada_lock);
-	radix_tree_preload_end();
-
-	return zone;
-}
-
-static struct reada_extent *reada_find_extent(struct btrfs_fs_info *fs_info,
-					      u64 logical,
-					      struct btrfs_key *top,
-					      u64 owner_root, int level)
-{
-	int ret;
-	struct reada_extent *re = NULL;
-	struct reada_extent *re_exist = NULL;
-	struct btrfs_io_context *bioc = NULL;
-	struct btrfs_device *dev;
-	struct btrfs_device *prev_dev;
-	u64 length;
-	int real_stripes;
-	int nzones = 0;
-	unsigned long index = logical >> fs_info->sectorsize_bits;
-	int dev_replace_is_ongoing;
-	int have_zone = 0;
-
-	spin_lock(&fs_info->reada_lock);
-	re = radix_tree_lookup(&fs_info->reada_tree, index);
-	if (re)
-		re->refcnt++;
-	spin_unlock(&fs_info->reada_lock);
-
-	if (re)
-		return re;
-
-	re = kzalloc(sizeof(*re), GFP_KERNEL);
-	if (!re)
-		return NULL;
-
-	re->logical = logical;
-	re->top = *top;
-	INIT_LIST_HEAD(&re->extctl);
-	spin_lock_init(&re->lock);
-	re->refcnt = 1;
-	re->owner_root = owner_root;
-	re->level = level;
-
-	/*
-	 * map block
-	 */
-	length = fs_info->nodesize;
-	ret = btrfs_map_block(fs_info, BTRFS_MAP_GET_READ_MIRRORS, logical,
-			      &length, &bioc, 0);
-	if (ret || !bioc || length < fs_info->nodesize)
-		goto error;
-
-	if (bioc->num_stripes > BTRFS_MAX_MIRRORS) {
-		btrfs_err(fs_info,
-			   "readahead: more than %d copies not supported",
-			   BTRFS_MAX_MIRRORS);
-		goto error;
-	}
-
-	real_stripes = bioc->num_stripes - bioc->num_tgtdevs;
-	for (nzones = 0; nzones < real_stripes; ++nzones) {
-		struct reada_zone *zone;
-
-		dev = bioc->stripes[nzones].dev;
-
-		/* cannot read ahead on missing device. */
-		if (!dev->bdev)
-			continue;
-
-		zone = reada_find_zone(dev, logical, bioc);
-		if (!zone)
-			continue;
-
-		re->zones[re->nzones++] = zone;
-		spin_lock(&zone->lock);
-		if (!zone->elems)
-			kref_get(&zone->refcnt);
-		++zone->elems;
-		spin_unlock(&zone->lock);
-		spin_lock(&fs_info->reada_lock);
-		kref_put(&zone->refcnt, reada_zone_release);
-		spin_unlock(&fs_info->reada_lock);
-	}
-	if (re->nzones == 0) {
-		/* not a single zone found, error and out */
-		goto error;
-	}
-
-	/* Insert extent in reada tree + all per-device trees, all or nothing */
-	down_read(&fs_info->dev_replace.rwsem);
-	ret = radix_tree_preload(GFP_KERNEL);
-	if (ret) {
-		up_read(&fs_info->dev_replace.rwsem);
-		goto error;
-	}
-
-	spin_lock(&fs_info->reada_lock);
-	ret = radix_tree_insert(&fs_info->reada_tree, index, re);
-	if (ret == -EEXIST) {
-		re_exist = radix_tree_lookup(&fs_info->reada_tree, index);
-		re_exist->refcnt++;
-		spin_unlock(&fs_info->reada_lock);
-		radix_tree_preload_end();
-		up_read(&fs_info->dev_replace.rwsem);
-		goto error;
-	}
-	if (ret) {
-		spin_unlock(&fs_info->reada_lock);
-		radix_tree_preload_end();
-		up_read(&fs_info->dev_replace.rwsem);
-		goto error;
-	}
-	radix_tree_preload_end();
-	prev_dev = NULL;
-	dev_replace_is_ongoing = btrfs_dev_replace_is_ongoing(
-			&fs_info->dev_replace);
-	for (nzones = 0; nzones < re->nzones; ++nzones) {
-		dev = re->zones[nzones]->device;
-
-		if (dev == prev_dev) {
-			/*
-			 * in case of DUP, just add the first zone. As both
-			 * are on the same device, there's nothing to gain
-			 * from adding both.
-			 * Also, it wouldn't work, as the tree is per device
-			 * and adding would fail with EEXIST
-			 */
-			continue;
-		}
-		if (!dev->bdev)
-			continue;
-
-		if (test_bit(BTRFS_DEV_STATE_NO_READA, &dev->dev_state))
-			continue;
-
-		if (dev_replace_is_ongoing &&
-		    dev == fs_info->dev_replace.tgtdev) {
-			/*
-			 * as this device is selected for reading only as
-			 * a last resort, skip it for read ahead.
-			 */
-			continue;
-		}
-		prev_dev = dev;
-		ret = radix_tree_insert(&dev->reada_extents, index, re);
-		if (ret) {
-			while (--nzones >= 0) {
-				dev = re->zones[nzones]->device;
-				BUG_ON(dev == NULL);
-				/* ignore whether the entry was inserted */
-				radix_tree_delete(&dev->reada_extents, index);
-			}
-			radix_tree_delete(&fs_info->reada_tree, index);
-			spin_unlock(&fs_info->reada_lock);
-			up_read(&fs_info->dev_replace.rwsem);
-			goto error;
-		}
-		have_zone = 1;
-	}
-	if (!have_zone)
-		radix_tree_delete(&fs_info->reada_tree, index);
-	spin_unlock(&fs_info->reada_lock);
-	up_read(&fs_info->dev_replace.rwsem);
-
-	if (!have_zone)
-		goto error;
-
-	btrfs_put_bioc(bioc);
-	return re;
-
-error:
-	for (nzones = 0; nzones < re->nzones; ++nzones) {
-		struct reada_zone *zone;
-
-		zone = re->zones[nzones];
-		kref_get(&zone->refcnt);
-		spin_lock(&zone->lock);
-		--zone->elems;
-		if (zone->elems == 0) {
-			/*
-			 * no fs_info->reada_lock needed, as this can't be
-			 * the last ref
-			 */
-			kref_put(&zone->refcnt, reada_zone_release);
-		}
-		spin_unlock(&zone->lock);
-
-		spin_lock(&fs_info->reada_lock);
-		kref_put(&zone->refcnt, reada_zone_release);
-		spin_unlock(&fs_info->reada_lock);
-	}
-	btrfs_put_bioc(bioc);
-	kfree(re);
-	return re_exist;
-}
-
-static void reada_extent_put(struct btrfs_fs_info *fs_info,
-			     struct reada_extent *re)
-{
-	int i;
-	unsigned long index = re->logical >> fs_info->sectorsize_bits;
-
-	spin_lock(&fs_info->reada_lock);
-	if (--re->refcnt) {
-		spin_unlock(&fs_info->reada_lock);
-		return;
-	}
-
-	radix_tree_delete(&fs_info->reada_tree, index);
-	for (i = 0; i < re->nzones; ++i) {
-		struct reada_zone *zone = re->zones[i];
-
-		radix_tree_delete(&zone->device->reada_extents, index);
-	}
-
-	spin_unlock(&fs_info->reada_lock);
-
-	for (i = 0; i < re->nzones; ++i) {
-		struct reada_zone *zone = re->zones[i];
-
-		kref_get(&zone->refcnt);
-		spin_lock(&zone->lock);
-		--zone->elems;
-		if (zone->elems == 0) {
-			/* no fs_info->reada_lock needed, as this can't be
-			 * the last ref */
-			kref_put(&zone->refcnt, reada_zone_release);
-		}
-		spin_unlock(&zone->lock);
-
-		spin_lock(&fs_info->reada_lock);
-		kref_put(&zone->refcnt, reada_zone_release);
-		spin_unlock(&fs_info->reada_lock);
-	}
-
-	kfree(re);
-}
-
-static void reada_zone_release(struct kref *kref)
-{
-	struct reada_zone *zone = container_of(kref, struct reada_zone, refcnt);
-	struct btrfs_fs_info *fs_info = zone->device->fs_info;
-
-	lockdep_assert_held(&fs_info->reada_lock);
-
-	radix_tree_delete(&zone->device->reada_zones,
-			  zone->end >> fs_info->sectorsize_bits);
-
-	kfree(zone);
-}
-
-static void reada_control_release(struct kref *kref)
-{
-	struct reada_control *rc = container_of(kref, struct reada_control,
-						refcnt);
-
-	kfree(rc);
-}
-
-static int reada_add_block(struct reada_control *rc, u64 logical,
-			   struct btrfs_key *top, u64 owner_root,
-			   u64 generation, int level)
-{
-	struct btrfs_fs_info *fs_info = rc->fs_info;
-	struct reada_extent *re;
-	struct reada_extctl *rec;
-
-	/* takes one ref */
-	re = reada_find_extent(fs_info, logical, top, owner_root, level);
-	if (!re)
-		return -1;
-
-	rec = kzalloc(sizeof(*rec), GFP_KERNEL);
-	if (!rec) {
-		reada_extent_put(fs_info, re);
-		return -ENOMEM;
-	}
-
-	rec->rc = rc;
-	rec->generation = generation;
-	atomic_inc(&rc->elems);
-
-	spin_lock(&re->lock);
-	list_add_tail(&rec->list, &re->extctl);
-	spin_unlock(&re->lock);
-
-	/* leave the ref on the extent */
-
-	return 0;
-}
-
-/*
- * called with fs_info->reada_lock held
- */
-static void reada_peer_zones_set_lock(struct reada_zone *zone, int lock)
-{
-	int i;
-	unsigned long index = zone->end >> zone->device->fs_info->sectorsize_bits;
-
-	for (i = 0; i < zone->ndevs; ++i) {
-		struct reada_zone *peer;
-		peer = radix_tree_lookup(&zone->devs[i]->reada_zones, index);
-		if (peer && peer->device != zone->device)
-			peer->locked = lock;
-	}
-}
-
-/*
- * called with fs_info->reada_lock held
- */
-static int reada_pick_zone(struct btrfs_device *dev)
-{
-	struct reada_zone *top_zone = NULL;
-	struct reada_zone *top_locked_zone = NULL;
-	u64 top_elems = 0;
-	u64 top_locked_elems = 0;
-	unsigned long index = 0;
-	int ret;
-
-	if (dev->reada_curr_zone) {
-		reada_peer_zones_set_lock(dev->reada_curr_zone, 0);
-		kref_put(&dev->reada_curr_zone->refcnt, reada_zone_release);
-		dev->reada_curr_zone = NULL;
-	}
-	/* pick the zone with the most elements */
-	while (1) {
-		struct reada_zone *zone;
-
-		ret = radix_tree_gang_lookup(&dev->reada_zones,
-					     (void **)&zone, index, 1);
-		if (ret == 0)
-			break;
-		index = (zone->end >> dev->fs_info->sectorsize_bits) + 1;
-		if (zone->locked) {
-			if (zone->elems > top_locked_elems) {
-				top_locked_elems = zone->elems;
-				top_locked_zone = zone;
-			}
-		} else {
-			if (zone->elems > top_elems) {
-				top_elems = zone->elems;
-				top_zone = zone;
-			}
-		}
-	}
-	if (top_zone)
-		dev->reada_curr_zone = top_zone;
-	else if (top_locked_zone)
-		dev->reada_curr_zone = top_locked_zone;
-	else
-		return 0;
-
-	dev->reada_next = dev->reada_curr_zone->start;
-	kref_get(&dev->reada_curr_zone->refcnt);
-	reada_peer_zones_set_lock(dev->reada_curr_zone, 1);
-
-	return 1;
-}
-
-static int reada_tree_block_flagged(struct btrfs_fs_info *fs_info, u64 bytenr,
-				    u64 owner_root, int level, int mirror_num,
-				    struct extent_buffer **eb)
-{
-	struct extent_buffer *buf = NULL;
-	int ret;
-
-	buf = btrfs_find_create_tree_block(fs_info, bytenr, owner_root, level);
-	if (IS_ERR(buf))
-		return 0;
-
-	set_bit(EXTENT_BUFFER_READAHEAD, &buf->bflags);
-
-	ret = read_extent_buffer_pages(buf, WAIT_PAGE_LOCK, mirror_num);
-	if (ret) {
-		free_extent_buffer_stale(buf);
-		return ret;
-	}
-
-	if (test_bit(EXTENT_BUFFER_CORRUPT, &buf->bflags)) {
-		free_extent_buffer_stale(buf);
-		return -EIO;
-	} else if (extent_buffer_uptodate(buf)) {
-		*eb = buf;
-	} else {
-		free_extent_buffer(buf);
-	}
-	return 0;
-}
-
-static int reada_start_machine_dev(struct btrfs_device *dev)
-{
-	struct btrfs_fs_info *fs_info = dev->fs_info;
-	struct reada_extent *re = NULL;
-	int mirror_num = 0;
-	struct extent_buffer *eb = NULL;
-	u64 logical;
-	int ret;
-	int i;
-
-	spin_lock(&fs_info->reada_lock);
-	if (dev->reada_curr_zone == NULL) {
-		ret = reada_pick_zone(dev);
-		if (!ret) {
-			spin_unlock(&fs_info->reada_lock);
-			return 0;
-		}
-	}
-	/*
-	 * FIXME currently we issue the reads one extent at a time. If we have
-	 * a contiguous block of extents, we could also coagulate them or use
-	 * plugging to speed things up
-	 */
-	ret = radix_tree_gang_lookup(&dev->reada_extents, (void **)&re,
-				dev->reada_next >> fs_info->sectorsize_bits, 1);
-	if (ret == 0 || re->logical > dev->reada_curr_zone->end) {
-		ret = reada_pick_zone(dev);
-		if (!ret) {
-			spin_unlock(&fs_info->reada_lock);
-			return 0;
-		}
-		re = NULL;
-		ret = radix_tree_gang_lookup(&dev->reada_extents, (void **)&re,
-				dev->reada_next >> fs_info->sectorsize_bits, 1);
-	}
-	if (ret == 0) {
-		spin_unlock(&fs_info->reada_lock);
-		return 0;
-	}
-	dev->reada_next = re->logical + fs_info->nodesize;
-	re->refcnt++;
-
-	spin_unlock(&fs_info->reada_lock);
-
-	spin_lock(&re->lock);
-	if (re->scheduled || list_empty(&re->extctl)) {
-		spin_unlock(&re->lock);
-		reada_extent_put(fs_info, re);
-		return 0;
-	}
-	re->scheduled = 1;
-	spin_unlock(&re->lock);
-
-	/*
-	 * find mirror num
-	 */
-	for (i = 0; i < re->nzones; ++i) {
-		if (re->zones[i]->device == dev) {
-			mirror_num = i + 1;
-			break;
-		}
-	}
-	logical = re->logical;
-
-	atomic_inc(&dev->reada_in_flight);
-	ret = reada_tree_block_flagged(fs_info, logical, re->owner_root,
-				       re->level, mirror_num, &eb);
-	if (ret)
-		__readahead_hook(fs_info, re, NULL, ret);
-	else if (eb)
-		__readahead_hook(fs_info, re, eb, ret);
-
-	if (eb)
-		free_extent_buffer(eb);
-
-	atomic_dec(&dev->reada_in_flight);
-	reada_extent_put(fs_info, re);
-
-	return 1;
-
-}
-
-static void reada_start_machine_worker(struct btrfs_work *work)
-{
-	struct reada_machine_work *rmw;
-	int old_ioprio;
-
-	rmw = container_of(work, struct reada_machine_work, work);
-
-	old_ioprio = IOPRIO_PRIO_VALUE(task_nice_ioclass(current),
-				       task_nice_ioprio(current));
-	set_task_ioprio(current, BTRFS_IOPRIO_READA);
-	__reada_start_machine(rmw->fs_info);
-	set_task_ioprio(current, old_ioprio);
-
-	atomic_dec(&rmw->fs_info->reada_works_cnt);
-
-	kfree(rmw);
-}
-
-/* Try to start up to 10k READA requests for a group of devices */
-static int reada_start_for_fsdevs(struct btrfs_fs_devices *fs_devices)
-{
-	u64 enqueued;
-	u64 total = 0;
-	struct btrfs_device *device;
-
-	do {
-		enqueued = 0;
-		list_for_each_entry(device, &fs_devices->devices, dev_list) {
-			if (atomic_read(&device->reada_in_flight) <
-			    MAX_IN_FLIGHT)
-				enqueued += reada_start_machine_dev(device);
-		}
-		total += enqueued;
-	} while (enqueued && total < 10000);
-
-	return total;
-}
-
-static void __reada_start_machine(struct btrfs_fs_info *fs_info)
-{
-	struct btrfs_fs_devices *fs_devices = fs_info->fs_devices, *seed_devs;
-	int i;
-	u64 enqueued = 0;
-
-	mutex_lock(&fs_devices->device_list_mutex);
-
-	enqueued += reada_start_for_fsdevs(fs_devices);
-	list_for_each_entry(seed_devs, &fs_devices->seed_list, seed_list)
-		enqueued += reada_start_for_fsdevs(seed_devs);
-
-	mutex_unlock(&fs_devices->device_list_mutex);
-	if (enqueued == 0)
-		return;
-
-	/*
-	 * If everything is already in the cache, this is effectively single
-	 * threaded. To a) not hold the caller for too long and b) to utilize
-	 * more cores, we broke the loop above after 10000 iterations and now
-	 * enqueue to workers to finish it. This will distribute the load to
-	 * the cores.
-	 */
-	for (i = 0; i < 2; ++i) {
-		reada_start_machine(fs_info);
-		if (atomic_read(&fs_info->reada_works_cnt) >
-		    BTRFS_MAX_MIRRORS * 2)
-			break;
-	}
-}
-
-static void reada_start_machine(struct btrfs_fs_info *fs_info)
-{
-	struct reada_machine_work *rmw;
-
-	rmw = kzalloc(sizeof(*rmw), GFP_KERNEL);
-	if (!rmw) {
-		/* FIXME we cannot handle this properly right now */
-		BUG();
-	}
-	btrfs_init_work(&rmw->work, reada_start_machine_worker, NULL, NULL);
-	rmw->fs_info = fs_info;
-
-	btrfs_queue_work(fs_info->readahead_workers, &rmw->work);
-	atomic_inc(&fs_info->reada_works_cnt);
-}
-
-#ifdef DEBUG
-static void dump_devs(struct btrfs_fs_info *fs_info, int all)
-{
-	struct btrfs_device *device;
-	struct btrfs_fs_devices *fs_devices = fs_info->fs_devices;
-	unsigned long index;
-	int ret;
-	int i;
-	int j;
-	int cnt;
-
-	spin_lock(&fs_info->reada_lock);
-	list_for_each_entry(device, &fs_devices->devices, dev_list) {
-		btrfs_debug(fs_info, "dev %lld has %d in flight", device->devid,
-			atomic_read(&device->reada_in_flight));
-		index = 0;
-		while (1) {
-			struct reada_zone *zone;
-			ret = radix_tree_gang_lookup(&device->reada_zones,
-						     (void **)&zone, index, 1);
-			if (ret == 0)
-				break;
-			pr_debug("  zone %llu-%llu elems %llu locked %d devs",
-				    zone->start, zone->end, zone->elems,
-				    zone->locked);
-			for (j = 0; j < zone->ndevs; ++j) {
-				pr_cont(" %lld",
-					zone->devs[j]->devid);
-			}
-			if (device->reada_curr_zone == zone)
-				pr_cont(" curr off %llu",
-					device->reada_next - zone->start);
-			pr_cont("\n");
-			index = (zone->end >> fs_info->sectorsize_bits) + 1;
-		}
-		cnt = 0;
-		index = 0;
-		while (all) {
-			struct reada_extent *re = NULL;
-
-			ret = radix_tree_gang_lookup(&device->reada_extents,
-						     (void **)&re, index, 1);
-			if (ret == 0)
-				break;
-			pr_debug("  re: logical %llu size %u empty %d scheduled %d",
-				re->logical, fs_info->nodesize,
-				list_empty(&re->extctl), re->scheduled);
-
-			for (i = 0; i < re->nzones; ++i) {
-				pr_cont(" zone %llu-%llu devs",
-					re->zones[i]->start,
-					re->zones[i]->end);
-				for (j = 0; j < re->zones[i]->ndevs; ++j) {
-					pr_cont(" %lld",
-						re->zones[i]->devs[j]->devid);
-				}
-			}
-			pr_cont("\n");
-			index = (re->logical >> fs_info->sectorsize_bits) + 1;
-			if (++cnt > 15)
-				break;
-		}
-	}
-
-	index = 0;
-	cnt = 0;
-	while (all) {
-		struct reada_extent *re = NULL;
-
-		ret = radix_tree_gang_lookup(&fs_info->reada_tree, (void **)&re,
-					     index, 1);
-		if (ret == 0)
-			break;
-		if (!re->scheduled) {
-			index = (re->logical >> fs_info->sectorsize_bits) + 1;
-			continue;
-		}
-		pr_debug("re: logical %llu size %u list empty %d scheduled %d",
-			re->logical, fs_info->nodesize,
-			list_empty(&re->extctl), re->scheduled);
-		for (i = 0; i < re->nzones; ++i) {
-			pr_cont(" zone %llu-%llu devs",
-				re->zones[i]->start,
-				re->zones[i]->end);
-			for (j = 0; j < re->zones[i]->ndevs; ++j) {
-				pr_cont(" %lld",
-				       re->zones[i]->devs[j]->devid);
-			}
-		}
-		pr_cont("\n");
-		index = (re->logical >> fs_info->sectorsize_bits) + 1;
-	}
-	spin_unlock(&fs_info->reada_lock);
-}
-#endif
-
-/*
- * interface
- */
-struct reada_control *btrfs_reada_add(struct btrfs_root *root,
-			struct btrfs_key *key_start, struct btrfs_key *key_end)
-{
-	struct reada_control *rc;
-	u64 start;
-	u64 generation;
-	int ret;
-	int level;
-	struct extent_buffer *node;
-	static struct btrfs_key max_key = {
-		.objectid = (u64)-1,
-		.type = (u8)-1,
-		.offset = (u64)-1
-	};
-
-	rc = kzalloc(sizeof(*rc), GFP_KERNEL);
-	if (!rc)
-		return ERR_PTR(-ENOMEM);
-
-	rc->fs_info = root->fs_info;
-	rc->key_start = *key_start;
-	rc->key_end = *key_end;
-	atomic_set(&rc->elems, 0);
-	init_waitqueue_head(&rc->wait);
-	kref_init(&rc->refcnt);
-	kref_get(&rc->refcnt); /* one ref for having elements */
-
-	node = btrfs_root_node(root);
-	start = node->start;
-	generation = btrfs_header_generation(node);
-	level = btrfs_header_level(node);
-	free_extent_buffer(node);
-
-	ret = reada_add_block(rc, start, &max_key, root->root_key.objectid,
-			      generation, level);
-	if (ret) {
-		kfree(rc);
-		return ERR_PTR(ret);
-	}
-
-	reada_start_machine(root->fs_info);
-
-	return rc;
-}
-
-#ifdef DEBUG
-int btrfs_reada_wait(void *handle)
-{
-	struct reada_control *rc = handle;
-	struct btrfs_fs_info *fs_info = rc->fs_info;
-
-	while (atomic_read(&rc->elems)) {
-		if (!atomic_read(&fs_info->reada_works_cnt))
-			reada_start_machine(fs_info);
-		wait_event_timeout(rc->wait, atomic_read(&rc->elems) == 0,
-				   5 * HZ);
-		dump_devs(fs_info, atomic_read(&rc->elems) < 10 ? 1 : 0);
-	}
-
-	dump_devs(fs_info, atomic_read(&rc->elems) < 10 ? 1 : 0);
-
-	kref_put(&rc->refcnt, reada_control_release);
-
-	return 0;
-}
-#else
-int btrfs_reada_wait(void *handle)
-{
-	struct reada_control *rc = handle;
-	struct btrfs_fs_info *fs_info = rc->fs_info;
-
-	while (atomic_read(&rc->elems)) {
-		if (!atomic_read(&fs_info->reada_works_cnt))
-			reada_start_machine(fs_info);
-		wait_event_timeout(rc->wait, atomic_read(&rc->elems) == 0,
-				   (HZ + 9) / 10);
-	}
-
-	kref_put(&rc->refcnt, reada_control_release);
-
-	return 0;
-}
-#endif
-
-void btrfs_reada_detach(void *handle)
-{
-	struct reada_control *rc = handle;
-
-	kref_put(&rc->refcnt, reada_control_release);
-}
-
-/*
- * Before removing a device (device replace or device remove ioctls), call this
- * function to wait for all existing readahead requests on the device and to
- * make sure no one queues more readahead requests for the device.
- *
- * Must be called without holding neither the device list mutex nor the device
- * replace semaphore, otherwise it will deadlock.
- */
-void btrfs_reada_remove_dev(struct btrfs_device *dev)
-{
-	struct btrfs_fs_info *fs_info = dev->fs_info;
-
-	/* Serialize with readahead extent creation at reada_find_extent(). */
-	spin_lock(&fs_info->reada_lock);
-	set_bit(BTRFS_DEV_STATE_NO_READA, &dev->dev_state);
-	spin_unlock(&fs_info->reada_lock);
-
-	/*
-	 * There might be readahead requests added to the radix trees which
-	 * were not yet added to the readahead work queue. We need to start
-	 * them and wait for their completion, otherwise we can end up with
-	 * use-after-free problems when dropping the last reference on the
-	 * readahead extents and their zones, as they need to access the
-	 * device structure.
-	 */
-	reada_start_machine(fs_info);
-	btrfs_flush_workqueue(fs_info->readahead_workers);
-}
-
-/*
- * If when removing a device (device replace or device remove ioctls) an error
- * happens after calling btrfs_reada_remove_dev(), call this to undo what that
- * function did. This is safe to call even if btrfs_reada_remove_dev() was not
- * called before.
- */
-void btrfs_reada_undo_remove_dev(struct btrfs_device *dev)
-{
-	spin_lock(&dev->fs_info->reada_lock);
-	clear_bit(BTRFS_DEV_STATE_NO_READA, &dev->dev_state);
-	spin_unlock(&dev->fs_info->reada_lock);
-}
diff --git a/fs/btrfs/scrub.c b/fs/btrfs/scrub.c
index a3bd3dfd5e8b..e1c8e8812793 100644
--- a/fs/btrfs/scrub.c
+++ b/fs/btrfs/scrub.c
@@ -3193,10 +3193,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 	u64 physical_end;
 	u64 generation;
 	int mirror_num;
-	struct reada_control *reada1;
-	struct reada_control *reada2;
 	struct btrfs_key key;
-	struct btrfs_key key_end;
 	u64 increment = map->stripe_len;
 	u64 offset;
 	u64 extent_logical;
@@ -3246,11 +3243,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 	path->search_commit_root = 1;
 	path->skip_locking = 1;
 
-	/*
-	 * trigger the readahead for extent tree csum tree and wait for
-	 * completion. During readahead, the scrub is officially paused
-	 * to not hold off transaction commits
-	 */
 	logical = base + offset;
 	physical_end = physical + nstripes * map->stripe_len;
 	if (map->type & BTRFS_BLOCK_GROUP_RAID56_MASK) {
@@ -3265,36 +3257,8 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 	scrub_blocked_if_needed(fs_info);
 
 	root = btrfs_extent_root(fs_info, logical);
-
-	/* FIXME it might be better to start readahead at commit root */
-	key.objectid = logical;
-	key.type = BTRFS_EXTENT_ITEM_KEY;
-	key.offset = (u64)0;
-	key_end.objectid = logic_end;
-	key_end.type = BTRFS_METADATA_ITEM_KEY;
-	key_end.offset = (u64)-1;
-	reada1 = btrfs_reada_add(root, &key, &key_end);
-
 	csum_root = btrfs_csum_root(fs_info, logical);
 
-	if (cache->flags & BTRFS_BLOCK_GROUP_DATA) {
-		key.objectid = BTRFS_EXTENT_CSUM_OBJECTID;
-		key.type = BTRFS_EXTENT_CSUM_KEY;
-		key.offset = logical;
-		key_end.objectid = BTRFS_EXTENT_CSUM_OBJECTID;
-		key_end.type = BTRFS_EXTENT_CSUM_KEY;
-		key_end.offset = logic_end;
-		reada2 = btrfs_reada_add(csum_root, &key, &key_end);
-	} else {
-		reada2 = NULL;
-	}
-
-	if (!IS_ERR(reada1))
-		btrfs_reada_wait(reada1);
-	if (!IS_ERR_OR_NULL(reada2))
-		btrfs_reada_wait(reada2);
-
-
 	/*
 	 * collect all data csums for the stripe to avoid seeking during
 	 * the scrub. This might currently (crc32) end up to be about 1MB
diff --git a/fs/btrfs/super.c b/fs/btrfs/super.c
index a1c54a2c787c..0ec09fe01be6 100644
--- a/fs/btrfs/super.c
+++ b/fs/btrfs/super.c
@@ -1842,7 +1842,6 @@ static void btrfs_resize_thread_pool(struct btrfs_fs_info *fs_info,
 	btrfs_workqueue_set_max(fs_info->endio_write_workers, new_pool_size);
 	btrfs_workqueue_set_max(fs_info->endio_freespace_worker, new_pool_size);
 	btrfs_workqueue_set_max(fs_info->delayed_workers, new_pool_size);
-	btrfs_workqueue_set_max(fs_info->readahead_workers, new_pool_size);
 	btrfs_workqueue_set_max(fs_info->scrub_wr_completion_workers,
 				new_pool_size);
 }
diff --git a/fs/btrfs/volumes.c b/fs/btrfs/volumes.c
index f38c230111be..141fd7b4b018 100644
--- a/fs/btrfs/volumes.c
+++ b/fs/btrfs/volumes.c
@@ -1166,7 +1166,6 @@ static void btrfs_close_one_device(struct btrfs_device *device)
 	ASSERT(!test_bit(BTRFS_DEV_STATE_REPLACE_TGT, &device->dev_state));
 	ASSERT(list_empty(&device->dev_alloc_list));
 	ASSERT(list_empty(&device->post_commit_list));
-	ASSERT(atomic_read(&device->reada_in_flight) == 0);
 }
 
 static void close_fs_devices(struct btrfs_fs_devices *fs_devices)
@@ -2148,8 +2147,6 @@ int btrfs_rm_device(struct btrfs_fs_info *fs_info,
 	}
 
 	ret = btrfs_shrink_device(device, 0);
-	if (!ret)
-		btrfs_reada_remove_dev(device);
 	if (ret)
 		goto error_undo;
 
@@ -2247,7 +2244,6 @@ int btrfs_rm_device(struct btrfs_fs_info *fs_info,
 	return ret;
 
 error_undo:
-	btrfs_reada_undo_remove_dev(device);
 	if (test_bit(BTRFS_DEV_STATE_WRITEABLE, &device->dev_state)) {
 		mutex_lock(&fs_info->chunk_mutex);
 		list_add(&device->dev_alloc_list,
@@ -6978,11 +6974,8 @@ struct btrfs_device *btrfs_alloc_device(struct btrfs_fs_info *fs_info,
 	INIT_LIST_HEAD(&dev->dev_alloc_list);
 	INIT_LIST_HEAD(&dev->post_commit_list);
 
-	atomic_set(&dev->reada_in_flight, 0);
 	atomic_set(&dev->dev_stats_ccnt, 0);
 	btrfs_device_data_ordered_init(dev);
-	INIT_RADIX_TREE(&dev->reada_zones, GFP_NOFS & ~__GFP_DIRECT_RECLAIM);
-	INIT_RADIX_TREE(&dev->reada_extents, GFP_NOFS & ~__GFP_DIRECT_RECLAIM);
 	extent_io_tree_init(fs_info, &dev->alloc_state,
 			    IO_TREE_DEVICE_ALLOC_STATE, NULL);
 
diff --git a/fs/btrfs/volumes.h b/fs/btrfs/volumes.h
index 3b8130680749..b2467a9159d0 100644
--- a/fs/btrfs/volumes.h
+++ b/fs/btrfs/volumes.h
@@ -123,13 +123,6 @@ struct btrfs_device {
 	/* per-device scrub information */
 	struct scrub_ctx *scrub_ctx;
 
-	/* readahead state */
-	atomic_t reada_in_flight;
-	u64 reada_next;
-	struct reada_zone *reada_curr_zone;
-	struct radix_tree_root reada_zones;
-	struct radix_tree_root reada_extents;
-
 	/* disk I/O failure stats. For detailed description refer to
 	 * enum btrfs_dev_stat_values in ioctl.h */
 	int dev_stats_valid;
-- 
2.34.1


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

* [PATCH 3/3] btrfs: use btrfs_path::reada for scrub extent tree readahead
  2021-12-13 13:10 [PATCH 0/3] btrfs: use btrfs_path::reada to replace the Qu Wenruo
  2021-12-13 13:10 ` [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity() Qu Wenruo
  2021-12-13 13:10 ` [PATCH 2/3] btrfs: remove reada mechanism Qu Wenruo
@ 2021-12-13 13:10 ` Qu Wenruo
  2021-12-14 12:41 ` [PATCH 0/3] btrfs: use btrfs_path::reada to replace the David Sterba
  3 siblings, 0 replies; 7+ messages in thread
From: Qu Wenruo @ 2021-12-13 13:10 UTC (permalink / raw)
  To: linux-btrfs

For scrub, we trigger two readahead for two trees, extent tree to get
where to scrub, and csum tree to get the data checksum.

For csum tree we already trigger readahead in
btrfs_lookup_csums_range(), by setting path->reada.
But for extent tree we don't have any path based readahead.

This patch will add the readahead for extent tree.

With path based readahead only, on SSDs the scrub speed is the same as
previous btrfs_reada_add() mechanism.
While on HDDs, the scrub speed get degraded by at most 5%, although not
the best case, it's still pretty acceptable, especially considering how
many bugs there are in the old btrfs_reada_add() mechanism recently.

Signed-off-by: Qu Wenruo <wqu@suse.com>
---
 fs/btrfs/scrub.c | 1 +
 1 file changed, 1 insertion(+)

diff --git a/fs/btrfs/scrub.c b/fs/btrfs/scrub.c
index e1c8e8812793..aca80a12a930 100644
--- a/fs/btrfs/scrub.c
+++ b/fs/btrfs/scrub.c
@@ -3242,6 +3242,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
 	 */
 	path->search_commit_root = 1;
 	path->skip_locking = 1;
+	path->reada = READA_FORWARD;
 
 	logical = base + offset;
 	physical_end = physical + nstripes * map->stripe_len;
-- 
2.34.1


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

* Re: [PATCH 0/3] btrfs: use btrfs_path::reada to replace the
  2021-12-13 13:10 [PATCH 0/3] btrfs: use btrfs_path::reada to replace the Qu Wenruo
                   ` (2 preceding siblings ...)
  2021-12-13 13:10 ` [PATCH 3/3] btrfs: use btrfs_path::reada for scrub extent tree readahead Qu Wenruo
@ 2021-12-14 12:41 ` David Sterba
  2021-12-14 12:45   ` Qu Wenruo
  3 siblings, 1 reply; 7+ messages in thread
From: David Sterba @ 2021-12-14 12:41 UTC (permalink / raw)
  To: Qu Wenruo; +Cc: linux-btrfs

On Mon, Dec 13, 2021 at 09:10:51PM +0800, Qu Wenruo wrote:
> [PROBLEMS]
> The metadata readahead code is introduced in 2011 (surprisingly, the
> commit message even contains changelog), but now there is only one user
> for it, and even for the only one user, the readahead mechanism can't
> provide all it needs:
> 
> - No support for commit tree readahead
>   Only support readahead on current tree.
> 
> - Bad layer separation
>   To manage on-fly bios, btrfs_reada_add() mechanism internally manages
>   a kinda complex zone system, and it's bind to per-device.
> 
>   This is against the common layer separation, as metadata should all be
>   in btrfs logical address space, should not be bound to device physical
>   layer.
> 
>   In fact, this is the cause of all recent reada related bugs.
> 
> - No caller for asynchronous metadata readahead
>   Even btrfs_reada_add() is designed to be fully asynchronous, scrub
>   only calls it in a synchronous way (call btrfs_reada_add() and
>   immediately call btrfs_reada_wait()).
>   Thus rendering a lot of code unnecessary.

I agree with removing the reada.c code, it's overengineered perhaps with
intentions to use it in more places but this hasn't happened and nobody
is interested doing the work. We have the path readahead so it's not
we'd lose readahead capabilities at all.

Thanks for benchmarking it, the drop is acceptable and we know people
are more interested in limiting the load anyway.

> [BENCHMARK]
> The conclusion looks like this:
> 
> For the worst case (no dirty metadata, slow HDD), there will be around 5%
> performance drop for scrub.
> For other cases (even SATA SSD), there is no distinguishable performance
> difference.
> 
> The number is reported scrub speed, in MiB/s.
> The resolution is limited by the reported duration, which only has a
> resolution of 1 second.
> 
> 	Old		New		Diff
> SSD	455.3		466.332		+2.42%
> HDD	103.927 	98.012		-5.69%

I'll copy this information to the last patch changelog.

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

* Re: [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity()
  2021-12-13 13:10 ` [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity() Qu Wenruo
@ 2021-12-14 12:44   ` Qu Wenruo
  0 siblings, 0 replies; 7+ messages in thread
From: Qu Wenruo @ 2021-12-14 12:44 UTC (permalink / raw)
  To: linux-btrfs



On 2021/12/13 21:10, Qu Wenruo wrote:
> In function scrub_stripe() we allocated two btrfs_path, one @path for
> extent tree search and another @ppath for full stripe extent tree search
> for RAID56.
> 
> This is totally uncessary, as the @ppath usage is completely inside
> scrub_raid56_parity(), thus we can move the path allocation into
> scrub_raid56_parity() completely.
> 
> Signed-off-by: Qu Wenruo <wqu@suse.com>
> ---
>   fs/btrfs/scrub.c | 28 ++++++++++++++--------------
>   1 file changed, 14 insertions(+), 14 deletions(-)
> 
> diff --git a/fs/btrfs/scrub.c b/fs/btrfs/scrub.c
> index 15a123e67108..a3bd3dfd5e8b 100644
> --- a/fs/btrfs/scrub.c
> +++ b/fs/btrfs/scrub.c
> @@ -2892,7 +2892,6 @@ static void scrub_parity_put(struct scrub_parity *sparity)
>   static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
>   						  struct map_lookup *map,
>   						  struct btrfs_device *sdev,
> -						  struct btrfs_path *path,
>   						  u64 logic_start,
>   						  u64 logic_end)
>   {
> @@ -2901,6 +2900,7 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
>   	struct btrfs_root *csum_root;
>   	struct btrfs_extent_item *extent;
>   	struct btrfs_io_context *bioc = NULL;
> +	struct btrfs_path *path;
>   	u64 flags;
>   	int ret;
>   	int slot;
> @@ -2919,6 +2919,14 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
>   	int extent_mirror_num;
>   	int stop_loop = 0;
>   
> +	path = btrfs_alloc_path();
> +	if (!path) {
> +		spin_lock(&sctx->stat_lock);
> +		sctx->stat.malloc_errors++;
> +		spin_unlock(&sctx->stat_lock);
> +		return -ENOMEM;
> +	}
> +
>   	ASSERT(map->stripe_len <= U32_MAX);
>   	nsectors = map->stripe_len >> fs_info->sectorsize_bits;
>   	bitmap_len = scrub_calc_parity_bitmap_len(nsectors);
> @@ -2928,6 +2936,7 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
>   		spin_lock(&sctx->stat_lock);
>   		sctx->stat.malloc_errors++;
>   		spin_unlock(&sctx->stat_lock);
> +		btrfs_free_path(path);
>   		return -ENOMEM;
>   	}
>   
> @@ -3117,7 +3126,7 @@ static noinline_for_stack int scrub_raid56_parity(struct scrub_ctx *sctx,
>   	scrub_wr_submit(sctx);
>   	mutex_unlock(&sctx->wr_lock);
>   
> -	btrfs_release_path(path);
> +	btrfs_free_path(path);
>   	return ret < 0 ? ret : 0;
>   }
>   
> @@ -3167,7 +3176,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
>   					   int num, u64 base, u64 length,
>   					   struct btrfs_block_group *cache)
>   {
> -	struct btrfs_path *path, *ppath;
> +	struct btrfs_path *path;
>   	struct btrfs_fs_info *fs_info = sctx->fs_info;
>   	struct btrfs_root *root;
>   	struct btrfs_root *csum_root;
> @@ -3229,12 +3238,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
>   	if (!path)
>   		return -ENOMEM;
>   
> -	ppath = btrfs_alloc_path();
> -	if (!ppath) {
> -		btrfs_free_path(path);
> -		return -ENOMEM;
> -	}
> -
>   	/*
>   	 * work on commit root. The related disk blocks are static as
>   	 * long as COW is applied. This means, it is save to rewrite
> @@ -3243,8 +3246,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
>   	path->search_commit_root = 1;
>   	path->skip_locking = 1;
>   
> -	ppath->search_commit_root = 1;
> -	ppath->skip_locking = 1;

The new path in the raid56 functions doesn't have this.

This is causing false alerts in raid56 dev-replace test cases, like 
btrfs/072.
Which reports csum mismatch for metadata.

Would fix it in next update.

Thanks,
Qu

>   	/*
>   	 * trigger the readahead for extent tree csum tree and wait for
>   	 * completion. During readahead, the scrub is officially paused
> @@ -3347,7 +3348,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
>   				stripe_logical += base;
>   				stripe_end = stripe_logical + increment;
>   				ret = scrub_raid56_parity(sctx, map, scrub_dev,
> -							  ppath, stripe_logical,
> +							  stripe_logical,
>   							  stripe_end);
>   				if (ret)
>   					goto out;
> @@ -3518,7 +3519,7 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
>   						stripe_end = stripe_logical +
>   								increment;
>   						ret = scrub_raid56_parity(sctx,
> -							map, scrub_dev, ppath,
> +							map, scrub_dev,
>   							stripe_logical,
>   							stripe_end);
>   						if (ret)
> @@ -3565,7 +3566,6 @@ static noinline_for_stack int scrub_stripe(struct scrub_ctx *sctx,
>   
>   	blk_finish_plug(&plug);
>   	btrfs_free_path(path);
> -	btrfs_free_path(ppath);
>   
>   	if (sctx->is_dev_replace && ret >= 0) {
>   		int ret2;


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

* Re: [PATCH 0/3] btrfs: use btrfs_path::reada to replace the
  2021-12-14 12:41 ` [PATCH 0/3] btrfs: use btrfs_path::reada to replace the David Sterba
@ 2021-12-14 12:45   ` Qu Wenruo
  0 siblings, 0 replies; 7+ messages in thread
From: Qu Wenruo @ 2021-12-14 12:45 UTC (permalink / raw)
  To: dsterba, Qu Wenruo, linux-btrfs



On 2021/12/14 20:41, David Sterba wrote:
> On Mon, Dec 13, 2021 at 09:10:51PM +0800, Qu Wenruo wrote:
>> [PROBLEMS]
>> The metadata readahead code is introduced in 2011 (surprisingly, the
>> commit message even contains changelog), but now there is only one user
>> for it, and even for the only one user, the readahead mechanism can't
>> provide all it needs:
>>
>> - No support for commit tree readahead
>>    Only support readahead on current tree.
>>
>> - Bad layer separation
>>    To manage on-fly bios, btrfs_reada_add() mechanism internally manages
>>    a kinda complex zone system, and it's bind to per-device.
>>
>>    This is against the common layer separation, as metadata should all be
>>    in btrfs logical address space, should not be bound to device physical
>>    layer.
>>
>>    In fact, this is the cause of all recent reada related bugs.
>>
>> - No caller for asynchronous metadata readahead
>>    Even btrfs_reada_add() is designed to be fully asynchronous, scrub
>>    only calls it in a synchronous way (call btrfs_reada_add() and
>>    immediately call btrfs_reada_wait()).
>>    Thus rendering a lot of code unnecessary.
>
> I agree with removing the reada.c code, it's overengineered perhaps with
> intentions to use it in more places but this hasn't happened and nobody
> is interested doing the work. We have the path readahead so it's not
> we'd lose readahead capabilities at all.
>
> Thanks for benchmarking it, the drop is acceptable and we know people
> are more interested in limiting the load anyway.
>
>> [BENCHMARK]
>> The conclusion looks like this:
>>
>> For the worst case (no dirty metadata, slow HDD), there will be around 5%
>> performance drop for scrub.
>> For other cases (even SATA SSD), there is no distinguishable performance
>> difference.
>>
>> The number is reported scrub speed, in MiB/s.
>> The resolution is limited by the reported duration, which only has a
>> resolution of 1 second.
>>
>> 	Old		New		Diff
>> SSD	455.3		466.332		+2.42%
>> HDD	103.927 	98.012		-5.69%
>
> I'll copy this information to the last patch changelog.

Since I found a bug in the first patch, let me have the chance to
re-order the patches, and put this into the patch removing reada.

Thanks,
Qu

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

end of thread, other threads:[~2021-12-14 12:46 UTC | newest]

Thread overview: 7+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-12-13 13:10 [PATCH 0/3] btrfs: use btrfs_path::reada to replace the Qu Wenruo
2021-12-13 13:10 ` [PATCH 1/3] btrfs: remove the unnecessary path parameter for scrub_raid56_parity() Qu Wenruo
2021-12-14 12:44   ` Qu Wenruo
2021-12-13 13:10 ` [PATCH 2/3] btrfs: remove reada mechanism Qu Wenruo
2021-12-13 13:10 ` [PATCH 3/3] btrfs: use btrfs_path::reada for scrub extent tree readahead Qu Wenruo
2021-12-14 12:41 ` [PATCH 0/3] btrfs: use btrfs_path::reada to replace the David Sterba
2021-12-14 12:45   ` Qu Wenruo

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).