aboutsummaryrefslogtreecommitdiff
path: root/drivers/md
diff options
context:
space:
mode:
authorLinus Torvalds <torvalds@linux-foundation.org>2013-09-10 13:03:41 -0700
committerLinus Torvalds <torvalds@linux-foundation.org>2013-09-10 13:03:41 -0700
commit4d7696f1b05f4aeb586c74868fe3da2731daca4b (patch)
treedd6cf4d41df2c0a1f52a85a3f8b8af5a9ebdeb5d /drivers/md
parentb05430fc9341fea7a6228a3611c850a476809596 (diff)
parentbfc90cb0936f5b972706625f38f72c7cb726c20a (diff)
Merge tag 'md/3.12' of git://neil.brown.name/md
Pull md update from Neil Brown: "Headline item is multithreading for RAID5 so that more IO/sec can be supported on fast (SSD) devices. Also TILE-Gx SIMD suppor for RAID6 calculations and an assortment of bug fixes" * tag 'md/3.12' of git://neil.brown.name/md: raid5: only wakeup necessary threads md/raid5: flush out all pending requests before proceeding with reshape. md/raid5: use seqcount to protect access to shape in make_request. raid5: sysfs entry to control worker thread number raid5: offload stripe handle to workqueue raid5: fix stripe release order raid5: make release_stripe lockless md: avoid deadlock when dirty buffers during md_stop. md: Don't test all of mddev->flags at once. md: Fix apparent cut-and-paste error in super_90_validate raid6/test: replace echo -e with printf RAID: add tilegx SIMD implementation of raid6 md: fix safe_mode buglet. md: don't call md_allow_write in get_bitmap_file.
Diffstat (limited to 'drivers/md')
-rw-r--r--drivers/md/md.c54
-rw-r--r--drivers/md/md.h8
-rw-r--r--drivers/md/raid5.c362
-rw-r--r--drivers/md/raid5.h22
4 files changed, 406 insertions, 40 deletions
diff --git a/drivers/md/md.c b/drivers/md/md.c
index 9f13e13506e..adf4d7e1d5e 100644
--- a/drivers/md/md.c
+++ b/drivers/md/md.c
@@ -1180,7 +1180,7 @@ static int super_90_validate(struct mddev *mddev, struct md_rdev *rdev)
mddev->bitmap_info.offset =
mddev->bitmap_info.default_offset;
mddev->bitmap_info.space =
- mddev->bitmap_info.space;
+ mddev->bitmap_info.default_space;
}
} else if (mddev->pers == NULL) {
@@ -3429,7 +3429,7 @@ safe_delay_store(struct mddev *mddev, const char *cbuf, size_t len)
mddev->safemode_delay = (msec*HZ)/1000;
if (mddev->safemode_delay == 0)
mddev->safemode_delay = 1;
- if (mddev->safemode_delay < old_delay)
+ if (mddev->safemode_delay < old_delay || old_delay == 0)
md_safemode_timeout((unsigned long)mddev);
}
return len;
@@ -5144,7 +5144,7 @@ int md_run(struct mddev *mddev)
set_bit(MD_RECOVERY_NEEDED, &mddev->recovery);
- if (mddev->flags)
+ if (mddev->flags & MD_UPDATE_SB_FLAGS)
md_update_sb(mddev, 0);
md_new_event(mddev);
@@ -5289,7 +5289,7 @@ static void __md_stop_writes(struct mddev *mddev)
md_super_wait(mddev);
if (mddev->ro == 0 &&
- (!mddev->in_sync || mddev->flags)) {
+ (!mddev->in_sync || (mddev->flags & MD_UPDATE_SB_FLAGS))) {
/* mark array as shutdown cleanly */
mddev->in_sync = 1;
md_update_sb(mddev, 1);
@@ -5337,8 +5337,14 @@ static int md_set_readonly(struct mddev *mddev, struct block_device *bdev)
err = -EBUSY;
goto out;
}
- if (bdev)
- sync_blockdev(bdev);
+ if (bdev && !test_bit(MD_STILL_CLOSED, &mddev->flags)) {
+ /* Someone opened the device since we flushed it
+ * so page cache could be dirty and it is too late
+ * to flush. So abort
+ */
+ mutex_unlock(&mddev->open_mutex);
+ return -EBUSY;
+ }
if (mddev->pers) {
__md_stop_writes(mddev);
@@ -5373,14 +5379,14 @@ static int do_md_stop(struct mddev * mddev, int mode,
mutex_unlock(&mddev->open_mutex);
return -EBUSY;
}
- if (bdev)
- /* It is possible IO was issued on some other
- * open file which was closed before we took ->open_mutex.
- * As that was not the last close __blkdev_put will not
- * have called sync_blockdev, so we must.
+ if (bdev && !test_bit(MD_STILL_CLOSED, &mddev->flags)) {
+ /* Someone opened the device since we flushed it
+ * so page cache could be dirty and it is too late
+ * to flush. So abort
*/
- sync_blockdev(bdev);
-
+ mutex_unlock(&mddev->open_mutex);
+ return -EBUSY;
+ }
if (mddev->pers) {
if (mddev->ro)
set_disk_ro(disk, 0);
@@ -5628,10 +5634,7 @@ static int get_bitmap_file(struct mddev * mddev, void __user * arg)
char *ptr, *buf = NULL;
int err = -ENOMEM;
- if (md_allow_write(mddev))
- file = kmalloc(sizeof(*file), GFP_NOIO);
- else
- file = kmalloc(sizeof(*file), GFP_KERNEL);
+ file = kmalloc(sizeof(*file), GFP_NOIO);
if (!file)
goto out;
@@ -6420,6 +6423,20 @@ static int md_ioctl(struct block_device *bdev, fmode_t mode,
!test_bit(MD_RECOVERY_NEEDED,
&mddev->flags),
msecs_to_jiffies(5000));
+ if (cmd == STOP_ARRAY || cmd == STOP_ARRAY_RO) {
+ /* Need to flush page cache, and ensure no-one else opens
+ * and writes
+ */
+ mutex_lock(&mddev->open_mutex);
+ if (atomic_read(&mddev->openers) > 1) {
+ mutex_unlock(&mddev->open_mutex);
+ err = -EBUSY;
+ goto abort;
+ }
+ set_bit(MD_STILL_CLOSED, &mddev->flags);
+ mutex_unlock(&mddev->open_mutex);
+ sync_blockdev(bdev);
+ }
err = mddev_lock(mddev);
if (err) {
printk(KERN_INFO
@@ -6673,6 +6690,7 @@ static int md_open(struct block_device *bdev, fmode_t mode)
err = 0;
atomic_inc(&mddev->openers);
+ clear_bit(MD_STILL_CLOSED, &mddev->flags);
mutex_unlock(&mddev->open_mutex);
check_disk_change(bdev);
@@ -7817,7 +7835,7 @@ void md_check_recovery(struct mddev *mddev)
sysfs_notify_dirent_safe(mddev->sysfs_state);
}
- if (mddev->flags)
+ if (mddev->flags & MD_UPDATE_SB_FLAGS)
md_update_sb(mddev, 0);
if (test_bit(MD_RECOVERY_RUNNING, &mddev->recovery) &&
diff --git a/drivers/md/md.h b/drivers/md/md.h
index 20f02c0b5f2..608050c43f1 100644
--- a/drivers/md/md.h
+++ b/drivers/md/md.h
@@ -204,12 +204,16 @@ struct mddev {
struct md_personality *pers;
dev_t unit;
int md_minor;
- struct list_head disks;
+ struct list_head disks;
unsigned long flags;
#define MD_CHANGE_DEVS 0 /* Some device status has changed */
#define MD_CHANGE_CLEAN 1 /* transition to or from 'clean' */
#define MD_CHANGE_PENDING 2 /* switch from 'clean' to 'active' in progress */
+#define MD_UPDATE_SB_FLAGS (1 | 2 | 4) /* If these are set, md_update_sb needed */
#define MD_ARRAY_FIRST_USE 3 /* First use of array, needs initialization */
+#define MD_STILL_CLOSED 4 /* If set, then array has not been opened since
+ * md_ioctl checked on it.
+ */
int suspended;
atomic_t active_io;
@@ -218,7 +222,7 @@ struct mddev {
* are happening, so run/
* takeover/stop are not safe
*/
- int ready; /* See when safe to pass
+ int ready; /* See when safe to pass
* IO requests down */
struct gendisk *gendisk;
diff --git a/drivers/md/raid5.c b/drivers/md/raid5.c
index 78ea44336e7..7ff4f252ca1 100644
--- a/drivers/md/raid5.c
+++ b/drivers/md/raid5.c
@@ -53,6 +53,7 @@
#include <linux/cpu.h>
#include <linux/slab.h>
#include <linux/ratelimit.h>
+#include <linux/nodemask.h>
#include <trace/events/block.h>
#include "md.h"
@@ -60,6 +61,10 @@
#include "raid0.h"
#include "bitmap.h"
+#define cpu_to_group(cpu) cpu_to_node(cpu)
+#define ANY_GROUP NUMA_NO_NODE
+
+static struct workqueue_struct *raid5_wq;
/*
* Stripe cache
*/
@@ -72,6 +77,7 @@
#define BYPASS_THRESHOLD 1
#define NR_HASH (PAGE_SIZE / sizeof(struct hlist_head))
#define HASH_MASK (NR_HASH - 1)
+#define MAX_STRIPE_BATCH 8
static inline struct hlist_head *stripe_hash(struct r5conf *conf, sector_t sect)
{
@@ -200,6 +206,49 @@ static int stripe_operations_active(struct stripe_head *sh)
test_bit(STRIPE_COMPUTE_RUN, &sh->state);
}
+static void raid5_wakeup_stripe_thread(struct stripe_head *sh)
+{
+ struct r5conf *conf = sh->raid_conf;
+ struct r5worker_group *group;
+ int thread_cnt;
+ int i, cpu = sh->cpu;
+
+ if (!cpu_online(cpu)) {
+ cpu = cpumask_any(cpu_online_mask);
+ sh->cpu = cpu;
+ }
+
+ if (list_empty(&sh->lru)) {
+ struct r5worker_group *group;
+ group = conf->worker_groups + cpu_to_group(cpu);
+ list_add_tail(&sh->lru, &group->handle_list);
+ group->stripes_cnt++;
+ sh->group = group;
+ }
+
+ if (conf->worker_cnt_per_group == 0) {
+ md_wakeup_thread(conf->mddev->thread);
+ return;
+ }
+
+ group = conf->worker_groups + cpu_to_group(sh->cpu);
+
+ group->workers[0].working = true;
+ /* at least one worker should run to avoid race */
+ queue_work_on(sh->cpu, raid5_wq, &group->workers[0].work);
+
+ thread_cnt = group->stripes_cnt / MAX_STRIPE_BATCH - 1;
+ /* wakeup more workers */
+ for (i = 1; i < conf->worker_cnt_per_group && thread_cnt > 0; i++) {
+ if (group->workers[i].working == false) {
+ group->workers[i].working = true;
+ queue_work_on(sh->cpu, raid5_wq,
+ &group->workers[i].work);
+ thread_cnt--;
+ }
+ }
+}
+
static void do_release_stripe(struct r5conf *conf, struct stripe_head *sh)
{
BUG_ON(!list_empty(&sh->lru));
@@ -214,7 +263,12 @@ static void do_release_stripe(struct r5conf *conf, struct stripe_head *sh)
else {
clear_bit(STRIPE_DELAYED, &sh->state);
clear_bit(STRIPE_BIT_DELAY, &sh->state);
- list_add_tail(&sh->lru, &conf->handle_list);
+ if (conf->worker_cnt_per_group == 0) {
+ list_add_tail(&sh->lru, &conf->handle_list);
+ } else {
+ raid5_wakeup_stripe_thread(sh);
+ return;
+ }
}
md_wakeup_thread(conf->mddev->thread);
} else {
@@ -239,12 +293,62 @@ static void __release_stripe(struct r5conf *conf, struct stripe_head *sh)
do_release_stripe(conf, sh);
}
+static struct llist_node *llist_reverse_order(struct llist_node *head)
+{
+ struct llist_node *new_head = NULL;
+
+ while (head) {
+ struct llist_node *tmp = head;
+ head = head->next;
+ tmp->next = new_head;
+ new_head = tmp;
+ }
+
+ return new_head;
+}
+
+/* should hold conf->device_lock already */
+static int release_stripe_list(struct r5conf *conf)
+{
+ struct stripe_head *sh;
+ int count = 0;
+ struct llist_node *head;
+
+ head = llist_del_all(&conf->released_stripes);
+ head = llist_reverse_order(head);
+ while (head) {
+ sh = llist_entry(head, struct stripe_head, release_list);
+ head = llist_next(head);
+ /* sh could be readded after STRIPE_ON_RELEASE_LIST is cleard */
+ smp_mb();
+ clear_bit(STRIPE_ON_RELEASE_LIST, &sh->state);
+ /*
+ * Don't worry the bit is set here, because if the bit is set
+ * again, the count is always > 1. This is true for
+ * STRIPE_ON_UNPLUG_LIST bit too.
+ */
+ __release_stripe(conf, sh);
+ count++;
+ }
+
+ return count;
+}
+
static void release_stripe(struct stripe_head *sh)
{
struct r5conf *conf = sh->raid_conf;
unsigned long flags;
+ bool wakeup;
+ if (test_and_set_bit(STRIPE_ON_RELEASE_LIST, &sh->state))
+ goto slow_path;
+ wakeup = llist_add(&sh->release_list, &conf->released_stripes);
+ if (wakeup)
+ md_wakeup_thread(conf->mddev->thread);
+ return;
+slow_path:
local_irq_save(flags);
+ /* we are ok here if STRIPE_ON_RELEASE_LIST is set or not */
if (atomic_dec_and_lock(&sh->count, &conf->device_lock)) {
do_release_stripe(conf, sh);
spin_unlock(&conf->device_lock);
@@ -359,6 +463,7 @@ static void init_stripe(struct stripe_head *sh, sector_t sector, int previous)
raid5_build_block(sh, i, previous);
}
insert_hash(conf, sh);
+ sh->cpu = smp_processor_id();
}
static struct stripe_head *__find_stripe(struct r5conf *conf, sector_t sector,
@@ -491,7 +596,8 @@ get_active_stripe(struct r5conf *conf, sector_t sector,
if (atomic_read(&sh->count)) {
BUG_ON(!list_empty(&sh->lru)
&& !test_bit(STRIPE_EXPANDING, &sh->state)
- && !test_bit(STRIPE_ON_UNPLUG_LIST, &sh->state));
+ && !test_bit(STRIPE_ON_UNPLUG_LIST, &sh->state)
+ && !test_bit(STRIPE_ON_RELEASE_LIST, &sh->state));
} else {
if (!test_bit(STRIPE_HANDLE, &sh->state))
atomic_inc(&conf->active_stripes);
@@ -499,6 +605,10 @@ get_active_stripe(struct r5conf *conf, sector_t sector,
!test_bit(STRIPE_EXPANDING, &sh->state))
BUG();
list_del_init(&sh->lru);
+ if (sh->group) {
+ sh->group->stripes_cnt--;
+ sh->group = NULL;
+ }
}
}
} while (sh == NULL);
@@ -3779,6 +3889,7 @@ static void raid5_activate_delayed(struct r5conf *conf)
if (!test_and_set_bit(STRIPE_PREREAD_ACTIVE, &sh->state))
atomic_inc(&conf->preread_active_stripes);
list_add_tail(&sh->lru, &conf->hold_list);
+ raid5_wakeup_stripe_thread(sh);
}
}
}
@@ -4058,18 +4169,35 @@ static int chunk_aligned_read(struct mddev *mddev, struct bio * raid_bio)
* head of the hold_list has changed, i.e. the head was promoted to the
* handle_list.
*/
-static struct stripe_head *__get_priority_stripe(struct r5conf *conf)
+static struct stripe_head *__get_priority_stripe(struct r5conf *conf, int group)
{
- struct stripe_head *sh;
+ struct stripe_head *sh = NULL, *tmp;
+ struct list_head *handle_list = NULL;
+ struct r5worker_group *wg = NULL;
+
+ if (conf->worker_cnt_per_group == 0) {
+ handle_list = &conf->handle_list;
+ } else if (group != ANY_GROUP) {
+ handle_list = &conf->worker_groups[group].handle_list;
+ wg = &conf->worker_groups[group];
+ } else {
+ int i;
+ for (i = 0; i < conf->group_cnt; i++) {
+ handle_list = &conf->worker_groups[i].handle_list;
+ wg = &conf->worker_groups[i];
+ if (!list_empty(handle_list))
+ break;
+ }
+ }
pr_debug("%s: handle: %s hold: %s full_writes: %d bypass_count: %d\n",
__func__,
- list_empty(&conf->handle_list) ? "empty" : "busy",
+ list_empty(handle_list) ? "empty" : "busy",
list_empty(&conf->hold_list) ? "empty" : "busy",
atomic_read(&conf->pending_full_writes), conf->bypass_count);
- if (!list_empty(&conf->handle_list)) {
- sh = list_entry(conf->handle_list.next, typeof(*sh), lru);
+ if (!list_empty(handle_list)) {
+ sh = list_entry(handle_list->next, typeof(*sh), lru);
if (list_empty(&conf->hold_list))
conf->bypass_count = 0;
@@ -4087,14 +4215,32 @@ static struct stripe_head *__get_priority_stripe(struct r5conf *conf)
((conf->bypass_threshold &&
conf->bypass_count > conf->bypass_threshold) ||
atomic_read(&conf->pending_full_writes) == 0)) {
- sh = list_entry(conf->hold_list.next,
- typeof(*sh), lru);
- conf->bypass_count -= conf->bypass_threshold;
- if (conf->bypass_count < 0)
- conf->bypass_count = 0;
- } else
+
+ list_for_each_entry(tmp, &conf->hold_list, lru) {
+ if (conf->worker_cnt_per_group == 0 ||
+ group == ANY_GROUP ||
+ !cpu_online(tmp->cpu) ||
+ cpu_to_group(tmp->cpu) == group) {
+ sh = tmp;
+ break;
+ }
+ }
+
+ if (sh) {
+ conf->bypass_count -= conf->bypass_threshold;
+ if (conf->bypass_count < 0)
+ conf->bypass_count = 0;
+ }
+ wg = NULL;
+ }
+
+ if (!sh)
return NULL;
+ if (wg) {
+ wg->stripes_cnt--;
+ sh->group = NULL;
+ }
list_del_init(&sh->lru);
atomic_inc(&sh->count);
BUG_ON(atomic_read(&sh->count) != 1);
@@ -4127,6 +4273,10 @@ static void raid5_unplug(struct blk_plug_cb *blk_cb, bool from_schedule)
*/
smp_mb__before_clear_bit();
clear_bit(STRIPE_ON_UNPLUG_LIST, &sh->state);
+ /*
+ * STRIPE_ON_RELEASE_LIST could be set here. In that
+ * case, the count is always > 1 here
+ */
__release_stripe(conf, sh);
cnt++;
}
@@ -4286,8 +4436,10 @@ static void make_request(struct mddev *mddev, struct bio * bi)
for (;logical_sector < last_sector; logical_sector += STRIPE_SECTORS) {
DEFINE_WAIT(w);
int previous;
+ int seq;
retry:
+ seq = read_seqcount_begin(&conf->gen_lock);
previous = 0;
prepare_to_wait(&conf->wait_for_overlap, &w, TASK_UNINTERRUPTIBLE);
if (unlikely(conf->reshape_progress != MaxSector)) {
@@ -4320,7 +4472,7 @@ static void make_request(struct mddev *mddev, struct bio * bi)
previous,
&dd_idx, NULL);
pr_debug("raid456: make_request, sector %llu logical %llu\n",
- (unsigned long long)new_sector,
+ (unsigned long long)new_sector,
(unsigned long long)logical_sector);
sh = get_active_stripe(conf, new_sector, previous,
@@ -4349,6 +4501,13 @@ static void make_request(struct mddev *mddev, struct bio * bi)
goto retry;
}
}
+ if (read_seqcount_retry(&conf->gen_lock, seq)) {
+ /* Might have got the wrong stripe_head
+ * by accident
+ */
+ release_stripe(sh);
+ goto retry;
+ }
if (rw == WRITE &&
logical_sector >= mddev->suspend_lo &&
@@ -4788,14 +4947,14 @@ static int retry_aligned_read(struct r5conf *conf, struct bio *raid_bio)
return handled;
}
-#define MAX_STRIPE_BATCH 8
-static int handle_active_stripes(struct r5conf *conf)
+static int handle_active_stripes(struct r5conf *conf, int group,
+ struct r5worker *worker)
{
struct stripe_head *batch[MAX_STRIPE_BATCH], *sh;
int i, batch_size = 0;
while (batch_size < MAX_STRIPE_BATCH &&
- (sh = __get_priority_stripe(conf)) != NULL)
+ (sh = __get_priority_stripe(conf, group)) != NULL)
batch[batch_size++] = sh;
if (batch_size == 0)
@@ -4813,6 +4972,39 @@ static int handle_active_stripes(struct r5conf *conf)
return batch_size;
}
+static void raid5_do_work(struct work_struct *work)
+{
+ struct r5worker *worker = container_of(work, struct r5worker, work);
+ struct r5worker_group *group = worker->group;
+ struct r5conf *conf = group->conf;
+ int group_id = group - conf->worker_groups;
+ int handled;
+ struct blk_plug plug;
+
+ pr_debug("+++ raid5worker active\n");
+
+ blk_start_plug(&plug);
+ handled = 0;
+ spin_lock_irq(&conf->device_lock);
+ while (1) {
+ int batch_size, released;
+
+ released = release_stripe_list(conf);
+
+ batch_size = handle_active_stripes(conf, group_id, worker);
+ worker->working = false;
+ if (!batch_size && !released)
+ break;
+ handled += batch_size;
+ }
+ pr_debug("%d stripes handled\n", handled);
+
+ spin_unlock_irq(&conf->device_lock);
+ blk_finish_plug(&plug);
+
+ pr_debug("--- raid5worker inactive\n");
+}
+
/*
* This is our raid5 kernel thread.
*
@@ -4836,7 +5028,9 @@ static void raid5d(struct md_thread *thread)
spin_lock_irq(&conf->device_lock);
while (1) {
struct bio *bio;
- int batch_size;
+ int batch_size, released;
+
+ released = release_stripe_list(conf);
if (
!list_empty(&conf->bitmap_list)) {
@@ -4860,8 +5054,8 @@ static void raid5d(struct md_thread *thread)
handled++;
}
- batch_size = handle_active_stripes(conf);
- if (!batch_size)
+ batch_size = handle_active_stripes(conf, ANY_GROUP, NULL);
+ if (!batch_size && !released)
break;
handled += batch_size;
@@ -4989,10 +5183,70 @@ stripe_cache_active_show(struct mddev *mddev, char *page)
static struct md_sysfs_entry
raid5_stripecache_active = __ATTR_RO(stripe_cache_active);
+static ssize_t
+raid5_show_group_thread_cnt(struct mddev *mddev, char *page)
+{
+ struct r5conf *conf = mddev->private;
+ if (conf)
+ return sprintf(page, "%d\n", conf->worker_cnt_per_group);
+ else
+ return 0;
+}
+
+static int alloc_thread_groups(struct r5conf *conf, int cnt);
+static ssize_t
+raid5_store_group_thread_cnt(struct mddev *mddev, const char *page, size_t len)
+{
+ struct r5conf *conf = mddev->private;
+ unsigned long new;
+ int err;
+ struct r5worker_group *old_groups;
+ int old_group_cnt;
+
+ if (len >= PAGE_SIZE)
+ return -EINVAL;
+ if (!conf)
+ return -ENODEV;
+
+ if (kstrtoul(page, 10, &new))
+ return -EINVAL;
+
+ if (new == conf->worker_cnt_per_group)
+ return len;
+
+ mddev_suspend(mddev);
+
+ old_groups = conf->worker_groups;
+ old_group_cnt = conf->worker_cnt_per_group;
+
+ conf->worker_groups = NULL;
+ err = alloc_thread_groups(conf, new);
+ if (err) {
+ conf->worker_groups = old_groups;
+ conf->worker_cnt_per_group = old_group_cnt;
+ } else {
+ if (old_groups)
+ kfree(old_groups[0].workers);
+ kfree(old_groups);
+ }
+
+ mddev_resume(mddev);
+
+ if (err)
+ return err;
+ return len;
+}
+
+static struct md_sysfs_entry
+raid5_group_thread_cnt = __ATTR(group_thread_cnt, S_IRUGO | S_IWUSR,
+ raid5_show_group_thread_cnt,
+ raid5_store_group_thread_cnt);
+
static struct attribute *raid5_attrs[] = {
&raid5_stripecache_size.attr,
&raid5_stripecache_active.attr,
&raid5_preread_bypass_threshold.attr,
+ &raid5_group_thread_cnt.attr,
NULL,
};
static struct attribute_group raid5_attrs_group = {
@@ -5000,6 +5254,54 @@ static struct attribute_group raid5_attrs_group = {
.attrs = raid5_attrs,
};
+static int alloc_thread_groups(struct r5conf *conf, int cnt)
+{
+ int i, j;
+ ssize_t size;
+ struct r5worker *workers;
+
+ conf->worker_cnt_per_group = cnt;
+ if (cnt == 0) {
+ conf->worker_groups = NULL;
+ return 0;
+ }
+ conf->group_cnt = num_possible_nodes();
+ size = sizeof(struct r5worker) * cnt;
+ workers = kzalloc(size * conf->group_cnt, GFP_NOIO);
+ conf->worker_groups = kzalloc(sizeof(struct r5worker_group) *
+ conf->group_cnt, GFP_NOIO);
+ if (!conf->worker_groups || !workers) {
+ kfree(workers);
+ kfree(conf->worker_groups);
+ conf->worker_groups = NULL;
+ return -ENOMEM;
+ }
+
+ for (i = 0; i < conf->group_cnt; i++) {
+ struct r5worker_group *group;
+
+ group = &conf->worker_groups[i];
+ INIT_LIST_HEAD(&group->handle_list);
+ group->conf = conf;
+ group->workers = workers + i * cnt;
+
+ for (j = 0; j < cnt; j++) {
+ group->workers[j].group = group;
+ INIT_WORK(&group->workers[j].work, raid5_do_work);
+ }
+ }
+
+ return 0;
+}
+
+static void free_thread_groups(struct r5conf *conf)
+{
+ if (conf->worker_groups)
+ kfree(conf->worker_groups[0].workers);
+ kfree(conf->worker_groups);
+ conf->worker_groups = NULL;
+}
+
static sector_t
raid5_size(struct mddev *mddev, sector_t sectors, int raid_disks)
{
@@ -5040,6 +5342,7 @@ static void raid5_free_percpu(struct r5conf *conf)
static void free_conf(struct r5conf *conf)
{
+ free_thread_groups(conf);
shrink_stripes(conf);
raid5_free_percpu(conf);
kfree(conf->disks);
@@ -5168,7 +5471,11 @@ static struct r5conf *setup_conf(struct mddev *mddev)
conf = kzalloc(sizeof(struct r5conf), GFP_KERNEL);
if (conf == NULL)
goto abort;
+ /* Don't enable multi-threading by default*/
+ if (alloc_thread_groups(conf, 0))
+ goto abort;
spin_lock_init(&conf->device_lock);
+ seqcount_init(&conf->gen_lock);
init_waitqueue_head(&conf->wait_for_stripe);
init_waitqueue_head(&conf->wait_for_overlap);
INIT_LIST_HEAD(&conf->handle_list);
@@ -5176,6 +5483,7 @@ static struct r5conf *setup_conf(struct mddev *mddev)
INIT_LIST_HEAD(&conf->delayed_list);
INIT_LIST_HEAD(&conf->bitmap_list);
INIT_LIST_HEAD(&conf->inactive_list);
+ init_llist_head(&conf->released_stripes);
atomic_set(&conf->active_stripes, 0);
atomic_set(&conf->preread_active_stripes, 0);
atomic_set(&conf->active_aligned_reads, 0);
@@ -5980,6 +6288,7 @@ static int raid5_start_reshape(struct mddev *mddev)
atomic_set(&conf->reshape_stripes, 0);
spin_lock_irq(&conf->device_lock);
+ write_seqcount_begin(&conf->gen_lock);
conf->previous_raid_disks = conf->raid_disks;
conf->raid_disks += mddev->delta_disks;
conf->prev_chunk_sectors = conf->chunk_sectors;
@@ -5996,8 +6305,16 @@ static int raid5_start_reshape(struct mddev *mddev)
else
conf->reshape_progress = 0;
conf->reshape_safe = conf->reshape_progress;
+ write_seqcount_end(&conf->gen_lock);
spin_unlock_irq(&conf->device_lock);
+ /* Now make sure any requests that proceeded on the assumption
+ * the reshape wasn't running - like Discard or Read - have
+ * completed.
+ */
+ mddev_suspend(mddev);
+ mddev_resume(mddev);
+
/* Add some new drives, as many as will fit.
* We know there are enough to make the newly sized array work.
* Don't add devices if we are reducing the number of
@@ -6472,6 +6789,10 @@ static struct md_personality raid4_personality =
static int __init raid5_init(void)
{
+ raid5_wq = alloc_workqueue("raid5wq",
+ WQ_UNBOUND|WQ_MEM_RECLAIM|WQ_CPU_INTENSIVE|WQ_SYSFS, 0);
+ if (!raid5_wq)
+ return -ENOMEM;
register_md_personality(&raid6_personality);
register_md_personality(&raid5_personality);
register_md_personality(&raid4_personality);
@@ -6483,6 +6804,7 @@ static void raid5_exit(void)
unregister_md_personality(&raid6_personality);
unregister_md_personality(&raid5_personality);
unregister_md_personality(&raid4_personality);
+ destroy_workqueue(raid5_wq);
}
module_init(raid5_init);
diff --git a/drivers/md/raid5.h b/drivers/md/raid5.h
index 70c49329ca9..2113ffa82c7 100644
--- a/drivers/md/raid5.h
+++ b/drivers/md/raid5.h
@@ -197,6 +197,7 @@ enum reconstruct_states {
struct stripe_head {
struct hlist_node hash;
struct list_head lru; /* inactive_list or handle_list */
+ struct llist_node release_list;
struct r5conf *raid_conf;
short generation; /* increments with every
* reshape */
@@ -211,6 +212,8 @@ struct stripe_head {
enum check_states check_state;
enum reconstruct_states reconstruct_state;
spinlock_t stripe_lock;
+ int cpu;
+ struct r5worker_group *group;
/**
* struct stripe_operations
* @target - STRIPE_OP_COMPUTE_BLK target
@@ -321,6 +324,7 @@ enum {
STRIPE_OPS_REQ_PENDING,
STRIPE_ON_UNPLUG_LIST,
STRIPE_DISCARD,
+ STRIPE_ON_RELEASE_LIST,
};
/*
@@ -363,6 +367,19 @@ struct disk_info {
struct md_rdev *rdev, *replacement;
};
+struct r5worker {
+ struct work_struct work;
+ struct r5worker_group *group;
+ bool working;
+};
+
+struct r5worker_group {
+ struct list_head handle_list;
+ struct r5conf *conf;
+ struct r5worker *workers;
+ int stripes_cnt;
+};
+
struct r5conf {
struct hlist_head *stripe_hashtbl;
struct mddev *mddev;
@@ -386,6 +403,7 @@ struct r5conf {
int prev_chunk_sectors;
int prev_algo;
short generation; /* increments with every reshape */
+ seqcount_t gen_lock; /* lock against generation changes */
unsigned long reshape_checkpoint; /* Time we last updated
* metadata */
long long min_offset_diff; /* minimum difference between
@@ -445,6 +463,7 @@ struct r5conf {
*/
atomic_t active_stripes;
struct list_head inactive_list;
+ struct llist_head released_stripes;
wait_queue_head_t wait_for_stripe;
wait_queue_head_t wait_for_overlap;
int inactive_blocked; /* release of inactive stripes blocked,
@@ -458,6 +477,9 @@ struct r5conf {
* the new thread here until we fully activate the array.
*/
struct md_thread *thread;
+ struct r5worker_group *worker_groups;
+ int group_cnt;
+ int worker_cnt_per_group;
};
/*