All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH V4 00/13] MD: a caching layer for raid5/6
@ 2015-06-23 21:37 Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 01/13] MD: add a new disk role to present cache device Shaohua Li
                   ` (14 more replies)
  0 siblings, 15 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

Hi,

This is the V4 version of the raid5/6 caching layer patches. The patches add
a caching layer for raid5/6. The caching layer uses a SSD as a cache for a raid
5/6. It works like the similar way of a hardware raid controller. The purpose
is to improve raid performance (reduce read-modify-write) and fix write hole
issue.

I split the patch to into smaller ones and hopefuly they are easier to
understand. The splitted patches will not break bisect. Functions of main parts
are divided well, though some data structures not.

I detached the multiple reclaim thread patch, the patch set is aimed to make
basic logic ready, and we can improve performance later (as long as we can make
sure current logic is flexible to have improvement space for performance).

Neil,

For the issue if flush_start block is required, I double checked it. You are
right we don't really need it, the data/parity checksum stored in cache disk
will guarantee data integrity, but we can't use a simple disk cache flush as
there is ordering issue. So the flush_start block does increase overhead, but
might not too much. I didn't delete it yet, but I'm open to do it.

Thanks,
Shaohua

V4:
-split patche into smaller ones
-add more comments into code and some code cleanup
-bug fixes in recovery code
-fix the feature bit

V3:
-make reclaim multi-thread
-add statistics in sysfs
-bug fixes

V2:
-metadata write doesn't use FUA
-discard request is only issued when necessary
-bug fixes and cleanup

Shaohua Li (12):
  raid5: directly use mddev->queue
  raid5: cache log handling
  raid5: cache part of raid5 cache
  raid5: cache reclaim support
  raid5: cache IO error handling
  raid5: cache device quiesce support
  raid5: cache recovery support
  raid5: add some sysfs entries
  raid5: don't allow resize/reshape with cache support
  raid5: guarantee cache release stripes in correct way
  raid5: enable cache for raid array with cache disk
  raid5: skip resync if caching is enabled

Song Liu (1):
  MD: add a new disk role to present cache device

 drivers/md/Makefile            |    2 +-
 drivers/md/md.c                |   24 +-
 drivers/md/md.h                |    4 +
 drivers/md/raid5-cache.c       | 3755 ++++++++++++++++++++++++++++++++++++++++
 drivers/md/raid5.c             |  176 +-
 drivers/md/raid5.h             |   24 +
 include/uapi/linux/raid/md_p.h |   79 +
 7 files changed, 4016 insertions(+), 48 deletions(-)
 create mode 100644 drivers/md/raid5-cache.c

-- 
1.8.1


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

* [PATCH V4 01/13] MD: add a new disk role to present cache device
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 02/13] raid5: directly use mddev->queue Shaohua Li
                   ` (13 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

From: Song Liu <songliubraving@fb.com>

Next patches will use a disk as raid5/6 caching. We need a new disk role
to present the cache device and add MD_FEATURE_WRITE_CACHE to
feature_map for backward compability.

Signed-off-by: Song Liu <songliubraving@fb.com>
Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/md.c                | 24 ++++++++++++++++++++++--
 drivers/md/md.h                |  4 ++++
 include/uapi/linux/raid/md_p.h |  3 +++
 3 files changed, 29 insertions(+), 2 deletions(-)

diff --git a/drivers/md/md.c b/drivers/md/md.c
index 4dbed4a..599525c 100644
--- a/drivers/md/md.c
+++ b/drivers/md/md.c
@@ -1656,6 +1656,16 @@ static int super_1_validate(struct mddev *mddev, struct md_rdev *rdev)
 		case 0xfffe: /* faulty */
 			set_bit(Faulty, &rdev->flags);
 			break;
+		case 0xfffd: /* cache device */
+			if (!(sb->feature_map & MD_FEATURE_WRITE_CACHE)) {
+				/* cache device without cache feature */
+				printk(KERN_WARNING
+				  "md: cache device provided without write "
+				  "cache feature, ignoring the device\n");
+				return -EINVAL;
+			}
+			set_bit(WriteCache, &rdev->flags);
+			break;
 		default:
 			rdev->saved_raid_disk = role;
 			if ((le32_to_cpu(sb->feature_map) &
@@ -1811,7 +1821,10 @@ static void super_1_sync(struct mddev *mddev, struct md_rdev *rdev)
 			sb->dev_roles[i] = cpu_to_le16(0xfffe);
 		else if (test_bit(In_sync, &rdev2->flags))
 			sb->dev_roles[i] = cpu_to_le16(rdev2->raid_disk);
-		else if (rdev2->raid_disk >= 0)
+		else if (test_bit(WriteCache, &rdev2->flags)) {
+			sb->dev_roles[i] = cpu_to_le16(0xfffd);
+			sb->feature_map |= cpu_to_le32(MD_FEATURE_WRITE_CACHE);
+		} else if (rdev2->raid_disk >= 0)
 			sb->dev_roles[i] = cpu_to_le16(rdev2->raid_disk);
 		else
 			sb->dev_roles[i] = cpu_to_le16(0xffff);
@@ -5781,7 +5794,8 @@ static int get_disk_info(struct mddev *mddev, void __user * arg)
 		else if (test_bit(In_sync, &rdev->flags)) {
 			info.state |= (1<<MD_DISK_ACTIVE);
 			info.state |= (1<<MD_DISK_SYNC);
-		}
+		} else if (test_bit(WriteCache, &rdev->flags))
+			info.state |= (1<<MD_DISK_WRITECACHE);
 		if (test_bit(WriteMostly, &rdev->flags))
 			info.state |= (1<<MD_DISK_WRITEMOSTLY);
 	} else {
@@ -5896,6 +5910,8 @@ static int add_new_disk(struct mddev *mddev, mdu_disk_info_t *info)
 		else
 			clear_bit(WriteMostly, &rdev->flags);
 
+		if (info->state & (1<<MD_DISK_WRITECACHE))
+			set_bit(WriteCache, &rdev->flags);
 		/*
 		 * check whether the device shows up in other nodes
 		 */
@@ -7264,6 +7280,10 @@ static int md_seq_show(struct seq_file *seq, void *v)
 				seq_printf(seq, "(F)");
 				continue;
 			}
+			if (test_bit(WriteCache, &rdev->flags)) {
+				seq_printf(seq, "(C)");
+				continue;
+			}
 			if (rdev->raid_disk < 0)
 				seq_printf(seq, "(S)"); /* spare */
 			if (test_bit(Replacement, &rdev->flags))
diff --git a/drivers/md/md.h b/drivers/md/md.h
index 4046a6c..6857592 100644
--- a/drivers/md/md.h
+++ b/drivers/md/md.h
@@ -175,6 +175,10 @@ enum flag_bits {
 				 * This device is seen locally but not
 				 * by the whole cluster
 				 */
+	WriteCache,		/* This device is used as write cache.
+				 * Usually, this device should be faster
+				 * than other devices in the array
+				 */
 };
 
 #define BB_LEN_MASK	(0x00000000000001FFULL)
diff --git a/include/uapi/linux/raid/md_p.h b/include/uapi/linux/raid/md_p.h
index 2ae6131..8c8e12c 100644
--- a/include/uapi/linux/raid/md_p.h
+++ b/include/uapi/linux/raid/md_p.h
@@ -89,6 +89,7 @@
 				   * read requests will only be sent here in
 				   * dire need
 				   */
+#define MD_DISK_WRITECACHE      18 /* disk is used as the write cache in RAID-5/6 */
 
 typedef struct mdp_device_descriptor_s {
 	__u32 number;		/* 0 Device number in the entire set	      */
@@ -302,6 +303,7 @@ struct mdp_superblock_1 {
 #define	MD_FEATURE_RECOVERY_BITMAP	128 /* recovery that is happening
 					     * is guided by bitmap.
 					     */
+#define	MD_FEATURE_WRITE_CACHE		256 /* support write cache */
 #define	MD_FEATURE_ALL			(MD_FEATURE_BITMAP_OFFSET	\
 					|MD_FEATURE_RECOVERY_OFFSET	\
 					|MD_FEATURE_RESHAPE_ACTIVE	\
@@ -310,6 +312,7 @@ struct mdp_superblock_1 {
 					|MD_FEATURE_RESHAPE_BACKWARDS	\
 					|MD_FEATURE_NEW_OFFSET		\
 					|MD_FEATURE_RECOVERY_BITMAP	\
+					|MD_FEATURE_WRITE_CACHE		\
 					)
 
 #endif
-- 
1.8.1


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

* [PATCH V4 02/13] raid5: directly use mddev->queue
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 01/13] MD: add a new disk role to present cache device Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 03/13] raid5: cache log handling Shaohua Li
                   ` (12 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

When the cache layer flushes data from cache disk to raid disks, it will
dipsatch IO to raid disks. At that time, we don't have a block device
attached to the bio, so directly use mddev->queue. That should not
impact IO dispatched to rdev, which has rdev block device attached.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5.c | 12 +++++-------
 1 file changed, 5 insertions(+), 7 deletions(-)

diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index b6793d2..8160a63 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -223,7 +223,7 @@ static int raid6_idx_to_slot(int idx, struct stripe_head *sh,
 	return slot;
 }
 
-static void return_io(struct bio *return_bi)
+static void return_io(struct r5conf *conf, struct bio *return_bi)
 {
 	struct bio *bi = return_bi;
 	while (bi) {
@@ -231,8 +231,7 @@ static void return_io(struct bio *return_bi)
 		return_bi = bi->bi_next;
 		bi->bi_next = NULL;
 		bi->bi_iter.bi_size = 0;
-		trace_block_bio_complete(bdev_get_queue(bi->bi_bdev),
-					 bi, 0);
+		trace_block_bio_complete(conf->mddev->queue, bi, 0);
 		bio_endio(bi, 0);
 		bi = return_bi;
 	}
@@ -1200,7 +1199,7 @@ static void ops_complete_biofill(void *stripe_head_ref)
 	}
 	clear_bit(STRIPE_BIOFILL_RUN, &sh->state);
 
-	return_io(return_bi);
+	return_io(sh->raid_conf, return_bi);
 
 	set_bit(STRIPE_HANDLE, &sh->state);
 	release_stripe(sh);
@@ -4594,7 +4593,7 @@ static void handle_stripe(struct stripe_head *sh)
 			md_wakeup_thread(conf->mddev->thread);
 	}
 
-	return_io(s.return_bi);
+	return_io(conf, s.return_bi);
 
 	clear_bit_unlock(STRIPE_ACTIVE, &sh->state);
 }
@@ -5298,8 +5297,7 @@ static void make_request(struct mddev *mddev, struct bio * bi)
 		if ( rw == WRITE )
 			md_write_end(mddev);
 
-		trace_block_bio_complete(bdev_get_queue(bi->bi_bdev),
-					 bi, 0);
+		trace_block_bio_complete(mddev->queue, bi, 0);
 		bio_endio(bi, 0);
 	}
 }
-- 
1.8.1


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

* [PATCH V4 03/13] raid5: cache log handling
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 01/13] MD: add a new disk role to present cache device Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 02/13] raid5: directly use mddev->queue Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 04/13] raid5: cache part of raid5 cache Shaohua Li
                   ` (11 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

This is the log handling part of raid5 cache. I divided cache into two
parts. One is log, which handles IO to cache disk and isn't aware how we
track cache data. The other is cache, which handles data tracking and
in-memory caching, it delegates IO store to log. The structures and
functions with r5l_ are for log, and those with r5c_ are for cache, but
there might be some exceptions we can't restrictly destinguish.

The cache disk will be organized as a simple ring buffer log. Logically
it looks like this:
|super|meta data|data|...|meta data|parity data|...|flush_start|...|flush_end|

super/meta data/flush_start/flush_end always use a single block. Block
size could be 512 to 4k.

super records raid array charactics to make sure raid array and cache
disk match. It also records some other parameters like log tail and
block size.

For IO data, meta data is a tuple of (raid_sector, io length, checksum),
which will be appended to the log, data follows; for raid 5/6 parity,
meta data is a tuple of (stripe_sector, parity length, checksum), which
will be appended to the log, parity data follows. meta data is stored in
a meta data block(r5l_meta_block).

meta data block is essentially a tuple (sequence, metadata length). The
sequence number is monotonously increased and can be used for integrity
checking and ordering track. A meta data block can have a mix of data
meta data tuple and parity meta data tuple. How many tuples will be
stored in meta data block depends on the inflight data/parity should be
dispatched. meta data block might only store one tuple.

We don't have in-disk index for the data appended to the log. So either
we can rebuild an in-memory index at startup with scanning the whole
log, or we can flush all data from cache disk to raid disks at shutdown
so cache disk has no valid data. Current code chooses the second option,
but this can be easily changed.

flush_start/flush_end block look like meta data block, but are to track
reclaim stream. Data stored there is stripe sector which starts/ends
flush to raid disks. More details are in reclaim and recover patches.

Crash recovery or startup will scan the log to read the metadata and
rebuild in-memory index. If metadata is broken at the head of the log,
even metadata afterward is ok, the scanning will not work well. So we
take some steps to mitigate the issue:
-A flush request is sent to cache disk every second to relieve the issue
(this isn't a must-have but a worst case defence, since reclaim will
always flush disk cache)
-FLUSH/FUA request is carefully handled. FLUSH/FUA will only be
dispatched after all previous requests finish.

To maintain the ordering, all IO are added into a list, flush/fua IO can
only be dispatched when they are the first entry of the list. Other IO
don't have such limitation. More details can be found in the comments of
r5l_io_unit.

r5l_task is the bridge between log part and cache part. Cache part
dispatches a task to log for IO. After task is finished, task returns
corresponding data/metadata position and sequence number where the task
data is stored in log.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c       | 1317 ++++++++++++++++++++++++++++++++++++++++
 include/uapi/linux/raid/md_p.h |   76 +++
 2 files changed, 1393 insertions(+)
 create mode 100644 drivers/md/raid5-cache.c

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
new file mode 100644
index 0000000..d6b7c64
--- /dev/null
+++ b/drivers/md/raid5-cache.c
@@ -0,0 +1,1317 @@
+/*
+ * A caching layer for raid 4/5/6
+ *
+ * Copyright (C) 2015 Shaohua Li <shli@fb.com>
+ *
+ * This program is free software; you can redistribute it and/or modify it
+ * under the terms and conditions of the GNU General Public License,
+ * version 2, as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope it will be useful, but WITHOUT
+ * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
+ * FITNESS FOR A PARTICULAR PURPOSE.  See the GNU General Public License for
+ * more details.
+ *
+ * You should have received a copy of the GNU General Public License along with
+ * this program; if not, write to the Free Software Foundation, Inc.,
+ * 51 Franklin St - Fifth Floor, Boston, MA 02110-1301 USA.
+ */
+#include <linux/kernel.h>
+#include <linux/wait.h>
+#include <linux/blkdev.h>
+#include <linux/slab.h>
+#include <linux/raid/md_p.h>
+#include <linux/crc32.h>
+#include <linux/list_sort.h>
+#include "md.h"
+#include "raid5.h"
+
+/* the log disk cache will be flushed forcely every second */
+#define LOG_FLUSH_TIME HZ
+#define LOG_SUPER_WRITE_TIME (5 * HZ)
+
+typedef u64 r5blk_t; /* log blocks, could be 512B - 4k */
+
+/*
+ * cache disk is treated as a log. data structures and routinues with r5l_ are
+ * for log. log is just for data storing in cache disk. It's not aware of how
+ * the data is tracked in memory
+ * */
+struct r5l_log {
+	struct md_rdev *rdev;
+
+	struct page *super_page;
+	u8 uuid[16];
+	u32 uuid_checksum_data;
+	u32 uuid_checksum_meta;
+
+	unsigned int block_size; /* bytes */
+	unsigned int block_sector_shift;
+	unsigned int page_block_shift; /* page to block */
+	unsigned int stripe_data_size; /* chunk_size * data_disks, sector */
+	unsigned int chunk_size; /* sector */
+	unsigned int stripe_size; /* chunk_size * all_disks, sector */
+	unsigned int parity_disks; /* parity disks of a stripe */
+
+	r5blk_t total_blocks;
+	r5blk_t first_block;
+	r5blk_t last_block;
+
+	/* background reclaim starts if free log space hits the watermark */
+	r5blk_t low_watermark;
+	/* background reclaim stops if free log space hits the watermark */
+	r5blk_t high_watermark;
+
+	r5blk_t last_checkpoint; /* log tail */
+	/* last_checkpoint to last_freepoint is freed but not discard yet */
+	r5blk_t last_freepoint;
+	r5blk_t super_point; /* checkpoint recorded in super block */
+	u64 last_cp_seq; /* log tail sequence */
+	u64 last_freeseq; /* sequence in last_freepoint */
+	unsigned long last_cp_time;
+	struct work_struct discard_work;
+
+	int do_discard; /* if do discard */
+
+	u64 seq; /* current sequence */
+	r5blk_t log_start; /* current log position */
+
+	u8 data_checksum_type;
+	u8 meta_checksum_type;
+
+	unsigned int reserved_blocks;
+	wait_queue_head_t space_waitq;
+
+	struct mutex io_mutex;
+	struct r5l_io_unit *current_io;
+
+	spinlock_t io_list_lock;
+	struct list_head running_ios; /* running io_unit list */
+
+	struct list_head task_list; /* pending bio task list */
+	struct list_head parity_task_list; /* pending parity task list */
+	spinlock_t task_lock;
+
+	struct kmem_cache *io_kc;
+	mempool_t *io_pool;
+	struct bio_set *bio_set;
+
+	unsigned long next_flush_time; /* force disk flush time */
+	struct work_struct flush_work;
+};
+
+/*
+ * r5l_task is an abstract of IO r5c_cache dispatched undearneath. It could be
+ * a bio or several parity pages. In either case, we append the bio or parity
+ * pages into log with appropriate metadata and return where we store it. The
+ * IO handling is transparent to r5c_cache
+ *
+ * If the task is for a bio, the bio is cloned. This is to simplify io error
+ * handling. The bio dispatched to cache disk can fail. IO error handling will
+ * resend it to raid disks for retry
+ * */
+struct r5l_task;
+/* end function must free task */
+typedef void (r5l_task_end_fn)(struct r5l_task *task, int error);
+struct r5l_task {
+	struct list_head list;
+	int type;
+	struct bio *bio;
+	union {
+		struct bio *orig_bio;
+		struct {
+			sector_t stripe_sector;
+			struct page *page_p;
+			struct page *page_q;
+		};
+	};
+	/* tasks in a single r5l_io_unit will have the same seq and meta_start */
+	sector_t meta_start;
+	sector_t data_start;
+	u64 seq;
+	r5l_task_end_fn *fn;
+	void *private;
+	unsigned int reserved_blocks;
+	u32 checksum[];
+};
+
+/*
+ * r5l_io_unit is a combination of a meta page (r5l_meta_block or
+ * r5l_flush_block) and several r5l_tasks. the meta page will record meta data
+ * of the r5l_tasks. Log can be thought of a set of r5l_io_units. Each
+ * r5l_io_unit is treated as a whole in IO handling. Each io_unit has a sequence
+ * number in ascend order too, tasks in the io_unit will have the same sequence
+ * number. r5c_cache will use the sequence number to sort io ranges and track
+ * log tail. Recovery uses the sequence number to determine log integrity.
+ *
+ * Data and metadata in log is written with normal IO write. A power failure
+ * can cause data loss. To relieve the issue, log disk cache will be flushed
+ * forcely every second.
+ *
+ * Note IO is finished out of order. If metadata corrupts in the middle,
+ * recovery can't work well even metadata/data at the tail is good. IO in log
+ * tail could finish earlier than IO ahead, so we must be very careful to
+ * handle FLUSH/FUA bio.
+ *
+ * FLUSH bio: the bio will be dispatched after all previous IO finish. FLUSH
+ * syntax doesn't require pending IO finish, but pending IO might be a metadata
+ * in the middle of log, we must force this order.
+ *
+ * FUA bio: same like FLUSH bio, previous meta IO must be finished before
+ * dispatching this bio. To simplify implementation, we wait all previous IO.
+ * And we must add a FLUSH to make sure previous IO hit disk media. metadata
+ * of the bio and the bio itself must be written with FUA.
+ *
+ * The r5l_io_unit is added to r5l_log running_ios list in submiting order.
+ * FLUSH/FUA io_unit isn't allowed to be executed ahead of other io_units, so
+ * above ordering is guaranteed.
+ **/
+struct r5l_io_unit {
+	struct r5l_log *log;
+	struct list_head log_sibling; /* link to log->running_ios */
+
+	struct page *meta_page;
+	sector_t meta_sector;
+	int meta_offset; /* current offset in meta_page */
+	u64 seq;
+	struct bio *meta_bio;
+
+	struct list_head tasks;
+	struct bio *current_bio; /* current_bio accepting pages */
+	atomic_t refcnt;
+
+	unsigned int has_flush:1; /* include flush request */
+	unsigned int has_fua:1; /* include fua request */
+	unsigned int has_null_flush:1; /* include empty flush request */
+	/*
+	 * io isn't sent yet, flush/fua request can only be submitted till it's
+	 * the first IO in running_ios list
+	 * */
+	unsigned int io_deferred:1;
+	int error;
+};
+
+/* reclaim reason */
+enum {
+	RECLAIM_MEM = 0, /* work hard to reclaim memory */
+	RECLAIM_MEM_BACKGROUND = 1, /* try to reclaim memory */
+	RECLAIM_MEM_FULL = 2, /* only reclaim full stripe */
+	RECLAIM_DISK = 8, /* work hard to reclaim disk */
+	RECLAIM_DISK_BACKGROUND = 9, /* try to reclaim disk */
+	RECLAIM_FLUSH_ALL = 16, /* flush all data to raid */
+};
+
+#define PAGE_SECTOR_SHIFT (PAGE_SHIFT - 9)
+
+static inline struct r5c_cache *r5l_cache(struct r5l_log *log)
+{
+	return container_of(log, struct r5c_cache, log);
+}
+
+static inline struct block_device *r5l_bdev(struct r5l_log *log)
+{
+	return log->rdev->bdev;
+}
+
+static inline sector_t r5l_block_to_sector(struct r5l_log *log, r5blk_t block)
+{
+	return block << log->block_sector_shift;
+}
+
+static inline r5blk_t r5l_sector_to_block(struct r5l_log *log, sector_t s)
+{
+	return s >> log->block_sector_shift;
+}
+
+static inline int r5l_page_blocks(struct r5l_log *log, int pages)
+{
+	return pages << log->page_block_shift;
+}
+
+static u32 r5l_calculate_checksum(struct r5l_log *log, u32 crc,
+	void *buf, size_t size, bool data)
+{
+
+	/*
+	 * We currently only support CRC32, other checksum algorithm might be
+	 * added later
+	 * */
+	if (log->data_checksum_type != R5LOG_CHECKSUM_CRC32)
+		BUG();
+	if (log->meta_checksum_type != R5LOG_CHECKSUM_CRC32)
+		BUG();
+	return crc32_le(crc, buf, size);
+}
+
+static r5blk_t r5l_ring_add(struct r5l_log *log, r5blk_t block, int inc)
+{
+	block += inc;
+	if (block >= log->last_block)
+		block = block - log->total_blocks;
+	return block;
+}
+
+static void r5c_wake_wait_reclaimer(struct r5c_cache *cache, int reason);
+static void r5c_wake_reclaimer(struct r5c_cache *cache, int reason);
+
+static r5blk_t r5l_free_space(struct r5l_log *log)
+{
+	r5blk_t used_size;
+
+	if (log->log_start >= log->last_checkpoint)
+		used_size = log->log_start - log->last_checkpoint;
+	else
+		used_size = log->log_start + log->total_blocks -
+			log->last_checkpoint;
+
+	if (log->total_blocks > used_size + log->reserved_blocks)
+		return log->total_blocks - used_size - log->reserved_blocks;
+	return 0;
+}
+
+/* Make sure we have enough free space in log device */
+static void r5l_get_reserve(struct r5l_log *log, unsigned int size)
+{
+	r5blk_t free;
+
+	mutex_lock(&log->io_mutex);
+
+	free = r5l_free_space(log);
+	if (free >= size) {
+		log->reserved_blocks += size;
+
+		if (free - size < log->low_watermark)
+			r5c_wake_reclaimer(r5l_cache(log),
+				RECLAIM_DISK_BACKGROUND);
+		mutex_unlock(&log->io_mutex);
+		return;
+	}
+
+	log->reserved_blocks += size;
+	mutex_unlock(&log->io_mutex);
+
+	schedule_work(&log->discard_work);
+	r5c_wake_reclaimer(r5l_cache(log), RECLAIM_DISK);
+
+	mutex_lock(&log->io_mutex);
+	wait_event_cmd(log->space_waitq, r5l_free_space(log) > 0,
+		mutex_unlock(&log->io_mutex), mutex_lock(&log->io_mutex));
+	mutex_unlock(&log->io_mutex);
+}
+
+static void __r5l_put_reserve(struct r5l_log *log, unsigned int reserved_blocks)
+{
+	BUG_ON(!mutex_is_locked(&log->io_mutex));
+
+	log->reserved_blocks -= reserved_blocks;
+	if (r5l_free_space(log) > 0)
+		wake_up(&log->space_waitq);
+}
+
+static void r5l_put_reserve(struct r5l_log *log, unsigned int reserved_blocks)
+{
+	mutex_lock(&log->io_mutex);
+	__r5l_put_reserve(log, reserved_blocks);
+	mutex_unlock(&log->io_mutex);
+}
+
+#define LOG_POOL_SIZE 8
+static void *__io_pool_alloc(gfp_t gfp, void *pool_data)
+{
+	struct r5l_log *log = pool_data;
+	struct r5l_io_unit *io;
+
+	io = kmem_cache_alloc(log->io_kc, gfp);
+	if (!io)
+		return NULL;
+	io->meta_page = alloc_page(gfp);
+	if (!io->meta_page) {
+		kmem_cache_free(log->io_kc, io);
+		return NULL;
+	}
+	return io;
+}
+
+static void __io_pool_free(void *e, void *pool_data)
+{
+	struct r5l_log *log = pool_data;
+	struct r5l_io_unit *io = e;
+
+	__free_page(io->meta_page);
+	kmem_cache_free(log->io_kc, io);
+}
+
+static struct r5l_io_unit *r5l_alloc_io_unit(struct r5l_log *log)
+{
+	struct r5l_io_unit *io;
+	struct page *page;
+
+	io = mempool_alloc(log->io_pool, GFP_NOIO);
+	page = io->meta_page;
+	memset(io, 0, sizeof(*io));
+	io->meta_page = page;
+
+	io->log = log;
+	INIT_LIST_HEAD(&io->tasks);
+	atomic_set(&io->refcnt, 1);
+	clear_highpage(io->meta_page);
+	return io;
+}
+
+static void r5l_free_io_unit(struct r5l_log *log, struct r5l_io_unit *io)
+{
+	mempool_free(io, log->io_pool);
+}
+
+static void r5l_submit_bio(struct r5l_log *log, int rw, struct bio *bio)
+{
+	/* all IO must start from rdev->data_offset */
+	struct bio *split;
+
+	if (bio_end_sector(bio) > (r5l_block_to_sector(log,
+				   log->last_block))) {
+		split = bio_split(bio, r5l_block_to_sector(log,
+			log->last_block) - bio->bi_iter.bi_sector,
+			GFP_NOIO, log->bio_set);
+		bio_chain(split, bio);
+		bio->bi_iter.bi_sector = r5l_block_to_sector(log,
+					log->first_block);
+		split->bi_iter.bi_sector += log->rdev->data_offset;
+		submit_bio(rw, split);
+	}
+
+	bio->bi_iter.bi_sector += log->rdev->data_offset;
+	submit_bio(rw, bio);
+}
+
+static void r5l_put_io_unit(struct r5l_io_unit *io, int error)
+{
+	struct r5l_log *log = io->log;
+	struct r5l_io_unit *io_deferred;
+	unsigned long flags;
+	struct r5l_task *task;
+
+	if (error)
+		io->error = error;
+
+	if (!atomic_dec_and_test(&io->refcnt))
+		return;
+
+	spin_lock_irqsave(&log->io_list_lock, flags);
+	list_del(&io->log_sibling);
+	if (!list_empty(&log->running_ios)) {
+		/*
+		 * FLUSH/FUA io_unit is deferred because of ordering, now we
+		 * can dispatch it
+		 * */
+		io_deferred = list_first_entry(&log->running_ios,
+			struct r5l_io_unit, log_sibling);
+		if (io_deferred->io_deferred)
+			schedule_work(&log->flush_work);
+	}
+	spin_unlock_irqrestore(&log->io_list_lock, flags);
+
+	while (!list_empty(&io->tasks)) {
+		task = list_first_entry(&io->tasks, struct r5l_task,
+			list);
+		list_del(&task->list);
+		task->fn(task, io->error);
+	}
+	r5l_free_io_unit(log, io);
+}
+
+static void r5l_log_endio(struct bio *bio, int error)
+{
+	struct r5l_io_unit *io = bio->bi_private;
+
+	bio_put(bio);
+	r5l_put_io_unit(io, error);
+}
+
+static void r5l_do_submit_io(struct r5l_log *log, struct r5l_io_unit *io)
+{
+	struct r5l_task *task;
+	int rw = WRITE;
+
+	if (io->has_null_flush) {
+		log->next_flush_time = jiffies + LOG_FLUSH_TIME;
+		task = list_first_entry(&io->tasks, struct r5l_task, list);
+		submit_bio(WRITE_FLUSH, task->bio);
+		r5l_put_io_unit(io, 0);
+		return;
+	}
+
+	if (io->has_flush)
+		rw |= REQ_FLUSH;
+	if (io->has_fua)
+		rw |= REQ_FUA | REQ_FLUSH;
+	if (time_before(log->next_flush_time, jiffies))
+		rw |= REQ_FLUSH;
+	if (rw & REQ_FLUSH)
+		log->next_flush_time = jiffies + LOG_FLUSH_TIME;
+
+	r5l_submit_bio(log, rw, io->meta_bio);
+
+	list_for_each_entry(task, &io->tasks, list) {
+		if (task->bio)
+			r5l_submit_bio(log, task->bio->bi_rw, task->bio);
+	}
+	r5l_put_io_unit(io, 0);
+}
+
+/*
+ * submit IO for a r5l_io_unit. For flush/fua, ordering is important. We don't
+ * submit such io_unit till all previous io_units finish. If io_unit isn't
+ * dispatched right now, it's marked as deferred
+ **/
+static void r5l_submit_io(struct r5l_log *log, struct r5l_io_unit *io)
+{
+	unsigned long flags;
+	bool do_io = true;
+
+	if (io->has_flush || io->has_fua) {
+		spin_lock_irqsave(&log->io_list_lock, flags);
+		if (io !=  list_first_entry(&log->running_ios,
+		    struct r5l_io_unit, log_sibling)) {
+			io->io_deferred = 1;
+			do_io = false;
+		}
+		spin_unlock_irqrestore(&log->io_list_lock, flags);
+		if (!do_io)
+			return;
+	}
+
+	r5l_do_submit_io(log, io);
+}
+
+static void r5l_submit_current_io(struct r5l_log *log)
+{
+	struct r5l_io_unit *io = log->current_io;
+	struct r5l_meta_header *header;
+	u32 crc;
+
+	if (!io)
+		return;
+
+	header = page_address(io->meta_page);
+	header->meta_size = cpu_to_le32(io->meta_offset);
+	crc = r5l_calculate_checksum(log, log->uuid_checksum_meta,
+		header, log->block_size, false);
+	header->checksum = cpu_to_le32(crc);
+
+	log->current_io = NULL;
+
+	r5l_submit_io(log, io);
+}
+
+/* deferred io_unit will be dispatched here */
+static void r5l_submit_io_async(struct work_struct *work)
+{
+	struct r5l_log *log = container_of(work, struct r5l_log, flush_work);
+	struct r5l_io_unit *io = NULL;
+
+	spin_lock_irq(&log->io_list_lock);
+	if (!list_empty(&log->running_ios)) {
+		io = list_first_entry(&log->running_ios, struct r5l_io_unit,
+				log_sibling);
+		if (!io->io_deferred)
+			io = NULL;
+		else
+			io->io_deferred = 0;
+	}
+	spin_unlock_irq(&log->io_list_lock);
+	if (!io)
+		return;
+	r5l_do_submit_io(log, io);
+}
+
+static struct r5l_io_unit *r5l_new_meta(struct r5l_log *log)
+{
+	struct r5l_io_unit *io;
+
+	io = r5l_alloc_io_unit(log);
+
+	io->meta_sector = r5l_block_to_sector(log, log->log_start);
+	io->meta_offset = sizeof(struct r5l_meta_header);
+	io->seq = log->seq;
+
+	io->meta_bio = bio_alloc_bioset(GFP_NOIO,
+		bio_get_nr_vecs(r5l_bdev(log)), log->bio_set);
+	io->meta_bio->bi_bdev = r5l_bdev(log);
+	io->meta_bio->bi_iter.bi_sector = io->meta_sector;
+	bio_add_page(io->meta_bio, io->meta_page, log->block_size, 0);
+	io->meta_bio->bi_end_io = r5l_log_endio;
+	io->meta_bio->bi_private = io;
+
+	atomic_inc(&io->refcnt);
+
+	log->seq++;
+	log->log_start = r5l_ring_add(log, log->log_start, 1);
+	io->current_bio = io->meta_bio;
+
+	spin_lock_irq(&log->io_list_lock);
+	list_add_tail(&io->log_sibling, &log->running_ios);
+	spin_unlock_irq(&log->io_list_lock);
+
+	return io;
+}
+
+static int r5l_get_meta(struct r5l_log *log, unsigned int pages, bool is_bio)
+{
+	struct r5l_io_unit *io;
+	struct r5l_meta_header *header;
+	unsigned int meta_size;
+
+	meta_size = sizeof(struct r5l_meta_payload) +
+		sizeof(u32) * pages;
+	io = log->current_io;
+	if (io && io->meta_offset + meta_size > log->block_size)
+		r5l_submit_current_io(log);
+	io = log->current_io;
+	if (io) {
+		/* If task has bio, we can't use previous bio to add data */
+		if (is_bio)
+			io->current_bio = NULL;
+		return 0;
+	}
+
+	io = r5l_new_meta(log);
+	log->current_io = io;
+
+	if (is_bio)
+		io->current_bio = NULL;
+
+	header = page_address(io->meta_page);
+	header->magic = cpu_to_le32(R5LOG_MAGIC);
+	header->type = cpu_to_le32(R5LOG_TYPE_META);
+	header->seq = cpu_to_le64(log->seq - 1);
+	if (log->log_start == log->first_block)
+		header->position = cpu_to_le64(log->last_block - 1);
+	else
+		header->position = cpu_to_le64(log->log_start - 1);
+
+	return 0;
+}
+
+/* must hold io_mutex */
+static int r5l_run_data_task(struct r5l_log *log, struct r5l_task *task)
+{
+	unsigned int pages;
+	struct r5l_io_unit *io;
+	unsigned int reserved_blocks = task->reserved_blocks;
+	struct r5l_meta_payload *payload;
+	struct bio *bio = task->bio;
+	void *meta;
+	int i;
+
+	pages = bio_sectors(bio) >> PAGE_SECTOR_SHIFT;
+
+	r5l_get_meta(log, pages, true);
+
+	io = log->current_io;
+
+	if (bio->bi_rw & REQ_FLUSH)
+		io->has_flush = 1;
+	if (bio->bi_rw & REQ_FUA)
+		io->has_fua = 1;
+
+	meta = page_address(io->meta_page);
+	payload = meta + io->meta_offset;
+	payload->payload_type = cpu_to_le16(task->type);
+	payload->blocks = cpu_to_le32(r5l_page_blocks(log, pages));
+	payload->location = cpu_to_le64(bio->bi_iter.bi_sector);
+	for (i = 0; i < pages; i++)
+		payload->data_checksum[i] = cpu_to_le32(task->checksum[i]);
+
+	io->meta_offset += sizeof(struct r5l_meta_payload) +
+		sizeof(u32) * pages;
+	task->seq = io->seq;
+	task->meta_start = io->meta_sector;
+	task->data_start = r5l_block_to_sector(log, log->log_start);
+
+	/* submiting will split the bio if it hits log end */
+	bio->bi_iter.bi_sector = task->data_start;
+	bio->bi_end_io = r5l_log_endio;
+	bio->bi_private = io;
+	bio->bi_bdev = r5l_bdev(log);
+	atomic_inc(&io->refcnt);
+
+	log->log_start = r5l_ring_add(log, log->log_start,
+		r5l_page_blocks(log, pages));
+
+	list_add_tail(&task->list, &io->tasks);
+
+	__r5l_put_reserve(log, reserved_blocks);
+	return 0;
+}
+
+/*
+ * We probably can directly add the parity pages into previous bio of io_unit
+ * (the meta data bio or bio for previous parity pages. If that isn't possible,
+ * we allocate a new bio
+ * */
+static int r5l_task_add_parity_pages(struct r5l_log *log,
+	struct r5l_task *task)
+{
+	struct r5l_io_unit *io = log->current_io;
+	struct bio *bio;
+	struct page *pages[] = {task->page_p, task->page_q};
+	r5blk_t pos = log->log_start;
+	r5blk_t bio_pos = 0;
+	int i;
+
+	bio = io->current_bio;
+
+alloc_bio:
+	if (!bio) {
+		/* bio can only contain one page ? */
+		BUG_ON(task->bio);
+		bio = bio_alloc_bioset(GFP_NOIO,
+			bio_get_nr_vecs(r5l_bdev(log)), log->bio_set);
+		bio->bi_rw = WRITE;
+		bio->bi_bdev = r5l_bdev(log);
+		/*
+		 * Set sector to 0, so bio_add_page doesn't fail if we are the
+		 * blkdev tail. Later we will set the bi_sector again and split
+		 * bio if required
+		 * */
+		bio->bi_iter.bi_sector = 0;
+		bio_pos = pos;
+		task->bio = bio;
+		io->current_bio = bio;
+	}
+
+	for (i = 0; i < ARRAY_SIZE(pages); i++) {
+		if (!pages[i])
+			continue;
+		if (!bio_add_page(bio, pages[i], PAGE_SIZE, 0)) {
+			bio = NULL;
+			goto alloc_bio;
+		}
+		pages[i] = NULL;
+		pos = r5l_ring_add(log, pos, r5l_page_blocks(log, 1));
+	}
+	if (bio_pos != 0)
+		bio->bi_iter.bi_sector = r5l_block_to_sector(log, bio_pos);
+	return 0;
+}
+
+/* must hold io_mutex. parity task doesn't have a bio */
+static int r5l_run_parity_task(struct r5l_log *log, struct r5l_task *task)
+{
+	unsigned int pages;
+	struct r5l_io_unit *io;
+	struct r5l_meta_payload *payload;
+	struct bio *bio;
+	void *meta;
+	int i;
+
+	pages = !!task->page_p + !!task->page_q;
+
+	r5l_get_meta(log, pages, false);
+
+	io = log->current_io;
+
+	meta = page_address(io->meta_page);
+	payload = meta + io->meta_offset;
+	payload->payload_type = cpu_to_le16(task->type);
+	payload->blocks = cpu_to_le32(r5l_page_blocks(log, pages));
+	payload->location = cpu_to_le64(task->stripe_sector);
+
+	for (i = 0; i < pages; i++)
+		payload->data_checksum[i] = cpu_to_le32(task->checksum[i]);
+
+	io->meta_offset += sizeof(struct r5l_meta_payload) +
+		sizeof(u32) * pages;
+	task->seq = io->seq;
+	task->meta_start = io->meta_sector;
+	task->data_start = r5l_block_to_sector(log, log->log_start);
+
+	r5l_task_add_parity_pages(log, task);
+
+	/* this should only happen r5l_task_add_parity_pages allocates a bio */
+	if (task->bio) {
+		bio = task->bio;
+		bio->bi_end_io = r5l_log_endio;
+		bio->bi_private = io;
+		atomic_inc(&io->refcnt);
+	}
+
+	log->log_start = r5l_ring_add(log, log->log_start,
+		r5l_page_blocks(log, pages));
+
+	list_add_tail(&task->list, &io->tasks);
+
+	return 0;
+}
+
+/*
+ * We reserve space for parity, so reclaim doesn't deadlock because of no space
+ * to write parity. To make this work, reclaim (and conrresponding raid5 threads)
+ * should not run data tasks
+ **/
+static void r5l_run_tasks(struct r5l_log *log, bool parity)
+{
+	struct r5l_task *task;
+	LIST_HEAD(tasks);
+
+	spin_lock(&log->task_lock);
+	list_splice_init(&log->parity_task_list, &tasks);
+	if (!parity)
+		list_splice_tail_init(&log->task_list, &tasks);
+	spin_unlock(&log->task_lock);
+
+	if (list_empty(&tasks))
+		return;
+
+	mutex_lock(&log->io_mutex);
+	while (!list_empty(&tasks)) {
+		task = list_first_entry(&tasks, struct r5l_task, list);
+		list_del_init(&task->list);
+
+		if (task->type == R5LOG_PAYLOAD_PARITY)
+			r5l_run_parity_task(log, task);
+		else
+			r5l_run_data_task(log, task);
+	}
+	r5l_submit_current_io(log);
+	mutex_unlock(&log->io_mutex);
+}
+
+static void r5l_queue_task(struct r5l_log *log, struct r5l_task *task)
+{
+	spin_lock(&log->task_lock);
+	if (task->type == R5LOG_PAYLOAD_DATA)
+		list_add_tail(&task->list, &log->task_list);
+	else
+		list_add_tail(&task->list, &log->parity_task_list);
+	spin_unlock(&log->task_lock);
+}
+
+/*
+ * empty bio hasn't any payload, but our FLUSH bio ordering requirement holds
+ * here too. To simplify implementation, we fake io_unit/task, so such bio can
+ * be handled like normal bios
+ * */
+static int r5l_queue_empty_flush_bio(struct r5l_log *log, struct bio *bio,
+	r5l_task_end_fn fn, void *private)
+{
+	struct r5l_io_unit *io;
+	struct r5l_task *task;
+
+	task = kmalloc(sizeof(*task), GFP_NOIO);
+	if (!task)
+		return -ENOMEM;
+	task->type = R5LOG_PAYLOAD_DATA;
+	task->bio = bio_clone(bio, GFP_NOIO);
+	if (!task->bio) {
+		kfree(task);
+		return -ENOMEM;
+	}
+	task->orig_bio = bio;
+	task->fn = fn;
+	task->private = private;
+	task->reserved_blocks = 0;
+
+	io = r5l_alloc_io_unit(log);
+
+	task->bio->bi_bdev = r5l_bdev(log);
+	task->bio->bi_end_io = r5l_log_endio;
+	task->bio->bi_private = io;
+
+	io->has_flush = 1;
+	io->has_null_flush = 1;
+
+	mutex_lock(&log->io_mutex);
+	r5l_submit_current_io(log);
+
+	atomic_inc(&io->refcnt);
+	list_add_tail(&task->list, &io->tasks);
+
+	spin_lock_irq(&log->io_list_lock);
+	list_add_tail(&io->log_sibling, &log->running_ios);
+	spin_unlock_irq(&log->io_list_lock);
+
+	r5l_submit_io(log, io);
+
+	mutex_unlock(&log->io_mutex);
+	return 0;
+}
+
+static int r5l_queue_bio(struct r5l_log *log, struct bio *bio,
+	r5l_task_end_fn fn, void *private, unsigned int reserved_blocks)
+{
+	struct r5l_task *task;
+	int pages = bio_sectors(bio) >> PAGE_SECTOR_SHIFT;
+	u32 *checksum;
+	struct bio *src = bio;
+	struct bvec_iter iter;
+	struct bio_vec bv;
+	void *src_p;
+	unsigned int page_index = 0, page_offset = 0;
+	unsigned int bytes;
+
+	task = kmalloc(sizeof(*task) + sizeof(u32) * pages, GFP_NOIO);
+	if (!task)
+		return -ENOMEM;
+
+	INIT_LIST_HEAD(&task->list);
+	task->type = R5LOG_PAYLOAD_DATA;
+	task->bio = bio_clone(bio, GFP_NOIO);
+	if (!task->bio) {
+		kfree(task);
+		return -ENOMEM;
+	}
+	task->orig_bio = bio;
+	task->fn = fn;
+	task->private = private;
+	task->reserved_blocks = reserved_blocks;
+
+	checksum = (u32 *)(task + 1);
+
+	iter = src->bi_iter;
+
+	checksum[0] = log->uuid_checksum_data;
+	while (1) {
+		if (!iter.bi_size) {
+			src = src->bi_next;
+			if (!src)
+				break;
+
+			iter = src->bi_iter;
+		}
+
+		if (page_offset == PAGE_SIZE) {
+			page_index++;
+			page_offset = 0;
+			checksum[page_index] = log->uuid_checksum_data;
+		}
+
+		bv = bio_iter_iovec(src, iter);
+
+		bytes = min_t(unsigned int, bv.bv_len, PAGE_SIZE - page_offset);
+
+		src_p = kmap_atomic(bv.bv_page);
+
+		checksum[page_index] = r5l_calculate_checksum(log,
+			checksum[page_index], src_p + bv.bv_offset,
+			bytes, true);
+
+		kunmap_atomic(src_p);
+
+		bio_advance_iter(src, &iter, bytes);
+		page_offset += bytes;
+	}
+	r5l_queue_task(log, task);
+	return 0;
+}
+
+static int r5l_queue_parity(struct r5l_log *log,
+	sector_t stripe_sector, struct page *page_p,
+	struct page *page_q, r5l_task_end_fn fn, void *private)
+{
+	struct r5l_task *task;
+	void *addr;
+	u32 *checksum;
+
+	task = kmalloc(sizeof(*task) + sizeof(u32) * 2, GFP_NOIO);
+	if (!task)
+		return -ENOMEM;
+
+	INIT_LIST_HEAD(&task->list);
+	task->type = R5LOG_PAYLOAD_PARITY;
+	task->bio = NULL;
+	task->stripe_sector = stripe_sector;
+	task->page_p = page_p;
+	task->page_q = page_q;
+	task->fn = fn;
+	task->private = private;
+	task->reserved_blocks = 0;
+
+	checksum = (u32 *)(task + 1);
+	addr = kmap_atomic(page_p);
+	checksum[0] = r5l_calculate_checksum(log,
+		log->uuid_checksum_data, addr, PAGE_SIZE, true);
+	kunmap_atomic(addr);
+
+	if (page_q) {
+		addr = kmap_atomic(page_q);
+		checksum[1] = r5l_calculate_checksum(log,
+			log->uuid_checksum_data, addr, PAGE_SIZE, true);
+		kunmap_atomic(addr);
+	} else
+		checksum[1] = 0;
+
+	r5l_queue_task(log, task);
+	return 0;
+}
+
+static void r5l_fb_end(struct r5l_task *task, int error)
+{
+	struct completion *comp = task->private;
+	complete(comp);
+}
+
+/*
+ * ordering requirement holds here. To simplify implementation, we fake
+ * io_unit/task, so flush block can be handled like other meta data
+ * */
+static int r5l_flush_block(struct r5l_log *log, int type, __le64 *stripes,
+	size_t size, u64 *next_seq, sector_t *next_sec)
+{
+	struct r5l_io_unit *io;
+	struct r5l_flush_block *fb;
+	struct r5l_task task;
+	DECLARE_COMPLETION_ONSTACK(fb_complete);
+	u32 crc;
+
+	mutex_lock(&log->io_mutex);
+
+	r5l_submit_current_io(log);
+
+	io = r5l_new_meta(log);
+	task.fn = r5l_fb_end;
+	task.private = &fb_complete;
+	task.reserved_blocks = 0;
+	task.type = R5LOG_PAYLOAD_PARITY;
+	task.bio = NULL;
+
+	fb = page_address(io->meta_page);
+	fb->header.magic = cpu_to_le32(R5LOG_MAGIC);
+	fb->header.type = cpu_to_le32(type);
+	fb->header.seq = cpu_to_le64(log->seq - 1);
+	if (log->log_start == log->first_block)
+		fb->header.position = cpu_to_le64(log->last_block - 1);
+	else
+		fb->header.position = cpu_to_le64(log->log_start - 1);
+	fb->header.meta_size = cpu_to_le32(sizeof(struct r5l_flush_block)
+					+ size);
+	memcpy(fb->flush_stripes, stripes, size);
+	crc = r5l_calculate_checksum(log, log->uuid_checksum_meta, fb,
+			log->block_size, false);
+	fb->header.checksum = cpu_to_le32(crc);
+
+	list_add_tail(&task.list, &io->tasks);
+
+	io->has_flush = 1;
+	io->has_fua = 1;
+	r5l_submit_io(log, io);
+
+	*next_seq = log->seq;
+	*next_sec = r5l_block_to_sector(log, log->log_start);
+
+	mutex_unlock(&log->io_mutex);
+
+	wait_for_completion_io(&fb_complete);
+	return 0;
+}
+
+static void r5l_discard_blocks(struct r5l_log *log, r5blk_t start, r5blk_t end)
+{
+	if (!log->do_discard)
+		return;
+	if (start < end) {
+		blkdev_issue_discard(r5l_bdev(log),
+		    r5l_block_to_sector(log, start) + log->rdev->data_offset,
+		    r5l_block_to_sector(log, end - start), GFP_NOIO, 0);
+	} else {
+		blkdev_issue_discard(r5l_bdev(log),
+		    r5l_block_to_sector(log, start) + log->rdev->data_offset,
+		    r5l_block_to_sector(log, log->last_block - start),
+		    GFP_NOIO, 0);
+		blkdev_issue_discard(r5l_bdev(log),
+		    r5l_block_to_sector(log, log->first_block) +
+		    log->rdev->data_offset,
+		    r5l_block_to_sector(log, end - log->first_block),
+		    GFP_NOIO, 0);
+	}
+}
+
+static int r5l_do_write_super(struct r5l_log *log, u64 seq, r5blk_t cp)
+{
+	struct r5l_super_block *sb_blk;
+	struct timespec now = current_kernel_time();
+	u32 crc;
+
+	log->last_cp_time = jiffies + LOG_SUPER_WRITE_TIME;
+
+	if (log->super_point == cp)
+		return 0;
+	sb_blk = page_address(log->super_page);
+	sb_blk->header.seq = cpu_to_le64(seq);
+	sb_blk->last_checkpoint = cpu_to_le64(cp);
+	sb_blk->update_time_sec = cpu_to_le64(now.tv_sec);
+	sb_blk->update_time_nsec = cpu_to_le64(now.tv_nsec);
+	sb_blk->header.checksum = 0;
+	crc = r5l_calculate_checksum(log, log->uuid_checksum_meta,
+		sb_blk, log->block_size, false);
+	sb_blk->header.checksum = cpu_to_le32(crc);
+
+	if (!sync_page_io(log->rdev, 0, log->block_size, log->super_page,
+	     WRITE_FUA, false))
+		return -EIO;
+	log->super_point = cp;
+	return 0;
+}
+
+static void r5l_discard_work(struct work_struct *work)
+{
+	struct r5l_log *log = container_of(work, struct r5l_log,
+		discard_work);
+	struct r5c_cache *cache = r5l_cache(log);
+	r5blk_t free;
+
+	mutex_lock(&log->io_mutex);
+	r5l_do_write_super(log, log->last_freeseq, log->last_freepoint);
+
+	if (log->do_discard && log->last_checkpoint !=
+	    log->last_freepoint)
+		r5l_discard_blocks(log, log->last_checkpoint,
+			log->last_freepoint);
+
+	log->last_cp_seq = log->last_freeseq;
+	log->last_checkpoint = log->last_freepoint;
+
+	free = r5l_free_space(log);
+	if (free > 0) {
+		clear_bit(RECLAIM_DISK, &cache->reclaim_reason);
+		wake_up(&log->space_waitq);
+	}
+	if (free > log->high_watermark)
+		clear_bit(RECLAIM_DISK_BACKGROUND, &cache->reclaim_reason);
+
+	mutex_unlock(&log->io_mutex);
+}
+
+static void r5l_schedule_discard(struct r5l_log *log)
+{
+	r5blk_t size;
+
+	if (log->last_freepoint >= log->last_checkpoint)
+		size = log->last_freepoint - log->last_checkpoint;
+	else
+		size = log->last_freepoint + log->total_blocks -
+			log->last_checkpoint;
+
+	if (size * 10 >= log->total_blocks)
+		schedule_work(&log->discard_work);
+}
+
+static int r5l_write_super(struct r5l_log *log, u64 seq, sector_t cp)
+{
+	struct r5c_cache *cache = r5l_cache(log);
+	r5blk_t free;
+
+	r5l_schedule_discard(log);
+
+	mutex_lock(&log->io_mutex);
+	log->last_freepoint = r5l_sector_to_block(log, cp);
+	log->last_freeseq = seq;
+
+	/*
+	 * skiping superblock update doesn't impact the correctness since
+	 * recover can handle it. So we try to defer superblock write for
+	 * performance
+	 * */
+	if (seq == log->last_cp_seq || time_after(log->last_cp_time,
+	    jiffies)) {
+		mutex_unlock(&log->io_mutex);
+		return 0;
+	}
+
+	r5l_do_write_super(log, seq, r5l_sector_to_block(log, cp));
+
+	if (!log->do_discard) {
+		log->last_cp_seq = seq;
+		log->last_checkpoint = log->last_freepoint;
+	}
+
+	free = r5l_free_space(log);
+	if (free > 0) {
+		clear_bit(RECLAIM_DISK, &cache->reclaim_reason);
+		wake_up(&log->space_waitq);
+	}
+	if (free > log->high_watermark)
+		clear_bit(RECLAIM_DISK_BACKGROUND, &cache->reclaim_reason);
+
+	mutex_unlock(&log->io_mutex);
+
+	return 0;
+}
+
+static void r5c_wake_reclaimer(struct r5c_cache *cache, int reason)
+{
+}
+
+static void r5c_wake_wait_reclaimer(struct r5c_cache *cache, int reason)
+{
+}
+
+static int r5l_load_log(struct r5l_log *log)
+{
+	return 0;
+}
+
+static int r5l_read_super(struct r5l_log *log)
+{
+	struct md_rdev *rdev = log->rdev;
+	struct r5l_super_block *sb_blk;
+	struct page *page = log->super_page;
+	u32 crc, stored_crc;
+
+	if (!sync_page_io(rdev, 0, PAGE_SIZE, page, READ, false))
+		return -EIO;
+
+	sb_blk = page_address(page);
+
+	if (le32_to_cpu(sb_blk->version) != R5LOG_VERSION ||
+	    le32_to_cpu(sb_blk->header.magic) != R5LOG_MAGIC ||
+	    le32_to_cpu(sb_blk->header.type) != R5LOG_TYPE_SUPER ||
+	    le64_to_cpu(sb_blk->header.position) != 0 ||
+	    le32_to_cpu(sb_blk->header.meta_size) !=
+	     sizeof(struct r5l_super_block))
+		goto error;
+
+	log->last_cp_seq = le64_to_cpu(sb_blk->header.seq);
+
+	log->block_size = le32_to_cpu(sb_blk->block_size);
+	log->block_sector_shift = ilog2(log->block_size >> 9);
+	log->page_block_shift = PAGE_SHIFT - ilog2(log->block_size);
+
+	/* Only support this stripe size right now */
+	if (le32_to_cpu(sb_blk->stripe_cache_size) != PAGE_SIZE)
+		goto error;
+
+	if (log->block_size > PAGE_SIZE)
+		goto error;
+
+	log->stripe_data_size = le32_to_cpu(sb_blk->stripe_data_size) >> 9;
+	log->chunk_size = le32_to_cpu(sb_blk->chunk_size) >> 9;
+	log->stripe_size = le32_to_cpu(sb_blk->stripe_size) >> 9;
+	log->parity_disks = le32_to_cpu(sb_blk->parity_disks);
+
+	if (sb_blk->meta_checksum_type >= R5LOG_CHECKSUM_NR ||
+	    sb_blk->data_checksum_type >= R5LOG_CHECKSUM_NR)
+		goto error;
+	log->meta_checksum_type = sb_blk->meta_checksum_type;
+	log->data_checksum_type = sb_blk->data_checksum_type;
+
+	stored_crc = le32_to_cpu(sb_blk->header.checksum);
+	sb_blk->header.checksum = 0;
+	crc = r5l_calculate_checksum(log, ~0,
+		sb_blk->uuid, sizeof(sb_blk->uuid), false);
+	crc = r5l_calculate_checksum(log, crc,
+		sb_blk, log->block_size, false);
+	if (crc != stored_crc)
+		goto error;
+
+	if (memcmp(log->uuid, sb_blk->uuid, sizeof(log->uuid)))
+		goto error;
+
+	log->first_block = le64_to_cpu(sb_blk->first_block);
+	if (log->first_block != 1)
+		goto error;
+
+	log->total_blocks = le64_to_cpu(sb_blk->total_blocks);
+	log->last_block = log->first_block + log->total_blocks;
+	log->last_checkpoint = le64_to_cpu(sb_blk->last_checkpoint);
+	log->last_freepoint = log->last_checkpoint;
+	log->super_point = log->last_checkpoint;
+
+	return 0;
+error:
+	return -EINVAL;
+}
+
+static int r5l_init_log(struct r5c_cache *cache)
+{
+	struct r5l_log *log;
+	r5blk_t blocks;
+
+	log = &cache->log;
+
+	log->rdev = cache->rdev;
+
+	log->do_discard = blk_queue_discard(bdev_get_queue(r5l_bdev(log)));
+
+	log->super_page = alloc_page(GFP_KERNEL);
+	if (!log->super_page)
+		return -ENOMEM;
+	memcpy(log->uuid, cache->mddev->uuid, sizeof(log->uuid));
+
+	init_waitqueue_head(&log->space_waitq);
+	mutex_init(&log->io_mutex);
+
+	spin_lock_init(&log->io_list_lock);
+	INIT_LIST_HEAD(&log->running_ios);
+
+	INIT_LIST_HEAD(&log->task_list);
+	INIT_LIST_HEAD(&log->parity_task_list);
+	spin_lock_init(&log->task_lock);
+
+	INIT_WORK(&log->flush_work, r5l_submit_io_async);
+	INIT_WORK(&log->discard_work, r5l_discard_work);
+
+	log->io_kc = KMEM_CACHE(r5l_io_unit, 0);
+	if (!log->io_kc)
+		goto io_kc;
+	log->io_pool = mempool_create(LOG_POOL_SIZE, __io_pool_alloc,
+		__io_pool_free, log);
+	if (!log->io_pool)
+		goto io_pool;
+	log->bio_set = bioset_create(LOG_POOL_SIZE * 2, 0);
+	if (!log->bio_set)
+		goto bio_set;
+
+	if (r5l_read_super(log))
+		goto error;
+
+	log->uuid_checksum_data = r5l_calculate_checksum(log, ~0, log->uuid,
+		sizeof(log->uuid), true);
+	log->uuid_checksum_meta = r5l_calculate_checksum(log, ~0, log->uuid,
+		sizeof(log->uuid), false);
+
+	log->reserved_blocks = r5l_sector_to_block(log,
+		cache->reserved_space) + 1;
+
+	if (log->stripe_data_size != cache->stripe_data_size ||
+	    log->chunk_size != cache->chunk_size ||
+	    log->stripe_size != cache->stripe_size ||
+	    log->parity_disks != cache->parity_disks)
+		goto error;
+
+	r5l_load_log(log);
+
+	blocks = log->total_blocks;
+	do_div(blocks, 10);
+	if (blocks * log->block_size < 1024 * 1024 * 1024)
+		log->low_watermark = blocks;
+	else
+		log->low_watermark = (1024 * 1024 * 1024 / log->block_size);
+	log->high_watermark = (log->low_watermark * 3) >> 1;
+
+	r5l_discard_blocks(log, log->first_block, log->last_block);
+
+	return 0;
+error:
+	bioset_free(log->bio_set);
+bio_set:
+	mempool_destroy(log->io_pool);
+io_pool:
+	kmem_cache_destroy(log->io_kc);
+io_kc:
+	__free_page(log->super_page);
+	return -EINVAL;
+}
+
+static void r5l_exit_log(struct r5l_log *log)
+{
+	blkdev_issue_flush(r5l_bdev(log), GFP_NOIO, NULL);
+	schedule_work(&log->discard_work);
+	flush_work(&log->discard_work);
+
+	bioset_free(log->bio_set);
+	mempool_destroy(log->io_pool);
+	kmem_cache_destroy(log->io_kc);
+	__free_page(log->super_page);
+}
diff --git a/include/uapi/linux/raid/md_p.h b/include/uapi/linux/raid/md_p.h
index 8c8e12c..bc0f3c0 100644
--- a/include/uapi/linux/raid/md_p.h
+++ b/include/uapi/linux/raid/md_p.h
@@ -315,4 +315,80 @@ struct mdp_superblock_1 {
 					|MD_FEATURE_WRITE_CACHE		\
 					)
 
+/* all disk position of below struct start from rdev->start_offset */
+struct r5l_meta_header {
+	__le32 magic;
+	__le32 type;
+	__le32 checksum; /* checksum(metadata block + uuid) */
+	__le32 meta_size;
+	__le64 seq;
+	__le64 position; /* block number the meta is written */
+} __attribute__ ((__packed__));
+
+#define R5LOG_VERSION 0x1
+#define R5LOG_MAGIC 0x6433c509
+
+enum {
+	R5LOG_TYPE_META = 0,
+	R5LOG_TYPE_SUPER = 1,
+	R5LOG_TYPE_FLUSH_START = 2,
+	R5LOG_TYPE_FLUSH_END = 3,
+};
+
+struct r5l_super_block {
+	struct r5l_meta_header header;
+	__le32 version;
+	__le32 stripe_cache_size; /* in kernel stripe unit size, bytes */
+	__le32 block_size; /* log block size, bytes */
+	__le32 stripe_data_size; /* chunk_size * data_disks, bytes */
+	__le32 chunk_size; /* bytes */
+	__le32 stripe_size; /* chunk_size * all_raid_disks, bytes */
+	__le32 parity_disks; /* parity disks in each stripe */
+	__le32 zero_padding;
+	__le64 total_blocks; /* block */
+	__le64 first_block; /* block */
+	__le64 last_checkpoint; /* block */
+	__le64 update_time_sec;
+	__le64 update_time_nsec;
+	__u8 meta_checksum_type;
+	__u8 data_checksum_type;
+	__u8 uuid[16];
+	/* must fill with 0 below */
+} __attribute__ ((__packed__));
+
+enum {
+	R5LOG_CHECKSUM_CRC32 = 0,
+	R5LOG_CHECKSUM_NR = 1,
+};
+
+struct r5l_meta_payload {
+	__le16 payload_type;
+	__le16 payload_flags;
+	__le32 blocks; /* block. For parity, should be 1 or 2 pages */
+	__le64 location; /* sector. For data, it's raid sector.
+			    For parity, it's stripe sector */
+	__le32 data_checksum[]; /* checksum(data + uuid) */
+} __attribute__ ((__packed__));
+
+/* payload type */
+enum {
+	R5LOG_PAYLOAD_DATA = 0,
+	R5LOG_PAYLOAD_PARITY = 1,
+};
+
+/* payload flags */
+enum {
+	R5LOG_PAYLOAD_DISCARD = 1,
+};
+
+struct r5l_meta_block {
+	struct r5l_meta_header header;
+	struct r5l_meta_payload payloads[];
+} __attribute__ ((__packed__));
+
+struct r5l_flush_block {
+	struct r5l_meta_header header;
+	__le64 flush_stripes[]; /* stripe sector */
+} __attribute__ ((__packed__));
+
 #endif
-- 
1.8.1


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

* [PATCH V4 04/13] raid5: cache part of raid5 cache
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (2 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 03/13] raid5: cache log handling Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 05/13] raid5: cache reclaim support Shaohua Li
                   ` (10 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

This is the in-memory representation of cache. It includes a memory pool
to store data which isn't flushed to raid disks yet and index to track
data stored in cache disk.

When new request bio comes to the raid array with cache, we copy data of
the bio to a memory pool and write the bio to cache disk for write
request; copy data from memory pool to bio or directly dispatch the bio
to raid disks depending on if the bio has data in memory pool for read
request. stripe (see below) is the basic track unit, it's aligned to
stripe boundary and stripe size. we must make sure request has correct
alignment and size.

A stripe (r5c_stripe) tracks stripe data. Note, there is difference
between the stripe and existing raid5 stripe cache stripe. stripe here
tracks data of chunk size * raid disks, while stripe cache stripe tracks
data of a single 4k stripe accross all raid disks disks. There is a
radix-tree to map raid disk sector to a stripe. If we write a stripe, we
copy data to the stripe data_pages and write the data to log. To know
where we put the data to log, we have a r5c_io_range to track it. Each
stripe has a list of r5c_io_ranges. At some time, data of a stripe can
be written to raid disks (at reclaim time). After the data is in raid
disks, the stripe and its data pages will be freed.

Each io_range tracks a chunk of data stored in cache disk. io_range are
in two lists. One is in stripe list, the other a global log list. Both
are sorted in time order (or sequence order). The ordering in stripe
list lets us find latest data. The ordering in log list helps track log
tail, so we don't corrupt data which isn't written to raid disks.

We don't have a data structure to track parity of a stripe. We just
write parity to log in reclaim. After reclaim finishes, disk space of
the stripe (including data and parity) can be reused, so we don't need
to track parity in memory.

The cache part doesn't dispatch IO to cache disk. It's the
responsibility of log part. Cache part just sends task to log.

We don't use existing stripe cache mechanism to track stripe. Partly
because existing stripe cache state machine is complex enough we don't
want to make it worse. A new memory pool (and stripe raidx tree) is more
efficient for the cache layer. A separate memory pool is more flexible
too. A memory pool isn't a must-have, since all data is stored in cache
disk, we might remove the memory pool in the future (compared to cache
disk, memory pool is far smaller. The memory pool causes more reclaim
activity and can't absorb write latency). We can easily remove the
separate memory pool if necessary.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/Makefile      |   2 +-
 drivers/md/raid5-cache.c | 870 +++++++++++++++++++++++++++++++++++++++++++++++
 drivers/md/raid5.c       |  21 +-
 drivers/md/raid5.h       |   6 +
 4 files changed, 896 insertions(+), 3 deletions(-)

diff --git a/drivers/md/Makefile b/drivers/md/Makefile
index dba4db5..aeb4330 100644
--- a/drivers/md/Makefile
+++ b/drivers/md/Makefile
@@ -16,7 +16,7 @@ dm-cache-mq-y   += dm-cache-policy-mq.o
 dm-cache-cleaner-y += dm-cache-policy-cleaner.o
 dm-era-y	+= dm-era-target.o
 md-mod-y	+= md.o bitmap.o
-raid456-y	+= raid5.o
+raid456-y	+= raid5.o raid5-cache.o
 
 # Note: link order is important.  All raid personalities
 # and must come before md.o, as they each initialise 
diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index d6b7c64..d0950ae 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -30,6 +30,10 @@
 #define LOG_FLUSH_TIME HZ
 #define LOG_SUPER_WRITE_TIME (5 * HZ)
 
+#define MAX_MEM (256 * 1024 * 1024)
+#define RECLAIM_BATCH 16
+#define CHECKPOINT_TIMEOUT (5 * HZ)
+
 typedef u64 r5blk_t; /* log blocks, could be 512B - 4k */
 
 /*
@@ -191,6 +195,154 @@ struct r5l_io_unit {
 	int error;
 };
 
+/*
+ * a stripe can have data stored to several different chunks in log. each chunk
+ * is a bio (or r5l_task) dispatched to log. When r5l_task is finished,
+ * r5l_task will return position where the bio and corresponding metadata is
+ * stored in cache disk and a sequence number of its metadata. we use a data
+ * structure to index the chunk in memory, which is a r5c_io_range. 
+ *
+ * io_range will be in two lists. One is the stripe_sibling, which is linked to
+ * stripe list. The other is the log_sibling, which is linked to
+ * r5c_cache->log_list. Both lists are sorted, io_range is added to the lists
+ * in time order (or sequence order, since newer io_range has higher sequence
+ * order.
+ *
+ * In the stripe list, ordering can be used to destinguish if data is newer.
+ * newer data is at the tail of the list.
+ *
+ * In the log list, io_range is used to track cache disk usage. the first
+ * entry's meta_start is the log tail. Note, if a stripe is reclaimed, its
+ * io_ranges are deleted. Since each stripe can have its io_ranges stored
+ * sparsely in cache disk, the log list might have holes, but the first entry's
+ * meta_start is always the log tail. The ordering guarantees we can find log
+ * tail. We use meta_start for log tail tracking, because one r5l_meta_block
+ * records several chunks (or io_unit), the meta data can be freed only after
+ * all data pages it tracked are freed.
+ *
+ * When log dispatches IO to cache disk, several io_range (each is a bio) will
+ * be in one r5l_io_unit, so the io_ranges will have the same meta_start and
+ * sequence
+ * */
+struct r5c_io_range {
+	struct list_head log_sibling; /*link to cache->log_list */
+	struct list_head stripe_sibling; /* link to stripe->io_ranges */
+
+	u64 seq; /* sequence of its metadata */
+
+	/* cache disk position of corresponding r5l_meta_block */
+	sector_t meta_start;
+	sector_t data_start; /* cache disk position of data */
+	sector_t raid_start; /* raid position the data should be stored */
+	unsigned int data_sectors;
+
+	struct r5c_stripe *stripe;
+	union {
+		struct bio *bio;
+		u32 *checksum; /* only for recovery */
+	};
+};
+
+/*
+ * track a stripe (chunk_size * raid_disks), not a stripe_cache
+ *
+ * we must be careful about appending data to a stripe when the stripe is being
+ * reclaimed. If we append data to a stripe between stripe parity is in cache
+ * disk and stripe isn't fully reclaimed (reusable), the modified data doesn't
+ * match with parity, recovery will get confused. So if we are reclaiming a
+ * stripe, we freeze the stripe and don't allow new data added to it till the
+ * stripe is fully reclaimed
+ * */
+struct r5c_stripe {
+	u64 raid_index;
+	struct r5c_cache *cache;
+	atomic_t ref;
+	int state;
+	int recovery_state; /* just for recovery */
+
+	struct list_head io_ranges; /* ordered list of r5c_io_range */
+	union {
+		struct list_head stripes;
+		struct list_head parity_list; /* just for recovery */
+	};
+
+	struct list_head lru;
+
+	int existing_pages;
+	atomic_t dirty_stripes;
+	atomic_t pending_bios;
+	struct page **parity_pages; /* just for recovery */
+	struct page *data_pages[];
+};
+
+/* stripe state */
+enum {
+	STRIPE_RUNNING = 0, /* stripe can accept new data */
+	STRIPE_FROZEN = 1, /* Doesn't accept new data and is reclaiming */
+	STRIPE_PARITY_DONE = 2, /* stripe parity is cache disk */
+	STRIPE_INRAID = 3, /* stripe data/parity are in raid disks */
+	STRIPE_DEAD = 4, /* stripe can be reused */
+};
+
+#define STRIPE_LOCK_BITS 8
+struct r5c_cache {
+	struct mddev *mddev;
+	struct md_rdev *rdev;
+
+	struct r5l_log log;
+
+	spinlock_t tree_lock; /* protect stripe_tree, log_list, full_stripes */
+	struct radix_tree_root stripe_tree;
+	struct list_head log_list; /* sorted list of io_range */
+	struct list_head full_stripes; /* stripes have data full */
+	int full_stripe_cnt;
+
+	struct list_head page_pool;
+	spinlock_t pool_lock;
+	u64 free_pages;
+	u64 total_pages;
+	u64 max_pages;
+	/* background reclaim starts when free memory hits this watermark */
+	u64 low_watermark;
+	/* background reclaim stops when free memory hits this watermark */
+	u64 high_watermark;
+
+	unsigned int stripe_data_size; /* chunk_size * data_disks, sector */
+	unsigned int chunk_size; /* sector */
+	unsigned int stripe_size; /* chunk_size * all disks, sector */
+	unsigned int parity_disks; /* parity disks of a stripe */
+	unsigned int stripe_data_pages; /* total data pages of a stripe */
+	unsigned int stripe_parity_pages; /* total parity pages of a stripe */
+
+	unsigned int reclaim_batch;
+
+	unsigned int reserved_space; /* log reserved size, sector */
+
+	unsigned long reclaim_reason;
+	wait_queue_head_t reclaim_wait;
+	struct md_thread *reclaim_thread;
+	__le64 *stripe_flush_data;
+	int quiesce_state;
+
+	int in_recovery; /* in recovery mode */
+
+	struct work_struct pending_io_work;
+
+	spinlock_t stripe_locks[1 << STRIPE_LOCK_BITS];
+	wait_queue_head_t stripe_waitq[1 << STRIPE_LOCK_BITS];
+
+	int error_state;
+	int error_type;
+	int retry_cnt;
+	unsigned long next_retry_time;
+	struct bio_list retry_bio_list; /* bios should be retried after IO error */
+	wait_queue_head_t error_wait;
+
+	struct kmem_cache *io_range_kc;
+	struct kmem_cache *stripe_kc;
+	struct bio_set *bio_set;
+};
+
 /* reclaim reason */
 enum {
 	RECLAIM_MEM = 0, /* work hard to reclaim memory */
@@ -1139,6 +1291,539 @@ static int r5l_write_super(struct r5l_log *log, u64 seq, sector_t cp)
 	return 0;
 }
 
+static inline void r5c_sector_stripe_index_offset(struct r5c_cache *cache,
+	sector_t sect, u64 *index, unsigned int *offset)
+{
+	*offset = sector_div(sect, cache->stripe_data_size);
+	*index = sect;
+}
+
+static inline sector_t r5c_stripe_raid_sector(struct r5c_cache *cache,
+	struct r5c_stripe *stripe)
+{
+	return stripe->raid_index * cache->stripe_data_size;
+}
+
+static void r5c_lock_stripe(struct r5c_cache *cache, struct r5c_stripe *stripe,
+	unsigned long *flags)
+{
+	spinlock_t *lock;
+
+	lock = &cache->stripe_locks[hash_ptr(stripe, STRIPE_LOCK_BITS)];
+	spin_lock_irqsave(lock, *flags);
+}
+
+static void r5c_unlock_stripe(struct r5c_cache *cache,
+	struct r5c_stripe *stripe, unsigned long *flags)
+{
+	spinlock_t *lock;
+
+	lock = &cache->stripe_locks[hash_ptr(stripe, STRIPE_LOCK_BITS)];
+	spin_unlock_irqrestore(lock, *flags);
+}
+
+static wait_queue_head_t *r5c_stripe_waitq(struct r5c_cache *cache,
+	struct r5c_stripe *stripe)
+{
+	return &cache->stripe_waitq[hash_ptr(stripe, STRIPE_LOCK_BITS)];
+}
+
+static void r5c_stripe_wait_state(struct r5c_cache *cache,
+	struct r5c_stripe *stripe, int state)
+{
+	wait_event(*r5c_stripe_waitq(cache, stripe), stripe->state >= state);
+}
+
+static struct page *r5c_get_page(struct r5c_cache *cache)
+{
+	struct page *page;
+again:
+	spin_lock_irq(&cache->pool_lock);
+	if (!list_empty(&cache->page_pool)) {
+		page = list_first_entry(&cache->page_pool,
+			struct page, lru);
+		list_del_init(&page->lru);
+		cache->free_pages--;
+		if (cache->free_pages < cache->low_watermark)
+			r5c_wake_reclaimer(cache,
+				RECLAIM_MEM_BACKGROUND);
+	}
+	spin_unlock_irq(&cache->pool_lock);
+	if (page)
+		return page;
+	r5c_wake_wait_reclaimer(cache, RECLAIM_MEM);
+	goto again;
+}
+
+static void r5c_put_pages(struct r5c_cache *cache,
+	struct page *pages[], int size)
+{
+	unsigned long flags;
+	int free = 0;
+	int i;
+
+	spin_lock_irqsave(&cache->pool_lock, flags);
+	for (i = 0; i < size; i++) {
+		if (!pages[i])
+			continue;
+		if (cache->total_pages > cache->max_pages) {
+			__free_page(pages[i]);
+			cache->total_pages--;
+		} else {
+			list_add(&pages[i]->lru, &cache->page_pool);
+			free++;
+		}
+	}
+	cache->free_pages += free;
+	if (cache->free_pages >= cache->high_watermark)
+		clear_bit(RECLAIM_MEM_BACKGROUND, &cache->reclaim_reason);
+	clear_bit(RECLAIM_MEM, &cache->reclaim_reason);
+	spin_unlock_irqrestore(&cache->pool_lock, flags);
+
+	wake_up(&cache->reclaim_wait);
+}
+
+static struct r5c_stripe *
+r5c_search_stripe(struct r5c_cache *cache, u64 stripe_index)
+{
+	struct r5c_stripe *stripe;
+
+	spin_lock_irq(&cache->tree_lock);
+	stripe = radix_tree_lookup(&cache->stripe_tree, stripe_index);
+	spin_unlock_irq(&cache->tree_lock);
+	return stripe;
+}
+
+static void r5c_put_stripe(struct r5c_stripe *stripe);
+static struct r5c_stripe *
+r5c_get_stripe(struct r5c_cache *cache, u64 stripe_index)
+{
+	struct r5c_stripe *stripe;
+
+again:
+	spin_lock_irq(&cache->tree_lock);
+	stripe = radix_tree_lookup(&cache->stripe_tree, stripe_index);
+	if (stripe)
+		atomic_inc(&stripe->ref);
+	spin_unlock_irq(&cache->tree_lock);
+
+	if (!stripe)
+		return NULL;
+	BUG_ON(stripe->raid_index != stripe_index);
+
+	/*
+	 * The stripe is being reclaimed, wait reclaim finish, see the comment
+	 * in r5c_stripe
+	 * */
+	if (stripe->state >= STRIPE_FROZEN) {
+		r5c_stripe_wait_state(cache, stripe, STRIPE_DEAD);
+		r5c_put_stripe(stripe);
+		goto again;
+	}
+	return stripe;
+}
+
+static struct r5c_stripe *
+r5c_create_get_stripe(struct r5c_cache *cache, u64 stripe_index)
+{
+	struct r5c_stripe *stripe, *new;
+	int error;
+
+again:
+	spin_lock_irq(&cache->tree_lock);
+	stripe = radix_tree_lookup(&cache->stripe_tree, stripe_index);
+	if (stripe)
+		atomic_inc(&stripe->ref);
+	spin_unlock_irq(&cache->tree_lock);
+
+	if (stripe)
+		goto has_stripe;
+
+	new = kmem_cache_zalloc(cache->stripe_kc, GFP_NOIO);
+	if (!new)
+		return NULL;
+
+	new->raid_index = stripe_index;
+	atomic_set(&new->ref, 2);
+	new->state = STRIPE_RUNNING;
+	new->cache = cache;
+	INIT_LIST_HEAD(&new->io_ranges);
+	INIT_LIST_HEAD(&new->stripes);
+	INIT_LIST_HEAD(&new->lru);
+
+	error = radix_tree_preload(GFP_NOIO);
+	if (error) {
+		kmem_cache_free(cache->stripe_kc, new);
+		return NULL;
+	}
+
+	spin_lock_irq(&cache->tree_lock);
+	if (radix_tree_insert(&cache->stripe_tree, stripe_index, new)) {
+		stripe = radix_tree_lookup(&cache->stripe_tree, stripe_index);
+		atomic_inc(&stripe->ref);
+		kmem_cache_free(cache->stripe_kc, new);
+	} else
+		stripe = new;
+	spin_unlock_irq(&cache->tree_lock);
+
+	radix_tree_preload_end();
+
+	BUG_ON(stripe->raid_index != stripe_index);
+has_stripe:
+	/*
+	 * The stripe is being reclaimed, wait reclaim finish, see the comment
+	 * in r5c_stripe
+	 * */
+	if (stripe->state >= STRIPE_FROZEN) {
+		r5c_stripe_wait_state(cache, stripe, STRIPE_DEAD);
+		r5c_put_stripe(stripe);
+		goto again;
+	}
+
+	return stripe;
+}
+
+static void r5c_put_stripe(struct r5c_stripe *stripe)
+{
+	struct r5c_cache *cache = stripe->cache;
+	struct r5c_io_range *range;
+	unsigned long flags, flags2;
+
+	local_irq_save(flags);
+
+	/*
+	 * must take lock too, since we can't access stripe if its ref is
+	 * dropped to 0
+	 **/
+	if (!atomic_dec_and_lock(&stripe->ref, &cache->tree_lock)) {
+		local_irq_restore(flags);
+		/* the stripe is freezing, might wait for ref */
+		wake_up(r5c_stripe_waitq(cache, stripe));
+
+		return;
+	}
+
+	BUG_ON(radix_tree_delete(&cache->stripe_tree, stripe->raid_index)
+		!= stripe);
+
+	r5c_lock_stripe(cache, stripe, &flags2);
+	while (!list_empty(&stripe->io_ranges)) {
+		range = list_first_entry(&stripe->io_ranges,
+			struct r5c_io_range, stripe_sibling);
+		list_del(&range->stripe_sibling);
+
+		list_del(&range->log_sibling);
+
+		kmem_cache_free(cache->io_range_kc, range);
+	}
+	r5c_put_pages(cache, stripe->data_pages, cache->stripe_data_pages);
+	BUG_ON(stripe->parity_pages);
+
+	r5c_unlock_stripe(cache, stripe, &flags2);
+
+	spin_unlock_irqrestore(&cache->tree_lock, flags);
+
+	kmem_cache_free(cache->stripe_kc, stripe);
+}
+
+static void r5c_bio_task_end(struct r5l_task *task, int error)
+{
+	struct r5c_io_range *range = task->private;
+	struct r5c_stripe *stripe = range->stripe;
+	struct r5c_cache *cache = stripe->cache;
+	struct r5c_io_range *tmp;
+	unsigned long flags, flags2;
+	struct bio *orig_bio;
+
+	range->seq = task->seq;
+	range->meta_start = task->meta_start;
+	range->data_start = task->data_start;
+	orig_bio = task->orig_bio;
+	kfree(task);
+
+	spin_lock_irqsave(&cache->tree_lock, flags);
+
+	if (list_empty(&cache->log_list)) {
+		list_add_tail(&range->log_sibling, &cache->log_list);
+		goto out;
+	}
+
+	/*
+	 * generally tmp will be the last one. The order is important to track
+	 * log tail, see comments in r5c_io_range
+	 * */
+	list_for_each_entry_reverse(tmp, &cache->log_list, log_sibling) {
+		/*
+		 * later range has bigger seq and meta_start than previous
+		 * range
+		 **/
+		if (range->seq >= tmp->seq)
+			break;
+	}
+	list_add_tail(&range->log_sibling, &tmp->log_sibling);
+out:
+	r5c_lock_stripe(cache, stripe, &flags2);
+	/*
+	 * full stripes aren't reclaimed immediately till we have enough such
+	 * stripes, this is to improve performance
+	 * */
+	if (stripe->existing_pages == cache->stripe_data_pages &&
+	    list_empty(&stripe->lru)) {
+		list_add_tail(&stripe->lru, &cache->full_stripes);
+		cache->full_stripe_cnt++;
+		if (cache->full_stripe_cnt >= cache->reclaim_batch)
+			r5c_wake_reclaimer(cache,
+				RECLAIM_MEM_FULL);
+	}
+	r5c_unlock_stripe(cache, stripe, &flags2);
+
+	spin_unlock_irqrestore(&cache->tree_lock, flags);
+
+	if (!error) {
+		bio_endio(orig_bio, 0);
+		md_write_end(cache->mddev);
+	}
+
+	r5c_put_stripe(stripe);
+}
+
+static void r5c_copy_bio(struct bio *bio, struct page *pages[], bool tobio)
+{
+	struct bio *src = bio;
+	struct bvec_iter iter;
+	struct bio_vec bv;
+	void *src_p, *dst_p;
+	unsigned page_index = 0, page_offset = 0;
+	unsigned bytes;
+
+	iter = src->bi_iter;
+
+	while (1) {
+		if (!iter.bi_size) {
+			src = src->bi_next;
+			if (!src)
+				break;
+
+			iter = src->bi_iter;
+		}
+
+		if (page_offset == PAGE_SIZE) {
+			page_index++;
+			page_offset = 0;
+		}
+
+		bv = bio_iter_iovec(src, iter);
+
+		bytes = min_t(unsigned int, bv.bv_len, PAGE_SIZE - page_offset);
+
+		src_p = kmap_atomic(bv.bv_page);
+		dst_p = kmap_atomic(pages[page_index]);
+
+		if (tobio) {
+			memcpy(src_p + bv.bv_offset,
+			       dst_p + page_offset, bytes);
+		} else {
+			memcpy(dst_p + page_offset,
+			       src_p + bv.bv_offset, bytes);
+		}
+
+		kunmap_atomic(dst_p);
+		kunmap_atomic(src_p);
+
+		bio_advance_iter(src, &iter, bytes);
+		page_offset += bytes;
+	}
+}
+
+static void r5c_bio_flush_task_end(struct r5l_task *task, int error)
+{
+	struct r5c_cache *cache = task->private;
+	struct bio *orig_bio = task->orig_bio;
+	unsigned long flags;
+
+	kfree(task);
+
+	if (!error) {
+		bio_endio(orig_bio, 0);
+		md_write_end(cache->mddev);
+	}
+}
+
+static void r5c_write_bio(struct r5c_cache *cache, struct bio *bio)
+{
+	struct r5c_stripe *stripe;
+	struct r5c_io_range *io_range;
+	u64 index;
+	unsigned int offset;
+	int i, new_pages = 0;
+	unsigned long flags;
+	unsigned int reserved_blocks = 0;
+	unsigned int pages;
+
+	/* Doesn't support discard */
+	if (bio->bi_rw & REQ_DISCARD) {
+		bio_endio(bio, 0);
+		md_write_end(cache->mddev);
+		return;
+	}
+
+	if (bio->bi_iter.bi_size == 0) {
+		BUG_ON(!(bio->bi_rw & REQ_FLUSH));
+		if (r5l_queue_empty_flush_bio(&cache->log, bio,
+		    r5c_bio_flush_task_end, cache))
+			goto enter_error;
+		return;
+	}
+
+	pages = bio_sectors(bio) >> PAGE_SECTOR_SHIFT;
+	reserved_blocks = 1 + r5l_page_blocks(&cache->log, pages);
+	r5l_get_reserve(&cache->log, reserved_blocks);
+
+	r5c_sector_stripe_index_offset(cache, bio->bi_iter.bi_sector,
+		&index, &offset);
+
+	stripe = r5c_create_get_stripe(cache, index);
+	if (!stripe)
+		goto enter_error;
+
+	io_range = kmem_cache_alloc(cache->io_range_kc, GFP_NOIO);
+	if (!io_range)
+		goto put_error;
+
+	io_range->bio = bio;
+	io_range->raid_start = bio->bi_iter.bi_sector;
+	io_range->data_sectors = bio_sectors(bio);
+	io_range->stripe = stripe;
+
+	/* FIXME: if read comes in here, it might read random data */
+	offset >>= PAGE_SECTOR_SHIFT;
+	for (i = offset; i < offset + (bio_sectors(bio) >> PAGE_SECTOR_SHIFT);
+	     i++) {
+		if (stripe->data_pages[i])
+			continue;
+		stripe->data_pages[i] = r5c_get_page(cache);
+		new_pages++;
+	}
+
+	r5c_copy_bio(bio, &stripe->data_pages[offset], false);
+
+	r5c_lock_stripe(cache, stripe, &flags);
+	list_add_tail(&io_range->stripe_sibling, &stripe->io_ranges);
+	stripe->existing_pages += new_pages;
+	r5c_unlock_stripe(cache, stripe, &flags);
+
+	if (r5l_queue_bio(&cache->log, bio, r5c_bio_task_end, io_range,
+	    reserved_blocks))
+		goto put_error;
+	return;
+put_error:
+	r5c_put_stripe(stripe);
+enter_error:
+	r5l_put_reserve(&cache->log, reserved_blocks);
+	raid5_make_request(cache->mddev, bio);
+}
+
+static void r5c_read_bio(struct r5c_cache *cache, struct bio *bio)
+{
+	struct r5c_stripe *stripe;
+	u64 stripe_index;
+	int offset;
+	u64 start, end, tmp;
+	struct bio *split;
+
+	r5c_sector_stripe_index_offset(cache, bio->bi_iter.bi_sector,
+		&stripe_index, &offset);
+
+	stripe = r5c_get_stripe(cache, stripe_index);
+	if (!stripe) {
+		raid5_make_request(cache->mddev, bio);
+		return;
+	}
+
+	start = offset >> PAGE_SECTOR_SHIFT;
+	end = start + (bio_sectors(bio) >> PAGE_SECTOR_SHIFT);
+
+	while (start < end) {
+		if (stripe->data_pages[start]) {
+			tmp = start;
+			while (tmp < end && stripe->data_pages[tmp])
+				tmp++;
+			if (tmp < end) {
+				split = bio_split(bio,
+					(tmp - start) << PAGE_SECTOR_SHIFT,
+					GFP_NOIO, NULL);
+				bio_chain(split, bio);
+			} else /* all in cache */
+				split = bio;
+
+			r5c_copy_bio(split, &stripe->data_pages[start], true);
+
+			bio_endio(split, 0);
+
+			start = tmp;
+		} else {
+			tmp = start;
+			while (tmp < end && !stripe->data_pages[tmp])
+				tmp++;
+			if (tmp < end) {
+				split = bio_split(bio,
+					(tmp - start) << PAGE_SECTOR_SHIFT,
+					GFP_NOIO, NULL);
+				bio_chain(split, bio);
+			} else
+				split = bio;
+
+			raid5_make_request(cache->mddev, split);
+
+			start = tmp;
+		}
+	}
+	r5c_put_stripe(stripe);
+}
+
+static void r5c_flush_pending_io(struct r5c_cache *cache)
+{
+	r5l_run_tasks(&cache->log, false);
+}
+
+static void r5c_unplug_work(struct work_struct *work)
+{
+	struct r5c_cache *cache = container_of(work, struct r5c_cache,
+		pending_io_work);
+	r5c_flush_pending_io(cache);
+}
+
+static void r5c_unplug(struct blk_plug_cb *cb, bool from_schedule)
+{
+	struct r5c_cache *cache = cb->data;
+
+	kfree(cb);
+	if (from_schedule) {
+		schedule_work(&cache->pending_io_work);
+		return;
+	}
+	r5c_flush_pending_io(cache);
+}
+
+/* we aggregate bio to plug list, so each r5l_meta_block can track more data */
+void r5c_handle_bio(struct r5c_cache *cache, struct bio *bio)
+{
+	struct blk_plug_cb *cb;
+
+	if (bio_data_dir(bio) == READ) {
+		r5c_read_bio(cache, bio);
+		return;
+	}
+
+	md_write_start(cache->mddev, bio);
+
+	r5c_write_bio(cache, bio);
+
+	cb = blk_check_plugged(r5c_unplug, cache, sizeof(*cb));
+	if (!cb)
+		r5c_flush_pending_io(cache);
+}
+
 static void r5c_wake_reclaimer(struct r5c_cache *cache, int reason)
 {
 }
@@ -1315,3 +2000,188 @@ static void r5l_exit_log(struct r5l_log *log)
 	kmem_cache_destroy(log->io_kc);
 	__free_page(log->super_page);
 }
+
+static void r5c_calculate_watermark(struct r5c_cache *cache)
+{
+	u64 pages = cache->max_pages >> 3;
+
+	if (pages < 1024)
+		cache->low_watermark = 1024;
+	else
+		cache->low_watermark = pages;
+	cache->high_watermark = cache->low_watermark << 1;
+}
+
+static int r5c_shrink_cache_memory(struct r5c_cache *cache, unsigned long size)
+{
+	LIST_HEAD(page_list);
+	struct page *page, *tmp;
+
+	if (cache->total_pages <= size)
+		return 0;
+
+	spin_lock_irq(&cache->pool_lock);
+	cache->max_pages = size;
+	r5c_calculate_watermark(cache);
+	spin_unlock_irq(&cache->pool_lock);
+
+	while (cache->total_pages > size) {
+		spin_lock_irq(&cache->pool_lock);
+		if (cache->total_pages - cache->free_pages >= size) {
+			list_splice_init(&cache->page_pool, &page_list);
+			cache->total_pages -= cache->free_pages;
+			cache->free_pages = 0;
+		} else {
+			while (cache->total_pages > size) {
+				list_move(cache->page_pool.next, &page_list);
+				cache->total_pages--;
+				cache->free_pages--;
+			}
+		}
+		spin_unlock_irq(&cache->pool_lock);
+
+		list_for_each_entry_safe(page, tmp, &page_list, lru) {
+			list_del_init(&page->lru);
+			__free_page(page);
+		}
+
+		r5c_wake_wait_reclaimer(cache, RECLAIM_MEM_BACKGROUND);
+	}
+	return 0;
+}
+
+static void r5c_free_cache_data(struct r5c_cache *cache)
+{
+	struct r5c_stripe *stripe;
+	struct page *page, *tmp;
+	struct radix_tree_iter iter;
+	void **slot;
+
+	radix_tree_for_each_slot(slot, &cache->stripe_tree, &iter, 0) {
+		stripe = radix_tree_deref_slot(slot);
+		r5c_put_stripe(stripe);
+	}
+
+	BUG_ON(!list_empty(&cache->log_list));
+
+	list_for_each_entry_safe(page, tmp, &cache->page_pool, lru) {
+		list_del_init(&page->lru);
+		__free_page(page);
+	}
+}
+
+struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev)
+{
+	struct mddev *mddev = rdev->mddev;
+	struct r5c_cache *cache;
+	int i;
+
+	cache = kzalloc(sizeof(*cache), GFP_KERNEL);
+	if (!cache)
+		return NULL;
+	cache->mddev = mddev;
+	cache->rdev = rdev;
+
+	spin_lock_init(&cache->tree_lock);
+	INIT_RADIX_TREE(&cache->stripe_tree, GFP_ATOMIC);
+	INIT_LIST_HEAD(&cache->log_list);
+	INIT_LIST_HEAD(&cache->full_stripes);
+
+	INIT_LIST_HEAD(&cache->page_pool);
+	spin_lock_init(&cache->pool_lock);
+	INIT_WORK(&cache->pending_io_work, r5c_unplug_work);
+
+	cache->max_pages = MAX_MEM >> PAGE_SHIFT;
+
+	cache->stripe_data_size = conf->chunk_sectors * (conf->raid_disks -
+		conf->max_degraded);
+	cache->chunk_size = conf->chunk_sectors;
+	cache->stripe_size = conf->chunk_sectors * conf->raid_disks;
+	cache->parity_disks = conf->max_degraded;
+	cache->stripe_data_pages = cache->stripe_data_size >> PAGE_SECTOR_SHIFT;
+	cache->stripe_parity_pages = (cache->stripe_size -
+		cache->stripe_data_size) >> PAGE_SECTOR_SHIFT;
+
+	cache->stripe_flush_data = kmalloc(PAGE_SIZE, GFP_KERNEL);
+	if (!cache->stripe_flush_data)
+		goto io_range_kc;
+
+	cache->io_range_kc = KMEM_CACHE(r5c_io_range, 0);
+	if (!cache->io_range_kc)
+		goto io_range_kc;
+	cache->stripe_kc = kmem_cache_create("r5c_stripe",
+		sizeof(struct r5c_stripe) + sizeof(struct page *) *
+		cache->stripe_data_pages, __alignof__(struct r5c_stripe),
+		0, NULL);
+	if (!cache->stripe_kc)
+		goto stripe_kc;
+
+	cache->bio_set = bioset_create(RECLAIM_BATCH, 0);
+	if (!cache->bio_set)
+		goto bio_set;
+
+	cache->reclaim_batch = RECLAIM_BATCH;
+	i = (conf->chunk_sectors >> PAGE_SECTOR_SHIFT) * cache->reclaim_batch;
+	if (i > conf->max_nr_stripes)
+		raid5_set_cache_size(mddev, i);
+
+	/* make sure we can add checkpoint */
+	cache->reserved_space = (cache->stripe_parity_pages <<
+		PAGE_SECTOR_SHIFT) * RECLAIM_BATCH;
+
+	init_waitqueue_head(&cache->reclaim_wait);
+	init_waitqueue_head(&cache->error_wait);
+	bio_list_init(&cache->retry_bio_list);
+
+	for (i = 0; i < (1 << STRIPE_LOCK_BITS); i++) {
+		spin_lock_init(&cache->stripe_locks[i]);
+		init_waitqueue_head(&cache->stripe_waitq[i]);
+	}
+
+	if (r5l_init_log(cache))
+		goto init_log;
+
+	while (cache->total_pages < cache->max_pages) {
+		struct page *page = alloc_page(GFP_KERNEL | __GFP_HIGHMEM);
+
+		if (!page)
+			goto err_page;
+		list_add(&page->lru, &cache->page_pool);
+		cache->free_pages++;
+		cache->total_pages++;
+	}
+
+	r5c_calculate_watermark(cache);
+
+	r5c_shrink_cache_memory(cache, cache->max_pages);
+
+	return cache;
+err_page:
+	r5c_free_cache_data(cache);
+
+	r5l_exit_log(&cache->log);
+init_log:
+	bioset_free(cache->bio_set);
+bio_set:
+	kmem_cache_destroy(cache->stripe_kc);
+stripe_kc:
+	kmem_cache_destroy(cache->io_range_kc);
+io_range_kc:
+	kfree(cache->stripe_flush_data);
+	kfree(cache);
+	return NULL;
+}
+
+void r5c_exit_cache(struct r5c_cache *cache)
+{
+	r5l_exit_log(&cache->log);
+
+	r5c_free_cache_data(cache);
+
+	bioset_free(cache->bio_set);
+	kmem_cache_destroy(cache->stripe_kc);
+	kmem_cache_destroy(cache->io_range_kc);
+
+	kfree(cache->stripe_flush_data);
+	kfree(cache);
+}
diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index 8160a63..ca164ad 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -4661,17 +4661,24 @@ static int raid5_mergeable_bvec(struct mddev *mddev,
 	int max;
 	unsigned int chunk_sectors = mddev->chunk_sectors;
 	unsigned int bio_sectors = bvm->bi_size >> 9;
+	struct r5conf *conf = mddev->private;
 
 	/*
 	 * always allow writes to be mergeable, read as well if array
 	 * is degraded as we'll go through stripe cache anyway.
+	 * with cache, write must align within stripe
 	 */
-	if ((bvm->bi_rw & 1) == WRITE || mddev->degraded)
+	if (((bvm->bi_rw & 1) == WRITE && !conf->cache) || mddev->degraded)
 		return biovec->bv_len;
 
 	if (mddev->new_chunk_sectors < mddev->chunk_sectors)
 		chunk_sectors = mddev->new_chunk_sectors;
 	max =  (chunk_sectors - ((sector & (chunk_sectors - 1)) + bio_sectors)) << 9;
+	if (((bvm->bi_rw & 1) == WRITE) && conf->cache) {
+		chunk_sectors *= conf->raid_disks - conf->max_degraded;
+		max = (chunk_sectors - (sector_div(sector, chunk_sectors) +
+			bio_sectors)) << 9;
+	}
 	if (max < 0) max = 0;
 	if (max <= biovec->bv_len && bio_sectors == 0)
 		return biovec->bv_len;
@@ -5125,7 +5132,7 @@ static void make_discard_request(struct mddev *mddev, struct bio *bi)
 	}
 }
 
-static void make_request(struct mddev *mddev, struct bio * bi)
+void raid5_make_request(struct mddev *mddev, struct bio * bi)
 {
 	struct r5conf *conf = mddev->private;
 	int dd_idx;
@@ -5302,6 +5309,16 @@ static void make_request(struct mddev *mddev, struct bio * bi)
 	}
 }
 
+static void make_request(struct mddev *mddev, struct bio *bi)
+{
+	struct r5conf *conf = mddev->private;
+
+	if (conf->cache)
+		r5c_handle_bio(conf->cache, bi);
+	else
+		raid5_make_request(mddev, bi);
+}
+
 static sector_t raid5_size(struct mddev *mddev, sector_t sectors, int raid_disks);
 
 static sector_t reshape_request(struct mddev *mddev, sector_t sector_nr, int *skipped)
diff --git a/drivers/md/raid5.h b/drivers/md/raid5.h
index 896d603..14761c9 100644
--- a/drivers/md/raid5.h
+++ b/drivers/md/raid5.h
@@ -538,6 +538,7 @@ struct r5conf {
 	struct r5worker_group	*worker_groups;
 	int			group_cnt;
 	int			worker_cnt_per_group;
+	struct r5c_cache	*cache;
 };
 
 
@@ -604,4 +605,9 @@ static inline int algorithm_is_DDF(int layout)
 
 extern void md_raid5_kick_device(struct r5conf *conf);
 extern int raid5_set_cache_size(struct mddev *mddev, int size);
+
+void raid5_make_request(struct mddev *mddev, struct bio *bi);
+void r5c_handle_bio(struct r5c_cache *cache, struct bio *bi);
+struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev);
+void r5c_exit_cache(struct r5c_cache *cache);
 #endif
-- 
1.8.1


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

* [PATCH V4 05/13] raid5: cache reclaim support
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (3 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 04/13] raid5: cache part of raid5 cache Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 06/13] raid5: cache IO error handling Shaohua Li
                   ` (9 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

We have two limited resources, memory pool and cache disk space. If
resource is tight, we will do reclaim. In either case, we will flush
some data from cache disk to raid disks. However, we implement different
strategies. For memory reclaim, we prefer reclaiming full stripe. For
cache disk space reclaim, we prefer reclaiming io_range entry at the
head of the global io_range list. We always do reclaim in stripe unit.
Since data which isn't flushed to raid disks is in memory, we don't need
read stripe's data back from cache disk.

Reclaim process will be like:
1. select stripe to reclaim
2. start sending stripe IO to raid disks. raid5 will fetch required data
and calculate parity, and ops_run_io will try to write data/parity.
3. cache hijack ops_run_io, write parity data to cache log. ops_run_io
doesn't write data/parity to raid disks at this time
4. flush cache disk and write flush_start block
5. ops_run_io continues. data/parity will be written to raid disks
6. flush all raid disks cache
7. write flush_end block
8. delete in-memory index of the stripe, and advance superblock log checkpoint

In the process, parity is always appended to log after its corresponding
data. After parity is in cache disk, a flush_start block is appended to
log, which indicates stripe is flushing to raid disks. Data writing to
raid disks only happens after all data and parity are already in cache
disk. This will fix the write whole issue. After a stripe is flushed to
raid disks, a flush_end block is appended to log, which indicates a
stripe is settled down in raid disks.

flush_start isn't strictly required in above process. We calcualte
checksum for both data/parity. The checksum will guarantee data/parity
isn't corrupted, and if all parity of a stripe is in cache disk,
recovery can think the stripe is already running step 5, even it
doesn't. But this doesn't matter, it's not an error recovery writes the
data/parity to raid disks. So the flush_start (step 4) can be replaced
with a 'flush cache disk'. But the flush shouldn't be a simple flush
cache disk, we must maintain log IO ordering like other flush request,
see the patch of 'log part' for the ordering requirement. The
flush_start, however, makes recovery easily determines which stripe
starts step 5. I'm open to remove the flush_start block if necessary.

Reclaim could create holes in the log, eg, some io_range in the middle
is reclaimed, but io_range at the head remains. But this doesn't matter,
the first io_range is always the log tail. Superblock has a field
pointing to the position of log tail. The hole can waste a lot of disk
space though. In the future, we can implement a garbage collection to
mitigate the issue, eg, copy data from the index tail to head.

The performance of the raid will largely be determined by reclaim speed
at run time. Several stages of the reclaim process involves IO wait or
disk cache flush, which significantly impact the raid disk utilization
and performance. The flush_start and flush_end block make running
multiple reclaim possible. Each reclaim only records stripes in the
flush start/end block which it is reclaiming. Recovery can use the
information to correctly determine stripe's flush stage. An example of 2
reclaimer:

xxx1 belongs to stripe1 and the same for stripe2

reclaim 1:  |data1|...|parity1|...|flush_start1|...|flush_end1|
reclaim 2:      |data2|..|parity2|...............|flush_start2|...|flush_end2|

the reclaims only record its own stripes in flush block. If, for
exmaple, recovery finds flush_start1, it knows stripe1 is flushing to
raid disk. Recovery will ignore stripe2, since stripe2 isn't in
flush_start1.

Multiple reclaim will efficiently solve the performance issue, since
pipeline of different reclaim can fill the gap caused by wait. Note,
this doesn't need to be multiple reclaim. The same mechanism should work
for a single thread with multiple reclaim stream (here reclaim1 and
reclaim 2 can thought as a stream), which for example could use a state
machine.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c | 486 +++++++++++++++++++++++++++++++++++++++++++++++
 drivers/md/raid5.c       |  25 ++-
 drivers/md/raid5.h       |   7 +
 3 files changed, 510 insertions(+), 8 deletions(-)

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index d0950ae..84fcb4d 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -1526,6 +1526,26 @@ static void r5c_put_stripe(struct r5c_stripe *stripe)
 	kmem_cache_free(cache->stripe_kc, stripe);
 }
 
+/* must hold cache->tree_lock. Freeze a stripe so it can be reclaimed */
+static bool r5c_freeze_stripe(struct r5c_cache *cache,
+	struct r5c_stripe *stripe, bool blocking)
+{
+	if (atomic_read(&stripe->ref) == 1) {
+		stripe->state = STRIPE_FROZEN;
+		return true;
+	}
+	if (!blocking)
+		return false;
+	/* Make sure no IO running in stripe */
+	wait_event_lock_irq(*r5c_stripe_waitq(cache, stripe),
+			atomic_read(&stripe->ref) == 1 ||
+			stripe->state >= STRIPE_FROZEN, cache->tree_lock);
+	if (stripe->state >= STRIPE_FROZEN)
+		return false;
+	stripe->state = STRIPE_FROZEN;
+	return true;
+}
+
 static void r5c_bio_task_end(struct r5l_task *task, int error)
 {
 	struct r5c_io_range *range = task->private;
@@ -1781,6 +1801,17 @@ static void r5c_read_bio(struct r5c_cache *cache, struct bio *bio)
 	r5c_put_stripe(stripe);
 }
 
+/*
+ * we write parity to log in a batch way, so each r5l_meta_block can track more
+ * parity
+ * */
+void r5c_flush_pending_parity(struct r5c_cache *cache)
+{
+	if (!cache)
+		return;
+	r5l_run_tasks(&cache->log, true);
+}
+
 static void r5c_flush_pending_io(struct r5c_cache *cache)
 {
 	r5l_run_tasks(&cache->log, false);
@@ -1824,12 +1855,460 @@ void r5c_handle_bio(struct r5c_cache *cache, struct bio *bio)
 		r5c_flush_pending_io(cache);
 }
 
+void r5c_write_start(struct mddev *mddev, struct bio *bi)
+{
+	struct r5conf *conf = mddev->private;
+	if (!conf->cache)
+		md_write_start(mddev, bi);
+
+	/*
+	 * With cache, r5c_handle_bio already does md_write_start. This is
+	 * called in reclaim, since quiesce will stop reclaim, we do nothing
+	 * here
+	 * */
+}
+
+void r5c_write_end(struct mddev *mddev, struct bio *bi)
+{
+	struct r5conf *conf = mddev->private;
+	if (!conf->cache) {
+		md_write_end(mddev);
+		return;
+	}
+
+	/*
+	 * r5c_flush_one doesn't set bi_bdev for bio, which doesn't require
+	 * md_write_end. While bio from retry_bio_list (for io error handling)
+	 * has bi_bdev, which require md_write_end
+	 **/
+	if (bi->bi_bdev)
+		md_write_end(mddev);
+
+	/*
+	 * For normal bio, r5c_bio_task_end already does md_write_end. This
+	 * routine is called in reclaim, which doesn't need md_write_end
+	 **/
+}
+
+static void r5c_flush_endio(struct bio *bio, int err)
+{
+	struct r5c_stripe *stripe = bio->bi_private;
+	struct r5c_cache *cache = stripe->cache;
+
+	if (atomic_dec_and_test(&stripe->pending_bios)) {
+		stripe->state = STRIPE_INRAID;
+		wake_up(r5c_stripe_waitq(cache, stripe));
+	}
+
+	bio_put(bio);
+}
+
+static void r5c_flush_one(struct r5c_cache *cache, struct r5c_stripe *stripe,
+	int start, int pages)
+{
+	sector_t base;
+	struct bio *bio;
+	int max_vecs;
+	int i, j;
+
+	i = 0;
+	base = r5c_stripe_raid_sector(cache, stripe) +
+					(start << PAGE_SECTOR_SHIFT);
+	while (i < pages) {
+		/* Below is ugly, but We don't have a block_device */
+		max_vecs = min_t(int, pages - i, BIO_MAX_PAGES);
+		bio = bio_alloc_bioset(GFP_NOIO, max_vecs, cache->bio_set);
+		bio->bi_iter.bi_sector = base + (i << PAGE_SECTOR_SHIFT);
+		bio->bi_bdev = NULL;
+		bio->bi_private = stripe;
+		for (j = 0; j < max_vecs; j++) {
+			struct bio_vec *bvec;
+			bvec = &bio->bi_io_vec[bio->bi_vcnt];
+			bvec->bv_page = stripe->data_pages[i + start];
+			bvec->bv_len = PAGE_SIZE;
+			bvec->bv_offset = 0;
+			bio->bi_vcnt++;
+			bio->bi_phys_segments++;
+			bio->bi_iter.bi_size += PAGE_SIZE;
+			i++;
+		}
+		bio->bi_end_io = r5c_flush_endio;
+		bio->bi_rw = WRITE;
+		atomic_inc(&stripe->pending_bios);
+		raid5_make_request(cache->mddev, bio);
+	}
+}
+
+static void r5c_put_stripe_dirty(struct r5c_cache *cache,
+					struct r5c_stripe *stripe)
+{
+	if (!atomic_dec_return(&stripe->dirty_stripes)) {
+		stripe->state = STRIPE_PARITY_DONE;
+		wake_up(r5c_stripe_waitq(cache, stripe));
+	}
+}
+
+static void r5c_flush_stripe(struct r5c_cache *cache, struct r5c_stripe *stripe)
+{
+	unsigned long *stripe_bits;
+	int chunk_stripes;
+	int start;
+	int end = 0;
+	int i;
+
+	chunk_stripes = cache->chunk_size >> PAGE_SECTOR_SHIFT;
+	stripe_bits = kzalloc(BITS_TO_LONGS(chunk_stripes) * sizeof(long),
+		GFP_NOIO);
+	for (i = 0; i < cache->stripe_data_pages; i++) {
+		if (stripe->data_pages[i])
+			__set_bit(i % chunk_stripes, stripe_bits);
+	}
+	atomic_set(&stripe->dirty_stripes, bitmap_weight(stripe_bits,
+				chunk_stripes) + 1);
+	kfree(stripe_bits);
+
+	while (end < cache->stripe_data_pages) {
+		while (end < cache->stripe_data_pages &&
+		       !stripe->data_pages[end])
+			end++;
+		if (end >= cache->stripe_data_pages)
+			break;
+		start = end;
+		while (end < cache->stripe_data_pages &&
+		       stripe->data_pages[end])
+			end++;
+		r5c_flush_one(cache, stripe, start, end - start);
+	}
+	r5c_put_stripe_dirty(cache, stripe);
+}
+
+/* Since r5c_freeze_stripe can sleep, this isn't multithread safe yet */
+static int r5c_select_front_stripes(struct r5c_cache *cache,
+	struct list_head *list, unsigned int count, bool blocking)
+{
+	struct r5c_stripe *stripe;
+	struct r5c_io_range *range;
+	int stripes = 0;
+
+	list_for_each_entry(range, &cache->log_list, log_sibling) {
+		stripe = range->stripe;
+		if (stripe->state >= STRIPE_FROZEN)
+			continue;
+
+		if (!r5c_freeze_stripe(cache, stripe, blocking))
+			continue;
+
+		list_move_tail(&stripe->lru, list);
+		stripes++;
+		if (stripes >= count)
+			break;
+	}
+	return stripes;
+}
+
+static int r5c_select_full_stripes(struct r5c_cache *cache,
+	struct list_head *list, unsigned int count, bool blocking)
+{
+	struct r5c_stripe *stripe, *tmp;
+	int stripes = 0;
+
+	list_for_each_entry_safe(stripe, tmp, &cache->full_stripes, lru) {
+		if (stripe->state >= STRIPE_FROZEN)
+			continue;
+		if (!r5c_freeze_stripe(cache, stripe, blocking))
+			continue;
+
+		list_move_tail(&stripe->lru, list);
+		stripes++;
+		cache->full_stripe_cnt--;
+		if (stripes >= count)
+			break;
+	}
+	if (list_empty(&cache->full_stripes))
+		clear_bit(RECLAIM_MEM_FULL, &cache->reclaim_reason);
+
+	return stripes;
+}
+
+static void r5c_select_stripes(struct r5c_cache *cache, struct list_head *list)
+{
+	int stripes;
+	bool blocking;
+
+	/*
+	 * generally select full stripe, if no disk space, select first stripe
+	 */
+	spin_lock_irq(&cache->tree_lock);
+	/* Don't need stripe lock, as nobody is operating on the stripe */
+	if (test_bit(RECLAIM_FLUSH_ALL, &cache->reclaim_reason)) {
+		r5c_select_front_stripes(cache, list, cache->reclaim_batch,
+				true);
+	} else if (test_bit(RECLAIM_DISK, &cache->reclaim_reason) ||
+	    test_bit(RECLAIM_DISK_BACKGROUND, &cache->reclaim_reason)) {
+		blocking = test_bit(RECLAIM_DISK, &cache->reclaim_reason);
+		r5c_select_front_stripes(cache, list, cache->reclaim_batch,
+				blocking);
+	} else if (test_bit(RECLAIM_MEM, &cache->reclaim_reason) ||
+	           test_bit(RECLAIM_MEM_BACKGROUND, &cache->reclaim_reason)) {
+		stripes = r5c_select_full_stripes(cache, list,
+				cache->reclaim_batch, false);
+		if (stripes < cache->reclaim_batch)
+			stripes += r5c_select_front_stripes(cache, list,
+				cache->reclaim_batch - stripes, false);
+		if (stripes < cache->reclaim_batch)
+			stripes += r5c_select_full_stripes(cache, list,
+				cache->reclaim_batch - stripes, true);
+		if (stripes < cache->reclaim_batch)
+			stripes += r5c_select_front_stripes(cache, list,
+				cache->reclaim_batch - stripes, true);
+	} else if (test_bit(RECLAIM_MEM_FULL, &cache->reclaim_reason)) {
+		r5c_select_full_stripes(cache, list, cache->reclaim_batch,
+				false);
+	}
+
+	spin_unlock_irq(&cache->tree_lock);
+}
+
+static void r5c_disks_flush_end(struct bio *bio, int err)
+{
+	struct completion *io_complete = bio->bi_private;
+
+	complete(io_complete);
+	bio_put(bio);
+}
+
+static void r5c_flush_all_disks(struct r5c_cache *cache)
+{
+	struct mddev *mddev = cache->mddev;
+	struct bio *bi;
+	DECLARE_COMPLETION_ONSTACK(io_complete);
+
+	bi = bio_alloc_mddev(GFP_NOIO, 0, mddev);
+	bi->bi_end_io = r5c_disks_flush_end;
+	bi->bi_private = &io_complete;
+
+	/* If bio hasn't payload, this function will just flush all disks */
+	md_flush_request(mddev, bi);
+
+	wait_for_completion_io(&io_complete);
+}
+
+static int r5c_stripe_list_cmp(void *priv, struct list_head *a,
+	struct list_head *b)
+{
+	struct r5c_stripe *stripe_a = container_of(a, struct r5c_stripe,
+		lru);
+	struct r5c_stripe *stripe_b = container_of(b, struct r5c_stripe,
+		lru);
+
+	return !(stripe_a->raid_index < stripe_b->raid_index);
+}
+
+static void r5c_reclaim_stripe_list(struct r5c_cache *cache,
+	struct list_head *stripe_list)
+{
+	struct r5c_stripe *stripe;
+	struct r5c_io_range *range;
+	u64 seq;
+	sector_t meta;
+	struct blk_plug plug;
+	size_t size = 0;
+
+	if (list_empty(stripe_list))
+		return;
+	list_sort(NULL, stripe_list, r5c_stripe_list_cmp);
+	list_for_each_entry(stripe, stripe_list, lru) {
+		cache->stripe_flush_data[size] =
+				cpu_to_le64(stripe->raid_index);
+		size++;
+	}
+	size *= sizeof(__le64);
+
+	blk_start_plug(&plug);
+	/* step 1: start write to raid */
+	list_for_each_entry(stripe, stripe_list, lru)
+		r5c_flush_stripe(cache, stripe);
+	blk_finish_plug(&plug);
+
+	/* step 2: wait parity write to cache */
+	list_for_each_entry_reverse(stripe, stripe_list, lru)
+		r5c_stripe_wait_state(cache, stripe, STRIPE_PARITY_DONE);
+
+	/* step 3: make sure data and parity settle down */
+	r5l_flush_block(&cache->log, R5LOG_TYPE_FLUSH_START,
+		cache->stripe_flush_data, size, &seq, &meta);
+
+	/* step 4: continue write to raid */
+	list_for_each_entry(stripe, stripe_list, lru) {
+		while (!list_empty(&stripe->stripes)) {
+			struct stripe_head *sh;
+
+			sh = list_first_entry(&stripe->stripes,
+				struct stripe_head, stripe_list);
+			list_del(&sh->stripe_list);
+			set_bit(STRIPE_HANDLE, &sh->state);
+			release_stripe(sh);
+		}
+	}
+
+	/* step 5: wait to make sure stripe data is in raid */
+	list_for_each_entry_reverse(stripe, stripe_list, lru)
+		r5c_stripe_wait_state(cache, stripe, STRIPE_INRAID);
+
+	/* step 6: flush raid disks */
+	r5c_flush_all_disks(cache);
+
+	/* step 7: mark data is flushed to raid */
+	r5l_flush_block(&cache->log, R5LOG_TYPE_FLUSH_END,
+		cache->stripe_flush_data, size, &seq, &meta);
+
+	/* step 8: mark stripe as dead */
+	while (!list_empty(stripe_list)) {
+		stripe = list_first_entry(stripe_list, struct r5c_stripe,
+			lru);
+		list_del_init(&stripe->lru);
+
+		stripe->state = STRIPE_DEAD;
+		wake_up(r5c_stripe_waitq(cache, stripe));
+
+		r5c_put_stripe(stripe);
+	}
+
+	/* step 9: advance superblock checkpoint */
+	spin_lock_irq(&cache->tree_lock);
+	/* if no data, superblock records the next position of checkpoint */
+	if (!list_empty(&cache->log_list)) {
+		range = list_first_entry(&cache->log_list,
+			struct r5c_io_range, log_sibling);
+		/* can't cross checkpoint */
+		if (range->seq < seq) {
+			seq = range->seq;
+			meta = range->meta_start;
+		}
+	}
+	spin_unlock_irq(&cache->tree_lock);
+
+	r5l_write_super(&cache->log, seq, meta);
+}
+
+static void r5c_reclaim_thread(struct md_thread *thread)
+{
+	struct mddev *mddev = thread->mddev;
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+	LIST_HEAD(stripe_list);
+	bool retry;
+
+do_retry:
+	retry = false;
+
+	while (cache->reclaim_reason != 0) {
+		/*
+		 * select stripe will freeze stripe, which will guarantee no
+		 * new task pending in error mode
+		 * */
+		r5c_select_stripes(cache, &stripe_list);
+
+		if (list_empty(&stripe_list)) {
+			clear_bit(RECLAIM_MEM_FULL, &cache->reclaim_reason);
+			clear_bit(RECLAIM_MEM_BACKGROUND,
+					&cache->reclaim_reason);
+			clear_bit(RECLAIM_DISK_BACKGROUND,
+					&cache->reclaim_reason);
+			clear_bit(RECLAIM_FLUSH_ALL, &cache->reclaim_reason);
+		} else
+			r5c_reclaim_stripe_list(cache, &stripe_list);
+
+		wake_up(&cache->reclaim_wait);
+	}
+}
+
 static void r5c_wake_reclaimer(struct r5c_cache *cache, int reason)
 {
+	set_bit(reason, &cache->reclaim_reason);
+	md_wakeup_thread(cache->reclaim_thread);
 }
 
 static void r5c_wake_wait_reclaimer(struct r5c_cache *cache, int reason)
 {
+	r5c_wake_reclaimer(cache, reason);
+	wait_event(cache->reclaim_wait, !test_bit(reason,
+		&cache->reclaim_reason));
+}
+
+static void r5c_parity_task_end(struct r5l_task *task, int error)
+{
+	struct stripe_head *head_sh = task->private;
+	struct r5conf *conf = head_sh->raid_conf;
+	struct r5c_cache *cache = conf->cache;
+	struct r5c_stripe *stripe = head_sh->stripe;
+
+	/* parity IO error doesn't matter */
+	r5c_put_stripe_dirty(cache, stripe);
+	kfree(task);
+}
+
+/*
+ * we don't record parity range in cache->log_list, because after a success
+ * reclaim, parity always is discarded
+ */
+static void r5c_write_one_stripe_parity(struct r5c_cache *cache,
+	struct r5c_stripe *stripe, struct stripe_head *head_sh,
+	struct stripe_head *sh)
+{
+	u64 stripe_index;
+	int stripe_offset;
+
+	stripe_index = sh->sector;
+	stripe_offset = sector_div(stripe_index, cache->chunk_size);
+	stripe_offset >>= PAGE_SECTOR_SHIFT;
+
+	r5l_queue_parity(&cache->log, sh->sector,
+		sh->dev[sh->pd_idx].page,
+		sh->qd_idx >= 0 ? sh->dev[sh->qd_idx].page : NULL,
+		r5c_parity_task_end, head_sh);
+}
+
+int r5c_write_parity(struct r5c_cache *cache, struct stripe_head *head_sh)
+{
+	struct r5c_stripe *stripe;
+	struct stripe_head *sh;
+	u64 stripe_index;
+	int parity_cnt;
+	unsigned long flags;
+
+	/* parity is already written */
+	if (head_sh->stripe) {
+		head_sh->stripe = NULL;
+		return -EAGAIN;
+	}
+	if (!test_bit(R5_Wantwrite, &head_sh->dev[head_sh->pd_idx].flags) ||
+	     test_bit(STRIPE_SYNCING, &head_sh->state))
+		return -EAGAIN;
+
+	stripe_index = head_sh->sector;
+	sector_div(stripe_index, cache->chunk_size);
+
+	stripe = r5c_search_stripe(cache, stripe_index);
+	r5c_lock_stripe(cache, stripe, &flags);
+	list_add_tail(&head_sh->stripe_list, &stripe->stripes);
+	r5c_unlock_stripe(cache, stripe, &flags);
+	head_sh->stripe = stripe;
+
+	atomic_inc(&head_sh->count);
+
+	parity_cnt = !!(head_sh->pd_idx >= 0) + !!(head_sh->qd_idx >= 0);
+	BUG_ON(parity_cnt != cache->parity_disks);
+
+	sh = head_sh;
+	do {
+		r5c_write_one_stripe_parity(cache, stripe, head_sh, sh);
+		if (!head_sh->batch_head)
+			return 0;
+		sh = list_first_entry(&sh->batch_list, struct stripe_head,
+				      batch_list);
+	} while (sh != head_sh);
+	return 0;
 }
 
 static int r5l_load_log(struct r5l_log *log)
@@ -2153,6 +2632,12 @@ struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev)
 
 	r5c_calculate_watermark(cache);
 
+	cache->reclaim_thread = md_register_thread(r5c_reclaim_thread,
+		mddev, "reclaim");
+	if (!cache->reclaim_thread)
+		goto err_page;
+	cache->reclaim_thread->timeout = CHECKPOINT_TIMEOUT;
+
 	r5c_shrink_cache_memory(cache, cache->max_pages);
 
 	return cache;
@@ -2174,6 +2659,7 @@ struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev)
 
 void r5c_exit_cache(struct r5c_cache *cache)
 {
+	md_unregister_thread(&cache->reclaim_thread);
 	r5l_exit_log(&cache->log);
 
 	r5c_free_cache_data(cache);
diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index ca164ad..a9e10ff 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -409,7 +409,7 @@ static int release_stripe_list(struct r5conf *conf,
 	return count;
 }
 
-static void release_stripe(struct stripe_head *sh)
+void release_stripe(struct stripe_head *sh)
 {
 	struct r5conf *conf = sh->raid_conf;
 	unsigned long flags;
@@ -887,6 +887,11 @@ static void ops_run_io(struct stripe_head *sh, struct stripe_head_state *s)
 
 	might_sleep();
 
+	if (conf->cache) {
+		if (!r5c_write_parity(conf->cache, sh))
+			return;
+	}
+
 	for (i = disks; i--; ) {
 		int rw;
 		int replace_only = 0;
@@ -3097,7 +3102,7 @@ handle_failed_stripe(struct r5conf *conf, struct stripe_head *sh,
 			struct bio *nextbi = r5_next_bio(bi, sh->dev[i].sector);
 			clear_bit(BIO_UPTODATE, &bi->bi_flags);
 			if (!raid5_dec_bi_active_stripes(bi)) {
-				md_write_end(conf->mddev);
+				r5c_write_end(conf->mddev, bi);
 				bi->bi_next = *return_bi;
 				*return_bi = bi;
 			}
@@ -3121,7 +3126,7 @@ handle_failed_stripe(struct r5conf *conf, struct stripe_head *sh,
 			struct bio *bi2 = r5_next_bio(bi, sh->dev[i].sector);
 			clear_bit(BIO_UPTODATE, &bi->bi_flags);
 			if (!raid5_dec_bi_active_stripes(bi)) {
-				md_write_end(conf->mddev);
+				r5c_write_end(conf->mddev, bi);
 				bi->bi_next = *return_bi;
 				*return_bi = bi;
 			}
@@ -3460,7 +3465,7 @@ static void handle_stripe_clean_event(struct r5conf *conf,
 					dev->sector + STRIPE_SECTORS) {
 					wbi2 = r5_next_bio(wbi, dev->sector);
 					if (!raid5_dec_bi_active_stripes(wbi)) {
-						md_write_end(conf->mddev);
+						r5c_write_end(conf->mddev, wbi);
 						wbi->bi_next = *return_bi;
 						*return_bi = wbi;
 					}
@@ -5127,7 +5132,7 @@ static void make_discard_request(struct mddev *mddev, struct bio *bi)
 
 	remaining = raid5_dec_bi_active_stripes(bi);
 	if (remaining == 0) {
-		md_write_end(mddev);
+		r5c_write_end(mddev, bi);
 		bio_endio(bi, 0);
 	}
 }
@@ -5149,7 +5154,7 @@ void raid5_make_request(struct mddev *mddev, struct bio * bi)
 		return;
 	}
 
-	md_write_start(mddev, bi);
+	r5c_write_start(mddev, bi);
 
 	/*
 	 * If array is degraded, better not do chunk aligned read because
@@ -5285,8 +5290,9 @@ void raid5_make_request(struct mddev *mddev, struct bio * bi)
 			}
 			set_bit(STRIPE_HANDLE, &sh->state);
 			clear_bit(STRIPE_DELAYED, &sh->state);
+			/* there's no point to delay stripe with cache */
 			if ((!sh->batch_head || sh == sh->batch_head) &&
-			    (bi->bi_rw & REQ_SYNC) &&
+			    ((bi->bi_rw & REQ_SYNC) || conf->cache) &&
 			    !test_and_set_bit(STRIPE_PREREAD_ACTIVE, &sh->state))
 				atomic_inc(&conf->preread_active_stripes);
 			release_stripe_plug(mddev, sh);
@@ -5302,7 +5308,7 @@ void raid5_make_request(struct mddev *mddev, struct bio * bi)
 	if (remaining == 0) {
 
 		if ( rw == WRITE )
-			md_write_end(mddev);
+			r5c_write_end(mddev, bi);
 
 		trace_block_bio_complete(mddev->queue, bi, 0);
 		bio_endio(bi, 0);
@@ -5703,6 +5709,7 @@ static int  retry_aligned_read(struct r5conf *conf, struct bio *raid_bio)
 		}
 
 		set_bit(R5_ReadNoMerge, &sh->dev[dd_idx].flags);
+		/* IO read should not trigger r5c IO */
 		handle_stripe(sh);
 		release_stripe(sh);
 		handled++;
@@ -5751,6 +5758,8 @@ static int handle_active_stripes(struct r5conf *conf, int group,
 	for (i = 0; i < batch_size; i++)
 		handle_stripe(batch[i]);
 
+	r5c_flush_pending_parity(conf->cache);
+
 	cond_resched();
 
 	spin_lock_irq(&conf->device_lock);
diff --git a/drivers/md/raid5.h b/drivers/md/raid5.h
index 14761c9..16bd54f 100644
--- a/drivers/md/raid5.h
+++ b/drivers/md/raid5.h
@@ -223,6 +223,8 @@ struct stripe_head {
 	struct stripe_head	*batch_head; /* protected by stripe lock */
 	spinlock_t		batch_lock; /* only header's lock is useful */
 	struct list_head	batch_list; /* protected by head's batch lock*/
+	struct r5c_stripe	*stripe;
+	struct list_head	stripe_list;
 	/**
 	 * struct stripe_operations
 	 * @target - STRIPE_OP_COMPUTE_BLK target
@@ -606,8 +608,13 @@ static inline int algorithm_is_DDF(int layout)
 extern void md_raid5_kick_device(struct r5conf *conf);
 extern int raid5_set_cache_size(struct mddev *mddev, int size);
 
+void release_stripe(struct stripe_head *sh);
+int r5c_write_parity(struct r5c_cache *cache, struct stripe_head *sh);
+void r5c_flush_pending_parity(struct r5c_cache *cache);
 void raid5_make_request(struct mddev *mddev, struct bio *bi);
 void r5c_handle_bio(struct r5c_cache *cache, struct bio *bi);
 struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev);
 void r5c_exit_cache(struct r5c_cache *cache);
+void r5c_write_start(struct mddev *mddev, struct bio *bi);
+void r5c_write_end(struct mddev *mddev, struct bio *bi);
 #endif
-- 
1.8.1


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

* [PATCH V4 06/13] raid5: cache IO error handling
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (4 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 05/13] raid5: cache reclaim support Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 07/13] raid5: cache device quiesce support Shaohua Li
                   ` (8 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

Cache disk might have IO error, which makes it a single failure point.
To solve this issue. we allow the raid array enters bypass mode in IO
error. In this mode, raid array will dispatch IO to raid disks directly,
so the array can still run. Existing data in log disk isn't lost.  Since
all data which isn't flushed to raid disks is in memory, we don't read
log disk to read back the data. The IO error handling will just flush
the data to raid disks. In this way there is no risk we lose data, and
raid array is still available.

Of course if raid array enters bypass mode, we don't have protection of
write hole and lose cache speed up. System admin can always choose to
shutdown the array for safety.

IO error likely isn't permanent. So after some time, we will retry
caching. But if there are too many IO errors, we don't do retry any
more.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c | 114 +++++++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 111 insertions(+), 3 deletions(-)

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index 84fcb4d..c5de6dd 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -34,6 +34,10 @@
 #define RECLAIM_BATCH 16
 #define CHECKPOINT_TIMEOUT (5 * HZ)
 
+#define MAX_RETRY 10
+#define IOERR_RETRY_TIME (5 * 60 * HZ)
+#define MEMERR_RETRY_TIME (60 * HZ)
+
 typedef u64 r5blk_t; /* log blocks, could be 512B - 4k */
 
 /*
@@ -353,6 +357,13 @@ enum {
 	RECLAIM_FLUSH_ALL = 16, /* flush all data to raid */
 };
 
+/* cache error state */
+enum {
+	ERROR_NOERROR = 0,
+	ERROR_PREPARE = 1, /* Had an error, flushing cache to raid */
+	ERROR_FINISH = 2, /* Had an error, cache has no data */
+};
+
 #define PAGE_SECTOR_SHIFT (PAGE_SHIFT - 9)
 
 static inline struct r5c_cache *r5l_cache(struct r5l_log *log)
@@ -1546,6 +1557,37 @@ static bool r5c_freeze_stripe(struct r5c_cache *cache,
 	return true;
 }
 
+/*
+ * If IO error happens in log, we flush all data to raid disks. This is safe
+ * since all data which isn't flushed to raid disk is in memory pool. If we are
+ * in the process to flush data to raid disks, we stall incoming IO, as we
+ * can't dispatch it to raid disks right now (to avoid data corruption, for
+ * example, new IO write to stripe which is reclaiming). New IO will be
+ * dispatched to raid disks after all data is flushed to raid disks
+ * */
+static bool r5c_check_wait_error_mode(struct r5c_cache *cache)
+{
+	if (cache->error_state)
+		wait_event(cache->error_wait,
+			cache->error_state == ERROR_FINISH);
+	if (cache->next_retry_time &&
+	    time_before(cache->next_retry_time, jiffies))
+		md_wakeup_thread(cache->reclaim_thread);
+	return !!cache->error_state;
+}
+
+static void r5c_enter_error_mode(struct r5c_cache *cache, int error)
+{
+	if (cmpxchg(&cache->error_state, ERROR_NOERROR,
+	    ERROR_PREPARE) == ERROR_NOERROR) {
+		printk(KERN_ERR"md: %s caching has %s, enter bypass mode\n",
+			mdname(cache->mddev),
+			error == -ENOMEM ? "no memory" : "IO error");
+		cache->error_type = error;
+		r5c_wake_reclaimer(cache, RECLAIM_FLUSH_ALL);
+	}
+}
+
 static void r5c_bio_task_end(struct r5l_task *task, int error)
 {
 	struct r5c_io_range *range = task->private;
@@ -1596,7 +1638,10 @@ static void r5c_bio_task_end(struct r5l_task *task, int error)
 				RECLAIM_MEM_FULL);
 	}
 	r5c_unlock_stripe(cache, stripe, &flags2);
-
+	if (error) {
+		bio_list_add(&cache->retry_bio_list, orig_bio);
+		r5c_enter_error_mode(cache, -EIO);
+	}
 	spin_unlock_irqrestore(&cache->tree_lock, flags);
 
 	if (!error) {
@@ -1663,7 +1708,12 @@ static void r5c_bio_flush_task_end(struct r5l_task *task, int error)
 
 	kfree(task);
 
-	if (!error) {
+	if (error) {
+		spin_lock_irqsave(&cache->tree_lock, flags);
+		bio_list_add(&cache->retry_bio_list, orig_bio);
+		r5c_enter_error_mode(cache, -EIO);
+		spin_unlock_irqrestore(&cache->tree_lock, flags);
+	} else {
 		bio_endio(orig_bio, 0);
 		md_write_end(cache->mddev);
 	}
@@ -1687,6 +1737,9 @@ static void r5c_write_bio(struct r5c_cache *cache, struct bio *bio)
 		return;
 	}
 
+	if (r5c_check_wait_error_mode(cache))
+		goto error_mode;
+
 	if (bio->bi_iter.bi_size == 0) {
 		BUG_ON(!(bio->bi_rw & REQ_FLUSH));
 		if (r5l_queue_empty_flush_bio(&cache->log, bio,
@@ -1740,6 +1793,9 @@ static void r5c_write_bio(struct r5c_cache *cache, struct bio *bio)
 	r5c_put_stripe(stripe);
 enter_error:
 	r5l_put_reserve(&cache->log, reserved_blocks);
+	r5c_enter_error_mode(cache, -ENOMEM);
+	r5c_check_wait_error_mode(cache);
+error_mode:
 	raid5_make_request(cache->mddev, bio);
 }
 
@@ -2221,6 +2277,53 @@ static void r5c_reclaim_thread(struct md_thread *thread)
 
 		wake_up(&cache->reclaim_wait);
 	}
+
+	if (cache->error_state == ERROR_PREPARE) {
+		struct bio *bio;
+		spin_lock_irq(&cache->tree_lock);
+		if (!list_empty(&cache->log_list))
+			retry = true;
+		spin_unlock_irq(&cache->tree_lock);
+		if (retry) {
+			clear_bit(RECLAIM_FLUSH_ALL, &cache->reclaim_reason);
+			goto do_retry;
+		}
+
+		cache->error_state = ERROR_FINISH;
+		wake_up(&cache->error_wait);
+
+		/*
+		 * after all data is flushed to raid disks from cache, we will
+		 * retry error IO and dispatch it to raid disks directly
+		 **/
+		while ((bio = bio_list_pop(&cache->retry_bio_list)) != NULL)
+			raid5_make_request(cache->mddev, bio);
+
+		if (++cache->retry_cnt < MAX_RETRY) {
+			cache->next_retry_time = jiffies +
+				(cache->error_type != -ENOMEM ?
+				IOERR_RETRY_TIME : MEMERR_RETRY_TIME);
+		}
+	}
+
+	/*
+	 * IO error to log might not be permanent, let's try log again if there
+	 * aren't too many IO errors
+	 * */
+	if (cache->next_retry_time &&
+		time_before(cache->next_retry_time, jiffies)) {
+
+		/*
+		 * must guarantee there are no pending IO running in raid
+		 * before we allow retry, since r5c_write_parity might get
+		 * confused for running stripes
+		 * */
+		mddev_suspend(cache->mddev);
+		cache->next_retry_time = 0;
+		cache->error_type = 0;
+		cache->error_state = 0;
+		mddev_resume(cache->mddev);
+	}
 }
 
 static void r5c_wake_reclaimer(struct r5c_cache *cache, int reason)
@@ -2263,6 +2366,11 @@ static void r5c_write_one_stripe_parity(struct r5c_cache *cache,
 	stripe_offset = sector_div(stripe_index, cache->chunk_size);
 	stripe_offset >>= PAGE_SECTOR_SHIFT;
 
+	if (cache->error_state) {
+		r5c_put_stripe_dirty(cache, stripe);
+		return;
+	}
+
 	r5l_queue_parity(&cache->log, sh->sector,
 		sh->dev[sh->pd_idx].page,
 		sh->qd_idx >= 0 ? sh->dev[sh->qd_idx].page : NULL,
@@ -2278,7 +2386,7 @@ int r5c_write_parity(struct r5c_cache *cache, struct stripe_head *head_sh)
 	unsigned long flags;
 
 	/* parity is already written */
-	if (head_sh->stripe) {
+	if (head_sh->stripe || cache->error_state == ERROR_FINISH) {
 		head_sh->stripe = NULL;
 		return -EAGAIN;
 	}
-- 
1.8.1


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

* [PATCH V4 07/13] raid5: cache device quiesce support
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (5 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 06/13] raid5: cache IO error handling Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 08/13] raid5: cache recovery support Shaohua Li
                   ` (7 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

In raid array quiesce, we should stop all IO. Existing mechanism stops
IO from upper layer but not IO from cache reclaim. This makes sure
reclaim thread will stop.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c | 35 +++++++++++++++++++++++++++++++++++
 drivers/md/raid5.c       |  2 ++
 drivers/md/raid5.h       |  1 +
 3 files changed, 38 insertions(+)

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index c5de6dd..35657cd 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -364,6 +364,13 @@ enum {
 	ERROR_FINISH = 2, /* Had an error, cache has no data */
 };
 
+/* device quiescent support */
+enum {
+	QUIESCE_NONE = 0,
+	QUIESCE_START = 1,
+	QUIESCE_END = 2,
+};
+
 #define PAGE_SECTOR_SHIFT (PAGE_SHIFT - 9)
 
 static inline struct r5c_cache *r5l_cache(struct r5l_log *log)
@@ -2324,6 +2331,15 @@ static void r5c_reclaim_thread(struct md_thread *thread)
 		cache->error_state = 0;
 		mddev_resume(cache->mddev);
 	}
+
+	if (cache->quiesce_state == QUIESCE_START) {
+		/* user IO already finished, we just stop reclaim */
+		cache->reclaim_reason = 0;
+		cache->quiesce_state = QUIESCE_END;
+		wake_up(&cache->reclaim_wait);
+		wait_event(cache->reclaim_wait, cache->quiesce_state ==
+			QUIESCE_NONE);
+	}
 }
 
 static void r5c_wake_reclaimer(struct r5c_cache *cache, int reason)
@@ -2339,6 +2355,25 @@ static void r5c_wake_wait_reclaimer(struct r5c_cache *cache, int reason)
 		&cache->reclaim_reason));
 }
 
+void r5c_quiesce(struct r5conf *conf, int state)
+{
+	struct r5c_cache *cache = conf->cache;
+
+	if (!cache || cache->error_state)
+		return;
+	if (state == 1) {
+		/* we choose to flush all data to raid disks, this is a policy */
+		r5c_wake_wait_reclaimer(cache, RECLAIM_FLUSH_ALL);
+		cache->quiesce_state = QUIESCE_START;
+		md_wakeup_thread(cache->reclaim_thread);
+		wait_event(cache->reclaim_wait, cache->quiesce_state ==
+			QUIESCE_END);
+	} else if (state == 0) {
+		cache->quiesce_state = QUIESCE_NONE;
+		wake_up(&cache->reclaim_wait);
+	}
+}
+
 static void r5c_parity_task_end(struct r5l_task *task, int error)
 {
 	struct stripe_head *head_sh = task->private;
diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index a9e10ff..bcd6c1f 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -7485,6 +7485,7 @@ static void raid5_quiesce(struct mddev *mddev, int state)
 		break;
 
 	case 1: /* stop all writes */
+		r5c_quiesce(conf, state);
 		lock_all_device_hash_locks_irq(conf);
 		/* '2' tells resync/reshape to pause so that all
 		 * active stripes can drain
@@ -7507,6 +7508,7 @@ static void raid5_quiesce(struct mddev *mddev, int state)
 		wake_up(&conf->wait_for_stripe);
 		wake_up(&conf->wait_for_overlap);
 		unlock_all_device_hash_locks_irq(conf);
+		r5c_quiesce(conf, state);
 		break;
 	}
 }
diff --git a/drivers/md/raid5.h b/drivers/md/raid5.h
index 16bd54f..534e5be 100644
--- a/drivers/md/raid5.h
+++ b/drivers/md/raid5.h
@@ -617,4 +617,5 @@ struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev);
 void r5c_exit_cache(struct r5c_cache *cache);
 void r5c_write_start(struct mddev *mddev, struct bio *bi);
 void r5c_write_end(struct mddev *mddev, struct bio *bi);
+void r5c_quiesce(struct r5conf *conf, int state);
 #endif
-- 
1.8.1


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

* [PATCH V4 08/13] raid5: cache recovery support
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (6 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 07/13] raid5: cache device quiesce support Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:37 ` [PATCH V4 09/13] raid5: add some sysfs entries Shaohua Li
                   ` (6 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

After a unclean shutdown, cache must be recovered. According to reclaim
process, we take different actions for stripes in different states.
-stripe without parity: read data to memory
-stripe with parity but not listed in flush_start block: read data to
memory and discard parity
-stripe with parity and listed in flush_start but not in flush_end
block: read data/parity to memory, recovery (flush them to raid disks)
-stripe listed in flush_end block: discard all its data and parity

The recovery process is quite straightforward. We just read all
data/parity into memory, do sanity check and flush stripes which reclaim
was flushint them to raid disks at crash.

There is one thing we must be careful about that recovery should use
parity stored in log instead of that calculated by raid5 stripe state
machine, since after crash, the raid array isn't in consistent state,
calculated parity could be wrong.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c | 640 ++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 639 insertions(+), 1 deletion(-)

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index 35657cd..143f333 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -288,6 +288,13 @@ enum {
 	STRIPE_DEAD = 4, /* stripe can be reused */
 };
 
+/* stripe recovery state */
+enum {
+	RECOVERY_NO_FLUSH = 0,
+	RECOVERY_FLUSH_START = 1, /* stripe in a start flush block */
+	RECOVERY_FLUSH_END = 2,
+};
+
 #define STRIPE_LOCK_BITS 8
 struct r5c_cache {
 	struct mddev *mddev;
@@ -1535,7 +1542,12 @@ static void r5c_put_stripe(struct r5c_stripe *stripe)
 		kmem_cache_free(cache->io_range_kc, range);
 	}
 	r5c_put_pages(cache, stripe->data_pages, cache->stripe_data_pages);
-	BUG_ON(stripe->parity_pages);
+	if (stripe->parity_pages) {
+		r5c_put_pages(cache, stripe->parity_pages,
+			cache->stripe_parity_pages);
+		kfree(stripe->parity_pages);
+		stripe->parity_pages = NULL;
+	}
 
 	r5c_unlock_stripe(cache, stripe, &flags2);
 
@@ -2401,6 +2413,26 @@ static void r5c_write_one_stripe_parity(struct r5c_cache *cache,
 	stripe_offset = sector_div(stripe_index, cache->chunk_size);
 	stripe_offset >>= PAGE_SECTOR_SHIFT;
 
+	/*
+	 * in recovery, raid array isn't in consistent state, we shouldn't use
+	 * calculated parity but those stored in log
+	 * */
+	if (cache->in_recovery) {
+		stripe_offset *= cache->parity_disks;
+
+		copy_highpage(sh->dev[sh->pd_idx].page,
+			stripe->parity_pages[stripe_offset]);
+		if (sh->qd_idx >= 0)
+			copy_highpage(sh->dev[sh->qd_idx].page,
+				stripe->parity_pages[stripe_offset + 1]);
+
+		if (!atomic_dec_return(&stripe->dirty_stripes)) {
+			stripe->state = STRIPE_PARITY_DONE;
+			wake_up(r5c_stripe_waitq(cache, stripe));
+		}
+		return;
+	}
+
 	if (cache->error_state) {
 		r5c_put_stripe_dirty(cache, stripe);
 		return;
@@ -2454,8 +2486,614 @@ int r5c_write_parity(struct r5c_cache *cache, struct stripe_head *head_sh)
 	return 0;
 }
 
+struct r5c_load_ctx {
+	struct r5c_cache *cache;
+	struct list_head io_ranges;
+	struct list_head stripes_with_parity;
+	struct list_head stripes_without_parity;
+	atomic_t io_cnt;
+	wait_queue_head_t io_wq;
+};
+
+/*
+ * If a stripe has parity in the log and is listed in a flush start block, the
+ * stripe is being flushed to raid. If the stripe is listed in flush end block,
+ * the striep is fully flushed to raid, all its data/parity can be discarded.
+ * Otherwise, the stripe must be recovered. If a stripe has parity data and is
+ * not listed in flush start block, it's not flushed to raid, its parity can be
+ * discarded.
+ *
+ * Note super block might point to anywhere. d(data), p(parity), f(flush start),
+ *  F(flush end), ^(superblock pointed log tail). assume 4d 2p:
+ *
+ *   d  d  d  d  p  p  f  F
+ * ^, normal case
+ *     ^ stripe must be in F, ignore data, parity
+ *                 ^ stripe must be in F, ignore parity
+ *                    ^ stripe in f/F can be ingored
+ *   d  d  d  d  p
+ * ^, f/F should not exsist if p isn't enough
+ *   d  d  d  d  p p
+ *        ^, invalid case, superblock is updated after F
+ *
+ * Each reclaim thread records its own flushed stripes and multiple reclaim
+ * threads can't handle one stripe in the meaning time. Multiple reclaim threads
+ * don't change the behavior.
+ * */
+static int r5c_add_flush_start(struct r5c_load_ctx *ctx,
+	struct r5l_flush_block *fb, u64 seq)
+{
+	struct r5c_stripe *stripe;
+	int size;
+	int i;
+	u64 stripe_index;
+
+	size = le32_to_cpu(fb->header.meta_size) -
+		sizeof(struct r5l_flush_block);
+	size /= sizeof(__le64);
+
+	for (i = 0; i < size; i++) {
+		stripe_index = le64_to_cpu(fb->flush_stripes[i]);
+		/* superblock skips the stripe */
+		stripe = r5c_search_stripe(ctx->cache, stripe_index);
+		if (!stripe)
+			continue;
+		stripe->recovery_state = RECOVERY_FLUSH_START;
+	}
+	return 0;
+}
+
+static int r5c_add_flush_end(struct r5c_load_ctx *ctx,
+	struct r5l_flush_block *fb, u64 seq)
+{
+	struct r5c_cache *cache = ctx->cache;
+	struct r5c_stripe *stripe;
+	struct r5c_io_range *range;
+	int size;
+	int i;
+	u64 stripe_index;
+
+	size = le32_to_cpu(fb->header.meta_size) -
+		sizeof(struct r5l_flush_block);
+	size /= sizeof(__le64);
+
+	for (i = 0; i < size; i++) {
+		stripe_index = le64_to_cpu(fb->flush_stripes[i]);
+		stripe = r5c_search_stripe(ctx->cache, stripe_index);
+		/* superblock skips the stripe */
+		if (!stripe)
+			continue;
+		if (stripe->recovery_state != RECOVERY_FLUSH_START)
+			return -EINVAL;
+
+		list_del(&stripe->lru);
+		/* essentially r5c_put_stripe */
+		radix_tree_delete(&cache->stripe_tree, stripe->raid_index);
+
+		while (!list_empty(&stripe->io_ranges)) {
+			range = list_first_entry(&stripe->io_ranges,
+				struct r5c_io_range, stripe_sibling);
+			list_del(&range->stripe_sibling);
+
+			list_del(&range->log_sibling);
+
+			kfree(range->checksum);
+			kmem_cache_free(cache->io_range_kc, range);
+		}
+		while (!list_empty(&stripe->parity_list)) {
+			range = list_first_entry(&stripe->parity_list,
+				struct r5c_io_range, stripe_sibling);
+			list_del(&range->stripe_sibling);
+
+			kfree(range->checksum);
+			kmem_cache_free(cache->io_range_kc, range);
+		}
+
+		kmem_cache_free(cache->stripe_kc, stripe);
+	}
+	return 0;
+}
+
+static int r5c_add_data(struct r5c_load_ctx *ctx, u64 seq,
+	sector_t meta_start, sector_t data_start, sector_t raid_start,
+	ssize_t sectors, u32 *checksum)
+{
+	struct r5c_stripe *stripe;
+	struct r5c_io_range *range;
+	u64 index;
+	int offset;
+
+	r5c_sector_stripe_index_offset(ctx->cache, raid_start, &index,
+		&offset);
+
+	stripe = r5c_create_get_stripe(ctx->cache, index);
+	if (stripe->recovery_state != RECOVERY_NO_FLUSH)
+		return -EINVAL;
+
+	range = kmem_cache_alloc(ctx->cache->io_range_kc, GFP_NOIO);
+	if (!range)
+		return -ENOMEM;
+
+	range->seq = seq;
+	range->meta_start = meta_start;
+	range->data_start = data_start;
+	range->raid_start = raid_start;
+	range->data_sectors = sectors;
+	range->stripe = stripe;
+	range->checksum = checksum;
+
+	list_add_tail(&range->stripe_sibling, &stripe->io_ranges);
+	list_add_tail(&range->log_sibling, &ctx->io_ranges);
+
+	if (list_empty(&stripe->lru))
+		list_add_tail(&stripe->lru, &ctx->stripes_without_parity);
+
+	r5c_put_stripe(stripe);
+	return 0;
+}
+
+static int r5c_add_parity(struct r5c_load_ctx *ctx, u64 seq,
+	sector_t meta_start, sector_t data_start, sector_t stripe_sect,
+	ssize_t sectors, u32 *checksum)
+{
+	struct r5c_stripe *stripe;
+	struct r5c_io_range *range;
+	sector_t index = stripe_sect;
+
+	sector_div(index, ctx->cache->log.chunk_size);
+	/* superblock skips this stripe */
+	stripe = r5c_search_stripe(ctx->cache, index);
+	if (!stripe)
+		return 0;
+	if (stripe->recovery_state != RECOVERY_NO_FLUSH)
+		return -EINVAL;
+
+	range = kmem_cache_alloc(ctx->cache->io_range_kc, GFP_NOIO);
+	if (!range)
+		return -ENOMEM;
+
+	range->seq = seq;
+	range->meta_start = meta_start;
+	range->data_start = data_start;
+	range->raid_start = stripe_sect;
+	range->data_sectors = sectors;
+	range->stripe = stripe;
+	range->checksum = checksum;
+
+	if (list_empty(&stripe->parity_list))
+		list_move_tail(&stripe->lru, &ctx->stripes_with_parity);
+	list_add_tail(&range->stripe_sibling, &stripe->parity_list);
+
+	return 0;
+}
+
+static void r5c_free_parity_ranges(struct r5c_stripe *stripe, bool skip_parity)
+{
+	struct r5c_cache *cache = stripe->cache;
+	struct r5c_io_range *range;
+
+	while (!list_empty(&stripe->parity_list)) {
+		range = list_first_entry(&stripe->parity_list,
+			struct r5c_io_range, stripe_sibling);
+		list_del(&range->stripe_sibling);
+
+		kfree(range->checksum);
+		kmem_cache_free(cache->io_range_kc, range);
+	}
+	if (skip_parity)
+		return;
+	if (stripe->parity_pages) {
+		r5c_put_pages(cache, stripe->parity_pages,
+			cache->stripe_parity_pages);
+		kfree(stripe->parity_pages);
+		stripe->parity_pages = NULL;
+	}
+}
+
+/*
+ * parities of some stripes aren't fully in log disk, such stripes havedn't
+ * started to be flushed to log yet, ignore parities of such stripes.
+ **/
+static void r5c_analyse_log(struct r5c_load_ctx *ctx)
+{
+	struct r5c_stripe *stripe, *tmp;
+	int recovery_state;
+	LIST_HEAD(list);
+
+	list_for_each_entry_safe(stripe, tmp, &ctx->stripes_with_parity, lru) {
+		recovery_state = stripe->recovery_state;
+		stripe->recovery_state = RECOVERY_NO_FLUSH;
+
+		if (recovery_state == RECOVERY_NO_FLUSH)
+			list_move_tail(&stripe->lru, &list);
+	}
+	if (list_empty(&list))
+		return;
+	list_for_each_entry(stripe, &list, lru)
+		r5c_free_parity_ranges(stripe, false);
+	list_splice_tail(&list, &ctx->stripes_without_parity);
+	printk(KERN_ERR
+		"md: %s, parities of some stripes aren't fully in cache, "
+		"discard them\n",
+		mdname(ctx->cache->mddev));
+}
+
+static void r5l_fetch_endio(struct bio *bio, int err)
+{
+	struct r5c_load_ctx *ctx = bio->bi_private;
+
+	bio_put(bio);
+	if (atomic_dec_and_test(&ctx->io_cnt))
+		wake_up(&ctx->io_wq);
+}
+
+static int r5l_fetch_one_range(struct r5c_load_ctx *ctx,
+	struct r5c_stripe *stripe, struct page **pages,
+	ssize_t page_cnt, sector_t sec, bool data)
+{
+	struct r5l_log *log = &ctx->cache->log;
+	struct bio *bio;
+	int page_index = 0;
+	int nvec;
+	int i;
+
+again:
+	nvec = min_t(int, page_cnt - page_index,
+		bio_get_nr_vecs(r5l_bdev(log)));
+	bio = bio_kmalloc(GFP_NOIO, nvec);
+	bio->bi_iter.bi_sector = sec;
+	bio->bi_end_io = r5l_fetch_endio;
+	bio->bi_private = ctx;
+	bio->bi_bdev = r5l_bdev(log);
+	atomic_inc(&ctx->io_cnt);
+
+	/* FIXME: we might use too many memory */
+	for (i = 0; i < nvec; i++) {
+		pages[page_index] = alloc_page(GFP_NOIO | __GFP_HIGHMEM);
+		if (!bio_add_page(bio, pages[page_index], PAGE_SIZE, 0)) {
+			__free_page(pages[page_index]);
+			pages[page_index] = NULL;
+			break;
+		}
+		page_index++;
+		if (data)
+			stripe->existing_pages++;
+		ctx->cache->total_pages++;
+	}
+
+	sec = r5l_block_to_sector(log, r5l_ring_add(log,
+		r5l_sector_to_block(log, sec), r5l_page_blocks(log, i)));
+	r5l_submit_bio(log, READ, bio);
+
+	if (page_index < page_cnt)
+		goto again;
+	return 0;
+}
+
+static int r5l_fetch_one_stripe(struct r5c_load_ctx *ctx,
+	struct r5c_stripe *stripe)
+{
+	struct r5c_cache *cache = stripe->cache;
+	struct r5c_io_range *range;
+	u64 index;
+	int offset, max, start, range_start;
+
+	/* new data is at the tail */
+	list_for_each_entry_reverse(range, &stripe->io_ranges, stripe_sibling) {
+		r5c_sector_stripe_index_offset(cache, range->raid_start,
+			&index, &offset);
+		offset >>= PAGE_SECTOR_SHIFT;
+		range_start = offset;
+		max = (range->data_sectors >> PAGE_SECTOR_SHIFT) + offset;
+again:
+		while (offset < max && stripe->data_pages[offset])
+			offset++;
+		start = offset;
+		while (offset < max && !stripe->data_pages[offset])
+			offset++;
+		if (start >= max)
+			continue;
+		r5l_fetch_one_range(ctx, stripe, &stripe->data_pages[start],
+			offset - start,
+			range->data_start + ((start - range_start) <<
+			PAGE_SECTOR_SHIFT), true);
+		if (offset < max)
+			goto again;
+	}
+
+	if (list_empty(&stripe->parity_list))
+		return 0;
+
+	stripe->parity_pages = kmalloc(cache->stripe_parity_pages *
+		sizeof(struct page *), GFP_NOIO | __GFP_ZERO);
+
+	list_for_each_entry(range, &stripe->parity_list, stripe_sibling) {
+		index = range->raid_start;
+		offset = sector_div(index, cache->chunk_size);
+		offset >>= PAGE_SECTOR_SHIFT;
+		offset *= cache->parity_disks;
+
+		r5l_fetch_one_range(ctx, stripe, &stripe->parity_pages[offset],
+			cache->parity_disks, range->data_start, false);
+	}
+	return 0;
+}
+
+static int r5l_fetch_stripes(struct r5c_load_ctx *ctx)
+{
+	struct r5c_stripe *stripe;
+	struct blk_plug plug;
+
+	blk_start_plug(&plug);
+	list_for_each_entry(stripe, &ctx->stripes_without_parity, lru) {
+		r5l_fetch_one_stripe(ctx, stripe);
+	}
+	list_for_each_entry(stripe, &ctx->stripes_with_parity, lru) {
+		r5l_fetch_one_stripe(ctx, stripe);
+	}
+	blk_finish_plug(&plug);
+
+	atomic_dec(&ctx->io_cnt);
+	wait_event(ctx->io_wq, atomic_read(&ctx->io_cnt) == 0);
+
+	return 0;
+}
+
+static u32 r5l_calculate_page_checksum(struct r5l_log *log, struct page *page)
+{
+	u32 csum;
+	void *addr = kmap_atomic(page);
+	csum = r5l_calculate_checksum(log, log->uuid_checksum_data,
+		addr, PAGE_SIZE, true);
+	kunmap_atomic(addr);
+	return csum;
+}
+
+static int r5l_check_one_stripe_checksum(struct r5c_stripe *stripe)
+{
+	struct r5c_cache *cache = stripe->cache;
+	struct r5l_log *log = &cache->log;
+	struct r5c_io_range *range;
+	struct page *page;
+	u64 index;
+	int offset;
+	int ret = 0;
+
+	/* FIXME: we currently ignore all data parity check */
+	list_for_each_entry(range, &stripe->io_ranges, stripe_sibling) {
+		kfree(range->checksum);
+		range->checksum = NULL;
+	}
+
+	if (list_empty(&stripe->parity_list)) {
+		r5c_free_parity_ranges(stripe, false);
+		return 0;
+	}
+
+	list_for_each_entry(range, &stripe->parity_list, stripe_sibling) {
+		index = range->raid_start;
+		offset = sector_div(index, cache->chunk_size);
+		offset >>= PAGE_SECTOR_SHIFT;
+		offset *= cache->parity_disks;
+
+		page = stripe->parity_pages[offset];
+
+		if (le32_to_cpu(range->checksum[0]) !=
+		    r5l_calculate_page_checksum(log, page)) {
+			ret = -EINVAL;
+			break;
+		}
+		if (cache->parity_disks > 1) {
+			page = stripe->parity_pages[offset + 1];
+			if (le32_to_cpu(range->checksum[1]) !=
+			    r5l_calculate_page_checksum(log, page)) {
+				ret = -EINVAL;
+				break;
+			}
+		}
+	}
+	/* If parity checksum is wrong, we don't recovery the stripe */
+	r5c_free_parity_ranges(stripe, !ret ? true : false);
+	return ret;
+}
+
+static int r5l_check_stripes_checksum(struct r5c_load_ctx *ctx)
+{
+	struct r5c_stripe *stripe, *tmp;
+
+	while (!list_empty(&ctx->stripes_without_parity)) {
+		stripe = list_first_entry(&ctx->stripes_without_parity,
+			struct r5c_stripe, lru);
+		list_del_init(&stripe->lru);
+
+		r5l_check_one_stripe_checksum(stripe);
+	}
+
+	list_for_each_entry_safe(stripe, tmp, &ctx->stripes_with_parity, lru) {
+		/* checksum error, don't recover the stripe */
+		if (r5l_check_one_stripe_checksum(stripe)) {
+			list_del_init(&stripe->lru);
+		}
+	}
+
+	return 0;
+}
+
+static int r5c_recover_stripes(struct r5c_load_ctx *ctx)
+{
+	struct r5c_cache *cache = ctx->cache;
+	LIST_HEAD(list);
+	int i;
+
+	r5l_check_stripes_checksum(ctx);
+
+	list_splice_tail(&ctx->io_ranges, &ctx->cache->log_list);
+
+	if (list_empty(&ctx->stripes_with_parity))
+		return 0;
+
+	/*
+	 * we already read all data/metadata in memory, just flush them to raid
+	 * disks.
+	 *
+	 * But note for parity, we should use that stored in cache disk instead
+	 * of that we calculated in recovery, since raid array isn't in
+	 * consistent state. The in_recovery is for this purpose.
+	 * */
+	cache->in_recovery = 1;
+
+	while (!list_empty(&ctx->stripes_with_parity)) {
+		i = 0;
+		/*
+		 * Can't handle large stripe list because of the size
+		 * limitation of flush_end block. We don't really need
+		 * flush_end though, but adding it makes we can share the
+		 * reclaim code
+		 * */
+		while (i < RECLAIM_BATCH &&
+		       !list_empty(&ctx->stripes_with_parity)) {
+			list_move_tail(ctx->stripes_with_parity.next,
+				&list);
+			i++;
+		}
+		r5c_reclaim_stripe_list(cache, &list);
+		BUG_ON(!list_empty(&list));
+	}
+
+	cache->in_recovery = 0;
+	return 0;
+}
+
+static void *r5l_read_meta_block(struct r5l_log *log, r5blk_t block,
+	u64 expected_seq, struct page **retpage)
+{
+	struct page *page = alloc_page(GFP_KERNEL|__GFP_ZERO);
+	struct r5l_meta_header *header;
+	u32 crc, stored_crc;
+
+	if (!sync_page_io(log->rdev, r5l_block_to_sector(log, block),
+	    log->block_size, page, READ, false))
+		return NULL;
+
+	header = page_address(page);
+	stored_crc = le32_to_cpu(header->checksum);
+	header->checksum = 0;
+
+	if (le32_to_cpu(header->magic) != R5LOG_MAGIC ||
+	    le64_to_cpu(header->seq) != expected_seq ||
+	    le64_to_cpu(header->position) != block)
+		goto error;
+	if (le32_to_cpu(header->type) != R5LOG_TYPE_META &&
+	    le32_to_cpu(header->type) != R5LOG_TYPE_FLUSH_START &&
+	    le32_to_cpu(header->type) != R5LOG_TYPE_FLUSH_END)
+		goto error;
+
+	crc = r5l_calculate_checksum(log, log->uuid_checksum_meta,
+			header, log->block_size, false);
+	if (stored_crc != crc)
+		goto error;
+
+	if (le32_to_cpu(header->meta_size) > log->block_size)
+		goto error;
+	*retpage = page;
+	return header;
+error:
+	__free_page(page);
+	return NULL;
+}
+
 static int r5l_load_log(struct r5l_log *log)
 {
+	r5blk_t last_cp = log->last_checkpoint;
+	u64 last_seq = log->last_cp_seq;
+	r5blk_t payload_block;
+	struct page *page = NULL;
+	struct r5l_meta_header *header;
+	void *meta;
+	struct r5l_meta_payload *payload;
+	struct r5c_load_ctx ctx;
+	int offset;
+
+	ctx.cache = r5l_cache(log);
+	INIT_LIST_HEAD(&ctx.io_ranges);
+	INIT_LIST_HEAD(&ctx.stripes_with_parity);
+	INIT_LIST_HEAD(&ctx.stripes_without_parity);
+	atomic_set(&ctx.io_cnt, 1);
+	init_waitqueue_head(&ctx.io_wq);
+
+again:
+	if (page) {
+		__free_page(page);
+		page = NULL;
+	}
+	header = r5l_read_meta_block(log, last_cp, last_seq, &page);
+	if (!header)
+		goto finish;
+	offset = sizeof(struct r5l_meta_header);
+
+	if (le32_to_cpu(header->type) == R5LOG_TYPE_FLUSH_START) {
+		r5c_add_flush_start(&ctx, (struct r5l_flush_block *)header,
+					last_seq);
+		last_cp = r5l_ring_add(log, last_cp, 1);
+		last_seq++;
+		goto again;
+	}
+
+	if (le32_to_cpu(header->type) == R5LOG_TYPE_FLUSH_END) {
+		r5c_add_flush_end(&ctx, (struct r5l_flush_block *)header,
+				last_seq);
+		last_cp = r5l_ring_add(log, last_cp, 1);
+		last_seq++;
+		goto again;
+	}
+
+	meta = header;
+	payload = meta + offset;
+	payload_block = r5l_ring_add(log, last_cp, 1);
+
+	while (offset < cpu_to_le32(header->meta_size)) {
+		u16 type = le16_to_cpu(payload->payload_type);
+		u16 entries = le32_to_cpu(payload->blocks) >>
+				log->page_block_shift;
+		u32 *checksum;
+
+		checksum = kmalloc(sizeof(u32) * entries, GFP_KERNEL);
+		memcpy(checksum, payload->data_checksum, sizeof(u32) * entries);
+		if (type == R5LOG_PAYLOAD_DATA) {
+			r5c_add_data(&ctx, last_seq,
+			  r5l_block_to_sector(log, last_cp),
+			  r5l_block_to_sector(log, payload_block),
+			  le64_to_cpu(payload->location),
+			  entries << PAGE_SECTOR_SHIFT, checksum);
+		} else {
+			r5c_add_parity(&ctx, last_seq,
+			  r5l_block_to_sector(log, last_cp),
+			  r5l_block_to_sector(log, payload_block),
+			  le64_to_cpu(payload->location),
+			  entries << PAGE_SECTOR_SHIFT, checksum);
+		}
+		payload_block = r5l_ring_add(log, payload_block,
+			r5l_page_blocks(log, entries));
+		offset += sizeof(struct r5l_meta_payload) +
+			entries * sizeof(u32);
+		payload = meta + offset;
+	}
+
+	last_seq++;
+	last_cp = payload_block;
+	goto again;
+finish:
+	if (page)
+		__free_page(page);
+
+	r5c_analyse_log(&ctx);
+
+	r5l_fetch_stripes(&ctx);
+
+	log->seq = last_seq;
+	log->log_start = last_cp;
+	r5c_recover_stripes(&ctx);
+
 	return 0;
 }
 
-- 
1.8.1


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

* [PATCH V4 09/13] raid5: add some sysfs entries
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (7 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 08/13] raid5: cache recovery support Shaohua Li
@ 2015-06-23 21:37 ` Shaohua Li
  2015-06-23 21:38 ` [PATCH V4 10/13] raid5: don't allow resize/reshape with cache support Shaohua Li
                   ` (5 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:37 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

Add some sysfs entries.
-cache_memory. Control the cache memory size.
-cache_reclaim_batch. Control how many stripes reclaim should run in one
time.
-cache_memory_watermark. The background reclaim runs if cache memory
hits the watermark and stops after hit 1.5x of the watermark.
-cache_disk_watermark. The background reclaim runs if cache disk space
hits the watermark and stops after hit 1.5x of the watermark.
-cache_stat. statistics about cache.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c | 299 ++++++++++++++++++++++++++++++++++++++++++++++-
 drivers/md/raid5.c       |   3 +
 drivers/md/raid5.h       |   1 +
 3 files changed, 302 insertions(+), 1 deletion(-)

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index 143f333..332230a 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -352,6 +352,12 @@ struct r5c_cache {
 	struct kmem_cache *io_range_kc;
 	struct kmem_cache *stripe_kc;
 	struct bio_set *bio_set;
+
+	atomic64_t in_cache_rq;
+	atomic64_t out_cache_rq;
+	atomic64_t in_cache_sectors;
+	atomic64_t out_cache_sectors;
+	atomic64_t read_cache_sectors;
 };
 
 /* reclaim reason */
@@ -405,6 +411,12 @@ static inline int r5l_page_blocks(struct r5l_log *log, int pages)
 	return pages << log->page_block_shift;
 }
 
+static inline int r5l_max_flush_stripes(struct r5l_log *log)
+{
+	return (log->block_size - sizeof(struct r5l_flush_block)) /
+		sizeof(__le64);
+}
+
 static u32 r5l_calculate_checksum(struct r5l_log *log, u32 crc,
 	void *buf, size_t size, bool data)
 {
@@ -1804,6 +1816,9 @@ static void r5c_write_bio(struct r5c_cache *cache, struct bio *bio)
 	stripe->existing_pages += new_pages;
 	r5c_unlock_stripe(cache, stripe, &flags);
 
+	atomic64_inc(&cache->in_cache_rq);
+	atomic64_add(bio_sectors(bio), &cache->in_cache_sectors);
+
 	if (r5l_queue_bio(&cache->log, bio, r5c_bio_task_end, io_range,
 	    reserved_blocks))
 		goto put_error;
@@ -1852,6 +1867,8 @@ static void r5c_read_bio(struct r5c_cache *cache, struct bio *bio)
 				split = bio;
 
 			r5c_copy_bio(split, &stripe->data_pages[start], true);
+			atomic64_add(bio_sectors(split),
+						&cache->read_cache_sectors);
 
 			bio_endio(split, 0);
 
@@ -2010,6 +2027,10 @@ static void r5c_flush_one(struct r5c_cache *cache, struct r5c_stripe *stripe,
 		bio->bi_end_io = r5c_flush_endio;
 		bio->bi_rw = WRITE;
 		atomic_inc(&stripe->pending_bios);
+
+		atomic64_inc(&cache->out_cache_rq);
+		atomic64_add(bio_sectors(bio), &cache->out_cache_sectors);
+
 		raid5_make_request(cache->mddev, bio);
 	}
 }
@@ -3310,6 +3331,278 @@ static int r5c_shrink_cache_memory(struct r5c_cache *cache, unsigned long size)
 	return 0;
 }
 
+static ssize_t r5c_show_cache_memory(struct mddev *mddev, char *page)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+
+	return sprintf(page, "%lld\n", cache->max_pages << PAGE_SHIFT);
+}
+
+static ssize_t r5c_store_cache_memory(struct mddev *mddev, const char *page,
+	size_t len)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+	unsigned long new;
+	LIST_HEAD(page_list);
+	u64 i;
+
+	if (len >= PAGE_SIZE)
+		return -EINVAL;
+	if (kstrtoul(page, 0, &new))
+		return -EINVAL;
+	new >>= PAGE_SHIFT;
+
+	if (new > cache->max_pages) {
+		i = cache->max_pages;
+		while (i < new) {
+			struct page *page = alloc_page(GFP_KERNEL);
+
+			if (!page)
+				break;
+			list_add(&page->lru, &page_list);
+			i++;
+		}
+
+		spin_lock_irq(&cache->pool_lock);
+		list_splice(&page_list, &cache->page_pool);
+		cache->free_pages += i - cache->max_pages;
+		cache->max_pages = i;
+		cache->total_pages = i;
+		r5c_calculate_watermark(cache);
+		spin_unlock_irq(&cache->pool_lock);
+		return len;
+	}
+	r5c_shrink_cache_memory(cache, new);
+	return len;
+}
+
+static struct md_sysfs_entry r5c_cache_memory = __ATTR(cache_memory,
+	S_IRUGO | S_IWUSR, r5c_show_cache_memory, r5c_store_cache_memory);
+
+/*
+ * we reclaim stripes in a batch way, so we must make sure there are enough
+ * stripe cache. Otherwise, reclaim will deadlock to wait some stripe caches
+ * free, but such stripe caches don't even run since reclaim is waitting
+ * */
+int r5c_min_stripe_cache_size(struct r5c_cache *cache)
+{
+	struct r5conf *conf = cache->mddev->private;
+	return (conf->chunk_sectors >> PAGE_SECTOR_SHIFT) *
+		cache->reclaim_batch;
+}
+
+static void r5c_set_reclaim_batch(struct r5c_cache *cache, int batch)
+{
+	struct mddev *mddev = cache->mddev;
+	struct r5conf *conf = mddev->private;
+	int size;
+
+	size = (cache->stripe_parity_pages << PAGE_SECTOR_SHIFT) * batch;
+	if (size > cache->reserved_space) {
+		cache->reserved_space = size;
+		mutex_lock(&cache->log.io_mutex);
+		cache->log.reserved_blocks = r5l_sector_to_block(&cache->log,
+			cache->reserved_space) + 1;
+		mutex_unlock(&cache->log.io_mutex);
+		r5c_wake_wait_reclaimer(cache,
+				RECLAIM_DISK_BACKGROUND);
+	} else {
+		mutex_lock(&cache->log.io_mutex);
+		cache->log.reserved_blocks -= r5l_sector_to_block(&cache->log,
+			cache->reserved_space - size);
+		mutex_unlock(&cache->log.io_mutex);
+		cache->reserved_space = size;
+	}
+
+	size = (conf->chunk_sectors >> PAGE_SECTOR_SHIFT) * batch;
+
+	mddev_lock(mddev);
+	if (size > conf->max_nr_stripes)
+		raid5_set_cache_size(mddev, size);
+	mddev_unlock(mddev);
+
+	cache->reclaim_batch = batch;
+}
+
+static ssize_t r5c_show_cache_reclaim_batch(struct mddev *mddev, char *page)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+
+	return sprintf(page, "%d\n", cache->reclaim_batch);
+}
+
+static ssize_t r5c_store_cache_reclaim_batch(struct mddev *mddev,
+	const char *page, size_t len)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+	unsigned long new;
+
+	if (len >= PAGE_SIZE)
+		return -EINVAL;
+	if (kstrtoul(page, 0, &new))
+		return -EINVAL;
+
+	if (new > r5l_max_flush_stripes(&cache->log))
+		new = r5l_max_flush_stripes(&cache->log);
+
+	if (new != cache->reclaim_batch)
+		r5c_set_reclaim_batch(cache, new);
+	return len;
+}
+
+static struct md_sysfs_entry r5c_cache_reclaim_batch =
+	__ATTR(cache_reclaim_batch, S_IRUGO | S_IWUSR,
+	r5c_show_cache_reclaim_batch, r5c_store_cache_reclaim_batch);
+
+static ssize_t r5c_show_cache_stat(struct mddev *mddev, char *page)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+
+	return sprintf(page, "%lld %lld %lld %lld %lld\n",
+		(u64)atomic64_read(&cache->in_cache_rq),
+		(u64)atomic64_read(&cache->in_cache_sectors),
+		(u64)atomic64_read(&cache->out_cache_rq),
+		(u64)atomic64_read(&cache->out_cache_sectors),
+		(u64)atomic64_read(&cache->read_cache_sectors));
+}
+
+static struct md_sysfs_entry r5c_cache_stat =
+	__ATTR(cache_stat, S_IRUGO, r5c_show_cache_stat, NULL);
+
+static ssize_t r5c_show_cache_disk_watermark(struct mddev *mddev, char *page)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+
+	return sprintf(page, "%lld\n", cache->log.low_watermark *
+		cache->log.block_size);
+}
+
+static ssize_t r5c_store_cache_disk_watermark(struct mddev *mddev,
+	const char *page, size_t len)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+	struct r5l_log *log = &cache->log;
+	unsigned long new;
+
+	if (len >= PAGE_SIZE)
+		return -EINVAL;
+	if (kstrtoul(page, 0, &new))
+		return -EINVAL;
+	new /= log->block_size;
+
+	if (new * 3 / 2 >= log->total_blocks)
+		return -EINVAL;
+
+	mutex_lock(&log->io_mutex);
+	log->low_watermark = new;
+	log->high_watermark = new * 3 / 2;
+	mutex_unlock(&log->io_mutex);
+	return len;
+}
+
+static struct md_sysfs_entry r5c_cache_disk_watermark =
+	__ATTR(cache_disk_watermark, S_IRUGO | S_IWUSR,
+	r5c_show_cache_disk_watermark, r5c_store_cache_disk_watermark);
+
+static ssize_t r5c_show_cache_memory_watermark(struct mddev *mddev, char *page)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+
+	return sprintf(page, "%lld\n", cache->low_watermark << PAGE_SHIFT);
+}
+
+static ssize_t r5c_store_cache_memory_watermark(struct mddev *mddev,
+	const char *page, size_t len)
+{
+	struct r5conf *conf = mddev->private;
+	struct r5c_cache *cache = conf->cache;
+	unsigned long new;
+
+	if (len >= PAGE_SIZE)
+		return -EINVAL;
+	if (kstrtoul(page, 0, &new))
+		return -EINVAL;
+	new >>= PAGE_SHIFT;
+
+	if (new * 2 >= cache->max_pages)
+		return -EINVAL;
+
+	spin_lock_irq(&cache->pool_lock);
+	cache->low_watermark = new;
+	cache->high_watermark = new << 1;
+	spin_unlock_irq(&cache->pool_lock);
+	return len;
+}
+
+static struct md_sysfs_entry r5c_cache_memory_watermark =
+	__ATTR(cache_memory_watermark, S_IRUGO | S_IWUSR,
+	r5c_show_cache_memory_watermark, r5c_store_cache_memory_watermark);
+
+static int r5c_init_sysfs(struct r5c_cache *cache)
+{
+	struct mddev *mddev = cache->mddev;
+	int ret;
+
+	ret = sysfs_add_file_to_group(&mddev->kobj, &r5c_cache_memory.attr,
+				      NULL);
+	if (ret)
+		return ret;
+	ret = sysfs_add_file_to_group(&mddev->kobj,
+				      &r5c_cache_reclaim_batch.attr, NULL);
+	if (ret)
+		goto err_reclaim;
+	ret = sysfs_add_file_to_group(&mddev->kobj,
+				      &r5c_cache_disk_watermark.attr, NULL);
+	if (ret)
+		goto disk_watermark;
+	ret = sysfs_add_file_to_group(&mddev->kobj,
+				      &r5c_cache_stat.attr, NULL);
+	if (ret)
+		goto stat;
+
+	ret = sysfs_add_file_to_group(&mddev->kobj,
+				      &r5c_cache_memory_watermark.attr, NULL);
+	if (ret)
+		goto memory_watermark;
+	return 0;
+memory_watermark:
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_stat.attr, NULL);
+stat:
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_disk_watermark.attr, NULL);
+disk_watermark:
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_reclaim_batch.attr, NULL);
+err_reclaim:
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_memory.attr, NULL);
+	return ret;
+}
+
+static void r5c_exit_sysfs(struct r5c_cache *cache)
+{
+	struct mddev *mddev = cache->mddev;
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_reclaim_batch.attr, NULL);
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_memory.attr, NULL);
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_disk_watermark.attr, NULL);
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_stat.attr, NULL);
+	sysfs_remove_file_from_group(&mddev->kobj,
+		&r5c_cache_memory_watermark.attr, NULL);
+}
+
 static void r5c_free_cache_data(struct r5c_cache *cache)
 {
 	struct r5c_stripe *stripe;
@@ -3420,8 +3713,11 @@ struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev)
 	cache->reclaim_thread->timeout = CHECKPOINT_TIMEOUT;
 
 	r5c_shrink_cache_memory(cache, cache->max_pages);
-
+	if (r5c_init_sysfs(cache))
+		goto err_sysfs;
 	return cache;
+err_sysfs:
+	md_unregister_thread(&cache->reclaim_thread);
 err_page:
 	r5c_free_cache_data(cache);
 
@@ -3440,6 +3736,7 @@ struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev)
 
 void r5c_exit_cache(struct r5c_cache *cache)
 {
+	r5c_exit_sysfs(cache);
 	md_unregister_thread(&cache->reclaim_thread);
 	r5l_exit_log(&cache->log);
 
diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index bcd6c1f..093611e 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -5906,6 +5906,9 @@ raid5_set_cache_size(struct mddev *mddev, int size)
 	if (size <= 16 || size > 32768)
 		return -EINVAL;
 
+	if (conf->cache && size < r5c_min_stripe_cache_size(conf->cache))
+		size = r5c_min_stripe_cache_size(conf->cache);
+
 	conf->min_nr_stripes = size;
 	while (size < conf->max_nr_stripes &&
 	       drop_one_stripe(conf))
diff --git a/drivers/md/raid5.h b/drivers/md/raid5.h
index 534e5be..25d9014 100644
--- a/drivers/md/raid5.h
+++ b/drivers/md/raid5.h
@@ -618,4 +618,5 @@ void r5c_exit_cache(struct r5c_cache *cache);
 void r5c_write_start(struct mddev *mddev, struct bio *bi);
 void r5c_write_end(struct mddev *mddev, struct bio *bi);
 void r5c_quiesce(struct r5conf *conf, int state);
+int r5c_min_stripe_cache_size(struct r5c_cache *cache);
 #endif
-- 
1.8.1


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

* [PATCH V4 10/13] raid5: don't allow resize/reshape with cache support
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (8 preceding siblings ...)
  2015-06-23 21:37 ` [PATCH V4 09/13] raid5: add some sysfs entries Shaohua Li
@ 2015-06-23 21:38 ` Shaohua Li
  2015-06-23 21:38 ` [PATCH V4 11/13] raid5: guarantee cache release stripes in correct way Shaohua Li
                   ` (4 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:38 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

If cache support is enabled, don't allow resize/reshape in current
stage. In the future, we can flush all data from cache to raid before
resize/reshape and then allow resize/reshape.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5.c | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index 093611e..f3b22f2 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -7186,6 +7186,10 @@ static int raid5_resize(struct mddev *mddev, sector_t sectors)
 	 * worth it.
 	 */
 	sector_t newsize;
+	struct r5conf *conf = mddev->private;
+
+	if (conf->cache)
+		return -EINVAL;
 	sectors &= ~((sector_t)mddev->chunk_sectors - 1);
 	newsize = raid5_size(mddev, sectors, mddev->raid_disks);
 	if (mddev->external_size &&
@@ -7237,6 +7241,8 @@ static int check_reshape(struct mddev *mddev)
 {
 	struct r5conf *conf = mddev->private;
 
+	if (conf->cache)
+		return -EINVAL;
 	if (mddev->delta_disks == 0 &&
 	    mddev->new_layout == mddev->layout &&
 	    mddev->new_chunk_sectors == mddev->chunk_sectors)
-- 
1.8.1


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

* [PATCH V4 11/13] raid5: guarantee cache release stripes in correct way
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (9 preceding siblings ...)
  2015-06-23 21:38 ` [PATCH V4 10/13] raid5: don't allow resize/reshape with cache support Shaohua Li
@ 2015-06-23 21:38 ` Shaohua Li
  2015-06-23 21:38 ` [PATCH V4 12/13] raid5: enable cache for raid array with cache disk Shaohua Li
                   ` (3 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:38 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

When cache trys to flush data to raid, we must be very careful about
stripe release. The issue is if we release a stripe and the stripe is
handling by raid5d and we add a new bio into the stripe later, the
stripe will enter the raid5 state machine several times, which
r5cache_write_parity can't handle well. This could happen if
get_active_stripe() sleeps in adding bio to several stripes. To solve
this issue, we guarantee stripe release after all bio are added to
corresponding stripes. This is a performance win too if the bug happens.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5-cache.c | 28 +++++++++-------
 drivers/md/raid5.c       | 84 +++++++++++++++++++++++++++++++-----------------
 drivers/md/raid5.h       | 11 ++++++-
 3 files changed, 80 insertions(+), 43 deletions(-)

diff --git a/drivers/md/raid5-cache.c b/drivers/md/raid5-cache.c
index 332230a..7d5292c 100644
--- a/drivers/md/raid5-cache.c
+++ b/drivers/md/raid5-cache.c
@@ -1830,7 +1830,7 @@ static void r5c_write_bio(struct r5c_cache *cache, struct bio *bio)
 	r5c_enter_error_mode(cache, -ENOMEM);
 	r5c_check_wait_error_mode(cache);
 error_mode:
-	raid5_make_request(cache->mddev, bio);
+	raid5_make_request(cache->mddev, bio, NULL);
 }
 
 static void r5c_read_bio(struct r5c_cache *cache, struct bio *bio)
@@ -1846,7 +1846,7 @@ static void r5c_read_bio(struct r5c_cache *cache, struct bio *bio)
 
 	stripe = r5c_get_stripe(cache, stripe_index);
 	if (!stripe) {
-		raid5_make_request(cache->mddev, bio);
+		raid5_make_request(cache->mddev, bio, NULL);
 		return;
 	}
 
@@ -1885,7 +1885,7 @@ static void r5c_read_bio(struct r5c_cache *cache, struct bio *bio)
 			} else
 				split = bio;
 
-			raid5_make_request(cache->mddev, split);
+			raid5_make_request(cache->mddev, split, NULL);
 
 			start = tmp;
 		}
@@ -1996,7 +1996,7 @@ static void r5c_flush_endio(struct bio *bio, int err)
 }
 
 static void r5c_flush_one(struct r5c_cache *cache, struct r5c_stripe *stripe,
-	int start, int pages)
+	int start, int pages, struct raid5_plug_context *pc)
 {
 	sector_t base;
 	struct bio *bio;
@@ -2031,7 +2031,7 @@ static void r5c_flush_one(struct r5c_cache *cache, struct r5c_stripe *stripe,
 		atomic64_inc(&cache->out_cache_rq);
 		atomic64_add(bio_sectors(bio), &cache->out_cache_sectors);
 
-		raid5_make_request(cache->mddev, bio);
+		raid5_make_request(cache->mddev, bio, pc);
 	}
 }
 
@@ -2044,7 +2044,8 @@ static void r5c_put_stripe_dirty(struct r5c_cache *cache,
 	}
 }
 
-static void r5c_flush_stripe(struct r5c_cache *cache, struct r5c_stripe *stripe)
+static void r5c_flush_stripe(struct r5c_cache *cache, struct r5c_stripe *stripe,
+	struct raid5_plug_context *pc)
 {
 	unsigned long *stripe_bits;
 	int chunk_stripes;
@@ -2073,7 +2074,7 @@ static void r5c_flush_stripe(struct r5c_cache *cache, struct r5c_stripe *stripe)
 		while (end < cache->stripe_data_pages &&
 		       stripe->data_pages[end])
 			end++;
-		r5c_flush_one(cache, stripe, start, end - start);
+		r5c_flush_one(cache, stripe, start, end - start, pc);
 	}
 	r5c_put_stripe_dirty(cache, stripe);
 }
@@ -2207,11 +2208,14 @@ static void r5c_reclaim_stripe_list(struct r5c_cache *cache,
 	struct r5c_io_range *range;
 	u64 seq;
 	sector_t meta;
-	struct blk_plug plug;
 	size_t size = 0;
+	struct raid5_plug_context pc;
 
 	if (list_empty(stripe_list))
 		return;
+
+	raid5_context_init(&pc);
+
 	list_sort(NULL, stripe_list, r5c_stripe_list_cmp);
 	list_for_each_entry(stripe, stripe_list, lru) {
 		cache->stripe_flush_data[size] =
@@ -2220,11 +2224,11 @@ static void r5c_reclaim_stripe_list(struct r5c_cache *cache,
 	}
 	size *= sizeof(__le64);
 
-	blk_start_plug(&plug);
 	/* step 1: start write to raid */
 	list_for_each_entry(stripe, stripe_list, lru)
-		r5c_flush_stripe(cache, stripe);
-	blk_finish_plug(&plug);
+		r5c_flush_stripe(cache, stripe, &pc);
+
+	raid5_context_unplug(&pc, cache->mddev, false);
 
 	/* step 2: wait parity write to cache */
 	list_for_each_entry_reverse(stripe, stripe_list, lru)
@@ -2337,7 +2341,7 @@ static void r5c_reclaim_thread(struct md_thread *thread)
 		 * retry error IO and dispatch it to raid disks directly
 		 **/
 		while ((bio = bio_list_pop(&cache->retry_bio_list)) != NULL)
-			raid5_make_request(cache->mddev, bio);
+			raid5_make_request(cache->mddev, bio, NULL);
 
 		if (++cache->retry_cnt < MAX_RETRY) {
 			cache->next_retry_time = jiffies +
diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index f3b22f2..75964e9 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -4972,26 +4972,26 @@ static struct stripe_head *__get_priority_stripe(struct r5conf *conf, int group)
 	return sh;
 }
 
-struct raid5_plug_cb {
-	struct blk_plug_cb	cb;
-	struct list_head	list;
-	struct list_head	temp_inactive_list[NR_STRIPE_HASH_LOCKS];
-};
+void raid5_context_init(struct raid5_plug_context *context)
+{
+	int i;
+	INIT_LIST_HEAD(&context->list);
+	for (i = 0; i < NR_STRIPE_HASH_LOCKS; i++)
+		INIT_LIST_HEAD(context->temp_inactive_list + i);
+}
 
-static void raid5_unplug(struct blk_plug_cb *blk_cb, bool from_schedule)
+void raid5_context_unplug(struct raid5_plug_context *context,
+	struct mddev *mddev, bool from_schedule)
 {
-	struct raid5_plug_cb *cb = container_of(
-		blk_cb, struct raid5_plug_cb, cb);
 	struct stripe_head *sh;
-	struct mddev *mddev = cb->cb.data;
 	struct r5conf *conf = mddev->private;
 	int cnt = 0;
 	int hash;
 
-	if (cb->list.next && !list_empty(&cb->list)) {
+	if (context->list.next && !list_empty(&context->list)) {
 		spin_lock_irq(&conf->device_lock);
-		while (!list_empty(&cb->list)) {
-			sh = list_first_entry(&cb->list, struct stripe_head, lru);
+		while (!list_empty(&context->list)) {
+			sh = list_first_entry(&context->list, struct stripe_head, lru);
 			list_del_init(&sh->lru);
 			/*
 			 * avoid race release_stripe_plug() sees
@@ -5005,15 +5005,38 @@ static void raid5_unplug(struct blk_plug_cb *blk_cb, bool from_schedule)
 			 * case, the count is always > 1 here
 			 */
 			hash = sh->hash_lock_index;
-			__release_stripe(conf, sh, &cb->temp_inactive_list[hash]);
+			__release_stripe(conf, sh, &context->temp_inactive_list[hash]);
 			cnt++;
 		}
 		spin_unlock_irq(&conf->device_lock);
 	}
-	release_inactive_stripe_list(conf, cb->temp_inactive_list,
+	release_inactive_stripe_list(conf, context->temp_inactive_list,
 				     NR_STRIPE_HASH_LOCKS);
 	if (mddev->queue)
 		trace_block_unplug(mddev->queue, cnt, !from_schedule);
+}
+
+static void raid5_context_plug(struct raid5_plug_context *context,
+			       struct stripe_head *sh)
+{
+	if (!test_and_set_bit(STRIPE_ON_UNPLUG_LIST, &sh->state))
+		list_add_tail(&sh->lru, &context->list);
+	else
+		release_stripe(sh);
+}
+
+struct raid5_plug_cb {
+	struct blk_plug_cb	cb;
+	struct raid5_plug_context context;
+};
+
+static void raid5_unplug(struct blk_plug_cb *blk_cb, bool from_schedule)
+{
+	struct raid5_plug_cb *cb = container_of(
+		blk_cb, struct raid5_plug_cb, cb);
+	struct mddev *mddev = cb->cb.data;
+
+	raid5_context_unplug(&cb->context, mddev, from_schedule);
 	kfree(cb);
 }
 
@@ -5032,20 +5055,14 @@ static void release_stripe_plug(struct mddev *mddev,
 
 	cb = container_of(blk_cb, struct raid5_plug_cb, cb);
 
-	if (cb->list.next == NULL) {
-		int i;
-		INIT_LIST_HEAD(&cb->list);
-		for (i = 0; i < NR_STRIPE_HASH_LOCKS; i++)
-			INIT_LIST_HEAD(cb->temp_inactive_list + i);
-	}
+	if (cb->context.list.next == NULL)
+		raid5_context_init(&cb->context);
 
-	if (!test_and_set_bit(STRIPE_ON_UNPLUG_LIST, &sh->state))
-		list_add_tail(&sh->lru, &cb->list);
-	else
-		release_stripe(sh);
+	raid5_context_plug(&cb->context, sh);
 }
 
-static void make_discard_request(struct mddev *mddev, struct bio *bi)
+static void make_discard_request(struct mddev *mddev, struct bio *bi,
+	struct raid5_plug_context *context)
 {
 	struct r5conf *conf = mddev->private;
 	sector_t logical_sector, last_sector;
@@ -5127,7 +5144,10 @@ static void make_discard_request(struct mddev *mddev, struct bio *bi)
 		clear_bit(STRIPE_DELAYED, &sh->state);
 		if (!test_and_set_bit(STRIPE_PREREAD_ACTIVE, &sh->state))
 			atomic_inc(&conf->preread_active_stripes);
-		release_stripe_plug(mddev, sh);
+		if (context)
+			raid5_context_plug(context, sh);
+		else
+			release_stripe_plug(mddev, sh);
 	}
 
 	remaining = raid5_dec_bi_active_stripes(bi);
@@ -5137,7 +5157,8 @@ static void make_discard_request(struct mddev *mddev, struct bio *bi)
 	}
 }
 
-void raid5_make_request(struct mddev *mddev, struct bio * bi)
+void raid5_make_request(struct mddev *mddev, struct bio * bi,
+	struct raid5_plug_context *context)
 {
 	struct r5conf *conf = mddev->private;
 	int dd_idx;
@@ -5167,7 +5188,7 @@ void raid5_make_request(struct mddev *mddev, struct bio * bi)
 		return;
 
 	if (unlikely(bi->bi_rw & REQ_DISCARD)) {
-		make_discard_request(mddev, bi);
+		make_discard_request(mddev, bi, context);
 		return;
 	}
 
@@ -5295,7 +5316,10 @@ void raid5_make_request(struct mddev *mddev, struct bio * bi)
 			    ((bi->bi_rw & REQ_SYNC) || conf->cache) &&
 			    !test_and_set_bit(STRIPE_PREREAD_ACTIVE, &sh->state))
 				atomic_inc(&conf->preread_active_stripes);
-			release_stripe_plug(mddev, sh);
+			if (context)
+				raid5_context_plug(context, sh);
+			else
+				release_stripe_plug(mddev, sh);
 		} else {
 			/* cannot get stripe for read-ahead, just give-up */
 			clear_bit(BIO_UPTODATE, &bi->bi_flags);
@@ -5322,7 +5346,7 @@ static void make_request(struct mddev *mddev, struct bio *bi)
 	if (conf->cache)
 		r5c_handle_bio(conf->cache, bi);
 	else
-		raid5_make_request(mddev, bi);
+		raid5_make_request(mddev, bi, NULL);
 }
 
 static sector_t raid5_size(struct mddev *mddev, sector_t sectors, int raid_disks);
diff --git a/drivers/md/raid5.h b/drivers/md/raid5.h
index 25d9014..2d78f1e 100644
--- a/drivers/md/raid5.h
+++ b/drivers/md/raid5.h
@@ -608,10 +608,19 @@ static inline int algorithm_is_DDF(int layout)
 extern void md_raid5_kick_device(struct r5conf *conf);
 extern int raid5_set_cache_size(struct mddev *mddev, int size);
 
+struct raid5_plug_context {
+	struct list_head	list;
+	struct list_head	temp_inactive_list[NR_STRIPE_HASH_LOCKS];
+};
+void raid5_context_init(struct raid5_plug_context *context);
+void raid5_context_unplug(struct raid5_plug_context *context,
+	struct mddev *mddev, bool from_schedule);
+
 void release_stripe(struct stripe_head *sh);
 int r5c_write_parity(struct r5c_cache *cache, struct stripe_head *sh);
 void r5c_flush_pending_parity(struct r5c_cache *cache);
-void raid5_make_request(struct mddev *mddev, struct bio *bi);
+void raid5_make_request(struct mddev *mddev, struct bio *bi,
+	struct raid5_plug_context *context);
 void r5c_handle_bio(struct r5c_cache *cache, struct bio *bi);
 struct r5c_cache *r5c_init_cache(struct r5conf *conf, struct md_rdev *rdev);
 void r5c_exit_cache(struct r5c_cache *cache);
-- 
1.8.1


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

* [PATCH V4 12/13] raid5: enable cache for raid array with cache disk
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (10 preceding siblings ...)
  2015-06-23 21:38 ` [PATCH V4 11/13] raid5: guarantee cache release stripes in correct way Shaohua Li
@ 2015-06-23 21:38 ` Shaohua Li
  2015-06-23 21:38 ` [PATCH V4 13/13] raid5: skip resync if caching is enabled Shaohua Li
                   ` (2 subsequent siblings)
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:38 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

Now cache is safe to enable for raid array with cache disk, let's do it.
We set the raid array sector size to 4k, so there is no partial write
which current cache code can't handling. This should not a big problem
because main filesystems are ok with 4k sector size.

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5.c | 20 ++++++++++++++++++++
 1 file changed, 20 insertions(+)

diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index 75964e9..9faa5c6 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -6340,6 +6340,9 @@ static void free_conf(struct r5conf *conf)
 {
 	if (conf->shrinker.seeks)
 		unregister_shrinker(&conf->shrinker);
+	if (conf->cache)
+		r5c_exit_cache(conf->cache);
+
 	free_thread_groups(conf);
 	shrink_stripes(conf);
 	raid5_free_percpu(conf);
@@ -6980,6 +6983,23 @@ static int run(struct mddev *mddev)
 						mddev->queue);
 	}
 
+	rdev_for_each(rdev, mddev) {
+		if (test_bit(WriteCache, &rdev->flags)) {
+			char b[BDEVNAME_SIZE];
+			printk(KERN_INFO"md/raid:%s: using device %s as cache\n",
+				mdname(mddev), bdevname(rdev->bdev, b));
+			conf->cache = r5c_init_cache(conf, rdev);
+			break;
+		}
+	}
+
+	if (conf->cache) {
+		/* Make sure all write do overwite in current stage */
+		if (mddev->queue)
+			blk_queue_logical_block_size(mddev->queue, STRIPE_SIZE);
+		conf->skip_copy = 1;
+	}
+
 	return 0;
 abort:
 	md_unregister_thread(&mddev->thread);
-- 
1.8.1


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

* [PATCH V4 13/13] raid5: skip resync if caching is enabled
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (11 preceding siblings ...)
  2015-06-23 21:38 ` [PATCH V4 12/13] raid5: enable cache for raid array with cache disk Shaohua Li
@ 2015-06-23 21:38 ` Shaohua Li
  2015-07-02  3:25 ` [PATCH V4 00/13] MD: a caching layer for raid5/6 Yuanhan Liu
  2015-07-08  1:56 ` NeilBrown
  14 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-06-23 21:38 UTC (permalink / raw)
  To: linux-raid; +Cc: songliubraving, hch, dan.j.williams, neilb

If caching is enabled, the caching layer will guarantee data
consistency, so skip resync for unclean shutdown

Signed-off-by: Shaohua Li <shli@fb.com>
---
 drivers/md/raid5.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index 9faa5c6..a91ff75 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -6998,6 +6998,13 @@ static int run(struct mddev *mddev)
 		if (mddev->queue)
 			blk_queue_logical_block_size(mddev->queue, STRIPE_SIZE);
 		conf->skip_copy = 1;
+
+		if (mddev->recovery_cp == 0) {
+			printk(KERN_NOTICE
+				"md/raid:%s: skip resync with caching enabled\n",
+				mdname(mddev));
+			mddev->recovery_cp = MaxSector;
+		}
 	}
 
 	return 0;
-- 
1.8.1


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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (12 preceding siblings ...)
  2015-06-23 21:38 ` [PATCH V4 13/13] raid5: skip resync if caching is enabled Shaohua Li
@ 2015-07-02  3:25 ` Yuanhan Liu
  2015-07-02 17:11   ` Shaohua Li
  2015-07-08  1:56 ` NeilBrown
  14 siblings, 1 reply; 44+ messages in thread
From: Yuanhan Liu @ 2015-07-02  3:25 UTC (permalink / raw)
  To: Shaohua Li
  Cc: linux-raid, songliubraving, hch, dan.j.williams, neilb, Yuanhan Liu

Hi Shaohua,

I gave it a quick test(dd if=/dev/zero of=/dev/md0) in a KVM, and
here I met a kernel oops.

    [   14.768563] BUG: unable to handle kernel NULL pointer dereference at   (null)
    [   14.769153] IP: [<c1383b13>] xor_sse_3_pf64+0x67/0x207
    [   14.769585] *pde = 00000000
    [   14.769822] Oops: 0000 [#1] SMP
    [   14.770092] Modules linked in:
    [   14.770349] CPU: 0 PID: 2234 Comm: md0_raid5 Not tainted 4.1.0+ #18
    [   14.770854] Hardware name: QEMU Standard PC (i440FX + PIIX, 1996), BIOS rel-1.7.5-0-ge51488c-20140602_164612-nilsson.home.kraxel.org 04/01/2014
    [   14.771859] task: f62c5e80 ti: f04ec000 task.ti: f04ec000
    [   14.772287] EIP: 0060:[<c1383b13>] EFLAGS: 00010246 CPU: 0
    [   14.772327] EIP is at xor_sse_3_pf64+0x67/0x207
    [   14.772327] EAX: 00000010 EBX: 00000000 ECX: e2a99000 EDX: f48ae000
    [   14.772327] ESI: f48ae000 EDI: 00000010 EBP: f04edcd8 ESP: f04edccc
    [   14.772327]  DS: 007b ES: 007b FS: 00d8 GS: 0000 SS: 0068
    [   14.772327] CR0: 80050033 CR2: 00000000 CR3: 22ac5000 CR4: 000006d0
    [   14.772327] Stack:
    [   14.772327]  c1c4d96c 00000000 00000001 f04edcf8 c1382bcf 00000000 00000002 c112d2e5
    [   14.772327]  00000000 f04edd98 00000002 f04edd30 c138554b f6a2e028 00000001 f6f51b30
    [   14.772327]  00000002 00000000 f48ae000 f6a2e028 00001000 f04edd40 00000002 00000000
    [   14.772327] Call Trace:
    [   14.772327]  [<c1382bcf>] xor_blocks+0x3a/0x6d
    [   14.772327]  [<c112d2e5>] ? page_address+0xb8/0xc0
    [   14.772327]  [<c138554b>] async_xor+0xf3/0x113
    [   14.772327]  [<c186d954>] raid_run_ops+0xf73/0x1025
    [   14.772327]  [<c186d954>] ? raid_run_ops+0xf73/0x1025
    [   14.772327]  [<c16ce8b9>] handle_stripe+0x120/0x199b
    [   14.772327]  [<c10c7df4>] ? clockevents_program_event+0xfb/0x118
    [   14.772327]  [<c10c94f7>] ? tick_program_event+0x5f/0x68
    [   14.772327]  [<c10be529>] ? hrtimer_interrupt+0xa3/0x134
    [   14.772327]  [<c16d0366>] handle_active_stripes.isra.37+0x232/0x2b5
    [   14.772327]  [<c16d0701>] raid5d+0x267/0x44c
    [   14.772327]  [<c18717e6>] ? schedule_timeout+0x1c/0x14d
    [   14.772327]  [<c1872011>] ? _raw_spin_lock_irqsave+0x1f/0x3d
    [   14.772327]  [<c16f684a>] md_thread+0x103/0x112
    [   14.772327]  [<c10a3146>] ? wait_woken+0x62/0x62
    [   14.772327]  [<c16f6747>] ? md_wait_for_blocked_rdev+0xda/0xda
    [   14.772327]  [<c108d44e>] kthread+0xa4/0xa9
    [   14.772327]  [<c1872401>] ret_from_kernel_thread+0x21/0x30
    [   14.772327]  [<c108d3aa>] ? kthread_create_on_node+0x104/0x104
    

I dug it a while, and came up with following fix. Does it make sense
to you?

----
diff --git a/crypto/async_tx/async_xor.c b/crypto/async_tx/async_xor.c
index e1bce26..063ac50 100644
--- a/crypto/async_tx/async_xor.c
+++ b/crypto/async_tx/async_xor.c
@@ -30,6 +30,7 @@
 #include <linux/dma-mapping.h>
 #include <linux/raid/xor.h>
 #include <linux/async_tx.h>
+#include <linux/highmem.h>

 /* do_async_xor - dma map the pages and perform the xor with an engine */
 static __async_inline struct dma_async_tx_descriptor *
@@ -127,7 +128,7 @@ do_sync_xor(struct page *dest, struct page **src_list, unsigned int offset,
        /* convert to buffer pointers */
        for (i = 0; i < src_cnt; i++)
                if (src_list[i])
-                       srcs[xor_src_cnt++] = page_address(src_list[i]) + offset;
+                       srcs[xor_src_cnt++] = kmap_atomic(src_list[i]) + offset;
        src_cnt = xor_src_cnt;
        /* set destination address */
        dest_buf = page_address(dest) + offset;
@@ -140,6 +141,9 @@ do_sync_xor(struct page *dest, struct page **src_list, unsigned int offset,
                xor_src_cnt = min(src_cnt, MAX_XOR_BLOCKS);
                xor_blocks(xor_src_cnt, len, dest_buf, &srcs[src_off]);

+               for(i = src_off; i < src_off + xor_src_cnt; i++)
+                       kunmap_atomic(srcs[i]);
+
                /* drop completed sources */
                src_cnt -= xor_src_cnt;
                src_off += xor_src_cnt;



Thanks.

	--yliu

On Tue, Jun 23, 2015 at 02:37:50PM -0700, Shaohua Li wrote:
> Hi,
> 
> This is the V4 version of the raid5/6 caching layer patches. The patches add
> a caching layer for raid5/6. The caching layer uses a SSD as a cache for a raid
> 5/6. It works like the similar way of a hardware raid controller. The purpose
> is to improve raid performance (reduce read-modify-write) and fix write hole
> issue.
> 
> I split the patch to into smaller ones and hopefuly they are easier to
> understand. The splitted patches will not break bisect. Functions of main parts
> are divided well, though some data structures not.
> 
> I detached the multiple reclaim thread patch, the patch set is aimed to make
> basic logic ready, and we can improve performance later (as long as we can make
> sure current logic is flexible to have improvement space for performance).
> 
> Neil,
> 
> For the issue if flush_start block is required, I double checked it. You are
> right we don't really need it, the data/parity checksum stored in cache disk
> will guarantee data integrity, but we can't use a simple disk cache flush as
> there is ordering issue. So the flush_start block does increase overhead, but
> might not too much. I didn't delete it yet, but I'm open to do it.
> 
> Thanks,
> Shaohua
> 
> V4:
> -split patche into smaller ones
> -add more comments into code and some code cleanup
> -bug fixes in recovery code
> -fix the feature bit
> 
> V3:
> -make reclaim multi-thread
> -add statistics in sysfs
> -bug fixes
> 
> V2:
> -metadata write doesn't use FUA
> -discard request is only issued when necessary
> -bug fixes and cleanup
> 
> Shaohua Li (12):
>   raid5: directly use mddev->queue
>   raid5: cache log handling
>   raid5: cache part of raid5 cache
>   raid5: cache reclaim support
>   raid5: cache IO error handling
>   raid5: cache device quiesce support
>   raid5: cache recovery support
>   raid5: add some sysfs entries
>   raid5: don't allow resize/reshape with cache support
>   raid5: guarantee cache release stripes in correct way
>   raid5: enable cache for raid array with cache disk
>   raid5: skip resync if caching is enabled
> 
> Song Liu (1):
>   MD: add a new disk role to present cache device
> 
>  drivers/md/Makefile            |    2 +-
>  drivers/md/md.c                |   24 +-
>  drivers/md/md.h                |    4 +
>  drivers/md/raid5-cache.c       | 3755 ++++++++++++++++++++++++++++++++++++++++
>  drivers/md/raid5.c             |  176 +-
>  drivers/md/raid5.h             |   24 +
>  include/uapi/linux/raid/md_p.h |   79 +
>  7 files changed, 4016 insertions(+), 48 deletions(-)
>  create mode 100644 drivers/md/raid5-cache.c
> 
> -- 
> 1.8.1
> 
> --
> To unsubscribe from this list: send the line "unsubscribe linux-raid" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-02  3:25 ` [PATCH V4 00/13] MD: a caching layer for raid5/6 Yuanhan Liu
@ 2015-07-02 17:11   ` Shaohua Li
  2015-07-03  2:18     ` Yuanhan Liu
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-02 17:11 UTC (permalink / raw)
  To: Yuanhan Liu; +Cc: linux-raid, songliubraving, hch, dan.j.williams, neilb

On Thu, Jul 02, 2015 at 11:25:16AM +0800, Yuanhan Liu wrote:
> Hi Shaohua,
> 
> I gave it a quick test(dd if=/dev/zero of=/dev/md0) in a KVM, and
> here I met a kernel oops.
> 
>     [   14.768563] BUG: unable to handle kernel NULL pointer dereference at   (null)
>     [   14.769153] IP: [<c1383b13>] xor_sse_3_pf64+0x67/0x207
>     [   14.769585] *pde = 00000000
>     [   14.769822] Oops: 0000 [#1] SMP
>     [   14.770092] Modules linked in:
>     [   14.770349] CPU: 0 PID: 2234 Comm: md0_raid5 Not tainted 4.1.0+ #18
>     [   14.770854] Hardware name: QEMU Standard PC (i440FX + PIIX, 1996), BIOS rel-1.7.5-0-ge51488c-20140602_164612-nilsson.home.kraxel.org 04/01/2014
>     [   14.771859] task: f62c5e80 ti: f04ec000 task.ti: f04ec000
>     [   14.772287] EIP: 0060:[<c1383b13>] EFLAGS: 00010246 CPU: 0
>     [   14.772327] EIP is at xor_sse_3_pf64+0x67/0x207
>     [   14.772327] EAX: 00000010 EBX: 00000000 ECX: e2a99000 EDX: f48ae000
>     [   14.772327] ESI: f48ae000 EDI: 00000010 EBP: f04edcd8 ESP: f04edccc
>     [   14.772327]  DS: 007b ES: 007b FS: 00d8 GS: 0000 SS: 0068
>     [   14.772327] CR0: 80050033 CR2: 00000000 CR3: 22ac5000 CR4: 000006d0
>     [   14.772327] Stack:
>     [   14.772327]  c1c4d96c 00000000 00000001 f04edcf8 c1382bcf 00000000 00000002 c112d2e5
>     [   14.772327]  00000000 f04edd98 00000002 f04edd30 c138554b f6a2e028 00000001 f6f51b30
>     [   14.772327]  00000002 00000000 f48ae000 f6a2e028 00001000 f04edd40 00000002 00000000
>     [   14.772327] Call Trace:
>     [   14.772327]  [<c1382bcf>] xor_blocks+0x3a/0x6d
>     [   14.772327]  [<c112d2e5>] ? page_address+0xb8/0xc0
>     [   14.772327]  [<c138554b>] async_xor+0xf3/0x113
>     [   14.772327]  [<c186d954>] raid_run_ops+0xf73/0x1025
>     [   14.772327]  [<c186d954>] ? raid_run_ops+0xf73/0x1025
>     [   14.772327]  [<c16ce8b9>] handle_stripe+0x120/0x199b
>     [   14.772327]  [<c10c7df4>] ? clockevents_program_event+0xfb/0x118
>     [   14.772327]  [<c10c94f7>] ? tick_program_event+0x5f/0x68
>     [   14.772327]  [<c10be529>] ? hrtimer_interrupt+0xa3/0x134
>     [   14.772327]  [<c16d0366>] handle_active_stripes.isra.37+0x232/0x2b5
>     [   14.772327]  [<c16d0701>] raid5d+0x267/0x44c
>     [   14.772327]  [<c18717e6>] ? schedule_timeout+0x1c/0x14d
>     [   14.772327]  [<c1872011>] ? _raw_spin_lock_irqsave+0x1f/0x3d
>     [   14.772327]  [<c16f684a>] md_thread+0x103/0x112
>     [   14.772327]  [<c10a3146>] ? wait_woken+0x62/0x62
>     [   14.772327]  [<c16f6747>] ? md_wait_for_blocked_rdev+0xda/0xda
>     [   14.772327]  [<c108d44e>] kthread+0xa4/0xa9
>     [   14.772327]  [<c1872401>] ret_from_kernel_thread+0x21/0x30
>     [   14.772327]  [<c108d3aa>] ? kthread_create_on_node+0x104/0x104
>     
> 
> I dug it a while, and came up with following fix. Does it make sense
> to you?
> 
> ----
> diff --git a/crypto/async_tx/async_xor.c b/crypto/async_tx/async_xor.c
> index e1bce26..063ac50 100644
> --- a/crypto/async_tx/async_xor.c
> +++ b/crypto/async_tx/async_xor.c
> @@ -30,6 +30,7 @@
>  #include <linux/dma-mapping.h>
>  #include <linux/raid/xor.h>
>  #include <linux/async_tx.h>
> +#include <linux/highmem.h>
> 
>  /* do_async_xor - dma map the pages and perform the xor with an engine */
>  static __async_inline struct dma_async_tx_descriptor *
> @@ -127,7 +128,7 @@ do_sync_xor(struct page *dest, struct page **src_list, unsigned int offset,
>         /* convert to buffer pointers */
>         for (i = 0; i < src_cnt; i++)
>                 if (src_list[i])
> -                       srcs[xor_src_cnt++] = page_address(src_list[i]) + offset;
> +                       srcs[xor_src_cnt++] = kmap_atomic(src_list[i]) + offset;
>         src_cnt = xor_src_cnt;
>         /* set destination address */
>         dest_buf = page_address(dest) + offset;
> @@ -140,6 +141,9 @@ do_sync_xor(struct page *dest, struct page **src_list, unsigned int offset,
>                 xor_src_cnt = min(src_cnt, MAX_XOR_BLOCKS);
>                 xor_blocks(xor_src_cnt, len, dest_buf, &srcs[src_off]);
> 
> +               for(i = src_off; i < src_off + xor_src_cnt; i++)
> +                       kunmap_atomic(srcs[i]);
> +
>                 /* drop completed sources */
>                 src_cnt -= xor_src_cnt;
>                 src_off += xor_src_cnt;

Thanks! This is the side effect of skip_copy, I think. skip_copy isn't
enabled by default, so we don't hit it before, while my previous test
was done with x86_64. We should see this with skip_copy enabled even
without the cache patches.

The patch does make sense. There are other async APIs using page_address
too: async_raid6_recov.c, async_pq.c, can you please fix them? You can
check them with raid6.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-02 17:11   ` Shaohua Li
@ 2015-07-03  2:18     ` Yuanhan Liu
  0 siblings, 0 replies; 44+ messages in thread
From: Yuanhan Liu @ 2015-07-03  2:18 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, neilb

On Thu, Jul 02, 2015 at 10:11:18AM -0700, Shaohua Li wrote:
> On Thu, Jul 02, 2015 at 11:25:16AM +0800, Yuanhan Liu wrote:
> > Hi Shaohua,
> > 
> > I gave it a quick test(dd if=/dev/zero of=/dev/md0) in a KVM, and
> > here I met a kernel oops.
> > 
> >     [   14.768563] BUG: unable to handle kernel NULL pointer dereference at   (null)
> >     [   14.769153] IP: [<c1383b13>] xor_sse_3_pf64+0x67/0x207
> >     [   14.769585] *pde = 00000000
> >     [   14.769822] Oops: 0000 [#1] SMP
> >     [   14.770092] Modules linked in:
> >     [   14.770349] CPU: 0 PID: 2234 Comm: md0_raid5 Not tainted 4.1.0+ #18
> >     [   14.770854] Hardware name: QEMU Standard PC (i440FX + PIIX, 1996), BIOS rel-1.7.5-0-ge51488c-20140602_164612-nilsson.home.kraxel.org 04/01/2014
> >     [   14.771859] task: f62c5e80 ti: f04ec000 task.ti: f04ec000
> >     [   14.772287] EIP: 0060:[<c1383b13>] EFLAGS: 00010246 CPU: 0
> >     [   14.772327] EIP is at xor_sse_3_pf64+0x67/0x207
> >     [   14.772327] EAX: 00000010 EBX: 00000000 ECX: e2a99000 EDX: f48ae000
> >     [   14.772327] ESI: f48ae000 EDI: 00000010 EBP: f04edcd8 ESP: f04edccc
> >     [   14.772327]  DS: 007b ES: 007b FS: 00d8 GS: 0000 SS: 0068
> >     [   14.772327] CR0: 80050033 CR2: 00000000 CR3: 22ac5000 CR4: 000006d0
> >     [   14.772327] Stack:
> >     [   14.772327]  c1c4d96c 00000000 00000001 f04edcf8 c1382bcf 00000000 00000002 c112d2e5
> >     [   14.772327]  00000000 f04edd98 00000002 f04edd30 c138554b f6a2e028 00000001 f6f51b30
> >     [   14.772327]  00000002 00000000 f48ae000 f6a2e028 00001000 f04edd40 00000002 00000000
> >     [   14.772327] Call Trace:
> >     [   14.772327]  [<c1382bcf>] xor_blocks+0x3a/0x6d
> >     [   14.772327]  [<c112d2e5>] ? page_address+0xb8/0xc0
> >     [   14.772327]  [<c138554b>] async_xor+0xf3/0x113
> >     [   14.772327]  [<c186d954>] raid_run_ops+0xf73/0x1025
> >     [   14.772327]  [<c186d954>] ? raid_run_ops+0xf73/0x1025
> >     [   14.772327]  [<c16ce8b9>] handle_stripe+0x120/0x199b
> >     [   14.772327]  [<c10c7df4>] ? clockevents_program_event+0xfb/0x118
> >     [   14.772327]  [<c10c94f7>] ? tick_program_event+0x5f/0x68
> >     [   14.772327]  [<c10be529>] ? hrtimer_interrupt+0xa3/0x134
> >     [   14.772327]  [<c16d0366>] handle_active_stripes.isra.37+0x232/0x2b5
> >     [   14.772327]  [<c16d0701>] raid5d+0x267/0x44c
> >     [   14.772327]  [<c18717e6>] ? schedule_timeout+0x1c/0x14d
> >     [   14.772327]  [<c1872011>] ? _raw_spin_lock_irqsave+0x1f/0x3d
> >     [   14.772327]  [<c16f684a>] md_thread+0x103/0x112
> >     [   14.772327]  [<c10a3146>] ? wait_woken+0x62/0x62
> >     [   14.772327]  [<c16f6747>] ? md_wait_for_blocked_rdev+0xda/0xda
> >     [   14.772327]  [<c108d44e>] kthread+0xa4/0xa9
> >     [   14.772327]  [<c1872401>] ret_from_kernel_thread+0x21/0x30
> >     [   14.772327]  [<c108d3aa>] ? kthread_create_on_node+0x104/0x104
> >     
> > 
> > I dug it a while, and came up with following fix. Does it make sense
> > to you?
> > 
> > ----
> > diff --git a/crypto/async_tx/async_xor.c b/crypto/async_tx/async_xor.c
> > index e1bce26..063ac50 100644
> > --- a/crypto/async_tx/async_xor.c
> > +++ b/crypto/async_tx/async_xor.c
> > @@ -30,6 +30,7 @@
> >  #include <linux/dma-mapping.h>
> >  #include <linux/raid/xor.h>
> >  #include <linux/async_tx.h>
> > +#include <linux/highmem.h>
> > 
> >  /* do_async_xor - dma map the pages and perform the xor with an engine */
> >  static __async_inline struct dma_async_tx_descriptor *
> > @@ -127,7 +128,7 @@ do_sync_xor(struct page *dest, struct page **src_list, unsigned int offset,
> >         /* convert to buffer pointers */
> >         for (i = 0; i < src_cnt; i++)
> >                 if (src_list[i])
> > -                       srcs[xor_src_cnt++] = page_address(src_list[i]) + offset;
> > +                       srcs[xor_src_cnt++] = kmap_atomic(src_list[i]) + offset;
> >         src_cnt = xor_src_cnt;
> >         /* set destination address */
> >         dest_buf = page_address(dest) + offset;
> > @@ -140,6 +141,9 @@ do_sync_xor(struct page *dest, struct page **src_list, unsigned int offset,
> >                 xor_src_cnt = min(src_cnt, MAX_XOR_BLOCKS);
> >                 xor_blocks(xor_src_cnt, len, dest_buf, &srcs[src_off]);
> > 
> > +               for(i = src_off; i < src_off + xor_src_cnt; i++)
> > +                       kunmap_atomic(srcs[i]);
> > +
> >                 /* drop completed sources */
> >                 src_cnt -= xor_src_cnt;
> >                 src_off += xor_src_cnt;
> 
> Thanks! This is the side effect of skip_copy, I think.

Yes, it is: the issue is gone when skip_copy is disabled.

> skip_copy isn't
> enabled by default, so we don't hit it before, while my previous test
> was done with x86_64. We should see this with skip_copy enabled even
> without the cache patches.

I gave it a quick test as well, at least I failed to trigger it. Seems
that no bio page is allocated with GFP_HIGHMEM flag.

> 
> The patch does make sense. There are other async APIs using page_address
> too: async_raid6_recov.c, async_pq.c, can you please fix them? You can
> check them with raid6.

Sure, and thanks.


	--yliu
> --
> To unsubscribe from this list: send the line "unsubscribe linux-raid" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
                   ` (13 preceding siblings ...)
  2015-07-02  3:25 ` [PATCH V4 00/13] MD: a caching layer for raid5/6 Yuanhan Liu
@ 2015-07-08  1:56 ` NeilBrown
  2015-07-08  5:44   ` Shaohua Li
  14 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-08  1:56 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams


Hi,
 I made some time to look at these this morning - sorry for the delay.

 Having it all broken down with more complete comments helps a lot -
 thanks.

 Unfortunately ... it helps confirm that I really don't like this.  It
 seems much more complex that it should be.  There certainly are a lot
 of details that need to be carefully considered, but the result needs
 to be simpler.

 A big part of my concern is that you choose to avoid making proper use
 of the current stripe cache.  Your argument is that it is already
 too complex.  That is probably true but I don't think you decrease
 total complexity by adding extra bits on the side that themselves are
 complex in completely different ways.

 I would like to start small and keep it as simple as possible.  I
 think the minimum useful functionality is closing the write-hole.
 Anything else could be done outside raid5 (e.g. bcache), but the
 write-hole needs integration.  Once that is closed, it may make sense
 to add more functionality.

 To do that we need a log, and it was quite sensible for you to put
 that first in the list of patches.  So let's start with that.

 I would like to propose a single metadata block format.  This block
 includes the general RAID5 parameters (both 'old' and 'new' for when a
 reshape is happening), it includes linkage to find nearby metadata.
 It includes a list of the data written after the metadata.  And it
 includes a list of device-addresses of parity blocks which are now safe
 on the RAID and so any log content is now ignored.

 The general mechanism for writing to the log is:
   - collect a list of bios.  As each arrives update the metadata
     block with index information.
   - when ready (there are various triggers) write the metadata block
     with FLUSH/FUA and write all the data blocks normally.  This
     metadata block plus data/parity blocks is a 'transaction'.

 We never trigger a log transaction until all writes for the previous
 transaction have completed. This means that once the metadata block is
 safe, all previous data must also be safe.  If this imposes too much
 waiting we could allow a "double-buffer" approach were each metadata
 block completes the previous-but-one transaction.

 When all of the previous writes complete we trigger a new log write
 if there is an outstanding SYNC write, or maybe an outstanding
 FLUSH/FUA, but otherwise we wait until the metadata block is full, or
 some amount of time has passed.

 This one metadata block serves all of the purposes that you
 identified.  It records where data is, it commits previous writes, and
 records which stripes are being written to RAID and which have been
 fully committed.

 With that log in place, we add the code to "hijack ops_run_io" to
 write all dirty blocks from the stripe_head to the cache.  This
 includes the parity of course.  Immediately that data is safe the
 write requests can be returned and the data is written the the RAID
 devices.

 "Recovery" at boot time simply involves:
  1. find and load first metadata block
  2. load the "next" metadata block. If it doesn't exist, stop.
  3. For each parity/data block listed in current block, get the
     stripe_head and read the block in.
  4. For each completed stripe listed, find the stripe_head and
     invalidate it.
  5. Make the "next" metadata block the current block and goto 2
  6. Flush out all dirty data in stripe cache.


 I think that all of this *needs* to use the stripe_cache.
 Intercepting normal processing between party computation and writing
 to the RAID devices must involve the stripe_cache.

 So this is a "simple" start.   Once this is written and agreed and
 tested and debugged, then we can look at the latency-hiding side of
 the cache.  I think this should still use the stripe_cache - just
 schedule the writes to the log a bit earlier.
 If a case can be made for a separate cache then I'm not completely
 closed to the idea but I don't really want to think about it until the
 write-hole closing code is done.

So:
  [PATCH V4 01/13] MD: add a new disk role to present cache device
    probably OK
  [PATCH V4 02/13] raid5: directly use mddev->queue
    OK
  [PATCH V4 03/13] raid5: cache log handling
    strip this down to a bare minimum.  It needs:
      -on-disk metadata block format
      -two or three active transactions which identify a list of bios
       that are currently in the stripe cache.  As the bio is added,
       its address info is added to the metadata block.
      -interface to "add a bio to a transaction" and "record a
       completed stripe".
      -timer flush the current transaction if previous one was not
       empty.

     I don't think there needs to be a separate superblock in the log.
     Each device in the array already has an 'md' superblock.  Use e.g
     the 'recovery_offset' field in there (which is per-device and
     meaningless for a log) to store the address of the most recent
     metadata blocks at the time the superblock was written.  Search
     forward and backward from there to find whole log.  Each metadata
     block holds everything else that might be useful.

  [PATCH V4 05/13] raid5: cache reclaim support
    Just want the "hijack ops_run_io" part of this so that when normal
    RAID5 processing wants to write a stripe, blocks get diverted to
    the log first and further processing of the stripe is delayed until
    those writes complete and the transaction is safe.

  [PATCH V4 08/13] raid5: cache recovery support
    Just load all the log into the stripe cache.


That should be much more manageable and would be a good start towards
getting all the functionality that you want.

Feel free to ask questions if I haven't explained things, or if you
want me to look over the metadata format or whatever.  I tend to answer
direct questions more quickly than I answer 100kB patchsets :-)

thanks,
NeilBrown


On Tue, 23 Jun 2015 14:37:50 -0700 Shaohua Li <shli@fb.com> wrote:

> Hi,
> 
> This is the V4 version of the raid5/6 caching layer patches. The patches add
> a caching layer for raid5/6. The caching layer uses a SSD as a cache for a raid
> 5/6. It works like the similar way of a hardware raid controller. The purpose
> is to improve raid performance (reduce read-modify-write) and fix write hole
> issue.
> 
> I split the patch to into smaller ones and hopefuly they are easier to
> understand. The splitted patches will not break bisect. Functions of main parts
> are divided well, though some data structures not.
> 
> I detached the multiple reclaim thread patch, the patch set is aimed to make
> basic logic ready, and we can improve performance later (as long as we can make
> sure current logic is flexible to have improvement space for performance).
> 
> Neil,
> 
> For the issue if flush_start block is required, I double checked it. You are
> right we don't really need it, the data/parity checksum stored in cache disk
> will guarantee data integrity, but we can't use a simple disk cache flush as
> there is ordering issue. So the flush_start block does increase overhead, but
> might not too much. I didn't delete it yet, but I'm open to do it.
> 
> Thanks,
> Shaohua
> 
> V4:
> -split patche into smaller ones
> -add more comments into code and some code cleanup
> -bug fixes in recovery code
> -fix the feature bit
> 
> V3:
> -make reclaim multi-thread
> -add statistics in sysfs
> -bug fixes
> 
> V2:
> -metadata write doesn't use FUA
> -discard request is only issued when necessary
> -bug fixes and cleanup
> 
> Shaohua Li (12):
>   raid5: directly use mddev->queue
>   raid5: cache log handling
>   raid5: cache part of raid5 cache
>   raid5: cache reclaim support
>   raid5: cache IO error handling
>   raid5: cache device quiesce support
>   raid5: cache recovery support
>   raid5: add some sysfs entries
>   raid5: don't allow resize/reshape with cache support
>   raid5: guarantee cache release stripes in correct way
>   raid5: enable cache for raid array with cache disk
>   raid5: skip resync if caching is enabled
> 
> Song Liu (1):
>   MD: add a new disk role to present cache device
> 
>  drivers/md/Makefile            |    2 +-
>  drivers/md/md.c                |   24 +-
>  drivers/md/md.h                |    4 +
>  drivers/md/raid5-cache.c       | 3755 ++++++++++++++++++++++++++++++++++++++++
>  drivers/md/raid5.c             |  176 +-
>  drivers/md/raid5.h             |   24 +
>  include/uapi/linux/raid/md_p.h |   79 +
>  7 files changed, 4016 insertions(+), 48 deletions(-)
>  create mode 100644 drivers/md/raid5-cache.c
> 


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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-08  1:56 ` NeilBrown
@ 2015-07-08  5:44   ` Shaohua Li
  2015-07-09 23:21     ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-08  5:44 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Wed, Jul 08, 2015 at 11:56:36AM +1000, NeilBrown wrote:
> 
> Hi,
>  I made some time to look at these this morning - sorry for the delay.
> 
>  Having it all broken down with more complete comments helps a lot -
>  thanks.
> 
>  Unfortunately ... it helps confirm that I really don't like this.  It
>  seems much more complex that it should be.  There certainly are a lot
>  of details that need to be carefully considered, but the result needs
>  to be simpler.
> 
>  A big part of my concern is that you choose to avoid making proper use
>  of the current stripe cache.  Your argument is that it is already
>  too complex.  That is probably true but I don't think you decrease
>  total complexity by adding extra bits on the side that themselves are
>  complex in completely different ways.
> 
>  I would like to start small and keep it as simple as possible.  I
>  think the minimum useful functionality is closing the write-hole.
>  Anything else could be done outside raid5 (e.g. bcache), but the
>  write-hole needs integration.  Once that is closed, it may make sense
>  to add more functionality.
> 
>  To do that we need a log, and it was quite sensible for you to put
>  that first in the list of patches.  So let's start with that.
> 
>  I would like to propose a single metadata block format.  This block
>  includes the general RAID5 parameters (both 'old' and 'new' for when a
>  reshape is happening), it includes linkage to find nearby metadata.
>  It includes a list of the data written after the metadata.  And it
>  includes a list of device-addresses of parity blocks which are now safe
>  on the RAID and so any log content is now ignored.
> 
>  The general mechanism for writing to the log is:
>    - collect a list of bios.  As each arrives update the metadata
>      block with index information.
>    - when ready (there are various triggers) write the metadata block
>      with FLUSH/FUA and write all the data blocks normally.  This
>      metadata block plus data/parity blocks is a 'transaction'.
> 
>  We never trigger a log transaction until all writes for the previous
>  transaction have completed. This means that once the metadata block is
>  safe, all previous data must also be safe.  If this imposes too much
>  waiting we could allow a "double-buffer" approach were each metadata
>  block completes the previous-but-one transaction.
> 
>  When all of the previous writes complete we trigger a new log write
>  if there is an outstanding SYNC write, or maybe an outstanding
>  FLUSH/FUA, but otherwise we wait until the metadata block is full, or
>  some amount of time has passed.
> 
>  This one metadata block serves all of the purposes that you
>  identified.  It records where data is, it commits previous writes, and
>  records which stripes are being written to RAID and which have been
>  fully committed.
> 
>  With that log in place, we add the code to "hijack ops_run_io" to
>  write all dirty blocks from the stripe_head to the cache.  This
>  includes the parity of course.  Immediately that data is safe the
>  write requests can be returned and the data is written the the RAID
>  devices.
> 
>  "Recovery" at boot time simply involves:
>   1. find and load first metadata block
>   2. load the "next" metadata block. If it doesn't exist, stop.
>   3. For each parity/data block listed in current block, get the
>      stripe_head and read the block in.
>   4. For each completed stripe listed, find the stripe_head and
>      invalidate it.
>   5. Make the "next" metadata block the current block and goto 2
>   6. Flush out all dirty data in stripe cache.
> 
> 
>  I think that all of this *needs* to use the stripe_cache.
>  Intercepting normal processing between party computation and writing
>  to the RAID devices must involve the stripe_cache.
> 
>  So this is a "simple" start.   Once this is written and agreed and
>  tested and debugged, then we can look at the latency-hiding side of
>  the cache.  I think this should still use the stripe_cache - just
>  schedule the writes to the log a bit earlier.
>  If a case can be made for a separate cache then I'm not completely
>  closed to the idea but I don't really want to think about it until the
>  write-hole closing code is done.
> 
> So:
>   [PATCH V4 01/13] MD: add a new disk role to present cache device
>     probably OK
>   [PATCH V4 02/13] raid5: directly use mddev->queue
>     OK
>   [PATCH V4 03/13] raid5: cache log handling
>     strip this down to a bare minimum.  It needs:
>       -on-disk metadata block format
>       -two or three active transactions which identify a list of bios
>        that are currently in the stripe cache.  As the bio is added,
>        its address info is added to the metadata block.
>       -interface to "add a bio to a transaction" and "record a
>        completed stripe".
>       -timer flush the current transaction if previous one was not
>        empty.
> 
>      I don't think there needs to be a separate superblock in the log.
>      Each device in the array already has an 'md' superblock.  Use e.g
>      the 'recovery_offset' field in there (which is per-device and
>      meaningless for a log) to store the address of the most recent
>      metadata blocks at the time the superblock was written.  Search
>      forward and backward from there to find whole log.  Each metadata
>      block holds everything else that might be useful.
> 
>   [PATCH V4 05/13] raid5: cache reclaim support
>     Just want the "hijack ops_run_io" part of this so that when normal
>     RAID5 processing wants to write a stripe, blocks get diverted to
>     the log first and further processing of the stripe is delayed until
>     those writes complete and the transaction is safe.
> 
>   [PATCH V4 08/13] raid5: cache recovery support
>     Just load all the log into the stripe cache.
> 
> 
> That should be much more manageable and would be a good start towards
> getting all the functionality that you want.
> 
> Feel free to ask questions if I haven't explained things, or if you
> want me to look over the metadata format or whatever.  I tend to answer
> direct questions more quickly than I answer 100kB patchsets :-)
Hi,

So we are back to the original point (eg, only fix the write hole issue) after
3 months development, which is really disappointed. I don't object just fixing
the write hole issue as a start, it's simple which is my arguement of first
post too, and probably a slight change of patch 3 meets the goal. The cache
does significantly increase complexity, which is well known when we move from
'just fix write hole' to 'fix write hole and do cache'. But the code is right
there, I'm wondering how bad it is. Just because of stripe_cache? I don't see
any significant technical merit stripe_cache is a must-have here. You said the
code is complex. Yes, it is, but it's not because of the new data cache. It's a
simple radix tree and takes less than 20% of the total code. The remaining code
does what any log device should do (write to log, reclaim when log is full,
recovery when log crash). Similar things must be done even stripe_cache is
used. And if we don't use stripe_cache, we don't deeply couple with current
state machine, the new data cache actually brings flexibility (for example, we
can choose not using cache, or we can easily put the cache to NVRAM).

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-08  5:44   ` Shaohua Li
@ 2015-07-09 23:21     ` NeilBrown
  2015-07-10  4:08       ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-09 23:21 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Tue, 7 Jul 2015 22:44:02 -0700 Shaohua Li <shli@fb.com> wrote:

> On Wed, Jul 08, 2015 at 11:56:36AM +1000, NeilBrown wrote:
> > 
> > Hi,
> >  I made some time to look at these this morning - sorry for the delay.
> > 
> >  Having it all broken down with more complete comments helps a lot -
> >  thanks.
> > 
> >  Unfortunately ... it helps confirm that I really don't like this.  It
> >  seems much more complex that it should be.  There certainly are a lot
> >  of details that need to be carefully considered, but the result needs
> >  to be simpler.
> > 
> >  A big part of my concern is that you choose to avoid making proper use
> >  of the current stripe cache.  Your argument is that it is already
> >  too complex.  That is probably true but I don't think you decrease
> >  total complexity by adding extra bits on the side that themselves are
> >  complex in completely different ways.
> > 
> >  I would like to start small and keep it as simple as possible.  I
> >  think the minimum useful functionality is closing the write-hole.
> >  Anything else could be done outside raid5 (e.g. bcache), but the
> >  write-hole needs integration.  Once that is closed, it may make sense
> >  to add more functionality.
> > 
> >  To do that we need a log, and it was quite sensible for you to put
> >  that first in the list of patches.  So let's start with that.
> > 
> >  I would like to propose a single metadata block format.  This block
> >  includes the general RAID5 parameters (both 'old' and 'new' for when a
> >  reshape is happening), it includes linkage to find nearby metadata.
> >  It includes a list of the data written after the metadata.  And it
> >  includes a list of device-addresses of parity blocks which are now safe
> >  on the RAID and so any log content is now ignored.
> > 
> >  The general mechanism for writing to the log is:
> >    - collect a list of bios.  As each arrives update the metadata
> >      block with index information.
> >    - when ready (there are various triggers) write the metadata block
> >      with FLUSH/FUA and write all the data blocks normally.  This
> >      metadata block plus data/parity blocks is a 'transaction'.
> > 
> >  We never trigger a log transaction until all writes for the previous
> >  transaction have completed. This means that once the metadata block is
> >  safe, all previous data must also be safe.  If this imposes too much
> >  waiting we could allow a "double-buffer" approach were each metadata
> >  block completes the previous-but-one transaction.
> > 
> >  When all of the previous writes complete we trigger a new log write
> >  if there is an outstanding SYNC write, or maybe an outstanding
> >  FLUSH/FUA, but otherwise we wait until the metadata block is full, or
> >  some amount of time has passed.
> > 
> >  This one metadata block serves all of the purposes that you
> >  identified.  It records where data is, it commits previous writes, and
> >  records which stripes are being written to RAID and which have been
> >  fully committed.
> > 
> >  With that log in place, we add the code to "hijack ops_run_io" to
> >  write all dirty blocks from the stripe_head to the cache.  This
> >  includes the parity of course.  Immediately that data is safe the
> >  write requests can be returned and the data is written the the RAID
> >  devices.
> > 
> >  "Recovery" at boot time simply involves:
> >   1. find and load first metadata block
> >   2. load the "next" metadata block. If it doesn't exist, stop.
> >   3. For each parity/data block listed in current block, get the
> >      stripe_head and read the block in.
> >   4. For each completed stripe listed, find the stripe_head and
> >      invalidate it.
> >   5. Make the "next" metadata block the current block and goto 2
> >   6. Flush out all dirty data in stripe cache.
> > 
> > 
> >  I think that all of this *needs* to use the stripe_cache.
> >  Intercepting normal processing between party computation and writing
> >  to the RAID devices must involve the stripe_cache.
> > 
> >  So this is a "simple" start.   Once this is written and agreed and
> >  tested and debugged, then we can look at the latency-hiding side of
> >  the cache.  I think this should still use the stripe_cache - just
> >  schedule the writes to the log a bit earlier.
> >  If a case can be made for a separate cache then I'm not completely
> >  closed to the idea but I don't really want to think about it until the
> >  write-hole closing code is done.
> > 
> > So:
> >   [PATCH V4 01/13] MD: add a new disk role to present cache device
> >     probably OK
> >   [PATCH V4 02/13] raid5: directly use mddev->queue
> >     OK
> >   [PATCH V4 03/13] raid5: cache log handling
> >     strip this down to a bare minimum.  It needs:
> >       -on-disk metadata block format
> >       -two or three active transactions which identify a list of bios
> >        that are currently in the stripe cache.  As the bio is added,
> >        its address info is added to the metadata block.
> >       -interface to "add a bio to a transaction" and "record a
> >        completed stripe".
> >       -timer flush the current transaction if previous one was not
> >        empty.
> > 
> >      I don't think there needs to be a separate superblock in the log.
> >      Each device in the array already has an 'md' superblock.  Use e.g
> >      the 'recovery_offset' field in there (which is per-device and
> >      meaningless for a log) to store the address of the most recent
> >      metadata blocks at the time the superblock was written.  Search
> >      forward and backward from there to find whole log.  Each metadata
> >      block holds everything else that might be useful.
> > 
> >   [PATCH V4 05/13] raid5: cache reclaim support
> >     Just want the "hijack ops_run_io" part of this so that when normal
> >     RAID5 processing wants to write a stripe, blocks get diverted to
> >     the log first and further processing of the stripe is delayed until
> >     those writes complete and the transaction is safe.
> > 
> >   [PATCH V4 08/13] raid5: cache recovery support
> >     Just load all the log into the stripe cache.
> > 
> > 
> > That should be much more manageable and would be a good start towards
> > getting all the functionality that you want.
> > 
> > Feel free to ask questions if I haven't explained things, or if you
> > want me to look over the metadata format or whatever.  I tend to answer
> > direct questions more quickly than I answer 100kB patchsets :-)
> Hi,
> 
> So we are back to the original point (eg, only fix the write hole issue) after
> 3 months development, which is really disappointed. I don't object just fixing
> the write hole issue as a start, it's simple which is my arguement of first
> post too, and probably a slight change of patch 3 meets the goal. The cache
> does significantly increase complexity, which is well known when we move from
> 'just fix write hole' to 'fix write hole and do cache'. But the code is right
> there, I'm wondering how bad it is. Just because of stripe_cache? I don't see
> any significant technical merit stripe_cache is a must-have here. You said the
> code is complex. Yes, it is, but it's not because of the new data cache. It's a
> simple radix tree and takes less than 20% of the total code. The remaining code
> does what any log device should do (write to log, reclaim when log is full,
> recovery when log crash). Similar things must be done even stripe_cache is
> used. And if we don't use stripe_cache, we don't deeply couple with current
> state machine, the new data cache actually brings flexibility (for example, we
> can choose not using cache, or we can easily put the cache to NVRAM).

I don't think we are completely back to the start.  The product of
recent months isn't just code - it is also your understanding and
expertise.   You've explored the problems and understand them.  You
could quickly see issues because you have struggled with them already.

I think the principal of "write one to throw away" is a good one - and
probably one we don't apply often enough.  The first time you implement
something you don't really know what you are doing, so the result is
bound to be imperfect.  The second time you have a much better idea of
the big picture.

The real big problem that I have with the current proposal is that I
just don't understand it.  Maybe that is my failing.  But all the
evidence suggests that I'll end up needing to maintain it, so I *have*
to understand it.

But let's not assume we are throwing everything away (but equally not
promise that we won't) and start with just the first (non trivial)
patch.  The log.

I think we do want a log.  I think you have identified the key
functionality that the log must provide.  But I think the current
layout is too complex.  This is something that will be persistent on
storage devices, so we want to get it as "right" as we can, early.
It is fairly cheap to throw away a log and start a new one, so fixing
design mistakes isn't too hard.  But we do have to support any format
we create indefinitely.
So simple is best.  Simple is most extensible.

As I said: A single metadata block format, with:
 - forward/backward linkage - and probably a link to the "first" block
   in the log at the time of writing
 - array state data - brieifly
 - list of data/parity block addresses and maybe checksums
 - list of "completed" stripes

The log is simply these metadata blocks mixed with the data/parity they
they describe.
Each metadata block commits all previous transactions (or maybe all but
the most recent).
The current md superblock contains the address of one of these metadata
blocks.

Does that seem reasonable?  Does that seem simpler than what you had?
Will that meet your perceived need?  It meets mine.
If we can do that, I'll apply it and we can move on to the next step.


(and just BTW, if we wanted to use NVRAM to help with this, I would
start out by just putting the log in NVRAM, and then see what needs to
be tweeked to make use of the easier addressing of NVRAM over a block
device).

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-09 23:21     ` NeilBrown
@ 2015-07-10  4:08       ` Shaohua Li
  2015-07-10  4:36         ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-10  4:08 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Fri, Jul 10, 2015 at 09:21:19AM +1000, NeilBrown wrote:
> On Tue, 7 Jul 2015 22:44:02 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Wed, Jul 08, 2015 at 11:56:36AM +1000, NeilBrown wrote:
> > > 
> > > Hi,
> > >  I made some time to look at these this morning - sorry for the delay.
> > > 
> > >  Having it all broken down with more complete comments helps a lot -
> > >  thanks.
> > > 
> > >  Unfortunately ... it helps confirm that I really don't like this.  It
> > >  seems much more complex that it should be.  There certainly are a lot
> > >  of details that need to be carefully considered, but the result needs
> > >  to be simpler.
> > > 
> > >  A big part of my concern is that you choose to avoid making proper use
> > >  of the current stripe cache.  Your argument is that it is already
> > >  too complex.  That is probably true but I don't think you decrease
> > >  total complexity by adding extra bits on the side that themselves are
> > >  complex in completely different ways.
> > > 
> > >  I would like to start small and keep it as simple as possible.  I
> > >  think the minimum useful functionality is closing the write-hole.
> > >  Anything else could be done outside raid5 (e.g. bcache), but the
> > >  write-hole needs integration.  Once that is closed, it may make sense
> > >  to add more functionality.
> > > 
> > >  To do that we need a log, and it was quite sensible for you to put
> > >  that first in the list of patches.  So let's start with that.
> > > 
> > >  I would like to propose a single metadata block format.  This block
> > >  includes the general RAID5 parameters (both 'old' and 'new' for when a
> > >  reshape is happening), it includes linkage to find nearby metadata.
> > >  It includes a list of the data written after the metadata.  And it
> > >  includes a list of device-addresses of parity blocks which are now safe
> > >  on the RAID and so any log content is now ignored.
> > > 
> > >  The general mechanism for writing to the log is:
> > >    - collect a list of bios.  As each arrives update the metadata
> > >      block with index information.
> > >    - when ready (there are various triggers) write the metadata block
> > >      with FLUSH/FUA and write all the data blocks normally.  This
> > >      metadata block plus data/parity blocks is a 'transaction'.
> > > 
> > >  We never trigger a log transaction until all writes for the previous
> > >  transaction have completed. This means that once the metadata block is
> > >  safe, all previous data must also be safe.  If this imposes too much
> > >  waiting we could allow a "double-buffer" approach were each metadata
> > >  block completes the previous-but-one transaction.
> > > 
> > >  When all of the previous writes complete we trigger a new log write
> > >  if there is an outstanding SYNC write, or maybe an outstanding
> > >  FLUSH/FUA, but otherwise we wait until the metadata block is full, or
> > >  some amount of time has passed.
> > > 
> > >  This one metadata block serves all of the purposes that you
> > >  identified.  It records where data is, it commits previous writes, and
> > >  records which stripes are being written to RAID and which have been
> > >  fully committed.
> > > 
> > >  With that log in place, we add the code to "hijack ops_run_io" to
> > >  write all dirty blocks from the stripe_head to the cache.  This
> > >  includes the parity of course.  Immediately that data is safe the
> > >  write requests can be returned and the data is written the the RAID
> > >  devices.
> > > 
> > >  "Recovery" at boot time simply involves:
> > >   1. find and load first metadata block
> > >   2. load the "next" metadata block. If it doesn't exist, stop.
> > >   3. For each parity/data block listed in current block, get the
> > >      stripe_head and read the block in.
> > >   4. For each completed stripe listed, find the stripe_head and
> > >      invalidate it.
> > >   5. Make the "next" metadata block the current block and goto 2
> > >   6. Flush out all dirty data in stripe cache.
> > > 
> > > 
> > >  I think that all of this *needs* to use the stripe_cache.
> > >  Intercepting normal processing between party computation and writing
> > >  to the RAID devices must involve the stripe_cache.
> > > 
> > >  So this is a "simple" start.   Once this is written and agreed and
> > >  tested and debugged, then we can look at the latency-hiding side of
> > >  the cache.  I think this should still use the stripe_cache - just
> > >  schedule the writes to the log a bit earlier.
> > >  If a case can be made for a separate cache then I'm not completely
> > >  closed to the idea but I don't really want to think about it until the
> > >  write-hole closing code is done.
> > > 
> > > So:
> > >   [PATCH V4 01/13] MD: add a new disk role to present cache device
> > >     probably OK
> > >   [PATCH V4 02/13] raid5: directly use mddev->queue
> > >     OK
> > >   [PATCH V4 03/13] raid5: cache log handling
> > >     strip this down to a bare minimum.  It needs:
> > >       -on-disk metadata block format
> > >       -two or three active transactions which identify a list of bios
> > >        that are currently in the stripe cache.  As the bio is added,
> > >        its address info is added to the metadata block.
> > >       -interface to "add a bio to a transaction" and "record a
> > >        completed stripe".
> > >       -timer flush the current transaction if previous one was not
> > >        empty.
> > > 
> > >      I don't think there needs to be a separate superblock in the log.
> > >      Each device in the array already has an 'md' superblock.  Use e.g
> > >      the 'recovery_offset' field in there (which is per-device and
> > >      meaningless for a log) to store the address of the most recent
> > >      metadata blocks at the time the superblock was written.  Search
> > >      forward and backward from there to find whole log.  Each metadata
> > >      block holds everything else that might be useful.
> > > 
> > >   [PATCH V4 05/13] raid5: cache reclaim support
> > >     Just want the "hijack ops_run_io" part of this so that when normal
> > >     RAID5 processing wants to write a stripe, blocks get diverted to
> > >     the log first and further processing of the stripe is delayed until
> > >     those writes complete and the transaction is safe.
> > > 
> > >   [PATCH V4 08/13] raid5: cache recovery support
> > >     Just load all the log into the stripe cache.
> > > 
> > > 
> > > That should be much more manageable and would be a good start towards
> > > getting all the functionality that you want.
> > > 
> > > Feel free to ask questions if I haven't explained things, or if you
> > > want me to look over the metadata format or whatever.  I tend to answer
> > > direct questions more quickly than I answer 100kB patchsets :-)
> > Hi,
> > 
> > So we are back to the original point (eg, only fix the write hole issue) after
> > 3 months development, which is really disappointed. I don't object just fixing
> > the write hole issue as a start, it's simple which is my arguement of first
> > post too, and probably a slight change of patch 3 meets the goal. The cache
> > does significantly increase complexity, which is well known when we move from
> > 'just fix write hole' to 'fix write hole and do cache'. But the code is right
> > there, I'm wondering how bad it is. Just because of stripe_cache? I don't see
> > any significant technical merit stripe_cache is a must-have here. You said the
> > code is complex. Yes, it is, but it's not because of the new data cache. It's a
> > simple radix tree and takes less than 20% of the total code. The remaining code
> > does what any log device should do (write to log, reclaim when log is full,
> > recovery when log crash). Similar things must be done even stripe_cache is
> > used. And if we don't use stripe_cache, we don't deeply couple with current
> > state machine, the new data cache actually brings flexibility (for example, we
> > can choose not using cache, or we can easily put the cache to NVRAM).
> 
> I don't think we are completely back to the start.  The product of
> recent months isn't just code - it is also your understanding and
> expertise.   You've explored the problems and understand them.  You
> could quickly see issues because you have struggled with them already.
> 
> I think the principal of "write one to throw away" is a good one - and
> probably one we don't apply often enough.  The first time you implement
> something you don't really know what you are doing, so the result is
> bound to be imperfect.  The second time you have a much better idea of
> the big picture.
> 
> The real big problem that I have with the current proposal is that I
> just don't understand it.  Maybe that is my failing.  But all the
> evidence suggests that I'll end up needing to maintain it, so I *have*
> to understand it.
> 
> But let's not assume we are throwing everything away (but equally not
> promise that we won't) and start with just the first (non trivial)
> patch.  The log.
> 
> I think we do want a log.  I think you have identified the key
> functionality that the log must provide.  But I think the current
> layout is too complex.  This is something that will be persistent on
> storage devices, so we want to get it as "right" as we can, early.
> It is fairly cheap to throw away a log and start a new one, so fixing
> design mistakes isn't too hard.  But we do have to support any format
> we create indefinitely.
> So simple is best.  Simple is most extensible.
> 
> As I said: A single metadata block format, with:
>  - forward/backward linkage - and probably a link to the "first" block
>    in the log at the time of writing
>  - array state data - brieifly
>  - list of data/parity block addresses and maybe checksums
>  - list of "completed" stripes
> 
> The log is simply these metadata blocks mixed with the data/parity they
> they describe.
> Each metadata block commits all previous transactions (or maybe all but
> the most recent).
> The current md superblock contains the address of one of these metadata
> blocks.
> 
> Does that seem reasonable?  Does that seem simpler than what you had?
> Will that meet your perceived need?  It meets mine.
> If we can do that, I'll apply it and we can move on to the next step.

My original layout has 3 types of blocks: super block, metadata block for data/parity,
flush block to list completed stripes.

Compared to your single metadata block format, your format
- delete super block. I'm fine with this. The only useful bits in my
  super block is log tail and seq. we can overload MD superblock
  recovery_offset/resync_offset to record seq/log tail.
- merge metadata block and flush block to one metadata block. That is
  the new metadata block can record a mix of data/parity/'completed'
  stripes. I don't object to this, it's doable, but don't think this
  makes things simpler.

If you insist, we can take this one. The new metadata block can record a
mix of data/parity/completed stripes, but it can also just record data
or completed stripes. So I'll change the disk format to cater this, but
I'll always put 'completed stripes' info into a separate metadata block
(that is not mixing with data/parity, but we still just have one
metadata type). That way I only need slight change of current code and
it works for the new format. We can, if necessary, let the new metadata
records any mix of data/parity/completed stripes later. Sounds good?

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  4:08       ` Shaohua Li
@ 2015-07-10  4:36         ` NeilBrown
  2015-07-10  4:52           ` Shaohua Li
  2015-07-15  0:45           ` Shaohua Li
  0 siblings, 2 replies; 44+ messages in thread
From: NeilBrown @ 2015-07-10  4:36 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:

> On Fri, Jul 10, 2015 at 09:21:19AM +1000, NeilBrown wrote:
> > On Tue, 7 Jul 2015 22:44:02 -0700 Shaohua Li <shli@fb.com> wrote:
> > 
> > > On Wed, Jul 08, 2015 at 11:56:36AM +1000, NeilBrown wrote:
> > > > 
> > > > Hi,
> > > >  I made some time to look at these this morning - sorry for the delay.
> > > > 
> > > >  Having it all broken down with more complete comments helps a lot -
> > > >  thanks.
> > > > 
> > > >  Unfortunately ... it helps confirm that I really don't like this.  It
> > > >  seems much more complex that it should be.  There certainly are a lot
> > > >  of details that need to be carefully considered, but the result needs
> > > >  to be simpler.
> > > > 
> > > >  A big part of my concern is that you choose to avoid making proper use
> > > >  of the current stripe cache.  Your argument is that it is already
> > > >  too complex.  That is probably true but I don't think you decrease
> > > >  total complexity by adding extra bits on the side that themselves are
> > > >  complex in completely different ways.
> > > > 
> > > >  I would like to start small and keep it as simple as possible.  I
> > > >  think the minimum useful functionality is closing the write-hole.
> > > >  Anything else could be done outside raid5 (e.g. bcache), but the
> > > >  write-hole needs integration.  Once that is closed, it may make sense
> > > >  to add more functionality.
> > > > 
> > > >  To do that we need a log, and it was quite sensible for you to put
> > > >  that first in the list of patches.  So let's start with that.
> > > > 
> > > >  I would like to propose a single metadata block format.  This block
> > > >  includes the general RAID5 parameters (both 'old' and 'new' for when a
> > > >  reshape is happening), it includes linkage to find nearby metadata.
> > > >  It includes a list of the data written after the metadata.  And it
> > > >  includes a list of device-addresses of parity blocks which are now safe
> > > >  on the RAID and so any log content is now ignored.
> > > > 
> > > >  The general mechanism for writing to the log is:
> > > >    - collect a list of bios.  As each arrives update the metadata
> > > >      block with index information.
> > > >    - when ready (there are various triggers) write the metadata block
> > > >      with FLUSH/FUA and write all the data blocks normally.  This
> > > >      metadata block plus data/parity blocks is a 'transaction'.
> > > > 
> > > >  We never trigger a log transaction until all writes for the previous
> > > >  transaction have completed. This means that once the metadata block is
> > > >  safe, all previous data must also be safe.  If this imposes too much
> > > >  waiting we could allow a "double-buffer" approach were each metadata
> > > >  block completes the previous-but-one transaction.
> > > > 
> > > >  When all of the previous writes complete we trigger a new log write
> > > >  if there is an outstanding SYNC write, or maybe an outstanding
> > > >  FLUSH/FUA, but otherwise we wait until the metadata block is full, or
> > > >  some amount of time has passed.
> > > > 
> > > >  This one metadata block serves all of the purposes that you
> > > >  identified.  It records where data is, it commits previous writes, and
> > > >  records which stripes are being written to RAID and which have been
> > > >  fully committed.
> > > > 
> > > >  With that log in place, we add the code to "hijack ops_run_io" to
> > > >  write all dirty blocks from the stripe_head to the cache.  This
> > > >  includes the parity of course.  Immediately that data is safe the
> > > >  write requests can be returned and the data is written the the RAID
> > > >  devices.
> > > > 
> > > >  "Recovery" at boot time simply involves:
> > > >   1. find and load first metadata block
> > > >   2. load the "next" metadata block. If it doesn't exist, stop.
> > > >   3. For each parity/data block listed in current block, get the
> > > >      stripe_head and read the block in.
> > > >   4. For each completed stripe listed, find the stripe_head and
> > > >      invalidate it.
> > > >   5. Make the "next" metadata block the current block and goto 2
> > > >   6. Flush out all dirty data in stripe cache.
> > > > 
> > > > 
> > > >  I think that all of this *needs* to use the stripe_cache.
> > > >  Intercepting normal processing between party computation and writing
> > > >  to the RAID devices must involve the stripe_cache.
> > > > 
> > > >  So this is a "simple" start.   Once this is written and agreed and
> > > >  tested and debugged, then we can look at the latency-hiding side of
> > > >  the cache.  I think this should still use the stripe_cache - just
> > > >  schedule the writes to the log a bit earlier.
> > > >  If a case can be made for a separate cache then I'm not completely
> > > >  closed to the idea but I don't really want to think about it until the
> > > >  write-hole closing code is done.
> > > > 
> > > > So:
> > > >   [PATCH V4 01/13] MD: add a new disk role to present cache device
> > > >     probably OK
> > > >   [PATCH V4 02/13] raid5: directly use mddev->queue
> > > >     OK
> > > >   [PATCH V4 03/13] raid5: cache log handling
> > > >     strip this down to a bare minimum.  It needs:
> > > >       -on-disk metadata block format
> > > >       -two or three active transactions which identify a list of bios
> > > >        that are currently in the stripe cache.  As the bio is added,
> > > >        its address info is added to the metadata block.
> > > >       -interface to "add a bio to a transaction" and "record a
> > > >        completed stripe".
> > > >       -timer flush the current transaction if previous one was not
> > > >        empty.
> > > > 
> > > >      I don't think there needs to be a separate superblock in the log.
> > > >      Each device in the array already has an 'md' superblock.  Use e.g
> > > >      the 'recovery_offset' field in there (which is per-device and
> > > >      meaningless for a log) to store the address of the most recent
> > > >      metadata blocks at the time the superblock was written.  Search
> > > >      forward and backward from there to find whole log.  Each metadata
> > > >      block holds everything else that might be useful.
> > > > 
> > > >   [PATCH V4 05/13] raid5: cache reclaim support
> > > >     Just want the "hijack ops_run_io" part of this so that when normal
> > > >     RAID5 processing wants to write a stripe, blocks get diverted to
> > > >     the log first and further processing of the stripe is delayed until
> > > >     those writes complete and the transaction is safe.
> > > > 
> > > >   [PATCH V4 08/13] raid5: cache recovery support
> > > >     Just load all the log into the stripe cache.
> > > > 
> > > > 
> > > > That should be much more manageable and would be a good start towards
> > > > getting all the functionality that you want.
> > > > 
> > > > Feel free to ask questions if I haven't explained things, or if you
> > > > want me to look over the metadata format or whatever.  I tend to answer
> > > > direct questions more quickly than I answer 100kB patchsets :-)
> > > Hi,
> > > 
> > > So we are back to the original point (eg, only fix the write hole issue) after
> > > 3 months development, which is really disappointed. I don't object just fixing
> > > the write hole issue as a start, it's simple which is my arguement of first
> > > post too, and probably a slight change of patch 3 meets the goal. The cache
> > > does significantly increase complexity, which is well known when we move from
> > > 'just fix write hole' to 'fix write hole and do cache'. But the code is right
> > > there, I'm wondering how bad it is. Just because of stripe_cache? I don't see
> > > any significant technical merit stripe_cache is a must-have here. You said the
> > > code is complex. Yes, it is, but it's not because of the new data cache. It's a
> > > simple radix tree and takes less than 20% of the total code. The remaining code
> > > does what any log device should do (write to log, reclaim when log is full,
> > > recovery when log crash). Similar things must be done even stripe_cache is
> > > used. And if we don't use stripe_cache, we don't deeply couple with current
> > > state machine, the new data cache actually brings flexibility (for example, we
> > > can choose not using cache, or we can easily put the cache to NVRAM).
> > 
> > I don't think we are completely back to the start.  The product of
> > recent months isn't just code - it is also your understanding and
> > expertise.   You've explored the problems and understand them.  You
> > could quickly see issues because you have struggled with them already.
> > 
> > I think the principal of "write one to throw away" is a good one - and
> > probably one we don't apply often enough.  The first time you implement
> > something you don't really know what you are doing, so the result is
> > bound to be imperfect.  The second time you have a much better idea of
> > the big picture.
> > 
> > The real big problem that I have with the current proposal is that I
> > just don't understand it.  Maybe that is my failing.  But all the
> > evidence suggests that I'll end up needing to maintain it, so I *have*
> > to understand it.
> > 
> > But let's not assume we are throwing everything away (but equally not
> > promise that we won't) and start with just the first (non trivial)
> > patch.  The log.
> > 
> > I think we do want a log.  I think you have identified the key
> > functionality that the log must provide.  But I think the current
> > layout is too complex.  This is something that will be persistent on
> > storage devices, so we want to get it as "right" as we can, early.
> > It is fairly cheap to throw away a log and start a new one, so fixing
> > design mistakes isn't too hard.  But we do have to support any format
> > we create indefinitely.
> > So simple is best.  Simple is most extensible.
> > 
> > As I said: A single metadata block format, with:
> >  - forward/backward linkage - and probably a link to the "first" block
> >    in the log at the time of writing
> >  - array state data - brieifly
> >  - list of data/parity block addresses and maybe checksums
> >  - list of "completed" stripes
> > 
> > The log is simply these metadata blocks mixed with the data/parity they
> > they describe.
> > Each metadata block commits all previous transactions (or maybe all but
> > the most recent).
> > The current md superblock contains the address of one of these metadata
> > blocks.
> > 
> > Does that seem reasonable?  Does that seem simpler than what you had?
> > Will that meet your perceived need?  It meets mine.
> > If we can do that, I'll apply it and we can move on to the next step.
> 
> My original layout has 3 types of blocks: super block, metadata block for data/parity,
> flush block to list completed stripes.
> 
> Compared to your single metadata block format, your format
> - delete super block. I'm fine with this. The only useful bits in my
>   super block is log tail and seq. we can overload MD superblock
>   recovery_offset/resync_offset to record seq/log tail.
> - merge metadata block and flush block to one metadata block. That is
>   the new metadata block can record a mix of data/parity/'completed'
>   stripes. I don't object to this, it's doable, but don't think this
>   makes things simpler.
> 
> If you insist, we can take this one. The new metadata block can record a
> mix of data/parity/completed stripes, but it can also just record data
> or completed stripes. So I'll change the disk format to cater this, but
> I'll always put 'completed stripes' info into a separate metadata block
> (that is not mixing with data/parity, but we still just have one
> metadata type). That way I only need slight change of current code and
> it works for the new format. We can, if necessary, let the new metadata
> records any mix of data/parity/completed stripes later. Sounds good?

Yes it does.  Having a single sort of metadata block is an important
part of the goal.  How the code actually chooses to use these is a
separate issue that can change harmlessly.

There is also the issue of what action commits a previous transaction.
I'm not sure what you had.  I'm suggesting that each metadata block
commits previous transactions.  Is that a close-enough match to what
you had?

Thanks,
NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  4:36         ` NeilBrown
@ 2015-07-10  4:52           ` Shaohua Li
  2015-07-10  5:10             ` NeilBrown
  2015-07-15  0:45           ` Shaohua Li
  1 sibling, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-10  4:52 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Fri, Jul 10, 2015 at 09:21:19AM +1000, NeilBrown wrote:
> > > On Tue, 7 Jul 2015 22:44:02 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Wed, Jul 08, 2015 at 11:56:36AM +1000, NeilBrown wrote:
> > > > > 
> > > > > Hi,
> > > > >  I made some time to look at these this morning - sorry for the delay.
> > > > > 
> > > > >  Having it all broken down with more complete comments helps a lot -
> > > > >  thanks.
> > > > > 
> > > > >  Unfortunately ... it helps confirm that I really don't like this.  It
> > > > >  seems much more complex that it should be.  There certainly are a lot
> > > > >  of details that need to be carefully considered, but the result needs
> > > > >  to be simpler.
> > > > > 
> > > > >  A big part of my concern is that you choose to avoid making proper use
> > > > >  of the current stripe cache.  Your argument is that it is already
> > > > >  too complex.  That is probably true but I don't think you decrease
> > > > >  total complexity by adding extra bits on the side that themselves are
> > > > >  complex in completely different ways.
> > > > > 
> > > > >  I would like to start small and keep it as simple as possible.  I
> > > > >  think the minimum useful functionality is closing the write-hole.
> > > > >  Anything else could be done outside raid5 (e.g. bcache), but the
> > > > >  write-hole needs integration.  Once that is closed, it may make sense
> > > > >  to add more functionality.
> > > > > 
> > > > >  To do that we need a log, and it was quite sensible for you to put
> > > > >  that first in the list of patches.  So let's start with that.
> > > > > 
> > > > >  I would like to propose a single metadata block format.  This block
> > > > >  includes the general RAID5 parameters (both 'old' and 'new' for when a
> > > > >  reshape is happening), it includes linkage to find nearby metadata.
> > > > >  It includes a list of the data written after the metadata.  And it
> > > > >  includes a list of device-addresses of parity blocks which are now safe
> > > > >  on the RAID and so any log content is now ignored.
> > > > > 
> > > > >  The general mechanism for writing to the log is:
> > > > >    - collect a list of bios.  As each arrives update the metadata
> > > > >      block with index information.
> > > > >    - when ready (there are various triggers) write the metadata block
> > > > >      with FLUSH/FUA and write all the data blocks normally.  This
> > > > >      metadata block plus data/parity blocks is a 'transaction'.
> > > > > 
> > > > >  We never trigger a log transaction until all writes for the previous
> > > > >  transaction have completed. This means that once the metadata block is
> > > > >  safe, all previous data must also be safe.  If this imposes too much
> > > > >  waiting we could allow a "double-buffer" approach were each metadata
> > > > >  block completes the previous-but-one transaction.
> > > > > 
> > > > >  When all of the previous writes complete we trigger a new log write
> > > > >  if there is an outstanding SYNC write, or maybe an outstanding
> > > > >  FLUSH/FUA, but otherwise we wait until the metadata block is full, or
> > > > >  some amount of time has passed.
> > > > > 
> > > > >  This one metadata block serves all of the purposes that you
> > > > >  identified.  It records where data is, it commits previous writes, and
> > > > >  records which stripes are being written to RAID and which have been
> > > > >  fully committed.
> > > > > 
> > > > >  With that log in place, we add the code to "hijack ops_run_io" to
> > > > >  write all dirty blocks from the stripe_head to the cache.  This
> > > > >  includes the parity of course.  Immediately that data is safe the
> > > > >  write requests can be returned and the data is written the the RAID
> > > > >  devices.
> > > > > 
> > > > >  "Recovery" at boot time simply involves:
> > > > >   1. find and load first metadata block
> > > > >   2. load the "next" metadata block. If it doesn't exist, stop.
> > > > >   3. For each parity/data block listed in current block, get the
> > > > >      stripe_head and read the block in.
> > > > >   4. For each completed stripe listed, find the stripe_head and
> > > > >      invalidate it.
> > > > >   5. Make the "next" metadata block the current block and goto 2
> > > > >   6. Flush out all dirty data in stripe cache.
> > > > > 
> > > > > 
> > > > >  I think that all of this *needs* to use the stripe_cache.
> > > > >  Intercepting normal processing between party computation and writing
> > > > >  to the RAID devices must involve the stripe_cache.
> > > > > 
> > > > >  So this is a "simple" start.   Once this is written and agreed and
> > > > >  tested and debugged, then we can look at the latency-hiding side of
> > > > >  the cache.  I think this should still use the stripe_cache - just
> > > > >  schedule the writes to the log a bit earlier.
> > > > >  If a case can be made for a separate cache then I'm not completely
> > > > >  closed to the idea but I don't really want to think about it until the
> > > > >  write-hole closing code is done.
> > > > > 
> > > > > So:
> > > > >   [PATCH V4 01/13] MD: add a new disk role to present cache device
> > > > >     probably OK
> > > > >   [PATCH V4 02/13] raid5: directly use mddev->queue
> > > > >     OK
> > > > >   [PATCH V4 03/13] raid5: cache log handling
> > > > >     strip this down to a bare minimum.  It needs:
> > > > >       -on-disk metadata block format
> > > > >       -two or three active transactions which identify a list of bios
> > > > >        that are currently in the stripe cache.  As the bio is added,
> > > > >        its address info is added to the metadata block.
> > > > >       -interface to "add a bio to a transaction" and "record a
> > > > >        completed stripe".
> > > > >       -timer flush the current transaction if previous one was not
> > > > >        empty.
> > > > > 
> > > > >      I don't think there needs to be a separate superblock in the log.
> > > > >      Each device in the array already has an 'md' superblock.  Use e.g
> > > > >      the 'recovery_offset' field in there (which is per-device and
> > > > >      meaningless for a log) to store the address of the most recent
> > > > >      metadata blocks at the time the superblock was written.  Search
> > > > >      forward and backward from there to find whole log.  Each metadata
> > > > >      block holds everything else that might be useful.
> > > > > 
> > > > >   [PATCH V4 05/13] raid5: cache reclaim support
> > > > >     Just want the "hijack ops_run_io" part of this so that when normal
> > > > >     RAID5 processing wants to write a stripe, blocks get diverted to
> > > > >     the log first and further processing of the stripe is delayed until
> > > > >     those writes complete and the transaction is safe.
> > > > > 
> > > > >   [PATCH V4 08/13] raid5: cache recovery support
> > > > >     Just load all the log into the stripe cache.
> > > > > 
> > > > > 
> > > > > That should be much more manageable and would be a good start towards
> > > > > getting all the functionality that you want.
> > > > > 
> > > > > Feel free to ask questions if I haven't explained things, or if you
> > > > > want me to look over the metadata format or whatever.  I tend to answer
> > > > > direct questions more quickly than I answer 100kB patchsets :-)
> > > > Hi,
> > > > 
> > > > So we are back to the original point (eg, only fix the write hole issue) after
> > > > 3 months development, which is really disappointed. I don't object just fixing
> > > > the write hole issue as a start, it's simple which is my arguement of first
> > > > post too, and probably a slight change of patch 3 meets the goal. The cache
> > > > does significantly increase complexity, which is well known when we move from
> > > > 'just fix write hole' to 'fix write hole and do cache'. But the code is right
> > > > there, I'm wondering how bad it is. Just because of stripe_cache? I don't see
> > > > any significant technical merit stripe_cache is a must-have here. You said the
> > > > code is complex. Yes, it is, but it's not because of the new data cache. It's a
> > > > simple radix tree and takes less than 20% of the total code. The remaining code
> > > > does what any log device should do (write to log, reclaim when log is full,
> > > > recovery when log crash). Similar things must be done even stripe_cache is
> > > > used. And if we don't use stripe_cache, we don't deeply couple with current
> > > > state machine, the new data cache actually brings flexibility (for example, we
> > > > can choose not using cache, or we can easily put the cache to NVRAM).
> > > 
> > > I don't think we are completely back to the start.  The product of
> > > recent months isn't just code - it is also your understanding and
> > > expertise.   You've explored the problems and understand them.  You
> > > could quickly see issues because you have struggled with them already.
> > > 
> > > I think the principal of "write one to throw away" is a good one - and
> > > probably one we don't apply often enough.  The first time you implement
> > > something you don't really know what you are doing, so the result is
> > > bound to be imperfect.  The second time you have a much better idea of
> > > the big picture.
> > > 
> > > The real big problem that I have with the current proposal is that I
> > > just don't understand it.  Maybe that is my failing.  But all the
> > > evidence suggests that I'll end up needing to maintain it, so I *have*
> > > to understand it.
> > > 
> > > But let's not assume we are throwing everything away (but equally not
> > > promise that we won't) and start with just the first (non trivial)
> > > patch.  The log.
> > > 
> > > I think we do want a log.  I think you have identified the key
> > > functionality that the log must provide.  But I think the current
> > > layout is too complex.  This is something that will be persistent on
> > > storage devices, so we want to get it as "right" as we can, early.
> > > It is fairly cheap to throw away a log and start a new one, so fixing
> > > design mistakes isn't too hard.  But we do have to support any format
> > > we create indefinitely.
> > > So simple is best.  Simple is most extensible.
> > > 
> > > As I said: A single metadata block format, with:
> > >  - forward/backward linkage - and probably a link to the "first" block
> > >    in the log at the time of writing
> > >  - array state data - brieifly
> > >  - list of data/parity block addresses and maybe checksums
> > >  - list of "completed" stripes
> > > 
> > > The log is simply these metadata blocks mixed with the data/parity they
> > > they describe.
> > > Each metadata block commits all previous transactions (or maybe all but
> > > the most recent).
> > > The current md superblock contains the address of one of these metadata
> > > blocks.
> > > 
> > > Does that seem reasonable?  Does that seem simpler than what you had?
> > > Will that meet your perceived need?  It meets mine.
> > > If we can do that, I'll apply it and we can move on to the next step.
> > 
> > My original layout has 3 types of blocks: super block, metadata block for data/parity,
> > flush block to list completed stripes.
> > 
> > Compared to your single metadata block format, your format
> > - delete super block. I'm fine with this. The only useful bits in my
> >   super block is log tail and seq. we can overload MD superblock
> >   recovery_offset/resync_offset to record seq/log tail.
> > - merge metadata block and flush block to one metadata block. That is
> >   the new metadata block can record a mix of data/parity/'completed'
> >   stripes. I don't object to this, it's doable, but don't think this
> >   makes things simpler.
> > 
> > If you insist, we can take this one. The new metadata block can record a
> > mix of data/parity/completed stripes, but it can also just record data
> > or completed stripes. So I'll change the disk format to cater this, but
> > I'll always put 'completed stripes' info into a separate metadata block
> > (that is not mixing with data/parity, but we still just have one
> > metadata type). That way I only need slight change of current code and
> > it works for the new format. We can, if necessary, let the new metadata
> > records any mix of data/parity/completed stripes later. Sounds good?
> 
> Yes it does.  Having a single sort of metadata block is an important
> part of the goal.  How the code actually chooses to use these is a
> separate issue that can change harmlessly.
> 
> There is also the issue of what action commits a previous transaction.
> I'm not sure what you had.  I'm suggesting that each metadata block
> commits previous transactions.  Is that a close-enough match to what
> you had?

What did you mean about a transaction? In my implementation, metadata
block and followed stripe data/parity consist of an io unit. io units can
be finished out of order. but if io unit has flush request (the data has
flush/flush bio or metadata is a flush block), the io unit can only
start after all previous io units and disk cache flush finish. Such io
unit is strictly ordered. The log patch describes this behavior. Does it
match?

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  4:52           ` Shaohua Li
@ 2015-07-10  5:10             ` NeilBrown
  2015-07-10  5:18               ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-10  5:10 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, 9 Jul 2015 21:52:43 -0700 Shaohua Li <shli@fb.com> wrote:

> On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> > 

> > There is also the issue of what action commits a previous transaction.
> > I'm not sure what you had.  I'm suggesting that each metadata block
> > commits previous transactions.  Is that a close-enough match to what
> > you had?
> 
> What did you mean about a transaction? In my implementation, metadata
> block and followed stripe data/parity consist of an io unit. io units can
> be finished out of order. but if io unit has flush request (the data has
> flush/flush bio or metadata is a flush block), the io unit can only
> start after all previous io units and disk cache flush finish. Such io
> unit is strictly ordered. The log patch describes this behavior. Does it
> match?

Yes, a "transaction" is an "io unit".  The flushing is the same.
I just couldn't remember how, when reading the log on restart, you
determined if a given "io unit" was reliably consistent, or whether it
should be ignored (having possibly only partially been written).

Thanks,
NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  5:10             ` NeilBrown
@ 2015-07-10  5:18               ` Shaohua Li
  2015-07-10  6:42                 ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-10  5:18 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Fri, Jul 10, 2015 at 03:10:44PM +1000, NeilBrown wrote:
> On Thu, 9 Jul 2015 21:52:43 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > > On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> 
> > > There is also the issue of what action commits a previous transaction.
> > > I'm not sure what you had.  I'm suggesting that each metadata block
> > > commits previous transactions.  Is that a close-enough match to what
> > > you had?
> > 
> > What did you mean about a transaction? In my implementation, metadata
> > block and followed stripe data/parity consist of an io unit. io units can
> > be finished out of order. but if io unit has flush request (the data has
> > flush/flush bio or metadata is a flush block), the io unit can only
> > start after all previous io units and disk cache flush finish. Such io
> > unit is strictly ordered. The log patch describes this behavior. Does it
> > match?
> 
> Yes, a "transaction" is an "io unit".  The flushing is the same.
> I just couldn't remember how, when reading the log on restart, you
> determined if a given "io unit" was reliably consistent, or whether it
> should be ignored (having possibly only partially been written).

The metadata block has a checksum for data of the block. data/parity has
checksum stored in metadata block. This way we can know if metadata and
data is consistent.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  5:18               ` Shaohua Li
@ 2015-07-10  6:42                 ` NeilBrown
  2015-07-10 17:48                   ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-10  6:42 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, 9 Jul 2015 22:18:15 -0700 Shaohua Li <shli@fb.com> wrote:

> On Fri, Jul 10, 2015 at 03:10:44PM +1000, NeilBrown wrote:
> > On Thu, 9 Jul 2015 21:52:43 -0700 Shaohua Li <shli@fb.com> wrote:
> > 
> > > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > > > On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > 
> > 
> > > > There is also the issue of what action commits a previous transaction.
> > > > I'm not sure what you had.  I'm suggesting that each metadata block
> > > > commits previous transactions.  Is that a close-enough match to what
> > > > you had?
> > > 
> > > What did you mean about a transaction? In my implementation, metadata
> > > block and followed stripe data/parity consist of an io unit. io units can
> > > be finished out of order. but if io unit has flush request (the data has
> > > flush/flush bio or metadata is a flush block), the io unit can only
> > > start after all previous io units and disk cache flush finish. Such io
> > > unit is strictly ordered. The log patch describes this behavior. Does it
> > > match?
> > 
> > Yes, a "transaction" is an "io unit".  The flushing is the same.
> > I just couldn't remember how, when reading the log on restart, you
> > determined if a given "io unit" was reliably consistent, or whether it
> > should be ignored (having possibly only partially been written).
> 
> The metadata block has a checksum for data of the block. data/parity has
> checksum stored in metadata block. This way we can know if metadata and
> data is consistent.
> 

OK .. though I'm not totally sold on the value of checksums.  When a
checksum doesn't match, that means something.  When a checksum does
match, it could just be a co-incidence.
I'd rather have a process that made checksums unnecessary, and only use
the checksums as a double-check.

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  6:42                 ` NeilBrown
@ 2015-07-10 17:48                   ` Shaohua Li
  2015-07-13 22:22                     ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-10 17:48 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Fri, Jul 10, 2015 at 04:42:09PM +1000, NeilBrown wrote:
> On Thu, 9 Jul 2015 22:18:15 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Fri, Jul 10, 2015 at 03:10:44PM +1000, NeilBrown wrote:
> > > On Thu, 9 Jul 2015 21:52:43 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > > > > On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > 
> > > 
> > > > > There is also the issue of what action commits a previous transaction.
> > > > > I'm not sure what you had.  I'm suggesting that each metadata block
> > > > > commits previous transactions.  Is that a close-enough match to what
> > > > > you had?
> > > > 
> > > > What did you mean about a transaction? In my implementation, metadata
> > > > block and followed stripe data/parity consist of an io unit. io units can
> > > > be finished out of order. but if io unit has flush request (the data has
> > > > flush/flush bio or metadata is a flush block), the io unit can only
> > > > start after all previous io units and disk cache flush finish. Such io
> > > > unit is strictly ordered. The log patch describes this behavior. Does it
> > > > match?
> > > 
> > > Yes, a "transaction" is an "io unit".  The flushing is the same.
> > > I just couldn't remember how, when reading the log on restart, you
> > > determined if a given "io unit" was reliably consistent, or whether it
> > > should be ignored (having possibly only partially been written).
> > 
> > The metadata block has a checksum for data of the block. data/parity has
> > checksum stored in metadata block. This way we can know if metadata and
> > data is consistent.
> > 
> 
> OK .. though I'm not totally sold on the value of checksums.  When a
> checksum doesn't match, that means something.  When a checksum does
> match, it could just be a co-incidence.
> I'd rather have a process that made checksums unnecessary, and only use
> the checksums as a double-check.

We could do something like: write metadata/data, wait, write another
metadata. the second metadata indicates the first is in disk. But this
can impact performance very much. I think checksum should be fine. It
might be just a coninsidence, but the rate should extremely low. jbd2 is
using checksum too now.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10 17:48                   ` Shaohua Li
@ 2015-07-13 22:22                     ` NeilBrown
  2015-07-13 22:35                       ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-13 22:22 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Fri, 10 Jul 2015 10:48:45 -0700 Shaohua Li <shli@fb.com> wrote:

> On Fri, Jul 10, 2015 at 04:42:09PM +1000, NeilBrown wrote:
> > On Thu, 9 Jul 2015 22:18:15 -0700 Shaohua Li <shli@fb.com> wrote:
> > 
> > > On Fri, Jul 10, 2015 at 03:10:44PM +1000, NeilBrown wrote:
> > > > On Thu, 9 Jul 2015 21:52:43 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > 
> > > > > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > > > > > On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > > 
> > > > 
> > > > > > There is also the issue of what action commits a previous transaction.
> > > > > > I'm not sure what you had.  I'm suggesting that each metadata block
> > > > > > commits previous transactions.  Is that a close-enough match to what
> > > > > > you had?
> > > > > 
> > > > > What did you mean about a transaction? In my implementation, metadata
> > > > > block and followed stripe data/parity consist of an io unit. io units can
> > > > > be finished out of order. but if io unit has flush request (the data has
> > > > > flush/flush bio or metadata is a flush block), the io unit can only
> > > > > start after all previous io units and disk cache flush finish. Such io
> > > > > unit is strictly ordered. The log patch describes this behavior. Does it
> > > > > match?
> > > > 
> > > > Yes, a "transaction" is an "io unit".  The flushing is the same.
> > > > I just couldn't remember how, when reading the log on restart, you
> > > > determined if a given "io unit" was reliably consistent, or whether it
> > > > should be ignored (having possibly only partially been written).
> > > 
> > > The metadata block has a checksum for data of the block. data/parity has
> > > checksum stored in metadata block. This way we can know if metadata and
> > > data is consistent.
> > > 
> > 
> > OK .. though I'm not totally sold on the value of checksums.  When a
> > checksum doesn't match, that means something.  When a checksum does
> > match, it could just be a co-incidence.
> > I'd rather have a process that made checksums unnecessary, and only use
> > the checksums as a double-check.
> 
> We could do something like: write metadata/data, wait, write another
> metadata. the second metadata indicates the first is in disk. But this
> can impact performance very much. 

The performance consideration is why I suggested a double-buffered
approach.  Write metadata1, data1, metadata2, data2, then don't write
metdata3 until metdata1 and data1 has been written.
I haven't actually tried that so I don't know for certain it would help.

>                                    I think checksum should be fine. It
> might be just a coninsidence, but the rate should extremely low. jbd2 is
> using checksum too now.

Maybe I'll have a look at jbd2 - do you know what sort of checksum it
uses?  I'd be surprised it didn't use something quite a bit stronger
than crc32 for a task like this.

NeilBrown


> 
> Thanks,
> Shaohua
> --
> To unsubscribe from this list: send the line "unsubscribe linux-raid" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html


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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-13 22:22                     ` NeilBrown
@ 2015-07-13 22:35                       ` Shaohua Li
  0 siblings, 0 replies; 44+ messages in thread
From: Shaohua Li @ 2015-07-13 22:35 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Tue, Jul 14, 2015 at 08:22:54AM +1000, NeilBrown wrote:
> On Fri, 10 Jul 2015 10:48:45 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Fri, Jul 10, 2015 at 04:42:09PM +1000, NeilBrown wrote:
> > > On Thu, 9 Jul 2015 22:18:15 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Fri, Jul 10, 2015 at 03:10:44PM +1000, NeilBrown wrote:
> > > > > On Thu, 9 Jul 2015 21:52:43 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > 
> > > > > > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > > > > > > On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > > > 
> > > > > 
> > > > > > > There is also the issue of what action commits a previous transaction.
> > > > > > > I'm not sure what you had.  I'm suggesting that each metadata block
> > > > > > > commits previous transactions.  Is that a close-enough match to what
> > > > > > > you had?
> > > > > > 
> > > > > > What did you mean about a transaction? In my implementation, metadata
> > > > > > block and followed stripe data/parity consist of an io unit. io units can
> > > > > > be finished out of order. but if io unit has flush request (the data has
> > > > > > flush/flush bio or metadata is a flush block), the io unit can only
> > > > > > start after all previous io units and disk cache flush finish. Such io
> > > > > > unit is strictly ordered. The log patch describes this behavior. Does it
> > > > > > match?
> > > > > 
> > > > > Yes, a "transaction" is an "io unit".  The flushing is the same.
> > > > > I just couldn't remember how, when reading the log on restart, you
> > > > > determined if a given "io unit" was reliably consistent, or whether it
> > > > > should be ignored (having possibly only partially been written).
> > > > 
> > > > The metadata block has a checksum for data of the block. data/parity has
> > > > checksum stored in metadata block. This way we can know if metadata and
> > > > data is consistent.
> > > > 
> > > 
> > > OK .. though I'm not totally sold on the value of checksums.  When a
> > > checksum doesn't match, that means something.  When a checksum does
> > > match, it could just be a co-incidence.
> > > I'd rather have a process that made checksums unnecessary, and only use
> > > the checksums as a double-check.
> > 
> > We could do something like: write metadata/data, wait, write another
> > metadata. the second metadata indicates the first is in disk. But this
> > can impact performance very much. 
> 
> The performance consideration is why I suggested a double-buffered
> approach.  Write metadata1, data1, metadata2, data2, then don't write
> metdata3 until metdata1 and data1 has been written.
> I haven't actually tried that so I don't know for certain it would help.

Not sure if double buffer works, but you can't write metadata1 till
data1 hits to disk, which has big penality. The only possible way is to
origanize data/metadata as a big transaction so wait doesn't hurt too
much like jbd does.

> >                                    I think checksum should be fine. It
> > might be just a coninsidence, but the rate should extremely low. jbd2 is
> > using checksum too now.
> 
> Maybe I'll have a look at jbd2 - do you know what sort of checksum it
> uses?  I'd be surprised it didn't use something quite a bit stronger
> than crc32 for a task like this.

It uses crc32. 32bits checksum for every 4k as far as I check.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-10  4:36         ` NeilBrown
  2015-07-10  4:52           ` Shaohua Li
@ 2015-07-15  0:45           ` Shaohua Li
  2015-07-15  2:12             ` NeilBrown
  1 sibling, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-15  0:45 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> On Thu, 9 Jul 2015 21:08:49 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Fri, Jul 10, 2015 at 09:21:19AM +1000, NeilBrown wrote:
> > > On Tue, 7 Jul 2015 22:44:02 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Wed, Jul 08, 2015 at 11:56:36AM +1000, NeilBrown wrote:
> > > > > 
> > > > > Hi,
> > > > >  I made some time to look at these this morning - sorry for the delay.
> > > > > 
> > > > >  Having it all broken down with more complete comments helps a lot -
> > > > >  thanks.
> > > > > 
> > > > >  Unfortunately ... it helps confirm that I really don't like this.  It
> > > > >  seems much more complex that it should be.  There certainly are a lot
> > > > >  of details that need to be carefully considered, but the result needs
> > > > >  to be simpler.
> > > > > 
> > > > >  A big part of my concern is that you choose to avoid making proper use
> > > > >  of the current stripe cache.  Your argument is that it is already
> > > > >  too complex.  That is probably true but I don't think you decrease
> > > > >  total complexity by adding extra bits on the side that themselves are
> > > > >  complex in completely different ways.
> > > > > 
> > > > >  I would like to start small and keep it as simple as possible.  I
> > > > >  think the minimum useful functionality is closing the write-hole.
> > > > >  Anything else could be done outside raid5 (e.g. bcache), but the
> > > > >  write-hole needs integration.  Once that is closed, it may make sense
> > > > >  to add more functionality.
> > > > > 
> > > > >  To do that we need a log, and it was quite sensible for you to put
> > > > >  that first in the list of patches.  So let's start with that.
> > > > > 
> > > > >  I would like to propose a single metadata block format.  This block
> > > > >  includes the general RAID5 parameters (both 'old' and 'new' for when a
> > > > >  reshape is happening), it includes linkage to find nearby metadata.
> > > > >  It includes a list of the data written after the metadata.  And it
> > > > >  includes a list of device-addresses of parity blocks which are now safe
> > > > >  on the RAID and so any log content is now ignored.
> > > > > 
> > > > >  The general mechanism for writing to the log is:
> > > > >    - collect a list of bios.  As each arrives update the metadata
> > > > >      block with index information.
> > > > >    - when ready (there are various triggers) write the metadata block
> > > > >      with FLUSH/FUA and write all the data blocks normally.  This
> > > > >      metadata block plus data/parity blocks is a 'transaction'.
> > > > > 
> > > > >  We never trigger a log transaction until all writes for the previous
> > > > >  transaction have completed. This means that once the metadata block is
> > > > >  safe, all previous data must also be safe.  If this imposes too much
> > > > >  waiting we could allow a "double-buffer" approach were each metadata
> > > > >  block completes the previous-but-one transaction.
> > > > > 
> > > > >  When all of the previous writes complete we trigger a new log write
> > > > >  if there is an outstanding SYNC write, or maybe an outstanding
> > > > >  FLUSH/FUA, but otherwise we wait until the metadata block is full, or
> > > > >  some amount of time has passed.
> > > > > 
> > > > >  This one metadata block serves all of the purposes that you
> > > > >  identified.  It records where data is, it commits previous writes, and
> > > > >  records which stripes are being written to RAID and which have been
> > > > >  fully committed.
> > > > > 
> > > > >  With that log in place, we add the code to "hijack ops_run_io" to
> > > > >  write all dirty blocks from the stripe_head to the cache.  This
> > > > >  includes the parity of course.  Immediately that data is safe the
> > > > >  write requests can be returned and the data is written the the RAID
> > > > >  devices.
> > > > > 
> > > > >  "Recovery" at boot time simply involves:
> > > > >   1. find and load first metadata block
> > > > >   2. load the "next" metadata block. If it doesn't exist, stop.
> > > > >   3. For each parity/data block listed in current block, get the
> > > > >      stripe_head and read the block in.
> > > > >   4. For each completed stripe listed, find the stripe_head and
> > > > >      invalidate it.
> > > > >   5. Make the "next" metadata block the current block and goto 2
> > > > >   6. Flush out all dirty data in stripe cache.
> > > > > 
> > > > > 
> > > > >  I think that all of this *needs* to use the stripe_cache.
> > > > >  Intercepting normal processing between party computation and writing
> > > > >  to the RAID devices must involve the stripe_cache.
> > > > > 
> > > > >  So this is a "simple" start.   Once this is written and agreed and
> > > > >  tested and debugged, then we can look at the latency-hiding side of
> > > > >  the cache.  I think this should still use the stripe_cache - just
> > > > >  schedule the writes to the log a bit earlier.
> > > > >  If a case can be made for a separate cache then I'm not completely
> > > > >  closed to the idea but I don't really want to think about it until the
> > > > >  write-hole closing code is done.
> > > > > 
> > > > > So:
> > > > >   [PATCH V4 01/13] MD: add a new disk role to present cache device
> > > > >     probably OK
> > > > >   [PATCH V4 02/13] raid5: directly use mddev->queue
> > > > >     OK
> > > > >   [PATCH V4 03/13] raid5: cache log handling
> > > > >     strip this down to a bare minimum.  It needs:
> > > > >       -on-disk metadata block format
> > > > >       -two or three active transactions which identify a list of bios
> > > > >        that are currently in the stripe cache.  As the bio is added,
> > > > >        its address info is added to the metadata block.
> > > > >       -interface to "add a bio to a transaction" and "record a
> > > > >        completed stripe".
> > > > >       -timer flush the current transaction if previous one was not
> > > > >        empty.
> > > > > 
> > > > >      I don't think there needs to be a separate superblock in the log.
> > > > >      Each device in the array already has an 'md' superblock.  Use e.g
> > > > >      the 'recovery_offset' field in there (which is per-device and
> > > > >      meaningless for a log) to store the address of the most recent
> > > > >      metadata blocks at the time the superblock was written.  Search
> > > > >      forward and backward from there to find whole log.  Each metadata
> > > > >      block holds everything else that might be useful.
> > > > > 
> > > > >   [PATCH V4 05/13] raid5: cache reclaim support
> > > > >     Just want the "hijack ops_run_io" part of this so that when normal
> > > > >     RAID5 processing wants to write a stripe, blocks get diverted to
> > > > >     the log first and further processing of the stripe is delayed until
> > > > >     those writes complete and the transaction is safe.
> > > > > 
> > > > >   [PATCH V4 08/13] raid5: cache recovery support
> > > > >     Just load all the log into the stripe cache.
> > > > > 
> > > > > 
> > > > > That should be much more manageable and would be a good start towards
> > > > > getting all the functionality that you want.
> > > > > 
> > > > > Feel free to ask questions if I haven't explained things, or if you
> > > > > want me to look over the metadata format or whatever.  I tend to answer
> > > > > direct questions more quickly than I answer 100kB patchsets :-)
> > > > Hi,
> > > > 
> > > > So we are back to the original point (eg, only fix the write hole issue) after
> > > > 3 months development, which is really disappointed. I don't object just fixing
> > > > the write hole issue as a start, it's simple which is my arguement of first
> > > > post too, and probably a slight change of patch 3 meets the goal. The cache
> > > > does significantly increase complexity, which is well known when we move from
> > > > 'just fix write hole' to 'fix write hole and do cache'. But the code is right
> > > > there, I'm wondering how bad it is. Just because of stripe_cache? I don't see
> > > > any significant technical merit stripe_cache is a must-have here. You said the
> > > > code is complex. Yes, it is, but it's not because of the new data cache. It's a
> > > > simple radix tree and takes less than 20% of the total code. The remaining code
> > > > does what any log device should do (write to log, reclaim when log is full,
> > > > recovery when log crash). Similar things must be done even stripe_cache is
> > > > used. And if we don't use stripe_cache, we don't deeply couple with current
> > > > state machine, the new data cache actually brings flexibility (for example, we
> > > > can choose not using cache, or we can easily put the cache to NVRAM).
> > > 
> > > I don't think we are completely back to the start.  The product of
> > > recent months isn't just code - it is also your understanding and
> > > expertise.   You've explored the problems and understand them.  You
> > > could quickly see issues because you have struggled with them already.
> > > 
> > > I think the principal of "write one to throw away" is a good one - and
> > > probably one we don't apply often enough.  The first time you implement
> > > something you don't really know what you are doing, so the result is
> > > bound to be imperfect.  The second time you have a much better idea of
> > > the big picture.
> > > 
> > > The real big problem that I have with the current proposal is that I
> > > just don't understand it.  Maybe that is my failing.  But all the
> > > evidence suggests that I'll end up needing to maintain it, so I *have*
> > > to understand it.
> > > 
> > > But let's not assume we are throwing everything away (but equally not
> > > promise that we won't) and start with just the first (non trivial)
> > > patch.  The log.
> > > 
> > > I think we do want a log.  I think you have identified the key
> > > functionality that the log must provide.  But I think the current
> > > layout is too complex.  This is something that will be persistent on
> > > storage devices, so we want to get it as "right" as we can, early.
> > > It is fairly cheap to throw away a log and start a new one, so fixing
> > > design mistakes isn't too hard.  But we do have to support any format
> > > we create indefinitely.
> > > So simple is best.  Simple is most extensible.
> > > 
> > > As I said: A single metadata block format, with:
> > >  - forward/backward linkage - and probably a link to the "first" block
> > >    in the log at the time of writing
> > >  - array state data - brieifly
> > >  - list of data/parity block addresses and maybe checksums
> > >  - list of "completed" stripes
> > > 
> > > The log is simply these metadata blocks mixed with the data/parity they
> > > they describe.
> > > Each metadata block commits all previous transactions (or maybe all but
> > > the most recent).
> > > The current md superblock contains the address of one of these metadata
> > > blocks.
> > > 
> > > Does that seem reasonable?  Does that seem simpler than what you had?
> > > Will that meet your perceived need?  It meets mine.
> > > If we can do that, I'll apply it and we can move on to the next step.
> > 
> > My original layout has 3 types of blocks: super block, metadata block for data/parity,
> > flush block to list completed stripes.
> > 
> > Compared to your single metadata block format, your format
> > - delete super block. I'm fine with this. The only useful bits in my
> >   super block is log tail and seq. we can overload MD superblock
> >   recovery_offset/resync_offset to record seq/log tail.
> > - merge metadata block and flush block to one metadata block. That is
> >   the new metadata block can record a mix of data/parity/'completed'
> >   stripes. I don't object to this, it's doable, but don't think this
> >   makes things simpler.
> > 
> > If you insist, we can take this one. The new metadata block can record a
> > mix of data/parity/completed stripes, but it can also just record data
> > or completed stripes. So I'll change the disk format to cater this, but
> > I'll always put 'completed stripes' info into a separate metadata block
> > (that is not mixing with data/parity, but we still just have one
> > metadata type). That way I only need slight change of current code and
> > it works for the new format. We can, if necessary, let the new metadata
> > records any mix of data/parity/completed stripes later. Sounds good?
> 
> Yes it does.  Having a single sort of metadata block is an important
> part of the goal.  How the code actually chooses to use these is a
> separate issue that can change harmlessly.

Taking a close look to reuse MD superblock for caching. It turns out to
be quite hacky. Suppose I use md_update_sb to update superblock when we
checkpoint the log. So I update corresponding fields of mddev
(resync_offset, recovery_offset). In md_update_sb, I must add a bunch of
'if (caching_disk) xxx' as raid disks shouldn't store the
resync_offset/recovery_offset. Or I can add a new cache_update_sb, but I
thought I must add the same hack code if we don't duplicate a lot of
code.

Adding a superblock for caching looks better. A 4k page data doesn't
increase any complexity. If you think we shouldn't duplicate too fileds
of md superblock to the cache superblock, we can store the most
necessary data in cache superblock.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-15  0:45           ` Shaohua Li
@ 2015-07-15  2:12             ` NeilBrown
  2015-07-15  3:16               ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-15  2:12 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Tue, 14 Jul 2015 17:45:04 -0700 Shaohua Li <shli@fb.com> wrote:

> On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:

> > Yes it does.  Having a single sort of metadata block is an important
> > part of the goal.  How the code actually chooses to use these is a
> > separate issue that can change harmlessly.
> 
> Taking a close look to reuse MD superblock for caching. It turns out to
> be quite hacky. Suppose I use md_update_sb to update superblock when we
> checkpoint the log. So I update corresponding fields of mddev
> (resync_offset, recovery_offset). In md_update_sb, I must add a bunch of
> 'if (caching_disk) xxx' as raid disks shouldn't store the
> resync_offset/recovery_offset. Or I can add a new cache_update_sb, but I
> thought I must add the same hack code if we don't duplicate a lot of
> code.

in md_update_sb, in the loop:


	/* First make sure individual recovery_offsets are correct */
	rdev_for_each(rdev, mddev) {
		if (rdev->raid_disk >= 0 &&
		    mddev->delta_disks >= 0 &&
		    !test_bit(In_sync, &rdev->flags) &&
		    mddev->curr_resync_completed > rdev->recovery_offset)
				rdev->recovery_offset = mddev->curr_resync_completed;

	}

add something like:
               else if (rdev->is_cache)
                        rdev->recovery_offset =
                        mddev->cache->latest_checkpoint


In super_1_sync, where the code:

	if (rdev->raid_disk >= 0 &&
	    !test_bit(In_sync, &rdev->flags)) {
		sb->feature_map |=
			cpu_to_le32(MD_FEATURE_RECOVERY_OFFSET);
		sb->recovery_offset =
			cpu_to_le64(rdev->recovery_offset);
		if (rdev->saved_raid_disk >= 0 && mddev->bitmap)
			sb->feature_map |=
				cpu_to_le32(MD_FEATURE_RECOVERY_BITMAP);
	}

is, add something like
	else if (rdev->is_a_cache_disk) {
              sb->feature_map |= MD_FEATURE_IMA_CACHE;
              sb->recovery_offset = cpu_to_le64(rdev->recovery_Offset);
        }

or just make the original code a little more general - I'm not sure
exactly how you flag the cache device.

You don't need to do this every time you checkpoint the log.  The
pointer just needs to point to somewhere in the log so that the
start/end can be found (each metadata block points to the next one).
You could leave it until the log wraps completely, though that probably
isn't ideal.

So when you checkpoint the log, if the ->recovery_offset of the cache
device is more than (say) 25% behind the new checkpoint location, just
set MD_CHANGE_PENDING and wake the md thread.

I don't see that as particularly hackish.

Thanks,
NeilBrown


> 
> Adding a superblock for caching looks better. A 4k page data doesn't
> increase any complexity. If you think we shouldn't duplicate too fileds
> of md superblock to the cache superblock, we can store the most
> necessary data in cache superblock.
> 
> Thanks,
> Shaohua


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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-15  2:12             ` NeilBrown
@ 2015-07-15  3:16               ` Shaohua Li
  2015-07-15  4:06                 ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-15  3:16 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Wed, Jul 15, 2015 at 12:12:34PM +1000, NeilBrown wrote:
> On Tue, 14 Jul 2015 17:45:04 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> 
> > > Yes it does.  Having a single sort of metadata block is an important
> > > part of the goal.  How the code actually chooses to use these is a
> > > separate issue that can change harmlessly.
> > 
> > Taking a close look to reuse MD superblock for caching. It turns out to
> > be quite hacky. Suppose I use md_update_sb to update superblock when we
> > checkpoint the log. So I update corresponding fields of mddev
> > (resync_offset, recovery_offset). In md_update_sb, I must add a bunch of
> > 'if (caching_disk) xxx' as raid disks shouldn't store the
> > resync_offset/recovery_offset. Or I can add a new cache_update_sb, but I
> > thought I must add the same hack code if we don't duplicate a lot of
> > code.
> 
> in md_update_sb, in the loop:
> 
> 
> 	/* First make sure individual recovery_offsets are correct */
> 	rdev_for_each(rdev, mddev) {
> 		if (rdev->raid_disk >= 0 &&
> 		    mddev->delta_disks >= 0 &&
> 		    !test_bit(In_sync, &rdev->flags) &&
> 		    mddev->curr_resync_completed > rdev->recovery_offset)
> 				rdev->recovery_offset = mddev->curr_resync_completed;
> 
> 	}
> 
> add something like:
>                else if (rdev->is_cache)
>                         rdev->recovery_offset =
>                         mddev->cache->latest_checkpoint
> 
> 
> In super_1_sync, where the code:
> 
> 	if (rdev->raid_disk >= 0 &&
> 	    !test_bit(In_sync, &rdev->flags)) {
> 		sb->feature_map |=
> 			cpu_to_le32(MD_FEATURE_RECOVERY_OFFSET);
> 		sb->recovery_offset =
> 			cpu_to_le64(rdev->recovery_offset);
> 		if (rdev->saved_raid_disk >= 0 && mddev->bitmap)
> 			sb->feature_map |=
> 				cpu_to_le32(MD_FEATURE_RECOVERY_BITMAP);
> 	}
> 
> is, add something like
> 	else if (rdev->is_a_cache_disk) {
>               sb->feature_map |= MD_FEATURE_IMA_CACHE;
>               sb->recovery_offset = cpu_to_le64(rdev->recovery_Offset);
>         }
> 
> or just make the original code a little more general - I'm not sure
> exactly how you flag the cache device.
> 
> You don't need to do this every time you checkpoint the log.  The
> pointer just needs to point to somewhere in the log so that the
> start/end can be found (each metadata block points to the next one).
> You could leave it until the log wraps completely, though that probably
> isn't ideal.
> 
> So when you checkpoint the log, if the ->recovery_offset of the cache
> device is more than (say) 25% behind the new checkpoint location, just
> set MD_CHANGE_PENDING and wake the md thread.
> 
> I don't see that as particularly hackish.

The policy above about when superblock should be written is fine with
me, but I'd like to focus on where/how superblock should be written
here. I'd say exporting a structure of cache (the
cache->latest_checkpoint) to generic MD layer is very hackish.
md_update_sb writes all raid disks, that's bad since we just want to
update cache disk. Overloading some fields of MD superblock and using
them with some 'if (cache)' stuff is not natural way too. I don't
understand why you object adding a superblock for cache. The advantage
is it's self contained. And there is nothing about
complexity/maintaince, as we can store the most necessary fields into
the superblock.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-15  3:16               ` Shaohua Li
@ 2015-07-15  4:06                 ` NeilBrown
  2015-07-15 19:49                   ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-15  4:06 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:

> On Wed, Jul 15, 2015 at 12:12:34PM +1000, NeilBrown wrote:
> > On Tue, 14 Jul 2015 17:45:04 -0700 Shaohua Li <shli@fb.com> wrote:
> > 
> > > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > 
> > > > Yes it does.  Having a single sort of metadata block is an important
> > > > part of the goal.  How the code actually chooses to use these is a
> > > > separate issue that can change harmlessly.
> > > 
> > > Taking a close look to reuse MD superblock for caching. It turns out to
> > > be quite hacky. Suppose I use md_update_sb to update superblock when we
> > > checkpoint the log. So I update corresponding fields of mddev
> > > (resync_offset, recovery_offset). In md_update_sb, I must add a bunch of
> > > 'if (caching_disk) xxx' as raid disks shouldn't store the
> > > resync_offset/recovery_offset. Or I can add a new cache_update_sb, but I
> > > thought I must add the same hack code if we don't duplicate a lot of
> > > code.
> > 
> > in md_update_sb, in the loop:
> > 
> > 
> > 	/* First make sure individual recovery_offsets are correct */
> > 	rdev_for_each(rdev, mddev) {
> > 		if (rdev->raid_disk >= 0 &&
> > 		    mddev->delta_disks >= 0 &&
> > 		    !test_bit(In_sync, &rdev->flags) &&
> > 		    mddev->curr_resync_completed > rdev->recovery_offset)
> > 				rdev->recovery_offset = mddev->curr_resync_completed;
> > 
> > 	}
> > 
> > add something like:
> >                else if (rdev->is_cache)
> >                         rdev->recovery_offset =
> >                         mddev->cache->latest_checkpoint
> > 
> > 
> > In super_1_sync, where the code:
> > 
> > 	if (rdev->raid_disk >= 0 &&
> > 	    !test_bit(In_sync, &rdev->flags)) {
> > 		sb->feature_map |=
> > 			cpu_to_le32(MD_FEATURE_RECOVERY_OFFSET);
> > 		sb->recovery_offset =
> > 			cpu_to_le64(rdev->recovery_offset);
> > 		if (rdev->saved_raid_disk >= 0 && mddev->bitmap)
> > 			sb->feature_map |=
> > 				cpu_to_le32(MD_FEATURE_RECOVERY_BITMAP);
> > 	}
> > 
> > is, add something like
> > 	else if (rdev->is_a_cache_disk) {
> >               sb->feature_map |= MD_FEATURE_IMA_CACHE;
> >               sb->recovery_offset = cpu_to_le64(rdev->recovery_Offset);
> >         }
> > 
> > or just make the original code a little more general - I'm not sure
> > exactly how you flag the cache device.
> > 
> > You don't need to do this every time you checkpoint the log.  The
> > pointer just needs to point to somewhere in the log so that the
> > start/end can be found (each metadata block points to the next one).
> > You could leave it until the log wraps completely, though that probably
> > isn't ideal.
> > 
> > So when you checkpoint the log, if the ->recovery_offset of the cache
> > device is more than (say) 25% behind the new checkpoint location, just
> > set MD_CHANGE_PENDING and wake the md thread.
> > 
> > I don't see that as particularly hackish.
> 
> The policy above about when superblock should be written is fine with
> me, but I'd like to focus on where/how superblock should be written
> here. I'd say exporting a structure of cache (the
> cache->latest_checkpoint) to generic MD layer is very hackish.

OK, get the cache code to write the desired value into the
recovery_offset field, so the md code only has to look at that.
The core md code does still need to know there is a cache, and which is
the cache device - it cannot be completely unaware...

> md_update_sb writes all raid disks, that's bad since we just want to
> update cache disk.

How bad?  How often?  Would you really be able to notice?

And having a per-device "update superblock" flag is not completely out
of the question.  RAID10 could benefit from the clean/dirty state being
localized to the device which was actually dirty.

>                    Overloading some fields of MD superblock and using
> them with some 'if (cache)' stuff is not natural way too.

If we could foresee everything, we could assign everything its own
field.  But unfortunately I didn't.  That is why we have feature bits.
Different feature bits mean different fields have different meanings.


>                                                           I don't
> understand why you object adding a superblock for cache. The advantage
> is it's self contained. And there is nothing about
> complexity/maintaince, as we can store the most necessary fields into
> the superblock.

Because there is precisely 1 number that needs to be stored in the
superblock, and there seems no point having a superblock just to store
one number.
It isn't much extra complexity, but any extra thing is still an extra
thing.
Having the data section of the log device containing just a log is
elegant.  Elegant is good.
If we decided that keeping two copies for superblocks was a good idea
(which I think it is, I just haven't created a "v1.3" layout yet), then
re-using the main superblock for the head-of-log pointer would instantly
give us two copies of that as well.

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-15  4:06                 ` NeilBrown
@ 2015-07-15 19:49                   ` Shaohua Li
  2015-07-15 23:16                     ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-15 19:49 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Wed, Jul 15, 2015 at 12:12:34PM +1000, NeilBrown wrote:
> > > On Tue, 14 Jul 2015 17:45:04 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Fri, Jul 10, 2015 at 02:36:56PM +1000, NeilBrown wrote:
> > > 
> > > > > Yes it does.  Having a single sort of metadata block is an important
> > > > > part of the goal.  How the code actually chooses to use these is a
> > > > > separate issue that can change harmlessly.
> > > > 
> > > > Taking a close look to reuse MD superblock for caching. It turns out to
> > > > be quite hacky. Suppose I use md_update_sb to update superblock when we
> > > > checkpoint the log. So I update corresponding fields of mddev
> > > > (resync_offset, recovery_offset). In md_update_sb, I must add a bunch of
> > > > 'if (caching_disk) xxx' as raid disks shouldn't store the
> > > > resync_offset/recovery_offset. Or I can add a new cache_update_sb, but I
> > > > thought I must add the same hack code if we don't duplicate a lot of
> > > > code.
> > > 
> > > in md_update_sb, in the loop:
> > > 
> > > 
> > > 	/* First make sure individual recovery_offsets are correct */
> > > 	rdev_for_each(rdev, mddev) {
> > > 		if (rdev->raid_disk >= 0 &&
> > > 		    mddev->delta_disks >= 0 &&
> > > 		    !test_bit(In_sync, &rdev->flags) &&
> > > 		    mddev->curr_resync_completed > rdev->recovery_offset)
> > > 				rdev->recovery_offset = mddev->curr_resync_completed;
> > > 
> > > 	}
> > > 
> > > add something like:
> > >                else if (rdev->is_cache)
> > >                         rdev->recovery_offset =
> > >                         mddev->cache->latest_checkpoint
> > > 
> > > 
> > > In super_1_sync, where the code:
> > > 
> > > 	if (rdev->raid_disk >= 0 &&
> > > 	    !test_bit(In_sync, &rdev->flags)) {
> > > 		sb->feature_map |=
> > > 			cpu_to_le32(MD_FEATURE_RECOVERY_OFFSET);
> > > 		sb->recovery_offset =
> > > 			cpu_to_le64(rdev->recovery_offset);
> > > 		if (rdev->saved_raid_disk >= 0 && mddev->bitmap)
> > > 			sb->feature_map |=
> > > 				cpu_to_le32(MD_FEATURE_RECOVERY_BITMAP);
> > > 	}
> > > 
> > > is, add something like
> > > 	else if (rdev->is_a_cache_disk) {
> > >               sb->feature_map |= MD_FEATURE_IMA_CACHE;
> > >               sb->recovery_offset = cpu_to_le64(rdev->recovery_Offset);
> > >         }
> > > 
> > > or just make the original code a little more general - I'm not sure
> > > exactly how you flag the cache device.
> > > 
> > > You don't need to do this every time you checkpoint the log.  The
> > > pointer just needs to point to somewhere in the log so that the
> > > start/end can be found (each metadata block points to the next one).
> > > You could leave it until the log wraps completely, though that probably
> > > isn't ideal.
> > > 
> > > So when you checkpoint the log, if the ->recovery_offset of the cache
> > > device is more than (say) 25% behind the new checkpoint location, just
> > > set MD_CHANGE_PENDING and wake the md thread.
> > > 
> > > I don't see that as particularly hackish.
> > 
> > The policy above about when superblock should be written is fine with
> > me, but I'd like to focus on where/how superblock should be written
> > here. I'd say exporting a structure of cache (the
> > cache->latest_checkpoint) to generic MD layer is very hackish.
> 
> OK, get the cache code to write the desired value into the
> recovery_offset field, so the md code only has to look at that.
> The core md code does still need to know there is a cache, and which is
> the cache device - it cannot be completely unaware...
> 
> > md_update_sb writes all raid disks, that's bad since we just want to
> > update cache disk.
> 
> How bad?  How often?  Would you really be able to notice?
> 
> And having a per-device "update superblock" flag is not completely out
> of the question.  RAID10 could benefit from the clean/dirty state being
> localized to the device which was actually dirty.
> 
> >                    Overloading some fields of MD superblock and using
> > them with some 'if (cache)' stuff is not natural way too.
> 
> If we could foresee everything, we could assign everything its own
> field.  But unfortunately I didn't.  That is why we have feature bits.
> Different feature bits mean different fields have different meanings.
> 
> 
> >                                                           I don't
> > understand why you object adding a superblock for cache. The advantage
> > is it's self contained. And there is nothing about
> > complexity/maintaince, as we can store the most necessary fields into
> > the superblock.
> 
> Because there is precisely 1 number that needs to be stored in the
> superblock, and there seems no point having a superblock just to store
> one number.
> It isn't much extra complexity, but any extra thing is still an extra
> thing.
> Having the data section of the log device containing just a log is
> elegant.  Elegant is good.
> If we decided that keeping two copies for superblocks was a good idea
> (which I think it is, I just haven't created a "v1.3" layout yet), then
> re-using the main superblock for the head-of-log pointer would instantly
> give us two copies of that as well.

I think I need 2 fields to find log head/tail in recovery. Currently
cache superblock records checkpoint disk position (log tail) and
checkpoint sequence number, which can be used to find log head. Just
recording log tail doesn't work well (it might work, for example,
zeroing sectors before log head, so we can identify log head. But it's
really ugly and not efficient). I only found recovery_offset can be
overloaded. Do you have idea other fileds can be overloaded in MD
superblock?

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-15 19:49                   ` Shaohua Li
@ 2015-07-15 23:16                     ` NeilBrown
  2015-07-16  0:07                       ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-15 23:16 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Wed, 15 Jul 2015 12:49:37 -0700 Shaohua Li <shli@fb.com> wrote:

> On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> > On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> > 

> > 
> > >                                                           I don't
> > > understand why you object adding a superblock for cache. The advantage
> > > is it's self contained. And there is nothing about
> > > complexity/maintaince, as we can store the most necessary fields into
> > > the superblock.
> > 
> > Because there is precisely 1 number that needs to be stored in the
> > superblock, and there seems no point having a superblock just to store
> > one number.
> > It isn't much extra complexity, but any extra thing is still an extra
> > thing.
> > Having the data section of the log device containing just a log is
> > elegant.  Elegant is good.
> > If we decided that keeping two copies for superblocks was a good idea
> > (which I think it is, I just haven't created a "v1.3" layout yet), then
> > re-using the main superblock for the head-of-log pointer would instantly
> > give us two copies of that as well.
> 
> I think I need 2 fields to find log head/tail in recovery. Currently
> cache superblock records checkpoint disk position (log tail) and
> checkpoint sequence number, which can be used to find log head. Just
> recording log tail doesn't work well (it might work, for example,
> zeroing sectors before log head, so we can identify log head. But it's
> really ugly and not efficient). I only found recovery_offset can be
> overloaded. Do you have idea other fileds can be overloaded in MD
> superblock?

If each metadata block contains
  - a magic number
  - a checksum of the block
  - a sequence number
  - a pointer to the "next" metadata block (which is equivalent to
    the size of all described data)
  - a pointer to the tail (oldest active metadata block).

Then given the address of any block in the log you can easily find the
head:  walk the "next" pointers forward until you find a block
that has the wrong magic or checksum or sequence or previous pointer.
The last block that was consistent is the head.

You can then find the tail directly, and walk forward processing the
log.

Efficiency is not really an issue.  On a clean shutdown (which should
be the norm), the md superblock will contain a pointer to the head, and
the "next" block after that can quickly be determined to be invalid.
On an unclean shutdown it is expected that we need to do a bit more
work, and skipping forward along the chain to find the head of the log
is the least of our worries.

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-15 23:16                     ` NeilBrown
@ 2015-07-16  0:07                       ` Shaohua Li
  2015-07-16  1:22                         ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-16  0:07 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, Jul 16, 2015 at 09:16:53AM +1000, NeilBrown wrote:
> On Wed, 15 Jul 2015 12:49:37 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> > > On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> 
> > > 
> > > >                                                           I don't
> > > > understand why you object adding a superblock for cache. The advantage
> > > > is it's self contained. And there is nothing about
> > > > complexity/maintaince, as we can store the most necessary fields into
> > > > the superblock.
> > > 
> > > Because there is precisely 1 number that needs to be stored in the
> > > superblock, and there seems no point having a superblock just to store
> > > one number.
> > > It isn't much extra complexity, but any extra thing is still an extra
> > > thing.
> > > Having the data section of the log device containing just a log is
> > > elegant.  Elegant is good.
> > > If we decided that keeping two copies for superblocks was a good idea
> > > (which I think it is, I just haven't created a "v1.3" layout yet), then
> > > re-using the main superblock for the head-of-log pointer would instantly
> > > give us two copies of that as well.
> > 
> > I think I need 2 fields to find log head/tail in recovery. Currently
> > cache superblock records checkpoint disk position (log tail) and
> > checkpoint sequence number, which can be used to find log head. Just
> > recording log tail doesn't work well (it might work, for example,
> > zeroing sectors before log head, so we can identify log head. But it's
> > really ugly and not efficient). I only found recovery_offset can be
> > overloaded. Do you have idea other fileds can be overloaded in MD
> > superblock?
> 
> If each metadata block contains
>   - a magic number
>   - a checksum of the block
>   - a sequence number
>   - a pointer to the "next" metadata block (which is equivalent to
>     the size of all described data)
>   - a pointer to the tail (oldest active metadata block).
> 
> Then given the address of any block in the log you can easily find the
> head:  walk the "next" pointers forward until you find a block
> that has the wrong magic or checksum or sequence or previous pointer.
> The last block that was consistent is the head.
> 
> You can then find the tail directly, and walk forward processing the
> log.
> 
> Efficiency is not really an issue.  On a clean shutdown (which should
> be the norm), the md superblock will contain a pointer to the head, and
> the "next" block after that can quickly be determined to be invalid.
> On an unclean shutdown it is expected that we need to do a bit more
> work, and skipping forward along the chain to find the head of the log
> is the least of our worries.

if superblock records 2 fileds (the log tail and the seq of log tail), metadata
block doesn't need 'a pointer to the tail (oldest active metadata block)'. The
log tail/seq pair can help us find log head easily. Adding a pointer to the
tail in every metadata block is definitionly worse than adding a filed in the
superblock.

Further, how can you handle the case when log winds. For example, initially the log is
................................
                ^ superblock points to here
then we add meta and wind
|meta n-1|meta n|meta 0|meta 1|....
                ^ superblock points to here

Next time we reload log, superblock points to a valid meta. recovery will think
this is an unclean shutdown, so we rescan the whole log disk (because all metas
are valid) and apply all the changes to raid array. this is terrible. But if
superblock stores both log tail and seq. we will find the meta 0 sequence
number doesn't match with superblock, recovery stops instantly.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16  0:07                       ` Shaohua Li
@ 2015-07-16  1:22                         ` NeilBrown
  2015-07-16  4:13                           ` Shaohua Li
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-16  1:22 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Wed, 15 Jul 2015 17:07:39 -0700 Shaohua Li <shli@fb.com> wrote:

> On Thu, Jul 16, 2015 at 09:16:53AM +1000, NeilBrown wrote:
> > On Wed, 15 Jul 2015 12:49:37 -0700 Shaohua Li <shli@fb.com> wrote:
> > 
> > > On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> > > > On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > 
> > 
> > > > 
> > > > >                                                           I don't
> > > > > understand why you object adding a superblock for cache. The advantage
> > > > > is it's self contained. And there is nothing about
> > > > > complexity/maintaince, as we can store the most necessary fields into
> > > > > the superblock.
> > > > 
> > > > Because there is precisely 1 number that needs to be stored in the
> > > > superblock, and there seems no point having a superblock just to store
> > > > one number.
> > > > It isn't much extra complexity, but any extra thing is still an extra
> > > > thing.
> > > > Having the data section of the log device containing just a log is
> > > > elegant.  Elegant is good.
> > > > If we decided that keeping two copies for superblocks was a good idea
> > > > (which I think it is, I just haven't created a "v1.3" layout yet), then
> > > > re-using the main superblock for the head-of-log pointer would instantly
> > > > give us two copies of that as well.
> > > 
> > > I think I need 2 fields to find log head/tail in recovery. Currently
> > > cache superblock records checkpoint disk position (log tail) and
> > > checkpoint sequence number, which can be used to find log head. Just
> > > recording log tail doesn't work well (it might work, for example,
> > > zeroing sectors before log head, so we can identify log head. But it's
> > > really ugly and not efficient). I only found recovery_offset can be
> > > overloaded. Do you have idea other fileds can be overloaded in MD
> > > superblock?
> > 
> > If each metadata block contains
> >   - a magic number
> >   - a checksum of the block
> >   - a sequence number
> >   - a pointer to the "next" metadata block (which is equivalent to
> >     the size of all described data)
> >   - a pointer to the tail (oldest active metadata block).
> > 
> > Then given the address of any block in the log you can easily find the
> > head:  walk the "next" pointers forward until you find a block
> > that has the wrong magic or checksum or sequence or previous pointer.
> > The last block that was consistent is the head.
> > 
> > You can then find the tail directly, and walk forward processing the
> > log.
> > 
> > Efficiency is not really an issue.  On a clean shutdown (which should
> > be the norm), the md superblock will contain a pointer to the head, and
> > the "next" block after that can quickly be determined to be invalid.
> > On an unclean shutdown it is expected that we need to do a bit more
> > work, and skipping forward along the chain to find the head of the log
> > is the least of our worries.
> 
> if superblock records 2 fileds (the log tail and the seq of log tail), metadata
> block doesn't need 'a pointer to the tail (oldest active metadata block)'. The
> log tail/seq pair can help us find log head easily. Adding a pointer to the
> tail in every metadata block is definitionly worse than adding a filed in the
> superblock.

I don't really follow... maybe we are confusing terms.
In my mind, the "head" is where new data gets written and the "tail" is
where the oldest data is - though I can see that the reverse could also
make sense.

Previously you said you wanted to record "checkpoint disk position and
checkpoint sequence number" which you also referred to as the "tail".
So maybe you mean the "tail" to be the most recent checkpoint?

In any case, wouldn't the checkpoint metadata records its own sequence
number?  So why do you need to start the sequence number in the
superblock as well.

If you record a specific end of the log in the superblock,
then you need to update the superblock any time that end moves (don't
you?).  If you just record some starting point for a search, then you
only need to update the superblock when that location might get
over-written soon.

I think it makes perfect sense for all the metadata blocks in the log
to be linked together.  Whether they link to the previous or to the
first doesn't make a lot of difference.
By "first" here, I mean the block that has been in the log for the
longest, but still refers to "live" data. 

> 
> Further, how can you handle the case when log winds. For example, initially the log is
> ................................
>                 ^ superblock points to here
> then we add meta and wind
> |meta n-1|meta n|meta 0|meta 1|....
>                 ^ superblock points to here
> 
> Next time we reload log, superblock points to a valid meta. recovery will think
> this is an unclean shutdown, so we rescan the whole log disk (because all metas
> are valid) and apply all the changes to raid array. this is terrible. But if
> superblock stores both log tail and seq. we will find the meta 0 sequence
> number doesn't match with superblock, recovery stops instantly.

There should always be at least one valid metadata block in the log,
and the superblock should point to one of those.
On restart, it is loaded and the 'next' pointers are followed until you
get to the end (most recently written) of the log.  You confirm you are
still in the log as sequence numbers will be increasing.
After a clean shutdown, the 'next' pointer won't lead any where useful.

From the last metadata block, you work backwards to find the first.
Maybe you walk all the way following backwards links.  Maybe each
metadata block has a link to the earliest block worth looking at.

Then you follow the log forward recording every block that is found,
and discarding blocks when you find a metadata block which says
something can be discarded.  Once you have walked all the way from the
start to the end (from the earliest block to the most recently written
block) you have a complete list of all the live blocks that are in the
log.  Then they can be written to the RAID.

I think you are suggesting that the thing stored in the superblock is
the address of the oldest valid block.  I don't exactly object to that,
though I feel it would require updating the superblock more often
than needed.  I don't think you need the seq number though.
On a clean shutdown you would write out a metadata with a new seq
number and store a pointer to that.
On restart, that metadata block is loaded and its sequence number
examined. Any other blocks in the log will have a lower sequence number
and so will be ignored.

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16  1:22                         ` NeilBrown
@ 2015-07-16  4:13                           ` Shaohua Li
  2015-07-16  6:07                             ` NeilBrown
  0 siblings, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-16  4:13 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, Jul 16, 2015 at 11:22:17AM +1000, NeilBrown wrote:
> On Wed, 15 Jul 2015 17:07:39 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Thu, Jul 16, 2015 at 09:16:53AM +1000, NeilBrown wrote:
> > > On Wed, 15 Jul 2015 12:49:37 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> > > > > On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > 
> > > 
> > > > > 
> > > > > >                                                           I don't
> > > > > > understand why you object adding a superblock for cache. The advantage
> > > > > > is it's self contained. And there is nothing about
> > > > > > complexity/maintaince, as we can store the most necessary fields into
> > > > > > the superblock.
> > > > > 
> > > > > Because there is precisely 1 number that needs to be stored in the
> > > > > superblock, and there seems no point having a superblock just to store
> > > > > one number.
> > > > > It isn't much extra complexity, but any extra thing is still an extra
> > > > > thing.
> > > > > Having the data section of the log device containing just a log is
> > > > > elegant.  Elegant is good.
> > > > > If we decided that keeping two copies for superblocks was a good idea
> > > > > (which I think it is, I just haven't created a "v1.3" layout yet), then
> > > > > re-using the main superblock for the head-of-log pointer would instantly
> > > > > give us two copies of that as well.
> > > > 
> > > > I think I need 2 fields to find log head/tail in recovery. Currently
> > > > cache superblock records checkpoint disk position (log tail) and
> > > > checkpoint sequence number, which can be used to find log head. Just
> > > > recording log tail doesn't work well (it might work, for example,
> > > > zeroing sectors before log head, so we can identify log head. But it's
> > > > really ugly and not efficient). I only found recovery_offset can be
> > > > overloaded. Do you have idea other fileds can be overloaded in MD
> > > > superblock?
> > > 
> > > If each metadata block contains
> > >   - a magic number
> > >   - a checksum of the block
> > >   - a sequence number
> > >   - a pointer to the "next" metadata block (which is equivalent to
> > >     the size of all described data)
> > >   - a pointer to the tail (oldest active metadata block).
> > > 
> > > Then given the address of any block in the log you can easily find the
> > > head:  walk the "next" pointers forward until you find a block
> > > that has the wrong magic or checksum or sequence or previous pointer.
> > > The last block that was consistent is the head.
> > > 
> > > You can then find the tail directly, and walk forward processing the
> > > log.
> > > 
> > > Efficiency is not really an issue.  On a clean shutdown (which should
> > > be the norm), the md superblock will contain a pointer to the head, and
> > > the "next" block after that can quickly be determined to be invalid.
> > > On an unclean shutdown it is expected that we need to do a bit more
> > > work, and skipping forward along the chain to find the head of the log
> > > is the least of our worries.
> > 
> > if superblock records 2 fileds (the log tail and the seq of log tail), metadata
> > block doesn't need 'a pointer to the tail (oldest active metadata block)'. The
> > log tail/seq pair can help us find log head easily. Adding a pointer to the
> > tail in every metadata block is definitionly worse than adding a filed in the
> > superblock.
> 
> I don't really follow... maybe we are confusing terms.
> In my mind, the "head" is where new data gets written and the "tail" is
> where the oldest data is - though I can see that the reverse could also
> make sense.

we are in the same page about the terms. 
> Previously you said you wanted to record "checkpoint disk position and
> checkpoint sequence number" which you also referred to as the "tail".
> So maybe you mean the "tail" to be the most recent checkpoint?

yes
> In any case, wouldn't the checkpoint metadata records its own sequence
> number?  So why do you need to start the sequence number in the
> superblock as well.

The problem is you don't know if the metadata is valid/invalid (or
checkpointed) according to the sequence number stored in the metadata
itself. But if we store a sequence number in superblock, comparing the
sequence numebr in superblock and sequence in metadata let us know if
the metadata is valid.

> If you record a specific end of the log in the superblock,
> then you need to update the superblock any time that end moves (don't
> you?).

we record the tail. recovery will search from the tail
> If you just record some starting point for a search, then you
> only need to update the superblock when that location might get
> over-written soon.

that's what I'm doing. we only update the superblock when the location
can be reused.

> I think it makes perfect sense for all the metadata blocks in the log
> to be linked together.  Whether they link to the previous or to the
> first doesn't make a lot of difference.
> By "first" here, I mean the block that has been in the log for the
> longest, but still refers to "live" data. 

I just don't find how it's useful at all.
> > 
> > Further, how can you handle the case when log winds. For example, initially the log is
> > ................................
> >                 ^ superblock points to here
> > then we add meta and wind
> > |meta n-1|meta n|meta 0|meta 1|....
> >                 ^ superblock points to here
> > 
> > Next time we reload log, superblock points to a valid meta. recovery will think
> > this is an unclean shutdown, so we rescan the whole log disk (because all metas
> > are valid) and apply all the changes to raid array. this is terrible. But if
> > superblock stores both log tail and seq. we will find the meta 0 sequence
> > number doesn't match with superblock, recovery stops instantly.
> 
> There should always be at least one valid metadata block in the log,
> and the superblock should point to one of those.
> On restart, it is loaded and the 'next' pointers are followed until you
> get to the end (most recently written) of the log.  You confirm you are
> still in the log as sequence numbers will be increasing.
> After a clean shutdown, the 'next' pointer won't lead any where useful.

This is exactly the problem. Just storing tail in superblock can't
handle the log wind issue efficiently, because we can't know if the
metadata superblock pointed to is checkpointed or not. So we need
workaround. Here your workaround is always keeping one valid metadata
block in the log. If we store both checkpoint and sequence number in
superblock, we don't need the workaround.

The log will look like:
initially:
............
^super point here, super seq = 0

we add meta:
|meta 0|meta 1|...
^super point here, super seq = 0

We do a checkpoint, so space of meta 0 can be reused:
|meta 0|meta 1|...
       ^super point here, super seq = 1

if log rewind, we already checkpoint meta x:
|meta x|meta 1|...
       ^super point here, super seq = x + 1

it's easy to know meta 1 should be ignored since 1 != x + 1
This is the classic log structure implementation.

> From the last metadata block, you work backwards to find the first.
> Maybe you walk all the way following backwards links.  Maybe each
> metadata block has a link to the earliest block worth looking at.
> 
> Then you follow the log forward recording every block that is found,
> and discarding blocks when you find a metadata block which says
> something can be discarded.  Once you have walked all the way from the
> start to the end (from the earliest block to the most recently written
> block) you have a complete list of all the live blocks that are in the
> log.  Then they can be written to the RAID.
> 
> I think you are suggesting that the thing stored in the superblock is
> the address of the oldest valid block.  I don't exactly object to that,
> though I feel it would require updating the superblock more often
> than needed.
No, we only updating superblock in checkpoint.

> I don't think you need the seq number though.
> On a clean shutdown you would write out a metadata with a new seq
> number and store a pointer to that.
> On restart, that metadata block is loaded and its sequence number
> examined. Any other blocks in the log will have a lower sequence number
> and so will be ignored.

As I said, you can workaround the issue without recording seq number in
superblock, writing an extra metadata as you proposed for example. But
recording seq number makes things much easier and cleaner.

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16  4:13                           ` Shaohua Li
@ 2015-07-16  6:07                             ` NeilBrown
  2015-07-16 15:07                               ` John Stoffel
  2015-07-16 17:40                               ` Shaohua Li
  0 siblings, 2 replies; 44+ messages in thread
From: NeilBrown @ 2015-07-16  6:07 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Wed, 15 Jul 2015 21:13:06 -0700 Shaohua Li <shli@fb.com> wrote:

> On Thu, Jul 16, 2015 at 11:22:17AM +1000, NeilBrown wrote:
> > On Wed, 15 Jul 2015 17:07:39 -0700 Shaohua Li <shli@fb.com> wrote:
> > 
> > > On Thu, Jul 16, 2015 at 09:16:53AM +1000, NeilBrown wrote:
> > > > On Wed, 15 Jul 2015 12:49:37 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > 
> > > > > On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> > > > > > On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > > 
> > > > 
> > > > > > 
> > > > > > >                                                           I don't
> > > > > > > understand why you object adding a superblock for cache. The advantage
> > > > > > > is it's self contained. And there is nothing about
> > > > > > > complexity/maintaince, as we can store the most necessary fields into
> > > > > > > the superblock.
> > > > > > 
> > > > > > Because there is precisely 1 number that needs to be stored in the
> > > > > > superblock, and there seems no point having a superblock just to store
> > > > > > one number.
> > > > > > It isn't much extra complexity, but any extra thing is still an extra
> > > > > > thing.
> > > > > > Having the data section of the log device containing just a log is
> > > > > > elegant.  Elegant is good.
> > > > > > If we decided that keeping two copies for superblocks was a good idea
> > > > > > (which I think it is, I just haven't created a "v1.3" layout yet), then
> > > > > > re-using the main superblock for the head-of-log pointer would instantly
> > > > > > give us two copies of that as well.
> > > > > 
> > > > > I think I need 2 fields to find log head/tail in recovery. Currently
> > > > > cache superblock records checkpoint disk position (log tail) and
> > > > > checkpoint sequence number, which can be used to find log head. Just
> > > > > recording log tail doesn't work well (it might work, for example,
> > > > > zeroing sectors before log head, so we can identify log head. But it's
> > > > > really ugly and not efficient). I only found recovery_offset can be
> > > > > overloaded. Do you have idea other fileds can be overloaded in MD
> > > > > superblock?
> > > > 
> > > > If each metadata block contains
> > > >   - a magic number
> > > >   - a checksum of the block
> > > >   - a sequence number
> > > >   - a pointer to the "next" metadata block (which is equivalent to
> > > >     the size of all described data)
> > > >   - a pointer to the tail (oldest active metadata block).
> > > > 
> > > > Then given the address of any block in the log you can easily find the
> > > > head:  walk the "next" pointers forward until you find a block
> > > > that has the wrong magic or checksum or sequence or previous pointer.
> > > > The last block that was consistent is the head.
> > > > 
> > > > You can then find the tail directly, and walk forward processing the
> > > > log.
> > > > 
> > > > Efficiency is not really an issue.  On a clean shutdown (which should
> > > > be the norm), the md superblock will contain a pointer to the head, and
> > > > the "next" block after that can quickly be determined to be invalid.
> > > > On an unclean shutdown it is expected that we need to do a bit more
> > > > work, and skipping forward along the chain to find the head of the log
> > > > is the least of our worries.
> > > 
> > > if superblock records 2 fileds (the log tail and the seq of log tail), metadata
> > > block doesn't need 'a pointer to the tail (oldest active metadata block)'. The
> > > log tail/seq pair can help us find log head easily. Adding a pointer to the
> > > tail in every metadata block is definitionly worse than adding a filed in the
> > > superblock.
> > 
> > I don't really follow... maybe we are confusing terms.
> > In my mind, the "head" is where new data gets written and the "tail" is
> > where the oldest data is - though I can see that the reverse could also
> > make sense.
> 
> we are in the same page about the terms. 
> > Previously you said you wanted to record "checkpoint disk position and
> > checkpoint sequence number" which you also referred to as the "tail".
> > So maybe you mean the "tail" to be the most recent checkpoint?
> 
> yes
> > In any case, wouldn't the checkpoint metadata records its own sequence
> > number?  So why do you need to start the sequence number in the
> > superblock as well.
> 
> The problem is you don't know if the metadata is valid/invalid (or
> checkpointed) according to the sequence number stored in the metadata
> itself. But if we store a sequence number in superblock, comparing the
> sequence numebr in superblock and sequence in metadata let us know if
> the metadata is valid.

So it basically comes down to:
  You think we should store extra information in the superblock to 
     validate the thing that the superblock points to.
  I think we should always have the superblock pointing to something
    that is valid.

> 
> > If you record a specific end of the log in the superblock,
> > then you need to update the superblock any time that end moves (don't
> > you?).
> 
> we record the tail. recovery will search from the tail
> > If you just record some starting point for a search, then you
> > only need to update the superblock when that location might get
> > over-written soon.
> 
> that's what I'm doing. we only update the superblock when the location
> can be reused.
> 
> > I think it makes perfect sense for all the metadata blocks in the log
> > to be linked together.  Whether they link to the previous or to the
> > first doesn't make a lot of difference.
> > By "first" here, I mean the block that has been in the log for the
> > longest, but still refers to "live" data. 
> 
> I just don't find how it's useful at all.

It's not a show-stopper.
However if you keep the backwards links, then the superblock is free to
point to any metadata block anywhere in the log.  That means we can let
the pointer in the superblock be updated lazily.  The cache module only
needs to force a superblock update if there hasn't been one for any
other reason.


> > > 
> > > Further, how can you handle the case when log winds. For example, initially the log is
> > > ................................
> > >                 ^ superblock points to here
> > > then we add meta and wind
> > > |meta n-1|meta n|meta 0|meta 1|....
> > >                 ^ superblock points to here
> > > 
> > > Next time we reload log, superblock points to a valid meta. recovery will think
> > > this is an unclean shutdown, so we rescan the whole log disk (because all metas
> > > are valid) and apply all the changes to raid array. this is terrible. But if
> > > superblock stores both log tail and seq. we will find the meta 0 sequence
> > > number doesn't match with superblock, recovery stops instantly.
> > 
> > There should always be at least one valid metadata block in the log,
> > and the superblock should point to one of those.
> > On restart, it is loaded and the 'next' pointers are followed until you
> > get to the end (most recently written) of the log.  You confirm you are
> > still in the log as sequence numbers will be increasing.
> > After a clean shutdown, the 'next' pointer won't lead any where useful.
> 
> This is exactly the problem. Just storing tail in superblock can't
> handle the log wind issue efficiently, because we can't know if the
> metadata superblock pointed to is checkpointed or not. So we need
> workaround. Here your workaround is always keeping one valid metadata
> block in the log. If we store both checkpoint and sequence number in
> superblock, we don't need the workaround.

I don't see that as a work-around.  But then I don't really trust
checksums and so would rather that we always write a metadata block
(with FUA or FLUSH) after writing data, and using that to know that the
data is safe.  If we did that, it would be natural to always have a
valid metadata block.

> 
> The log will look like:
> initially:
> ............
> ^super point here, super seq = 0
> 
> we add meta:
> |meta 0|meta 1|...
> ^super point here, super seq = 0
> 
> We do a checkpoint, so space of meta 0 can be reused:
> |meta 0|meta 1|...
>        ^super point here, super seq = 1
> 
> if log rewind, we already checkpoint meta x:
> |meta x|meta 1|...
>        ^super point here, super seq = x + 1
> 
> it's easy to know meta 1 should be ignored since 1 != x + 1
> This is the classic log structure implementation.
> 
> > From the last metadata block, you work backwards to find the first.
> > Maybe you walk all the way following backwards links.  Maybe each
> > metadata block has a link to the earliest block worth looking at.
> > 
> > Then you follow the log forward recording every block that is found,
> > and discarding blocks when you find a metadata block which says
> > something can be discarded.  Once you have walked all the way from the
> > start to the end (from the earliest block to the most recently written
> > block) you have a complete list of all the live blocks that are in the
> > log.  Then they can be written to the RAID.
> > 
> > I think you are suggesting that the thing stored in the superblock is
> > the address of the oldest valid block.  I don't exactly object to that,
> > though I feel it would require updating the superblock more often
> > than needed.
> No, we only updating superblock in checkpoint.

hm... what exactly do you mean by "checkpoint" here?

I imagine there would be more checkpoints than times when the
superblock really needs updating.

> 
> > I don't think you need the seq number though.
> > On a clean shutdown you would write out a metadata with a new seq
> > number and store a pointer to that.
> > On restart, that metadata block is loaded and its sequence number
> > examined. Any other blocks in the log will have a lower sequence number
> > and so will be ignored.
> 
> As I said, you can workaround the issue without recording seq number in
> superblock, writing an extra metadata as you proposed for example. But
> recording seq number makes things much easier and cleaner.

Clearly "cleaner" is in the eye of the beholder.  I think it is much
clean for the superblock to point to something that is definitely valid.

NeilBrown


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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16  6:07                             ` NeilBrown
@ 2015-07-16 15:07                               ` John Stoffel
  2015-07-20  0:03                                 ` NeilBrown
  2015-07-16 17:40                               ` Shaohua Li
  1 sibling, 1 reply; 44+ messages in thread
From: John Stoffel @ 2015-07-16 15:07 UTC (permalink / raw)
  To: NeilBrown
  Cc: Shaohua Li, linux-raid, songliubraving, hch, dan.j.williams, Kernel-team


So.... how will wrapping of sequence numbers be handled in the cache
metadata on disk?  They can't grow for ever...

John

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16  6:07                             ` NeilBrown
  2015-07-16 15:07                               ` John Stoffel
@ 2015-07-16 17:40                               ` Shaohua Li
  2015-07-17  3:47                                 ` NeilBrown
  1 sibling, 1 reply; 44+ messages in thread
From: Shaohua Li @ 2015-07-16 17:40 UTC (permalink / raw)
  To: NeilBrown; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, Jul 16, 2015 at 04:07:11PM +1000, NeilBrown wrote:
> On Wed, 15 Jul 2015 21:13:06 -0700 Shaohua Li <shli@fb.com> wrote:
> 
> > On Thu, Jul 16, 2015 at 11:22:17AM +1000, NeilBrown wrote:
> > > On Wed, 15 Jul 2015 17:07:39 -0700 Shaohua Li <shli@fb.com> wrote:
> > > 
> > > > On Thu, Jul 16, 2015 at 09:16:53AM +1000, NeilBrown wrote:
> > > > > On Wed, 15 Jul 2015 12:49:37 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > 
> > > > > > On Wed, Jul 15, 2015 at 02:06:41PM +1000, NeilBrown wrote:
> > > > > > > On Tue, 14 Jul 2015 20:16:17 -0700 Shaohua Li <shli@fb.com> wrote:
> > > > > > > 
> > > > > 
> > > > > > > 
> > > > > > > >                                                           I don't
> > > > > > > > understand why you object adding a superblock for cache. The advantage
> > > > > > > > is it's self contained. And there is nothing about
> > > > > > > > complexity/maintaince, as we can store the most necessary fields into
> > > > > > > > the superblock.
> > > > > > > 
> > > > > > > Because there is precisely 1 number that needs to be stored in the
> > > > > > > superblock, and there seems no point having a superblock just to store
> > > > > > > one number.
> > > > > > > It isn't much extra complexity, but any extra thing is still an extra
> > > > > > > thing.
> > > > > > > Having the data section of the log device containing just a log is
> > > > > > > elegant.  Elegant is good.
> > > > > > > If we decided that keeping two copies for superblocks was a good idea
> > > > > > > (which I think it is, I just haven't created a "v1.3" layout yet), then
> > > > > > > re-using the main superblock for the head-of-log pointer would instantly
> > > > > > > give us two copies of that as well.
> > > > > > 
> > > > > > I think I need 2 fields to find log head/tail in recovery. Currently
> > > > > > cache superblock records checkpoint disk position (log tail) and
> > > > > > checkpoint sequence number, which can be used to find log head. Just
> > > > > > recording log tail doesn't work well (it might work, for example,
> > > > > > zeroing sectors before log head, so we can identify log head. But it's
> > > > > > really ugly and not efficient). I only found recovery_offset can be
> > > > > > overloaded. Do you have idea other fileds can be overloaded in MD
> > > > > > superblock?
> > > > > 
> > > > > If each metadata block contains
> > > > >   - a magic number
> > > > >   - a checksum of the block
> > > > >   - a sequence number
> > > > >   - a pointer to the "next" metadata block (which is equivalent to
> > > > >     the size of all described data)
> > > > >   - a pointer to the tail (oldest active metadata block).
> > > > > 
> > > > > Then given the address of any block in the log you can easily find the
> > > > > head:  walk the "next" pointers forward until you find a block
> > > > > that has the wrong magic or checksum or sequence or previous pointer.
> > > > > The last block that was consistent is the head.
> > > > > 
> > > > > You can then find the tail directly, and walk forward processing the
> > > > > log.
> > > > > 
> > > > > Efficiency is not really an issue.  On a clean shutdown (which should
> > > > > be the norm), the md superblock will contain a pointer to the head, and
> > > > > the "next" block after that can quickly be determined to be invalid.
> > > > > On an unclean shutdown it is expected that we need to do a bit more
> > > > > work, and skipping forward along the chain to find the head of the log
> > > > > is the least of our worries.
> > > > 
> > > > if superblock records 2 fileds (the log tail and the seq of log tail), metadata
> > > > block doesn't need 'a pointer to the tail (oldest active metadata block)'. The
> > > > log tail/seq pair can help us find log head easily. Adding a pointer to the
> > > > tail in every metadata block is definitionly worse than adding a filed in the
> > > > superblock.
> > > 
> > > I don't really follow... maybe we are confusing terms.
> > > In my mind, the "head" is where new data gets written and the "tail" is
> > > where the oldest data is - though I can see that the reverse could also
> > > make sense.
> > 
> > we are in the same page about the terms. 
> > > Previously you said you wanted to record "checkpoint disk position and
> > > checkpoint sequence number" which you also referred to as the "tail".
> > > So maybe you mean the "tail" to be the most recent checkpoint?
> > 
> > yes
> > > In any case, wouldn't the checkpoint metadata records its own sequence
> > > number?  So why do you need to start the sequence number in the
> > > superblock as well.
> > 
> > The problem is you don't know if the metadata is valid/invalid (or
> > checkpointed) according to the sequence number stored in the metadata
> > itself. But if we store a sequence number in superblock, comparing the
> > sequence numebr in superblock and sequence in metadata let us know if
> > the metadata is valid.
> 
> So it basically comes down to:
>   You think we should store extra information in the superblock to 
>      validate the thing that the superblock points to.
>   I think we should always have the superblock pointing to something
>     that is valid.
> 
> > 
> > > If you record a specific end of the log in the superblock,
> > > then you need to update the superblock any time that end moves (don't
> > > you?).
> > 
> > we record the tail. recovery will search from the tail
> > > If you just record some starting point for a search, then you
> > > only need to update the superblock when that location might get
> > > over-written soon.
> > 
> > that's what I'm doing. we only update the superblock when the location
> > can be reused.
> > 
> > > I think it makes perfect sense for all the metadata blocks in the log
> > > to be linked together.  Whether they link to the previous or to the
> > > first doesn't make a lot of difference.
> > > By "first" here, I mean the block that has been in the log for the
> > > longest, but still refers to "live" data. 
> > 
> > I just don't find how it's useful at all.
> 
> It's not a show-stopper.
> However if you keep the backwards links, then the superblock is free to
> point to any metadata block anywhere in the log.  That means we can let
> the pointer in the superblock be updated lazily.  The cache module only
> needs to force a superblock update if there hasn't been one for any
> other reason.

superblock never need to be updated frequently. It's only updated when
some space will be reused or to make recovery scan less. We don't need
the backward link to achieve this. If you check my patch, superblock is
updated very rarely.

> > > > 
> > > > Further, how can you handle the case when log winds. For example, initially the log is
> > > > ................................
> > > >                 ^ superblock points to here
> > > > then we add meta and wind
> > > > |meta n-1|meta n|meta 0|meta 1|....
> > > >                 ^ superblock points to here
> > > > 
> > > > Next time we reload log, superblock points to a valid meta. recovery will think
> > > > this is an unclean shutdown, so we rescan the whole log disk (because all metas
> > > > are valid) and apply all the changes to raid array. this is terrible. But if
> > > > superblock stores both log tail and seq. we will find the meta 0 sequence
> > > > number doesn't match with superblock, recovery stops instantly.
> > > 
> > > There should always be at least one valid metadata block in the log,
> > > and the superblock should point to one of those.
> > > On restart, it is loaded and the 'next' pointers are followed until you
> > > get to the end (most recently written) of the log.  You confirm you are
> > > still in the log as sequence numbers will be increasing.
> > > After a clean shutdown, the 'next' pointer won't lead any where useful.
> > 
> > This is exactly the problem. Just storing tail in superblock can't
> > handle the log wind issue efficiently, because we can't know if the
> > metadata superblock pointed to is checkpointed or not. So we need
> > workaround. Here your workaround is always keeping one valid metadata
> > block in the log. If we store both checkpoint and sequence number in
> > superblock, we don't need the workaround.
> 
> I don't see that as a work-around.  But then I don't really trust
> checksums and so would rather that we always write a metadata block
> (with FUA or FLUSH) after writing data, and using that to know that the
> data is safe.  If we did that, it would be natural to always have a
> valid metadata block.

if you don't trust checksum, how can you make sure the metadata block is
valid?
> > 
> > The log will look like:
> > initially:
> > ............
> > ^super point here, super seq = 0
> > 
> > we add meta:
> > |meta 0|meta 1|...
> > ^super point here, super seq = 0
> > 
> > We do a checkpoint, so space of meta 0 can be reused:
> > |meta 0|meta 1|...
> >        ^super point here, super seq = 1
> > 
> > if log rewind, we already checkpoint meta x:
> > |meta x|meta 1|...
> >        ^super point here, super seq = x + 1
> > 
> > it's easy to know meta 1 should be ignored since 1 != x + 1
> > This is the classic log structure implementation.
> > 
> > > From the last metadata block, you work backwards to find the first.
> > > Maybe you walk all the way following backwards links.  Maybe each
> > > metadata block has a link to the earliest block worth looking at.
> > > 
> > > Then you follow the log forward recording every block that is found,
> > > and discarding blocks when you find a metadata block which says
> > > something can be discarded.  Once you have walked all the way from the
> > > start to the end (from the earliest block to the most recently written
> > > block) you have a complete list of all the live blocks that are in the
> > > log.  Then they can be written to the RAID.
> > > 
> > > I think you are suggesting that the thing stored in the superblock is
> > > the address of the oldest valid block.  I don't exactly object to that,
> > > though I feel it would require updating the superblock more often
> > > than needed.
> > No, we only updating superblock in checkpoint.
> 
> hm... what exactly do you mean by "checkpoint" here?
> 
> I imagine there would be more checkpoints than times when the
> superblock really needs updating.
> 
> > 
> > > I don't think you need the seq number though.
> > > On a clean shutdown you would write out a metadata with a new seq
> > > number and store a pointer to that.
> > > On restart, that metadata block is loaded and its sequence number
> > > examined. Any other blocks in the log will have a lower sequence number
> > > and so will be ignored.
> > 
> > As I said, you can workaround the issue without recording seq number in
> > superblock, writing an extra metadata as you proposed for example. But
> > recording seq number makes things much easier and cleaner.
> 
> Clearly "cleaner" is in the eye of the beholder.  I think it is much
> clean for the superblock to point to something that is definitely valid.

You object to add a superblock. Actually there is one with your
proposal, the 'always valid metadata block' is acting as the role of
superblock, since we must read metadata block to find log head. Creating
a raid array will need create such block too. In other words, we need do
everything a real superblock need to do with your proposal. So why don't
we have an explict superblock instead of an implicit superblock?

Thanks,
Shaohua

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16 17:40                               ` Shaohua Li
@ 2015-07-17  3:47                                 ` NeilBrown
  0 siblings, 0 replies; 44+ messages in thread
From: NeilBrown @ 2015-07-17  3:47 UTC (permalink / raw)
  To: Shaohua Li; +Cc: linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, 16 Jul 2015 10:40:17 -0700 Shaohua Li <shli@fb.com> wrote:

> On Thu, Jul 16, 2015 at 04:07:11PM +1000, NeilBrown wrote:
> > On Wed, 15 Jul 2015 21:13:06 -0700 Shaohua Li <shli@fb.com> wrote:
> > 

> > It's not a show-stopper.
> > However if you keep the backwards links, then the superblock is free to
> > point to any metadata block anywhere in the log.  That means we can let
> > the pointer in the superblock be updated lazily.  The cache module only
> > needs to force a superblock update if there hasn't been one for any
> > other reason.
> 
> superblock never need to be updated frequently. It's only updated when
> some space will be reused or to make recovery scan less. We don't need
> the backward link to achieve this. If you check my patch, superblock is
> updated very rarely.

OK.


> > I don't see that as a work-around.  But then I don't really trust
> > checksums and so would rather that we always write a metadata block
> > (with FUA or FLUSH) after writing data, and using that to know that the
> > data is safe.  If we did that, it would be natural to always have a
> > valid metadata block.
> 
> if you don't trust checksum, how can you make sure the metadata block is
> valid?

A checksum is useful when you have a block that you have good reason to
expect to be correct, and you want to double-check to guard against
error or corruption.
So if there is a pointer in the superblock that is supposed to always
point to a valid metadata block, then I strongly expect to find a
metadata block there.  If the magic number of checksum are wrong, then
that is very surprising.  Still worth checking because bugs happen at
all levels, but surprising.

If you have a pointer to something that may or may no be valid, then it
isn't particularly surprising if the checksum is wrong.  That is a very
different use case.

Famously reiserfs uses (or at least "used") checksums to identify
different sorts of blocks when performing an 'fsck'.  If you store the
image of a reiserfs filesystem in a file in reiserfs, then fsck could
get confused and think the data is metadata - the checksum matches.

That exact scenario would not apply to our log, but it still serves as
a warning not to put too much faith in checksums.  They can tell when
something if wrong, but not when it is right.

> 
> You object to add a superblock. Actually there is one with your
> proposal, the 'always valid metadata block' is acting as the role of
> superblock, since we must read metadata block to find log head. Creating
> a raid array will need create such block too. In other words, we need do
> everything a real superblock need to do with your proposal. So why don't
> we have an explict superblock instead of an implicit superblock?

I accept that a metadata block can be a de-facto superblock.
But why have two things when you can make do with one?

I'm a big fan of Occam's Razor:
   Entities must not be multiplied beyond necessity

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-16 15:07                               ` John Stoffel
@ 2015-07-20  0:03                                 ` NeilBrown
  2015-07-20 14:11                                   ` John Stoffel
  0 siblings, 1 reply; 44+ messages in thread
From: NeilBrown @ 2015-07-20  0:03 UTC (permalink / raw)
  To: John Stoffel
  Cc: Shaohua Li, linux-raid, songliubraving, hch, dan.j.williams, Kernel-team

On Thu, 16 Jul 2015 11:07:34 -0400 "John Stoffel" <john@stoffel.org>
wrote:

> 
> So.... how will wrapping of sequence numbers be handled in the cache
> metadata on disk?  They can't grow for ever...

Can't they?

A 64bit number incremented every millisecond can grow for half a
billion years.  As far as I'm concerned, that is forever.

But even if we used 32bit numbers, it is very easy to handle looping
providing you have few than half of the numbers in use at any time.
So as long as we don't have more than 2 billion active transactions in
the log at once, there is no problem.
The kernel does some internal time keeping using "jiffies" which, on
32bit systems, cycle about every fortnight.  A small amount of care is
enough that this doesn't cause problem.

NeilBrown

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

* Re: [PATCH V4 00/13] MD: a caching layer for raid5/6
  2015-07-20  0:03                                 ` NeilBrown
@ 2015-07-20 14:11                                   ` John Stoffel
  0 siblings, 0 replies; 44+ messages in thread
From: John Stoffel @ 2015-07-20 14:11 UTC (permalink / raw)
  To: NeilBrown
  Cc: John Stoffel, Shaohua Li, linux-raid, songliubraving, hch,
	dan.j.williams, Kernel-team

>>>>> "NeilBrown" == NeilBrown  <neilb@suse.com> writes:

NeilBrown> On Thu, 16 Jul 2015 11:07:34 -0400 "John Stoffel" <john@stoffel.org>
NeilBrown> wrote:

>> 
>> So.... how will wrapping of sequence numbers be handled in the cache
>> metadata on disk?  They can't grow for ever...

NeilBrown> Can't they?

NeilBrown> A 64bit number incremented every millisecond can grow for half a
NeilBrown> billion years.  As far as I'm concerned, that is forever.

LOL... that is true.  But I'm trying to just think about edge cases
and point out some possible pitfalls in case of corruption or errors.  

NeilBrown> But even if we used 32bit numbers, it is very easy to
NeilBrown> handle looping providing you have few than half of the
NeilBrown> numbers in use at any time.  So as long as we don't have
NeilBrown> more than 2 billion active transactions in the log at once,
NeilBrown> there is no problem.  The kernel does some internal time
NeilBrown> keeping using "jiffies" which, on 32bit systems, cycle
NeilBrown> about every fortnight.  A small amount of care is enough
NeilBrown> that this doesn't cause problem.

All I wanted to make sure, was that this was thought of and handled.
I like the idea of randomly picking a STARTING number, so that we
hopefully exercise all the corner cases properly early on in the process.

John


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

end of thread, other threads:[~2015-07-20 14:11 UTC | newest]

Thread overview: 44+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-06-23 21:37 [PATCH V4 00/13] MD: a caching layer for raid5/6 Shaohua Li
2015-06-23 21:37 ` [PATCH V4 01/13] MD: add a new disk role to present cache device Shaohua Li
2015-06-23 21:37 ` [PATCH V4 02/13] raid5: directly use mddev->queue Shaohua Li
2015-06-23 21:37 ` [PATCH V4 03/13] raid5: cache log handling Shaohua Li
2015-06-23 21:37 ` [PATCH V4 04/13] raid5: cache part of raid5 cache Shaohua Li
2015-06-23 21:37 ` [PATCH V4 05/13] raid5: cache reclaim support Shaohua Li
2015-06-23 21:37 ` [PATCH V4 06/13] raid5: cache IO error handling Shaohua Li
2015-06-23 21:37 ` [PATCH V4 07/13] raid5: cache device quiesce support Shaohua Li
2015-06-23 21:37 ` [PATCH V4 08/13] raid5: cache recovery support Shaohua Li
2015-06-23 21:37 ` [PATCH V4 09/13] raid5: add some sysfs entries Shaohua Li
2015-06-23 21:38 ` [PATCH V4 10/13] raid5: don't allow resize/reshape with cache support Shaohua Li
2015-06-23 21:38 ` [PATCH V4 11/13] raid5: guarantee cache release stripes in correct way Shaohua Li
2015-06-23 21:38 ` [PATCH V4 12/13] raid5: enable cache for raid array with cache disk Shaohua Li
2015-06-23 21:38 ` [PATCH V4 13/13] raid5: skip resync if caching is enabled Shaohua Li
2015-07-02  3:25 ` [PATCH V4 00/13] MD: a caching layer for raid5/6 Yuanhan Liu
2015-07-02 17:11   ` Shaohua Li
2015-07-03  2:18     ` Yuanhan Liu
2015-07-08  1:56 ` NeilBrown
2015-07-08  5:44   ` Shaohua Li
2015-07-09 23:21     ` NeilBrown
2015-07-10  4:08       ` Shaohua Li
2015-07-10  4:36         ` NeilBrown
2015-07-10  4:52           ` Shaohua Li
2015-07-10  5:10             ` NeilBrown
2015-07-10  5:18               ` Shaohua Li
2015-07-10  6:42                 ` NeilBrown
2015-07-10 17:48                   ` Shaohua Li
2015-07-13 22:22                     ` NeilBrown
2015-07-13 22:35                       ` Shaohua Li
2015-07-15  0:45           ` Shaohua Li
2015-07-15  2:12             ` NeilBrown
2015-07-15  3:16               ` Shaohua Li
2015-07-15  4:06                 ` NeilBrown
2015-07-15 19:49                   ` Shaohua Li
2015-07-15 23:16                     ` NeilBrown
2015-07-16  0:07                       ` Shaohua Li
2015-07-16  1:22                         ` NeilBrown
2015-07-16  4:13                           ` Shaohua Li
2015-07-16  6:07                             ` NeilBrown
2015-07-16 15:07                               ` John Stoffel
2015-07-20  0:03                                 ` NeilBrown
2015-07-20 14:11                                   ` John Stoffel
2015-07-16 17:40                               ` Shaohua Li
2015-07-17  3:47                                 ` NeilBrown

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.