aboutsummaryrefslogtreecommitdiff
path: root/drivers/md/raid1.c
diff options
context:
space:
mode:
Diffstat (limited to 'drivers/md/raid1.c')
-rw-r--r--drivers/md/raid1.c1185
1 files changed, 623 insertions, 562 deletions
diff --git a/drivers/md/raid1.c b/drivers/md/raid1.c
index 7b0f647bcccb..3febfc8391fb 100644
--- a/drivers/md/raid1.c
+++ b/drivers/md/raid1.c
@@ -37,14 +37,18 @@
#include <linux/module.h>
#include <linux/seq_file.h>
#include <linux/ratelimit.h>
+#include <linux/sched/signal.h>
+
#include <trace/events/block.h>
+
#include "md.h"
#include "raid1.h"
#include "bitmap.h"
#define UNSUPPORTED_MDDEV_FLAGS \
((1L << MD_HAS_JOURNAL) | \
- (1L << MD_JOURNAL_CLEAN))
+ (1L << MD_JOURNAL_CLEAN) | \
+ (1L << MD_HAS_PPL))
/*
* Number of guaranteed r1bios in case of extreme VM load:
@@ -71,13 +75,30 @@
*/
static int max_queued_requests = 1024;
-static void allow_barrier(struct r1conf *conf, sector_t start_next_window,
- sector_t bi_sector);
-static void lower_barrier(struct r1conf *conf);
+static void allow_barrier(struct r1conf *conf, sector_t sector_nr);
+static void lower_barrier(struct r1conf *conf, sector_t sector_nr);
#define raid1_log(md, fmt, args...) \
do { if ((md)->queue) blk_add_trace_msg((md)->queue, "raid1 " fmt, ##args); } while (0)
+/*
+ * 'strct resync_pages' stores actual pages used for doing the resync
+ * IO, and it is per-bio, so make .bi_private points to it.
+ */
+static inline struct resync_pages *get_resync_pages(struct bio *bio)
+{
+ return bio->bi_private;
+}
+
+/*
+ * for resync bio, r1bio pointer can be retrieved from the per-bio
+ * 'struct resync_pages'.
+ */
+static inline struct r1bio *get_resync_r1bio(struct bio *bio)
+{
+ return get_resync_pages(bio)->raid_bio;
+}
+
static void * r1bio_pool_alloc(gfp_t gfp_flags, void *data)
{
struct pool_info *pi = data;
@@ -92,15 +113,12 @@ static void r1bio_pool_free(void *r1_bio, void *data)
kfree(r1_bio);
}
-#define RESYNC_BLOCK_SIZE (64*1024)
#define RESYNC_DEPTH 32
#define RESYNC_SECTORS (RESYNC_BLOCK_SIZE >> 9)
-#define RESYNC_PAGES ((RESYNC_BLOCK_SIZE + PAGE_SIZE-1) / PAGE_SIZE)
#define RESYNC_WINDOW (RESYNC_BLOCK_SIZE * RESYNC_DEPTH)
#define RESYNC_WINDOW_SECTORS (RESYNC_WINDOW >> 9)
#define CLUSTER_RESYNC_WINDOW (16 * RESYNC_WINDOW)
#define CLUSTER_RESYNC_WINDOW_SECTORS (CLUSTER_RESYNC_WINDOW >> 9)
-#define NEXT_NORMALIO_DISTANCE (3 * RESYNC_WINDOW_SECTORS)
static void * r1buf_pool_alloc(gfp_t gfp_flags, void *data)
{
@@ -108,12 +126,18 @@ static void * r1buf_pool_alloc(gfp_t gfp_flags, void *data)
struct r1bio *r1_bio;
struct bio *bio;
int need_pages;
- int i, j;
+ int j;
+ struct resync_pages *rps;
r1_bio = r1bio_pool_alloc(gfp_flags, pi);
if (!r1_bio)
return NULL;
+ rps = kmalloc(sizeof(struct resync_pages) * pi->raid_disks,
+ gfp_flags);
+ if (!rps)
+ goto out_free_r1bio;
+
/*
* Allocate bios : 1 for reading, n-1 for writing
*/
@@ -133,19 +157,22 @@ static void * r1buf_pool_alloc(gfp_t gfp_flags, void *data)
need_pages = pi->raid_disks;
else
need_pages = 1;
- for (j = 0; j < need_pages; j++) {
+ for (j = 0; j < pi->raid_disks; j++) {
+ struct resync_pages *rp = &rps[j];
+
bio = r1_bio->bios[j];
- bio->bi_vcnt = RESYNC_PAGES;
- if (bio_alloc_pages(bio, gfp_flags))
- goto out_free_pages;
- }
- /* If not user-requests, copy the page pointers to all bios */
- if (!test_bit(MD_RECOVERY_REQUESTED, &pi->mddev->recovery)) {
- for (i=0; i<RESYNC_PAGES ; i++)
- for (j=1; j<pi->raid_disks; j++)
- r1_bio->bios[j]->bi_io_vec[i].bv_page =
- r1_bio->bios[0]->bi_io_vec[i].bv_page;
+ if (j < need_pages) {
+ if (resync_alloc_pages(rp, gfp_flags))
+ goto out_free_pages;
+ } else {
+ memcpy(rp, &rps[0], sizeof(*rp));
+ resync_get_all_pages(rp);
+ }
+
+ rp->idx = 0;
+ rp->raid_bio = r1_bio;
+ bio->bi_private = rp;
}
r1_bio->master_bio = NULL;
@@ -154,11 +181,14 @@ static void * r1buf_pool_alloc(gfp_t gfp_flags, void *data)
out_free_pages:
while (--j >= 0)
- bio_free_pages(r1_bio->bios[j]);
+ resync_free_pages(&rps[j]);
out_free_bio:
while (++j < pi->raid_disks)
bio_put(r1_bio->bios[j]);
+ kfree(rps);
+
+out_free_r1bio:
r1bio_pool_free(r1_bio, data);
return NULL;
}
@@ -166,18 +196,18 @@ out_free_bio:
static void r1buf_pool_free(void *__r1_bio, void *data)
{
struct pool_info *pi = data;
- int i,j;
+ int i;
struct r1bio *r1bio = __r1_bio;
+ struct resync_pages *rp = NULL;
- for (i = 0; i < RESYNC_PAGES; i++)
- for (j = pi->raid_disks; j-- ;) {
- if (j == 0 ||
- r1bio->bios[j]->bi_io_vec[i].bv_page !=
- r1bio->bios[0]->bi_io_vec[i].bv_page)
- safe_put_page(r1bio->bios[j]->bi_io_vec[i].bv_page);
- }
- for (i=0 ; i < pi->raid_disks; i++)
+ for (i = pi->raid_disks; i--; ) {
+ rp = get_resync_pages(r1bio->bios[i]);
+ resync_free_pages(rp);
bio_put(r1bio->bios[i]);
+ }
+
+ /* resync pages array stored in the 1st bio's .bi_private */
+ kfree(rp);
r1bio_pool_free(r1bio, data);
}
@@ -205,6 +235,7 @@ static void free_r1bio(struct r1bio *r1_bio)
static void put_buf(struct r1bio *r1_bio)
{
struct r1conf *conf = r1_bio->mddev->private;
+ sector_t sect = r1_bio->sector;
int i;
for (i = 0; i < conf->raid_disks * 2; i++) {
@@ -215,7 +246,7 @@ static void put_buf(struct r1bio *r1_bio)
mempool_free(r1_bio, conf->r1buf_pool);
- lower_barrier(conf);
+ lower_barrier(conf, sect);
}
static void reschedule_retry(struct r1bio *r1_bio)
@@ -223,10 +254,12 @@ static void reschedule_retry(struct r1bio *r1_bio)
unsigned long flags;
struct mddev *mddev = r1_bio->mddev;
struct r1conf *conf = mddev->private;
+ int idx;
+ idx = sector_to_idx(r1_bio->sector);
spin_lock_irqsave(&conf->device_lock, flags);
list_add(&r1_bio->retry_list, &conf->retry_list);
- conf->nr_queued ++;
+ atomic_inc(&conf->nr_queued[idx]);
spin_unlock_irqrestore(&conf->device_lock, flags);
wake_up(&conf->wait_barrier);
@@ -241,36 +274,17 @@ static void reschedule_retry(struct r1bio *r1_bio)
static void call_bio_endio(struct r1bio *r1_bio)
{
struct bio *bio = r1_bio->master_bio;
- int done;
struct r1conf *conf = r1_bio->mddev->private;
- sector_t start_next_window = r1_bio->start_next_window;
- sector_t bi_sector = bio->bi_iter.bi_sector;
-
- if (bio->bi_phys_segments) {
- unsigned long flags;
- spin_lock_irqsave(&conf->device_lock, flags);
- bio->bi_phys_segments--;
- done = (bio->bi_phys_segments == 0);
- spin_unlock_irqrestore(&conf->device_lock, flags);
- /*
- * make_request() might be waiting for
- * bi_phys_segments to decrease
- */
- wake_up(&conf->wait_barrier);
- } else
- done = 1;
if (!test_bit(R1BIO_Uptodate, &r1_bio->state))
- bio->bi_error = -EIO;
+ bio->bi_status = BLK_STS_IOERR;
- if (done) {
- bio_endio(bio);
- /*
- * Wake up any possible resync thread that waits for the device
- * to go idle.
- */
- allow_barrier(conf, start_next_window, bi_sector);
- }
+ bio_endio(bio);
+ /*
+ * Wake up any possible resync thread that waits for the device
+ * to go idle.
+ */
+ allow_barrier(conf, r1_bio->sector);
}
static void raid_end_bio_io(struct r1bio *r1_bio)
@@ -321,7 +335,7 @@ static int find_bio_disk(struct r1bio *r1_bio, struct bio *bio)
static void raid1_end_read_request(struct bio *bio)
{
- int uptodate = !bio->bi_error;
+ int uptodate = !bio->bi_status;
struct r1bio *r1_bio = bio->bi_private;
struct r1conf *conf = r1_bio->mddev->private;
struct md_rdev *rdev = conf->mirrors[r1_bio->read_disk].rdev;
@@ -374,12 +388,9 @@ static void close_write(struct r1bio *r1_bio)
{
/* it really is the end of this request */
if (test_bit(R1BIO_BehindIO, &r1_bio->state)) {
- /* free extra copy of the data pages */
- int i = r1_bio->behind_page_count;
- while (i--)
- safe_put_page(r1_bio->behind_bvecs[i].bv_page);
- kfree(r1_bio->behind_bvecs);
- r1_bio->behind_bvecs = NULL;
+ bio_free_pages(r1_bio->behind_master_bio);
+ bio_put(r1_bio->behind_master_bio);
+ r1_bio->behind_master_bio = NULL;
}
/* clear the bitmap if all writes complete successfully */
bitmap_endwrite(r1_bio->mddev->bitmap, r1_bio->sector,
@@ -415,12 +426,12 @@ static void raid1_end_write_request(struct bio *bio)
struct md_rdev *rdev = conf->mirrors[mirror].rdev;
bool discard_error;
- discard_error = bio->bi_error && bio_op(bio) == REQ_OP_DISCARD;
+ discard_error = bio->bi_status && bio_op(bio) == REQ_OP_DISCARD;
/*
* 'one mirror IO has finished' event handler:
*/
- if (bio->bi_error && !discard_error) {
+ if (bio->bi_status && !discard_error) {
set_bit(WriteErrorSeen, &rdev->flags);
if (!test_and_set_bit(WantReplacement, &rdev->flags))
set_bit(MD_RECOVERY_NEEDED, &
@@ -481,6 +492,10 @@ static void raid1_end_write_request(struct bio *bio)
}
if (behind) {
+ /* we release behind master bio when all write are done */
+ if (r1_bio->behind_master_bio == bio)
+ to_put = NULL;
+
if (test_bit(WriteMostly, &rdev->flags))
atomic_dec(&r1_bio->behind_remaining);
@@ -517,6 +532,25 @@ static void raid1_end_write_request(struct bio *bio)
bio_put(to_put);
}
+static sector_t align_to_barrier_unit_end(sector_t start_sector,
+ sector_t sectors)
+{
+ sector_t len;
+
+ WARN_ON(sectors == 0);
+ /*
+ * len is the number of sectors from start_sector to end of the
+ * barrier unit which start_sector belongs to.
+ */
+ len = round_up(start_sector + 1, BARRIER_UNIT_SECTOR_SIZE) -
+ start_sector;
+
+ if (len > sectors)
+ len = sectors;
+
+ return len;
+}
+
/*
* This routine returns the disk from which the requested read should
* be done. There is a per-array 'next expected sequential IO' sector
@@ -632,8 +666,11 @@ static int read_balance(struct r1conf *conf, struct r1bio *r1_bio, int *max_sect
break;
}
continue;
- } else
+ } else {
+ if ((sectors > best_good_sectors) && (best_disk >= 0))
+ best_disk = -1;
best_good_sectors = sectors;
+ }
if (best_disk >= 0)
/* At least two disks to choose from so failfast is OK */
@@ -744,15 +781,39 @@ static int raid1_congested(struct mddev *mddev, int bits)
* non-congested targets, it can be removed
*/
if ((bits & (1 << WB_async_congested)) || 1)
- ret |= bdi_congested(&q->backing_dev_info, bits);
+ ret |= bdi_congested(q->backing_dev_info, bits);
else
- ret &= bdi_congested(&q->backing_dev_info, bits);
+ ret &= bdi_congested(q->backing_dev_info, bits);
}
}
rcu_read_unlock();
return ret;
}
+static void flush_bio_list(struct r1conf *conf, struct bio *bio)
+{
+ /* flush any pending bitmap writes to disk before proceeding w/ I/O */
+ bitmap_unplug(conf->mddev->bitmap);
+ wake_up(&conf->wait_barrier);
+
+ while (bio) { /* submit pending writes */
+ struct bio *next = bio->bi_next;
+ struct md_rdev *rdev = (void*)bio->bi_bdev;
+ bio->bi_next = NULL;
+ bio->bi_bdev = rdev->bdev;
+ if (test_bit(Faulty, &rdev->flags)) {
+ bio->bi_status = BLK_STS_IOERR;
+ bio_endio(bio);
+ } else if (unlikely((bio_op(bio) == REQ_OP_DISCARD) &&
+ !blk_queue_discard(bdev_get_queue(bio->bi_bdev))))
+ /* Just ignore it */
+ bio_endio(bio);
+ else
+ generic_make_request(bio);
+ bio = next;
+ }
+}
+
static void flush_pending_writes(struct r1conf *conf)
{
/* Any writes that have been queued but are awaiting
@@ -765,27 +826,7 @@ static void flush_pending_writes(struct r1conf *conf)
bio = bio_list_get(&conf->pending_bio_list);
conf->pending_count = 0;
spin_unlock_irq(&conf->device_lock);
- /* flush any pending bitmap writes to
- * disk before proceeding w/ I/O */
- bitmap_unplug(conf->mddev->bitmap);
- wake_up(&conf->wait_barrier);
-
- while (bio) { /* submit pending writes */
- struct bio *next = bio->bi_next;
- struct md_rdev *rdev = (void*)bio->bi_bdev;
- bio->bi_next = NULL;
- bio->bi_bdev = rdev->bdev;
- if (test_bit(Faulty, &rdev->flags)) {
- bio->bi_error = -EIO;
- bio_endio(bio);
- } else if (unlikely((bio_op(bio) == REQ_OP_DISCARD) &&
- !blk_queue_discard(bdev_get_queue(bio->bi_bdev))))
- /* Just ignore it */
- bio_endio(bio);
- else
- generic_make_request(bio);
- bio = next;
- }
+ flush_bio_list(conf, bio);
} else
spin_unlock_irq(&conf->device_lock);
}
@@ -813,168 +854,229 @@ static void flush_pending_writes(struct r1conf *conf)
*/
static void raise_barrier(struct r1conf *conf, sector_t sector_nr)
{
+ int idx = sector_to_idx(sector_nr);
+
spin_lock_irq(&conf->resync_lock);
/* Wait until no block IO is waiting */
- wait_event_lock_irq(conf->wait_barrier, !conf->nr_waiting,
+ wait_event_lock_irq(conf->wait_barrier,
+ !atomic_read(&conf->nr_waiting[idx]),
conf->resync_lock);
/* block any new IO from starting */
- conf->barrier++;
- conf->next_resync = sector_nr;
+ atomic_inc(&conf->barrier[idx]);
+ /*
+ * In raise_barrier() we firstly increase conf->barrier[idx] then
+ * check conf->nr_pending[idx]. In _wait_barrier() we firstly
+ * increase conf->nr_pending[idx] then check conf->barrier[idx].
+ * A memory barrier here to make sure conf->nr_pending[idx] won't
+ * be fetched before conf->barrier[idx] is increased. Otherwise
+ * there will be a race between raise_barrier() and _wait_barrier().
+ */
+ smp_mb__after_atomic();
/* For these conditions we must wait:
* A: while the array is in frozen state
- * B: while barrier >= RESYNC_DEPTH, meaning resync reach
- * the max count which allowed.
- * C: next_resync + RESYNC_SECTORS > start_next_window, meaning
- * next resync will reach to the window which normal bios are
- * handling.
- * D: while there are any active requests in the current window.
+ * B: while conf->nr_pending[idx] is not 0, meaning regular I/O
+ * existing in corresponding I/O barrier bucket.
+ * C: while conf->barrier[idx] >= RESYNC_DEPTH, meaning reaches
+ * max resync count which allowed on current I/O barrier bucket.
*/
wait_event_lock_irq(conf->wait_barrier,
!conf->array_frozen &&
- conf->barrier < RESYNC_DEPTH &&
- conf->current_window_requests == 0 &&
- (conf->start_next_window >=
- conf->next_resync + RESYNC_SECTORS),
+ !atomic_read(&conf->nr_pending[idx]) &&
+ atomic_read(&conf->barrier[idx]) < RESYNC_DEPTH,
conf->resync_lock);
- conf->nr_pending++;
+ atomic_inc(&conf->nr_sync_pending);
spin_unlock_irq(&conf->resync_lock);
}
-static void lower_barrier(struct r1conf *conf)
+static void lower_barrier(struct r1conf *conf, sector_t sector_nr)
{
- unsigned long flags;
- BUG_ON(conf->barrier <= 0);
- spin_lock_irqsave(&conf->resync_lock, flags);
- conf->barrier--;
- conf->nr_pending--;
- spin_unlock_irqrestore(&conf->resync_lock, flags);
+ int idx = sector_to_idx(sector_nr);
+
+ BUG_ON(atomic_read(&conf->barrier[idx]) <= 0);
+
+ atomic_dec(&conf->barrier[idx]);
+ atomic_dec(&conf->nr_sync_pending);
wake_up(&conf->wait_barrier);
}
-static bool need_to_wait_for_sync(struct r1conf *conf, struct bio *bio)
+static void _wait_barrier(struct r1conf *conf, int idx)
{
- bool wait = false;
+ /*
+ * We need to increase conf->nr_pending[idx] very early here,
+ * then raise_barrier() can be blocked when it waits for
+ * conf->nr_pending[idx] to be 0. Then we can avoid holding
+ * conf->resync_lock when there is no barrier raised in same
+ * barrier unit bucket. Also if the array is frozen, I/O
+ * should be blocked until array is unfrozen.
+ */
+ atomic_inc(&conf->nr_pending[idx]);
+ /*
+ * In _wait_barrier() we firstly increase conf->nr_pending[idx], then
+ * check conf->barrier[idx]. In raise_barrier() we firstly increase
+ * conf->barrier[idx], then check conf->nr_pending[idx]. A memory
+ * barrier is necessary here to make sure conf->barrier[idx] won't be
+ * fetched before conf->nr_pending[idx] is increased. Otherwise there
+ * will be a race between _wait_barrier() and raise_barrier().
+ */
+ smp_mb__after_atomic();
- if (conf->array_frozen || !bio)
- wait = true;
- else if (conf->barrier && bio_data_dir(bio) == WRITE) {
- if ((conf->mddev->curr_resync_completed
- >= bio_end_sector(bio)) ||
- (conf->start_next_window + NEXT_NORMALIO_DISTANCE
- <= bio->bi_iter.bi_sector))
- wait = false;
- else
- wait = true;
- }
+ /*
+ * Don't worry about checking two atomic_t variables at same time
+ * here. If during we check conf->barrier[idx], the array is
+ * frozen (conf->array_frozen is 1), and chonf->barrier[idx] is
+ * 0, it is safe to return and make the I/O continue. Because the
+ * array is frozen, all I/O returned here will eventually complete
+ * or be queued, no race will happen. See code comment in
+ * frozen_array().
+ */
+ if (!READ_ONCE(conf->array_frozen) &&
+ !atomic_read(&conf->barrier[idx]))
+ return;
- return wait;
+ /*
+ * After holding conf->resync_lock, conf->nr_pending[idx]
+ * should be decreased before waiting for barrier to drop.
+ * Otherwise, we may encounter a race condition because
+ * raise_barrer() might be waiting for conf->nr_pending[idx]
+ * to be 0 at same time.
+ */
+ spin_lock_irq(&conf->resync_lock);
+ atomic_inc(&conf->nr_waiting[idx]);
+ atomic_dec(&conf->nr_pending[idx]);
+ /*
+ * In case freeze_array() is waiting for
+ * get_unqueued_pending() == extra
+ */
+ wake_up(&conf->wait_barrier);
+ /* Wait for the barrier in same barrier unit bucket to drop. */
+ wait_event_lock_irq(conf->wait_barrier,
+ !conf->array_frozen &&
+ !atomic_read(&conf->barrier[idx]),
+ conf->resync_lock);
+ atomic_inc(&conf->nr_pending[idx]);
+ atomic_dec(&conf->nr_waiting[idx]);
+ spin_unlock_irq(&conf->resync_lock);
}
-static sector_t wait_barrier(struct r1conf *conf, struct bio *bio)
+static void wait_read_barrier(struct r1conf *conf, sector_t sector_nr)
{
- sector_t sector = 0;
+ int idx = sector_to_idx(sector_nr);
- spin_lock_irq(&conf->resync_lock);
- if (need_to_wait_for_sync(conf, bio)) {
- conf->nr_waiting++;
- /* Wait for the barrier to drop.
- * However if there are already pending
- * requests (preventing the barrier from
- * rising completely), and the
- * per-process bio queue isn't empty,
- * then don't wait, as we need to empty
- * that queue to allow conf->start_next_window
- * to increase.
- */
- raid1_log(conf->mddev, "wait barrier");
- wait_event_lock_irq(conf->wait_barrier,
- !conf->array_frozen &&
- (!conf->barrier ||
- ((conf->start_next_window <
- conf->next_resync + RESYNC_SECTORS) &&
- current->bio_list &&
- !bio_list_empty(current->bio_list))),
- conf->resync_lock);
- conf->nr_waiting--;
- }
-
- if (bio && bio_data_dir(bio) == WRITE) {
- if (bio->bi_iter.bi_sector >= conf->next_resync) {
- if (conf->start_next_window == MaxSector)
- conf->start_next_window =
- conf->next_resync +
- NEXT_NORMALIO_DISTANCE;
-
- if ((conf->start_next_window + NEXT_NORMALIO_DISTANCE)
- <= bio->bi_iter.bi_sector)
- conf->next_window_requests++;
- else
- conf->current_window_requests++;
- sector = conf->start_next_window;
- }
- }
+ /*
+ * Very similar to _wait_barrier(). The difference is, for read
+ * I/O we don't need wait for sync I/O, but if the whole array
+ * is frozen, the read I/O still has to wait until the array is
+ * unfrozen. Since there is no ordering requirement with
+ * conf->barrier[idx] here, memory barrier is unnecessary as well.
+ */
+ atomic_inc(&conf->nr_pending[idx]);
+
+ if (!READ_ONCE(conf->array_frozen))
+ return;
- conf->nr_pending++;
+ spin_lock_irq(&conf->resync_lock);
+ atomic_inc(&conf->nr_waiting[idx]);
+ atomic_dec(&conf->nr_pending[idx]);
+ /*
+ * In case freeze_array() is waiting for
+ * get_unqueued_pending() == extra
+ */
+ wake_up(&conf->wait_barrier);
+ /* Wait for array to be unfrozen */
+ wait_event_lock_irq(conf->wait_barrier,
+ !conf->array_frozen,
+ conf->resync_lock);
+ atomic_inc(&conf->nr_pending[idx]);
+ atomic_dec(&conf->nr_waiting[idx]);
spin_unlock_irq(&conf->resync_lock);
- return sector;
}
-static void allow_barrier(struct r1conf *conf, sector_t start_next_window,
- sector_t bi_sector)
+static void wait_barrier(struct r1conf *conf, sector_t sector_nr)
{
- unsigned long flags;
+ int idx = sector_to_idx(sector_nr);
- spin_lock_irqsave(&conf->resync_lock, flags);
- conf->nr_pending--;
- if (start_next_window) {
- if (start_next_window == conf->start_next_window) {
- if (conf->start_next_window + NEXT_NORMALIO_DISTANCE
- <= bi_sector)
- conf->next_window_requests--;
- else
- conf->current_window_requests--;
- } else
- conf->current_window_requests--;
-
- if (!conf->current_window_requests) {
- if (conf->next_window_requests) {
- conf->current_window_requests =
- conf->next_window_requests;
- conf->next_window_requests = 0;
- conf->start_next_window +=
- NEXT_NORMALIO_DISTANCE;
- } else
- conf->start_next_window = MaxSector;
- }
- }
- spin_unlock_irqrestore(&conf->resync_lock, flags);
+ _wait_barrier(conf, idx);
+}
+
+static void wait_all_barriers(struct r1conf *conf)
+{
+ int idx;
+
+ for (idx = 0; idx < BARRIER_BUCKETS_NR; idx++)
+ _wait_barrier(conf, idx);
+}
+
+static void _allow_barrier(struct r1conf *conf, int idx)
+{
+ atomic_dec(&conf->nr_pending[idx]);
wake_up(&conf->wait_barrier);
}
+static void allow_barrier(struct r1conf *conf, sector_t sector_nr)
+{
+ int idx = sector_to_idx(sector_nr);
+
+ _allow_barrier(conf, idx);
+}
+
+static void allow_all_barriers(struct r1conf *conf)
+{
+ int idx;
+
+ for (idx = 0; idx < BARRIER_BUCKETS_NR; idx++)
+ _allow_barrier(conf, idx);
+}
+
+/* conf->resync_lock should be held */
+static int get_unqueued_pending(struct r1conf *conf)
+{
+ int idx, ret;
+
+ ret = atomic_read(&conf->nr_sync_pending);
+ for (idx = 0; idx < BARRIER_BUCKETS_NR; idx++)
+ ret += atomic_read(&conf->nr_pending[idx]) -
+ atomic_read(&conf->nr_queued[idx]);
+
+ return ret;
+}
+
static void freeze_array(struct r1conf *conf, int extra)
{
- /* stop syncio and normal IO and wait for everything to
- * go quite.
- * We wait until nr_pending match nr_queued+extra
- * This is called in the context of one normal IO request
- * that has failed. Thus any sync request that might be pending
- * will be blocked by nr_pending, and we need to wait for
- * pending IO requests to complete or be queued for re-try.
- * Thus the number queued (nr_queued) plus this request (extra)
- * must match the number of pending IOs (nr_pending) before
- * we continue.
+ /* Stop sync I/O and normal I/O and wait for everything to
+ * go quiet.
+ * This is called in two situations:
+ * 1) management command handlers (reshape, remove disk, quiesce).
+ * 2) one normal I/O request failed.
+
+ * After array_frozen is set to 1, new sync IO will be blocked at
+ * raise_barrier(), and new normal I/O will blocked at _wait_barrier()
+ * or wait_read_barrier(). The flying I/Os will either complete or be
+ * queued. When everything goes quite, there are only queued I/Os left.
+
+ * Every flying I/O contributes to a conf->nr_pending[idx], idx is the
+ * barrier bucket index which this I/O request hits. When all sync and
+ * normal I/O are queued, sum of all conf->nr_pending[] will match sum
+ * of all conf->nr_queued[]. But normal I/O failure is an exception,
+ * in handle_read_error(), we may call freeze_array() before trying to
+ * fix the read error. In this case, the error read I/O is not queued,
+ * so get_unqueued_pending() == 1.
+ *
+ * Therefore before this function returns, we need to wait until
+ * get_unqueued_pendings(conf) gets equal to extra. For
+ * normal I/O context, extra is 1, in rested situations extra is 0.
*/
spin_lock_irq(&conf->resync_lock);
conf->array_frozen = 1;
raid1_log(conf->mddev, "wait freeze");
- wait_event_lock_irq_cmd(conf->wait_barrier,
- conf->nr_pending == conf->nr_queued+extra,
- conf->resync_lock,
- flush_pending_writes(conf));
+ wait_event_lock_irq_cmd(
+ conf->wait_barrier,
+ get_unqueued_pending(conf) == extra,
+ conf->resync_lock,
+ flush_pending_writes(conf));
spin_unlock_irq(&conf->resync_lock);
}
static void unfreeze_array(struct r1conf *conf)
@@ -982,43 +1084,53 @@ static void unfreeze_array(struct r1conf *conf)
/* reverse the effect of the freeze */
spin_lock_irq(&conf->resync_lock);
conf->array_frozen = 0;
- wake_up(&conf->wait_barrier);
spin_unlock_irq(&conf->resync_lock);
+ wake_up(&conf->wait_barrier);
}
-/* duplicate the data pages for behind I/O
- */
-static void alloc_behind_pages(struct bio *bio, struct r1bio *r1_bio)
+static struct bio *alloc_behind_master_bio(struct r1bio *r1_bio,
+ struct bio *bio)
{
- int i;
- struct bio_vec *bvec;
- struct bio_vec *bvecs = kzalloc(bio->bi_vcnt * sizeof(struct bio_vec),
- GFP_NOIO);
- if (unlikely(!bvecs))
- return;
+ int size = bio->bi_iter.bi_size;
+ unsigned vcnt = (size + PAGE_SIZE - 1) >> PAGE_SHIFT;
+ int i = 0;
+ struct bio *behind_bio = NULL;
+
+ behind_bio = bio_alloc_mddev(GFP_NOIO, vcnt, r1_bio->mddev);
+ if (!behind_bio)
+ goto fail;
+
+ /* discard op, we don't support writezero/writesame yet */
+ if (!bio_has_data(bio))
+ goto skip_copy;
+
+ while (i < vcnt && size) {
+ struct page *page;
+ int len = min_t(int, PAGE_SIZE, size);
+
+ page = alloc_page(GFP_NOIO);
+ if (unlikely(!page))
+ goto free_pages;
- bio_for_each_segment_all(bvec, bio, i) {
- bvecs[i] = *bvec;
- bvecs[i].bv_page = alloc_page(GFP_NOIO);
- if (unlikely(!bvecs[i].bv_page))
- goto do_sync_io;
- memcpy(kmap(bvecs[i].bv_page) + bvec->bv_offset,
- kmap(bvec->bv_page) + bvec->bv_offset, bvec->bv_len);
- kunmap(bvecs[i].bv_page);
- kunmap(bvec->bv_page);
- }
- r1_bio->behind_bvecs = bvecs;
- r1_bio->behind_page_count = bio->bi_vcnt;
+ bio_add_page(behind_bio, page, len, 0);
+
+ size -= len;
+ i++;
+ }
+
+ bio_copy_data(behind_bio, bio);
+skip_copy:
+ r1_bio->behind_master_bio = behind_bio;;
set_bit(R1BIO_BehindIO, &r1_bio->state);
- return;
-do_sync_io:
- for (i = 0; i < bio->bi_vcnt; i++)
- if (bvecs[i].bv_page)
- put_page(bvecs[i].bv_page);
- kfree(bvecs);
+ return behind_bio;
+
+free_pages:
pr_debug("%dB behind alloc failed, doing sync I/O\n",
bio->bi_iter.bi_size);
+ bio_free_pages(behind_bio);
+fail:
+ return behind_bio;
}
struct raid1_plug_cb {
@@ -1048,30 +1160,34 @@ static void raid1_unplug(struct blk_plug_cb *cb, bool from_schedule)
/* we aren't scheduling, so we can do the write-out directly. */
bio = bio_list_get(&plug->pending);
- bitmap_unplug(mddev->bitmap);
- wake_up(&conf->wait_barrier);
-
- while (bio) { /* submit pending writes */
- struct bio *next = bio->bi_next;
- struct md_rdev *rdev = (void*)bio->bi_bdev;
- bio->bi_next = NULL;
- bio->bi_bdev = rdev->bdev;
- if (test_bit(Faulty, &rdev->flags)) {
- bio->bi_error = -EIO;
- bio_endio(bio);
- } else if (unlikely((bio_op(bio) == REQ_OP_DISCARD) &&
- !blk_queue_discard(bdev_get_queue(bio->bi_bdev))))
- /* Just ignore it */
- bio_endio(bio);
- else
- generic_make_request(bio);
- bio = next;
- }
+ flush_bio_list(conf, bio);
kfree(plug);
}
+static void init_r1bio(struct r1bio *r1_bio, struct mddev *mddev, struct bio *bio)
+{
+ r1_bio->master_bio = bio;
+ r1_bio->sectors = bio_sectors(bio);
+ r1_bio->state = 0;
+ r1_bio->mddev = mddev;
+ r1_bio->sector = bio->bi_iter.bi_sector;
+}
+
+static inline struct r1bio *
+alloc_r1bio(struct mddev *mddev, struct bio *bio)
+{
+ struct r1conf *conf = mddev->private;
+ struct r1bio *r1_bio;
+
+ r1_bio = mempool_alloc(conf->r1bio_pool, GFP_NOIO);
+ /* Ensure no bio records IO_BLOCKED */
+ memset(r1_bio->bios, 0, conf->raid_disks * sizeof(r1_bio->bios[0]));
+ init_r1bio(r1_bio, mddev, bio);
+ return r1_bio;
+}
+
static void raid1_read_request(struct mddev *mddev, struct bio *bio,
- struct r1bio *r1_bio)
+ int max_read_sectors, struct r1bio *r1_bio)
{
struct r1conf *conf = mddev->private;
struct raid1_info *mirror;
@@ -1079,22 +1195,67 @@ static void raid1_read_request(struct mddev *mddev, struct bio *bio,
struct bitmap *bitmap = mddev->bitmap;
const int op = bio_op(bio);
const unsigned long do_sync = (bio->bi_opf & REQ_SYNC);
- int sectors_handled;
int max_sectors;
int rdisk;
+ bool print_msg = !!r1_bio;
+ char b[BDEVNAME_SIZE];
+
+ /*
+ * If r1_bio is set, we are blocking the raid1d thread
+ * so there is a tiny risk of deadlock. So ask for
+ * emergency memory if needed.
+ */
+ gfp_t gfp = r1_bio ? (GFP_NOIO | __GFP_HIGH) : GFP_NOIO;
+
+ if (print_msg) {
+ /* Need to get the block device name carefully */
+ struct md_rdev *rdev;
+ rcu_read_lock();
+ rdev = rcu_dereference(conf->mirrors[r1_bio->read_disk].rdev);
+ if (rdev)
+ bdevname(rdev->bdev, b);
+ else
+ strcpy(b, "???");
+ rcu_read_unlock();
+ }
- wait_barrier(conf, bio);
+ /*
+ * Still need barrier for READ in case that whole
+ * array is frozen.
+ */
+ wait_read_barrier(conf, bio->bi_iter.bi_sector);
-read_again:
+ if (!r1_bio)
+ r1_bio = alloc_r1bio(mddev, bio);
+ else
+ init_r1bio(r1_bio, mddev, bio);
+ r1_bio->sectors = max_read_sectors;
+
+ /*
+ * make_request() can abort the operation when read-ahead is being
+ * used and no empty request is available.
+ */
rdisk = read_balance(conf, r1_bio, &max_sectors);
if (rdisk < 0) {
/* couldn't find anywhere to read from */
+ if (print_msg) {
+ pr_crit_ratelimited("md/raid1:%s: %s: unrecoverable I/O read error for block %llu\n",
+ mdname(mddev),
+ b,
+ (unsigned long long)r1_bio->sector);
+ }
raid_end_bio_io(r1_bio);
return;
}
mirror = conf->mirrors + rdisk;
+ if (print_msg)
+ pr_info_ratelimited("md/raid1:%s: redirecting sector %llu to other mirror: %s\n",
+ mdname(mddev),
+ (unsigned long long)r1_bio->sector,
+ bdevname(mirror->rdev->bdev, b));
+
if (test_bit(WriteMostly, &mirror->rdev->flags) &&
bitmap) {
/*
@@ -1105,12 +1266,20 @@ read_again:
wait_event(bitmap->behind_wait,
atomic_read(&bitmap->behind_writes) == 0);
}
+
+ if (max_sectors < bio_sectors(bio)) {
+ struct bio *split = bio_split(bio, max_sectors,
+ gfp, conf->bio_split);
+ bio_chain(split, bio);
+ generic_make_request(bio);
+ bio = split;
+ r1_bio->master_bio = bio;
+ r1_bio->sectors = max_sectors;
+ }
+
r1_bio->read_disk = rdisk;
- r1_bio->start_next_window = 0;
- read_bio = bio_clone_mddev(bio, GFP_NOIO, mddev);
- bio_trim(read_bio, r1_bio->sector - bio->bi_iter.bi_sector,
- max_sectors);
+ read_bio = bio_clone_fast(bio, gfp, mddev->bio_set);
r1_bio->bios[rdisk] = read_bio;
@@ -1129,58 +1298,22 @@ read_again:
read_bio, disk_devt(mddev->gendisk),
r1_bio->sector);
- if (max_sectors < r1_bio->sectors) {
- /*
- * could not read all from this device, so we will need another
- * r1_bio.
- */
- sectors_handled = (r1_bio->sector + max_sectors
- - bio->bi_iter.bi_sector);
- r1_bio->sectors = max_sectors;
- spin_lock_irq(&conf->device_lock);
- if (bio->bi_phys_segments == 0)
- bio->bi_phys_segments = 2;
- else
- bio->bi_phys_segments++;
- spin_unlock_irq(&conf->device_lock);
-
- /*
- * Cannot call generic_make_request directly as that will be
- * queued in __make_request and subsequent mempool_alloc might
- * block waiting for it. So hand bio over to raid1d.
- */
- reschedule_retry(r1_bio);
-
- r1_bio = mempool_alloc(conf->r1bio_pool, GFP_NOIO);
-
- r1_bio->master_bio = bio;
- r1_bio->sectors = bio_sectors(bio) - sectors_handled;
- r1_bio->state = 0;
- r1_bio->mddev = mddev;
- r1_bio->sector = bio->bi_iter.bi_sector + sectors_handled;
- goto read_again;
- } else
- generic_make_request(read_bio);
+ generic_make_request(read_bio);
}
static void raid1_write_request(struct mddev *mddev, struct bio *bio,
- struct r1bio *r1_bio)
+ int max_write_sectors)
{
struct r1conf *conf = mddev->private;
+ struct r1bio *r1_bio;
int i, disks;
struct bitmap *bitmap = mddev->bitmap;
unsigned long flags;
- const int op = bio_op(bio);
- const unsigned long do_sync = (bio->bi_opf & REQ_SYNC);
- const unsigned long do_flush_fua = (bio->bi_opf &
- (REQ_PREFLUSH | REQ_FUA));
struct md_rdev *blocked_rdev;
struct blk_plug_cb *cb;
struct raid1_plug_cb *plug = NULL;
int first_clone;
- int sectors_handled;
int max_sectors;
- sector_t start_next_window;
/*
* Register the new request and wait if the reconstruction
@@ -1188,7 +1321,6 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
* Continue immediately if no resync is active currently.
*/
- md_write_start(mddev, bio); /* wait on superblock update early */
if ((bio_end_sector(bio) > mddev->suspend_lo &&
bio->bi_iter.bi_sector < mddev->suspend_hi) ||
@@ -1202,7 +1334,7 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
*/
DEFINE_WAIT(w);
for (;;) {
- flush_signals(current);
+ sigset_t full, old;
prepare_to_wait(&conf->wait_barrier,
&w, TASK_INTERRUPTIBLE);
if (bio_end_sector(bio) <= mddev->suspend_lo ||
@@ -1212,11 +1344,17 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
bio->bi_iter.bi_sector,
bio_end_sector(bio))))
break;
+ sigfillset(&full);
+ sigprocmask(SIG_BLOCK, &full, &old);
schedule();
+ sigprocmask(SIG_SETMASK, &old, NULL);
}
finish_wait(&conf->wait_barrier, &w);
}
- start_next_window = wait_barrier(conf, bio);
+ wait_barrier(conf, bio->bi_iter.bi_sector);
+
+ r1_bio = alloc_r1bio(mddev, bio);
+ r1_bio->sectors = max_write_sectors;
if (conf->pending_count >= max_queued_requests) {
md_wakeup_thread(mddev->thread);
@@ -1237,7 +1375,6 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
disks = conf->raid_disks * 2;
retry_write:
- r1_bio->start_next_window = start_next_window;
blocked_rdev = NULL;
rcu_read_lock();
max_sectors = r1_bio->sectors;
@@ -1304,54 +1441,38 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
if (unlikely(blocked_rdev)) {
/* Wait for this device to become unblocked */
int j;
- sector_t old = start_next_window;
for (j = 0; j < i; j++)
if (r1_bio->bios[j])
rdev_dec_pending(conf->mirrors[j].rdev, mddev);
r1_bio->state = 0;
- allow_barrier(conf, start_next_window, bio->bi_iter.bi_sector);
+ allow_barrier(conf, bio->bi_iter.bi_sector);
raid1_log(mddev, "wait rdev %d blocked", blocked_rdev->raid_disk);
md_wait_for_blocked_rdev(blocked_rdev, mddev);
- start_next_window = wait_barrier(conf, bio);
- /*
- * We must make sure the multi r1bios of bio have
- * the same value of bi_phys_segments
- */
- if (bio->bi_phys_segments && old &&
- old != start_next_window)
- /* Wait for the former r1bio(s) to complete */
- wait_event(conf->wait_barrier,
- bio->bi_phys_segments == 1);
+ wait_barrier(conf, bio->bi_iter.bi_sector);
goto retry_write;
}
- if (max_sectors < r1_bio->sectors) {
- /* We are splitting this write into multiple parts, so
- * we need to prepare for allocating another r1_bio.
- */
+ if (max_sectors < bio_sectors(bio)) {
+ struct bio *split = bio_split(bio, max_sectors,
+ GFP_NOIO, conf->bio_split);
+ bio_chain(split, bio);
+ generic_make_request(bio);
+ bio = split;
+ r1_bio->master_bio = bio;
r1_bio->sectors = max_sectors;
- spin_lock_irq(&conf->device_lock);
- if (bio->bi_phys_segments == 0)
- bio->bi_phys_segments = 2;
- else
- bio->bi_phys_segments++;
- spin_unlock_irq(&conf->device_lock);
}
- sectors_handled = r1_bio->sector + max_sectors - bio->bi_iter.bi_sector;
atomic_set(&r1_bio->remaining, 1);
atomic_set(&r1_bio->behind_remaining, 0);
first_clone = 1;
+
for (i = 0; i < disks; i++) {
- struct bio *mbio;
+ struct bio *mbio = NULL;
if (!r1_bio->bios[i])
continue;
- mbio = bio_clone_mddev(bio, GFP_NOIO, mddev);
- bio_trim(mbio, r1_bio->sector - bio->bi_iter.bi_sector,
- max_sectors);
if (first_clone) {
/* do behind I/O ?
@@ -1361,8 +1482,9 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
if (bitmap &&
(atomic_read(&bitmap->behind_writes)
< mddev->bitmap_info.max_write_behind) &&
- !waitqueue_active(&bitmap->behind_wait))
- alloc_behind_pages(mbio, r1_bio);
+ !waitqueue_active(&bitmap->behind_wait)) {
+ mbio = alloc_behind_master_bio(r1_bio, bio);
+ }
bitmap_startwrite(bitmap, r1_bio->sector,
r1_bio->sectors,
@@ -1370,15 +1492,17 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
&r1_bio->state));
first_clone = 0;
}
- if (r1_bio->behind_bvecs) {
- struct bio_vec *bvec;
- int j;
- /*
- * We trimmed the bio, so _all is legit
- */
- bio_for_each_segment_all(bvec, mbio, j)
- bvec->bv_page = r1_bio->behind_bvecs[j].bv_page;
+ if (!mbio) {
+ if (r1_bio->behind_master_bio)
+ mbio = bio_clone_fast(r1_bio->behind_master_bio,
+ GFP_NOIO,
+ mddev->bio_set);
+ else
+ mbio = bio_clone_fast(bio, GFP_NOIO, mddev->bio_set);
+ }
+
+ if (r1_bio->behind_master_bio) {
if (test_bit(WriteMostly, &conf->mirrors[i].rdev->flags))
atomic_inc(&r1_bio->behind_remaining);
}
@@ -1389,7 +1513,7 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
conf->mirrors[i].rdev->data_offset);
mbio->bi_bdev = conf->mirrors[i].rdev->bdev;
mbio->bi_end_io = raid1_end_write_request;
- bio_set_op_attrs(mbio, op, do_flush_fua | do_sync);
+ mbio->bi_opf = bio_op(bio) | (bio->bi_opf & (REQ_SYNC | REQ_FUA));
if (test_bit(FailFast, &conf->mirrors[i].rdev->flags) &&
!test_bit(WriteMostly, &conf->mirrors[i].rdev->flags) &&
conf->raid_disks - mddev->degraded > 1)
@@ -1410,33 +1534,16 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
plug = container_of(cb, struct raid1_plug_cb, cb);
else
plug = NULL;
- spin_lock_irqsave(&conf->device_lock, flags);
if (plug) {
bio_list_add(&plug->pending, mbio);
plug->pending_cnt++;
} else {
+ spin_lock_irqsave(&conf->device_lock, flags);
bio_list_add(&conf->pending_bio_list, mbio);
conf->pending_count++;
- }
- spin_unlock_irqrestore(&conf->device_lock, flags);
- if (!plug)
+ spin_unlock_irqrestore(&conf->device_lock, flags);
md_wakeup_thread(mddev->thread);
- }
- /* Mustn't call r1_bio_write_done before this next test,
- * as it could result in the bio being freed.
- */
- if (sectors_handled < bio_sectors(bio)) {
- r1_bio_write_done(r1_bio);
- /* We need another r1_bio. It has already been counted
- * in bio->bi_phys_segments
- */
- r1_bio = mempool_alloc(conf->r1bio_pool, GFP_NOIO);
- r1_bio->master_bio = bio;
- r1_bio->sectors = bio_sectors(bio) - sectors_handled;
- r1_bio->state = 0;
- r1_bio->mddev = mddev;
- r1_bio->sector = bio->bi_iter.bi_sector + sectors_handled;
- goto retry_write;
+ }
}
r1_bio_write_done(r1_bio);
@@ -1445,38 +1552,33 @@ static void raid1_write_request(struct mddev *mddev, struct bio *bio,
wake_up(&conf->wait_barrier);
}
-static void raid1_make_request(struct mddev *mddev, struct bio *bio)
+static bool raid1_make_request(struct mddev *mddev, struct bio *bio)
{
- struct r1conf *conf = mddev->private;
- struct r1bio *r1_bio;
-
- /*
- * make_request() can abort the operation when read-ahead is being
- * used and no empty request is available.
- *
- */
- r1_bio = mempool_alloc(conf->r1bio_pool, GFP_NOIO);
+ sector_t sectors;
- r1_bio->master_bio = bio;
- r1_bio->sectors = bio_sectors(bio);
- r1_bio->state = 0;
- r1_bio->mddev = mddev;
- r1_bio->sector = bio->bi_iter.bi_sector;
+ if (unlikely(bio->bi_opf & REQ_PREFLUSH)) {
+ md_flush_request(mddev, bio);
+ return true;
+ }
/*
- * We might need to issue multiple reads to different devices if there
- * are bad blocks around, so we keep track of the number of reads in
- * bio->bi_phys_segments. If this is 0, there is only one r1_bio and
- * no locking will be needed when requests complete. If it is
- * non-zero, then it is the number of not-completed requests.
+ * There is a limit to the maximum size, but
+ * the read/write handler might find a lower limit
+ * due to bad blocks. To avoid multiple splits,
+ * we pass the maximum number of sectors down
+ * and let the lower level perform the split.
*/
- bio->bi_phys_segments = 0;
- bio_clear_flag(bio, BIO_SEG_VALID);
+ sectors = align_to_barrier_unit_end(
+ bio->bi_iter.bi_sector, bio_sectors(bio));
if (bio_data_dir(bio) == READ)
- raid1_read_request(mddev, bio, r1_bio);
- else
- raid1_write_request(mddev, bio, r1_bio);
+ raid1_read_request(mddev, bio, sectors, NULL);
+ else {
+ if (!md_write_start(mddev,bio))
+ return false;
+ raid1_write_request(mddev, bio, sectors);
+ }
+ return true;
}
static void raid1_status(struct seq_file *seq, struct mddev *mddev)
@@ -1567,19 +1669,11 @@ static void print_conf(struct r1conf *conf)
static void close_sync(struct r1conf *conf)
{
- wait_barrier(conf, NULL);
- allow_barrier(conf, 0, 0);
+ wait_all_barriers(conf);
+ allow_all_barriers(conf);
mempool_destroy(conf->r1buf_pool);
conf->r1buf_pool = NULL;
-
- spin_lock_irq(&conf->resync_lock);
- conf->next_resync = MaxSector - 2 * NEXT_NORMALIO_DISTANCE;
- conf->start_next_window = MaxSector;
- conf->current_window_requests +=
- conf->next_window_requests;
- conf->next_window_requests = 0;
- spin_unlock_irq(&conf->resync_lock);
}
static int raid1_spare_active(struct mddev *mddev)
@@ -1746,9 +1840,9 @@ static int raid1_remove_disk(struct mddev *mddev, struct md_rdev *rdev)
p->rdev = repl;
conf->mirrors[conf->raid_disks + number].rdev = NULL;
unfreeze_array(conf);
- clear_bit(WantReplacement, &rdev->flags);
- } else
- clear_bit(WantReplacement, &rdev->flags);
+ }
+
+ clear_bit(WantReplacement, &rdev->flags);
err = md_integrity_register(mddev);
}
abort:
@@ -1759,7 +1853,7 @@ abort:
static void end_sync_read(struct bio *bio)
{
- struct r1bio *r1_bio = bio->bi_private;
+ struct r1bio *r1_bio = get_resync_r1bio(bio);
update_head_pos(r1_bio->read_disk, r1_bio);
@@ -1768,7 +1862,7 @@ static void end_sync_read(struct bio *bio)
* or re-read if the read failed.
* We don't do much here, just schedule handling by raid1d
*/
- if (!bio->bi_error)
+ if (!bio->bi_status)
set_bit(R1BIO_Uptodate, &r1_bio->state);
if (atomic_dec_and_test(&r1_bio->remaining))
@@ -1777,8 +1871,8 @@ static void end_sync_read(struct bio *bio)
static void end_sync_write(struct bio *bio)
{
- int uptodate = !bio->bi_error;
- struct r1bio *r1_bio = bio->bi_private;
+ int uptodate = !bio->bi_status;
+ struct r1bio *r1_bio = get_resync_r1bio(bio);
struct mddev *mddev = r1_bio->mddev;
struct r1conf *conf = mddev->private;
sector_t first_bad;
@@ -1857,6 +1951,7 @@ static int fix_sync_read_error(struct r1bio *r1_bio)
struct mddev *mddev = r1_bio->mddev;
struct r1conf *conf = mddev->private;
struct bio *bio = r1_bio->bios[r1_bio->read_disk];
+ struct page **pages = get_resync_pages(bio)->pages;
sector_t sect = r1_bio->sector;
int sectors = r1_bio->sectors;
int idx = 0;
@@ -1890,7 +1985,7 @@ static int fix_sync_read_error(struct r1bio *r1_bio)
*/
rdev = conf->mirrors[d].rdev;
if (sync_page_io(rdev, sect, s<<9,
- bio->bi_io_vec[idx].bv_page,
+ pages[idx],
REQ_OP_READ, 0, false)) {
success = 1;
break;
@@ -1945,7 +2040,7 @@ static int fix_sync_read_error(struct r1bio *r1_bio)
continue;
rdev = conf->mirrors[d].rdev;
if (r1_sync_page_io(rdev, sect, s,
- bio->bi_io_vec[idx].bv_page,
+ pages[idx],
WRITE) == 0) {
r1_bio->bios[d]->bi_end_io = NULL;
rdev_dec_pending(rdev, mddev);
@@ -1960,7 +2055,7 @@ static int fix_sync_read_error(struct r1bio *r1_bio)
continue;
rdev = conf->mirrors[d].rdev;
if (r1_sync_page_io(rdev, sect, s,
- bio->bi_io_vec[idx].bv_page,
+ pages[idx],
READ) != 0)
atomic_add(s, &rdev->corrected_errors);
}
@@ -1969,7 +2064,7 @@ static int fix_sync_read_error(struct r1bio *r1_bio)
idx ++;
}
set_bit(R1BIO_Uptodate, &r1_bio->state);
- bio->bi_error = 0;
+ bio->bi_status = 0;
return 1;
}
@@ -1993,26 +2088,27 @@ static void process_checks(struct r1bio *r1_bio)
for (i = 0; i < conf->raid_disks * 2; i++) {
int j;
int size;
- int error;
+ blk_status_t status;
+ struct bio_vec *bi;
struct bio *b = r1_bio->bios[i];
+ struct resync_pages *rp = get_resync_pages(b);
if (b->bi_end_io != end_sync_read)
continue;
/* fixup the bio for reuse, but preserve errno */
- error = b->bi_error;
+ status = b->bi_status;
bio_reset(b);
- b->bi_error = error;
+ b->bi_status = status;
b->bi_vcnt = vcnt;
b->bi_iter.bi_size = r1_bio->sectors << 9;
b->bi_iter.bi_sector = r1_bio->sector +
conf->mirrors[i].rdev->data_offset;
b->bi_bdev = conf->mirrors[i].rdev->bdev;
b->bi_end_io = end_sync_read;
- b->bi_private = r1_bio;
+ rp->raid_bio = r1_bio;
+ b->bi_private = rp;
size = b->bi_iter.bi_size;
- for (j = 0; j < vcnt ; j++) {
- struct bio_vec *bi;
- bi = &b->bi_io_vec[j];
+ bio_for_each_segment_all(bi, b, j) {
bi->bv_offset = 0;
if (size > PAGE_SIZE)
bi->bv_len = PAGE_SIZE;
@@ -2023,7 +2119,7 @@ static void process_checks(struct r1bio *r1_bio)
}
for (primary = 0; primary < conf->raid_disks * 2; primary++)
if (r1_bio->bios[primary]->bi_end_io == end_sync_read &&
- !r1_bio->bios[primary]->bi_error) {
+ !r1_bio->bios[primary]->bi_status) {
r1_bio->bios[primary]->bi_end_io = NULL;
rdev_dec_pending(conf->mirrors[primary].rdev, mddev);
break;
@@ -2033,21 +2129,25 @@ static void process_checks(struct r1bio *r1_bio)
int j;
struct bio *pbio = r1_bio->bios[primary];
struct bio *sbio = r1_bio->bios[i];
- int error = sbio->bi_error;
+ blk_status_t status = sbio->bi_status;
+ struct page **ppages = get_resync_pages(pbio)->pages;
+ struct page **spages = get_resync_pages(sbio)->pages;
+ struct bio_vec *bi;
+ int page_len[RESYNC_PAGES] = { 0 };
if (sbio->bi_end_io != end_sync_read)
continue;
/* Now we can 'fixup' the error value */
- sbio->bi_error = 0;
+ sbio->bi_status = 0;
+
+ bio_for_each_segment_all(bi, sbio, j)
+ page_len[j] = bi->bv_len;
- if (!error) {
+ if (!status) {
for (j = vcnt; j-- ; ) {
- struct page *p, *s;
- p = pbio->bi_io_vec[j].bv_page;
- s = sbio->bi_io_vec[j].bv_page;
- if (memcmp(page_address(p),
- page_address(s),
- sbio->bi_io_vec[j].bv_len))
+ if (memcmp(page_address(ppages[j]),
+ page_address(spages[j]),
+ page_len[j]))
break;
}
} else
@@ -2055,7 +2155,7 @@ static void process_checks(struct r1bio *r1_bio)
if (j >= 0)
atomic64_add(r1_bio->sectors, &mddev->resync_mismatches);
if (j < 0 || (test_bit(MD_RECOVERY_CHECK, &mddev->recovery)
- && !error)) {
+ && !status)) {
/* No need to write to this device. */
sbio->bi_end_io = NULL;
rdev_dec_pending(conf->mirrors[i].rdev, mddev);
@@ -2071,9 +2171,7 @@ static void sync_request_write(struct mddev *mddev, struct r1bio *r1_bio)
struct r1conf *conf = mddev->private;
int i;
int disks = conf->raid_disks * 2;
- struct bio *bio, *wbio;
-
- bio = r1_bio->bios[r1_bio->read_disk];
+ struct bio *wbio;
if (!test_bit(R1BIO_Uptodate, &r1_bio->state))
/* ouch - failed to read all of that. */
@@ -2094,6 +2192,8 @@ static void sync_request_write(struct mddev *mddev, struct r1bio *r1_bio)
(i == r1_bio->read_disk ||
!test_bit(MD_RECOVERY_SYNC, &mddev->recovery))))
continue;
+ if (test_bit(Faulty, &conf->mirrors[i].rdev->flags))
+ continue;
bio_set_op_attrs(wbio, REQ_OP_WRITE, 0);
if (test_bit(FailFast, &conf->mirrors[i].rdev->flags))
@@ -2263,20 +2363,14 @@ static int narrow_write_error(struct r1bio *r1_bio, int i)
/* Write at 'sector' for 'sectors'*/
if (test_bit(R1BIO_BehindIO, &r1_bio->state)) {
- unsigned vcnt = r1_bio->behind_page_count;
- struct bio_vec *vec = r1_bio->behind_bvecs;
-
- while (!vec->bv_page) {
- vec++;
- vcnt--;
- }
-
- wbio = bio_alloc_mddev(GFP_NOIO, vcnt, mddev);
- memcpy(wbio->bi_io_vec, vec, vcnt * sizeof(struct bio_vec));
-
- wbio->bi_vcnt = vcnt;
+ wbio = bio_clone_fast(r1_bio->behind_master_bio,
+ GFP_NOIO,
+ mddev->bio_set);
+ /* We really need a _all clone */
+ wbio->bi_iter = (struct bvec_iter){ 0 };
} else {
- wbio = bio_clone_mddev(r1_bio->master_bio, GFP_NOIO, mddev);
+ wbio = bio_clone_fast(r1_bio->master_bio, GFP_NOIO,
+ mddev->bio_set);
}
bio_set_op_attrs(wbio, REQ_OP_WRITE, 0);
@@ -2310,11 +2404,11 @@ static void handle_sync_write_finished(struct r1conf *conf, struct r1bio *r1_bio
struct bio *bio = r1_bio->bios[m];
if (bio->bi_end_io == NULL)
continue;
- if (!bio->bi_error &&
+ if (!bio->bi_status &&
test_bit(R1BIO_MadeGood, &r1_bio->state)) {
rdev_clear_badblocks(rdev, r1_bio->sector, s, 0);
}
- if (bio->bi_error &&
+ if (bio->bi_status &&
test_bit(R1BIO_WriteError, &r1_bio->state)) {
if (!rdev_set_badblocks(rdev, r1_bio->sector, s, 0))
md_error(conf->mddev, rdev);
@@ -2326,8 +2420,9 @@ static void handle_sync_write_finished(struct r1conf *conf, struct r1bio *r1_bio
static void handle_write_finished(struct r1conf *conf, struct r1bio *r1_bio)
{
- int m;
+ int m, idx;
bool fail = false;
+
for (m = 0; m < conf->raid_disks * 2 ; m++)
if (r1_bio->bios[m] == IO_MADE_GOOD) {
struct md_rdev *rdev = conf->mirrors[m].rdev;
@@ -2353,8 +2448,14 @@ static void handle_write_finished(struct r1conf *conf, struct r1bio *r1_bio)
if (fail) {
spin_lock_irq(&conf->device_lock);
list_add(&r1_bio->retry_list, &conf->bio_end_io_list);
- conf->nr_queued++;
+ idx = sector_to_idx(r1_bio->sector);
+ atomic_inc(&conf->nr_queued[idx]);
spin_unlock_irq(&conf->device_lock);
+ /*
+ * In case freeze_array() is waiting for condition
+ * get_unqueued_pending() == extra to be true.
+ */
+ wake_up(&conf->wait_barrier);
md_wakeup_thread(conf->mddev->thread);
} else {
if (test_bit(R1BIO_WriteError, &r1_bio->state))
@@ -2365,11 +2466,8 @@ static void handle_write_finished(struct r1conf *conf, struct r1bio *r1_bio)
static void handle_read_error(struct r1conf *conf, struct r1bio *r1_bio)
{
- int disk;
- int max_sectors;
struct mddev *mddev = conf->mddev;
struct bio *bio;
- char b[BDEVNAME_SIZE];
struct md_rdev *rdev;
dev_t bio_dev;
sector_t bio_sector;
@@ -2385,7 +2483,6 @@ static void handle_read_error(struct r1conf *conf, struct r1bio *r1_bio)
*/
bio = r1_bio->bios[r1_bio->read_disk];
- bdevname(bio->bi_bdev, b);
bio_dev = bio->bi_bdev->bd_dev;
bio_sector = conf->mirrors[r1_bio->read_disk].rdev->data_offset + r1_bio->sector;
bio_put(bio);
@@ -2403,68 +2500,12 @@ static void handle_read_error(struct r1conf *conf, struct r1bio *r1_bio)
}
rdev_dec_pending(rdev, conf->mddev);
+ allow_barrier(conf, r1_bio->sector);
+ bio = r1_bio->master_bio;
-read_more:
- disk = read_balance(conf, r1_bio, &max_sectors);
- if (disk == -1) {
- pr_crit_ratelimited("md/raid1:%s: %s: unrecoverable I/O read error for block %llu\n",
- mdname(mddev), b, (unsigned long long)r1_bio->sector);
- raid_end_bio_io(r1_bio);
- } else {
- const unsigned long do_sync
- = r1_bio->master_bio->bi_opf & REQ_SYNC;
- r1_bio->read_disk = disk;
- bio = bio_clone_mddev(r1_bio->master_bio, GFP_NOIO, mddev);
- bio_trim(bio, r1_bio->sector - bio->bi_iter.bi_sector,
- max_sectors);
- r1_bio->bios[r1_bio->read_disk] = bio;
- rdev = conf->mirrors[disk].rdev;
- pr_info_ratelimited("md/raid1:%s: redirecting sector %llu to other mirror: %s\n",
- mdname(mddev),
- (unsigned long long)r1_bio->sector,
- bdevname(rdev->bdev, b));
- bio->bi_iter.bi_sector = r1_bio->sector + rdev->data_offset;
- bio->bi_bdev = rdev->bdev;
- bio->bi_end_io = raid1_end_read_request;
- bio_set_op_attrs(bio, REQ_OP_READ, do_sync);
- if (test_bit(FailFast, &rdev->flags) &&
- test_bit(R1BIO_FailFast, &r1_bio->state))
- bio->bi_opf |= MD_FAILFAST;
- bio->bi_private = r1_bio;
- if (max_sectors < r1_bio->sectors) {
- /* Drat - have to split this up more */
- struct bio *mbio = r1_bio->master_bio;
- int sectors_handled = (r1_bio->sector + max_sectors
- - mbio->bi_iter.bi_sector);
- r1_bio->sectors = max_sectors;
- spin_lock_irq(&conf->device_lock);
- if (mbio->bi_phys_segments == 0)
- mbio->bi_phys_segments = 2;
- else
- mbio->bi_phys_segments++;
- spin_unlock_irq(&conf->device_lock);
- trace_block_bio_remap(bdev_get_queue(bio->bi_bdev),
- bio, bio_dev, bio_sector);
- generic_make_request(bio);
- bio = NULL;
-
- r1_bio = mempool_alloc(conf->r1bio_pool, GFP_NOIO);
-
- r1_bio->master_bio = mbio;
- r1_bio->sectors = bio_sectors(mbio) - sectors_handled;
- r1_bio->state = 0;
- set_bit(R1BIO_ReadError, &r1_bio->state);
- r1_bio->mddev = mddev;
- r1_bio->sector = mbio->bi_iter.bi_sector +
- sectors_handled;
-
- goto read_more;
- } else {
- trace_block_bio_remap(bdev_get_queue(bio->bi_bdev),
- bio, bio_dev, bio_sector);
- generic_make_request(bio);
- }
- }
+ /* Reuse the old r1_bio so that the IO_BLOCKED settings are preserved */
+ r1_bio->state = 0;
+ raid1_read_request(mddev, bio, r1_bio->sectors, r1_bio);
}
static void raid1d(struct md_thread *thread)
@@ -2475,6 +2516,7 @@ static void raid1d(struct md_thread *thread)
struct r1conf *conf = mddev->private;
struct list_head *head = &conf->retry_list;
struct blk_plug plug;
+ int idx;
md_check_recovery(mddev);
@@ -2482,17 +2524,15 @@ static void raid1d(struct md_thread *thread)
!test_bit(MD_SB_CHANGE_PENDING, &mddev->sb_flags)) {
LIST_HEAD(tmp);
spin_lock_irqsave(&conf->device_lock, flags);
- if (!test_bit(MD_SB_CHANGE_PENDING, &mddev->sb_flags)) {
- while (!list_empty(&conf->bio_end_io_list)) {
- list_move(conf->bio_end_io_list.prev, &tmp);
- conf->nr_queued--;
- }
- }
+ if (!test_bit(MD_SB_CHANGE_PENDING, &mddev->sb_flags))
+ list_splice_init(&conf->bio_end_io_list, &tmp);
spin_unlock_irqrestore(&conf->device_lock, flags);
while (!list_empty(&tmp)) {
r1_bio = list_first_entry(&tmp, struct r1bio,
retry_list);
list_del(&r1_bio->retry_list);
+ idx = sector_to_idx(r1_bio->sector);
+ atomic_dec(&conf->nr_queued[idx]);
if (mddev->degraded)
set_bit(R1BIO_Degraded, &r1_bio->state);
if (test_bit(R1BIO_WriteError, &r1_bio->state))
@@ -2513,7 +2553,8 @@ static void raid1d(struct md_thread *thread)
}
r1_bio = list_entry(head->prev, struct r1bio, retry_list);
list_del(head->prev);
- conf->nr_queued--;
+ idx = sector_to_idx(r1_bio->sector);
+ atomic_dec(&conf->nr_queued[idx]);
spin_unlock_irqrestore(&conf->device_lock, flags);
mddev = r1_bio->mddev;
@@ -2530,10 +2571,7 @@ static void raid1d(struct md_thread *thread)
else if (test_bit(R1BIO_ReadError, &r1_bio->state))
handle_read_error(conf, r1_bio);
else
- /* just a partial read to be scheduled from separate
- * context
- */
- generic_make_request(r1_bio->bios[r1_bio->read_disk]);
+ WARN_ON_ONCE(1);
cond_resched();
if (mddev->sb_flags & ~(1<<MD_SB_CHANGE_PENDING))
@@ -2552,7 +2590,6 @@ static int init_resync(struct r1conf *conf)
conf->poolinfo);
if (!conf->r1buf_pool)
return -ENOMEM;
- conf->next_resync = 0;
return 0;
}
@@ -2581,6 +2618,7 @@ static sector_t raid1_sync_request(struct mddev *mddev, sector_t sector_nr,
int still_degraded = 0;
int good_sectors = RESYNC_SECTORS;
int min_bad = 0; /* number of sectors that are bad in all devices */
+ int idx = sector_to_idx(sector_nr);
if (!conf->r1buf_pool)
if (init_resync(conf))
@@ -2630,7 +2668,7 @@ static sector_t raid1_sync_request(struct mddev *mddev, sector_t sector_nr,
* If there is non-resync activity waiting for a turn, then let it
* though before starting on this new sync request.
*/
- if (conf->nr_waiting)
+ if (atomic_read(&conf->nr_waiting[idx]))
schedule_timeout_uninterruptible(1);
/* we are incrementing sector_nr below. To be safe, we check against
@@ -2657,11 +2695,12 @@ static sector_t raid1_sync_request(struct mddev *mddev, sector_t sector_nr,
r1_bio->sector = sector_nr;
r1_bio->state = 0;
set_bit(R1BIO_IsSync, &r1_bio->state);
+ /* make sure good_sectors won't go across barrier unit boundary */
+ good_sectors = align_to_barrier_unit_end(sector_nr, good_sectors);
for (i = 0; i < conf->raid_disks * 2; i++) {
struct md_rdev *rdev;
bio = r1_bio->bios[i];
- bio_reset(bio);
rdev = rcu_dereference(conf->mirrors[i].rdev);
if (rdev == NULL ||
@@ -2717,7 +2756,6 @@ static sector_t raid1_sync_request(struct mddev *mddev, sector_t sector_nr,
atomic_inc(&rdev->nr_pending);
bio->bi_iter.bi_sector = sector_nr + rdev->data_offset;
bio->bi_bdev = rdev->bdev;
- bio->bi_private = r1_bio;
if (test_bit(FailFast, &rdev->flags))
bio->bi_opf |= MD_FAILFAST;
}
@@ -2803,31 +2841,25 @@ static sector_t raid1_sync_request(struct mddev *mddev, sector_t sector_nr,
}
for (i = 0 ; i < conf->raid_disks * 2; i++) {
+ struct resync_pages *rp;
+
bio = r1_bio->bios[i];
+ rp = get_resync_pages(bio);
if (bio->bi_end_io) {
- page = bio->bi_io_vec[bio->bi_vcnt].bv_page;
- if (bio_add_page(bio, page, len, 0) == 0) {
- /* stop here */
- bio->bi_io_vec[bio->bi_vcnt].bv_page = page;
- while (i > 0) {
- i--;
- bio = r1_bio->bios[i];
- if (bio->bi_end_io==NULL)
- continue;
- /* remove last page from this bio */
- bio->bi_vcnt--;
- bio->bi_iter.bi_size -= len;
- bio_clear_flag(bio, BIO_SEG_VALID);
- }
- goto bio_full;
- }
+ page = resync_fetch_page(rp, rp->idx++);
+
+ /*
+ * won't fail because the vec table is big
+ * enough to hold all these pages
+ */
+ bio_add_page(bio, page, len, 0);
}
}
nr_sectors += len>>9;
sector_nr += len>>9;
sync_blocks -= (len>>9);
- } while (r1_bio->bios[disk]->bi_vcnt < RESYNC_PAGES);
- bio_full:
+ } while (get_resync_pages(r1_bio->bios[disk]->bi_private)->idx < RESYNC_PAGES);
+
r1_bio->sectors = nr_sectors;
if (mddev_is_clustered(mddev) &&
@@ -2887,6 +2919,26 @@ static struct r1conf *setup_conf(struct mddev *mddev)
if (!conf)
goto abort;
+ conf->nr_pending = kcalloc(BARRIER_BUCKETS_NR,
+ sizeof(atomic_t), GFP_KERNEL);
+ if (!conf->nr_pending)
+ goto abort;
+
+ conf->nr_waiting = kcalloc(BARRIER_BUCKETS_NR,
+ sizeof(atomic_t), GFP_KERNEL);
+ if (!conf->nr_waiting)
+ goto abort;
+
+ conf->nr_queued = kcalloc(BARRIER_BUCKETS_NR,
+ sizeof(atomic_t), GFP_KERNEL);
+ if (!conf->nr_queued)
+ goto abort;
+
+ conf->barrier = kcalloc(BARRIER_BUCKETS_NR,
+ sizeof(atomic_t), GFP_KERNEL);
+ if (!conf->barrier)
+ goto abort;
+
conf->mirrors = kzalloc(sizeof(struct raid1_info)
* mddev->raid_disks * 2,
GFP_KERNEL);
@@ -2907,12 +2959,15 @@ static struct r1conf *setup_conf(struct mddev *mddev)
if (!conf->r1bio_pool)
goto abort;
+ conf->bio_split = bioset_create(BIO_POOL_SIZE, 0, 0);
+ if (!conf->bio_split)
+ goto abort;
+
conf->poolinfo->mddev = mddev;
err = -EINVAL;
spin_lock_init(&conf->device_lock);
rdev_for_each(rdev, mddev) {
- struct request_queue *q;
int disk_idx = rdev->raid_disk;
if (disk_idx >= mddev->raid_disks
|| disk_idx < 0)
@@ -2925,8 +2980,6 @@ static struct r1conf *setup_conf(struct mddev *mddev)
if (disk->rdev)
goto abort;
disk->rdev = rdev;
- q = bdev_get_queue(rdev->bdev);
-
disk->head_position = 0;
disk->seq_start = MaxSector;
}
@@ -2942,9 +2995,6 @@ static struct r1conf *setup_conf(struct mddev *mddev)
conf->pending_count = 0;
conf->recovery_disabled = mddev->recovery_disabled - 1;
- conf->start_next_window = MaxSector;
- conf->current_window_requests = conf->next_window_requests = 0;
-
err = -EIO;
for (i = 0; i < conf->raid_disks * 2; i++) {
@@ -2987,6 +3037,12 @@ static struct r1conf *setup_conf(struct mddev *mddev)
kfree(conf->mirrors);
safe_put_page(conf->tmppage);
kfree(conf->poolinfo);
+ kfree(conf->nr_pending);
+ kfree(conf->nr_waiting);
+ kfree(conf->nr_queued);
+ kfree(conf->barrier);
+ if (conf->bio_split)
+ bioset_free(conf->bio_split);
kfree(conf);
}
return ERR_PTR(err);
@@ -3011,6 +3067,8 @@ static int raid1_run(struct mddev *mddev)
mdname(mddev));
return -EIO;
}
+ if (mddev_init_writes_pending(mddev) < 0)
+ return -ENOMEM;
/*
* copy the already verified devices into our private RAID1
* bookkeeping area. [whatever we allocate in run(),
@@ -3024,8 +3082,10 @@ static int raid1_run(struct mddev *mddev)
if (IS_ERR(conf))
return PTR_ERR(conf);
- if (mddev->queue)
+ if (mddev->queue) {
blk_queue_max_write_same_sectors(mddev->queue, 0);
+ blk_queue_max_write_zeroes_sectors(mddev->queue, 0);
+ }
rdev_for_each(rdev, mddev) {
if (!mddev->gendisk)
@@ -3088,6 +3148,12 @@ static void raid1_free(struct mddev *mddev, void *priv)
kfree(conf->mirrors);
safe_put_page(conf->tmppage);
kfree(conf->poolinfo);
+ kfree(conf->nr_pending);
+ kfree(conf->nr_waiting);
+ kfree(conf->nr_queued);
+ kfree(conf->barrier);
+ if (conf->bio_split)
+ bioset_free(conf->bio_split);
kfree(conf);
}
@@ -3110,8 +3176,6 @@ static int raid1_resize(struct mddev *mddev, sector_t sectors)
return ret;
}
md_set_array_sectors(mddev, newsize);
- set_capacity(mddev->gendisk, mddev->array_sectors);
- revalidate_disk(mddev->gendisk);
if (sectors > mddev->dev_sectors &&
mddev->recovery_cp > mddev->dev_sectors) {
mddev->recovery_cp = mddev->dev_sectors;
@@ -3141,7 +3205,7 @@ static int raid1_reshape(struct mddev *mddev)
struct r1conf *conf = mddev->private;
int cnt, raid_disks;
unsigned long flags;
- int d, d2, err;
+ int d, d2;
/* Cannot change chunk_size, layout, or level */
if (mddev->chunk_sectors != mddev->new_chunk_sectors ||
@@ -3153,11 +3217,8 @@ static int raid1_reshape(struct mddev *mddev)
return -EINVAL;
}
- if (!mddev_is_clustered(mddev)) {
- err = md_allow_write(mddev);
- if (err)
- return err;
- }
+ if (!mddev_is_clustered(mddev))
+ md_allow_write(mddev);
raid_disks = mddev->raid_disks + mddev->delta_disks;