Merge branch 'for-linus' of git://git.kernel.dk/linux-2.6-block

* 'for-linus' of git://git.kernel.dk/linux-2.6-block:
  block: add blk_run_queue_async
  block: blk_delay_queue() should use kblockd workqueue
  md: fix up raid1/raid10 unplugging.
  md: incorporate new plugging into raid5.
  md: provide generic support for handling unplug callbacks.
  md - remove old plugging code.
  md/dm - remove remains of plug_fn callback.
  md: use new plugging interface for RAID IO.
  block: drop queue lock before calling __blk_run_queue() for kblockd punt
  Revert "block: add callback function for unplug notification"
  block: Enhance new plugging support to support general callbacks
This commit is contained in:
Linus Torvalds 2011-04-18 13:21:18 -07:00
commit 8a83f33100
18 changed files with 186 additions and 187 deletions

View File

@ -204,7 +204,7 @@ static void blk_delay_work(struct work_struct *work)
q = container_of(work, struct request_queue, delay_work.work);
spin_lock_irq(q->queue_lock);
__blk_run_queue(q, false);
__blk_run_queue(q);
spin_unlock_irq(q->queue_lock);
}
@ -220,7 +220,8 @@ static void blk_delay_work(struct work_struct *work)
*/
void blk_delay_queue(struct request_queue *q, unsigned long msecs)
{
schedule_delayed_work(&q->delay_work, msecs_to_jiffies(msecs));
queue_delayed_work(kblockd_workqueue, &q->delay_work,
msecs_to_jiffies(msecs));
}
EXPORT_SYMBOL(blk_delay_queue);
@ -238,7 +239,7 @@ void blk_start_queue(struct request_queue *q)
WARN_ON(!irqs_disabled());
queue_flag_clear(QUEUE_FLAG_STOPPED, q);
__blk_run_queue(q, false);
__blk_run_queue(q);
}
EXPORT_SYMBOL(blk_start_queue);
@ -296,9 +297,8 @@ EXPORT_SYMBOL(blk_sync_queue);
* Description:
* See @blk_run_queue. This variant must be called with the queue lock
* held and interrupts disabled.
*
*/
void __blk_run_queue(struct request_queue *q, bool force_kblockd)
void __blk_run_queue(struct request_queue *q)
{
if (unlikely(blk_queue_stopped(q)))
return;
@ -307,7 +307,7 @@ void __blk_run_queue(struct request_queue *q, bool force_kblockd)
* Only recurse once to avoid overrunning the stack, let the unplug
* handling reinvoke the handler shortly if we already got there.
*/
if (!force_kblockd && !queue_flag_test_and_set(QUEUE_FLAG_REENTER, q)) {
if (!queue_flag_test_and_set(QUEUE_FLAG_REENTER, q)) {
q->request_fn(q);
queue_flag_clear(QUEUE_FLAG_REENTER, q);
} else
@ -315,6 +315,20 @@ void __blk_run_queue(struct request_queue *q, bool force_kblockd)
}
EXPORT_SYMBOL(__blk_run_queue);
/**
* blk_run_queue_async - run a single device queue in workqueue context
* @q: The queue to run
*
* Description:
* Tells kblockd to perform the equivalent of @blk_run_queue on behalf
* of us.
*/
void blk_run_queue_async(struct request_queue *q)
{
if (likely(!blk_queue_stopped(q)))
queue_delayed_work(kblockd_workqueue, &q->delay_work, 0);
}
/**
* blk_run_queue - run a single device queue
* @q: The queue to run
@ -328,7 +342,7 @@ void blk_run_queue(struct request_queue *q)
unsigned long flags;
spin_lock_irqsave(q->queue_lock, flags);
__blk_run_queue(q, false);
__blk_run_queue(q);
spin_unlock_irqrestore(q->queue_lock, flags);
}
EXPORT_SYMBOL(blk_run_queue);
@ -977,7 +991,7 @@ void blk_insert_request(struct request_queue *q, struct request *rq,
blk_queue_end_tag(q, rq);
add_acct_request(q, rq, where);
__blk_run_queue(q, false);
__blk_run_queue(q);
spin_unlock_irqrestore(q->queue_lock, flags);
}
EXPORT_SYMBOL(blk_insert_request);
@ -1321,7 +1335,7 @@ get_rq:
} else {
spin_lock_irq(q->queue_lock);
add_acct_request(q, req, where);
__blk_run_queue(q, false);
__blk_run_queue(q);
out_unlock:
spin_unlock_irq(q->queue_lock);
}
@ -2638,6 +2652,7 @@ void blk_start_plug(struct blk_plug *plug)
plug->magic = PLUG_MAGIC;
INIT_LIST_HEAD(&plug->list);
INIT_LIST_HEAD(&plug->cb_list);
plug->should_sort = 0;
/*
@ -2670,12 +2685,41 @@ static int plug_rq_cmp(void *priv, struct list_head *a, struct list_head *b)
*/
static void queue_unplugged(struct request_queue *q, unsigned int depth,
bool from_schedule)
__releases(q->queue_lock)
{
trace_block_unplug(q, depth, !from_schedule);
__blk_run_queue(q, from_schedule);
if (q->unplugged_fn)
q->unplugged_fn(q);
/*
* If we are punting this to kblockd, then we can safely drop
* the queue_lock before waking kblockd (which needs to take
* this lock).
*/
if (from_schedule) {
spin_unlock(q->queue_lock);
blk_run_queue_async(q);
} else {
__blk_run_queue(q);
spin_unlock(q->queue_lock);
}
}
static void flush_plug_callbacks(struct blk_plug *plug)
{
LIST_HEAD(callbacks);
if (list_empty(&plug->cb_list))
return;
list_splice_init(&plug->cb_list, &callbacks);
while (!list_empty(&callbacks)) {
struct blk_plug_cb *cb = list_first_entry(&callbacks,
struct blk_plug_cb,
list);
list_del(&cb->list);
cb->callback(cb);
}
}
void blk_flush_plug_list(struct blk_plug *plug, bool from_schedule)
@ -2688,6 +2732,7 @@ void blk_flush_plug_list(struct blk_plug *plug, bool from_schedule)
BUG_ON(plug->magic != PLUG_MAGIC);
flush_plug_callbacks(plug);
if (list_empty(&plug->list))
return;
@ -2712,10 +2757,11 @@ void blk_flush_plug_list(struct blk_plug *plug, bool from_schedule)
BUG_ON(!(rq->cmd_flags & REQ_ON_PLUG));
BUG_ON(!rq->q);
if (rq->q != q) {
if (q) {
/*
* This drops the queue lock
*/
if (q)
queue_unplugged(q, depth, from_schedule);
spin_unlock(q->queue_lock);
}
q = rq->q;
depth = 0;
spin_lock(q->queue_lock);
@ -2733,10 +2779,11 @@ void blk_flush_plug_list(struct blk_plug *plug, bool from_schedule)
depth++;
}
if (q) {
/*
* This drops the queue lock
*/
if (q)
queue_unplugged(q, depth, from_schedule);
spin_unlock(q->queue_lock);
}
local_irq_restore(flags);
}

View File

@ -55,7 +55,7 @@ void blk_execute_rq_nowait(struct request_queue *q, struct gendisk *bd_disk,
WARN_ON(irqs_disabled());
spin_lock_irq(q->queue_lock);
__elv_add_request(q, rq, where);
__blk_run_queue(q, false);
__blk_run_queue(q);
/* the queue is stopped so it won't be plugged+unplugged */
if (rq->cmd_type == REQ_TYPE_PM_RESUME)
q->request_fn(q);

View File

@ -218,7 +218,7 @@ static void flush_end_io(struct request *flush_rq, int error)
* request_fn may confuse the driver. Always use kblockd.
*/
if (queued)
__blk_run_queue(q, true);
blk_run_queue_async(q);
}
/**
@ -274,7 +274,7 @@ static void flush_data_end_io(struct request *rq, int error)
* the comment in flush_end_io().
*/
if (blk_flush_complete_seq(rq, REQ_FSEQ_DATA, error))
__blk_run_queue(q, true);
blk_run_queue_async(q);
}
/**

View File

@ -790,22 +790,6 @@ void blk_queue_flush(struct request_queue *q, unsigned int flush)
}
EXPORT_SYMBOL_GPL(blk_queue_flush);
/**
* blk_queue_unplugged - register a callback for an unplug event
* @q: the request queue for the device
* @fn: the function to call
*
* Some stacked drivers may need to know when IO is dispatched on an
* unplug event. By registrering a callback here, they will be notified
* when someone flushes their on-stack queue plug. The function will be
* called with the queue lock held.
*/
void blk_queue_unplugged(struct request_queue *q, unplugged_fn *fn)
{
q->unplugged_fn = fn;
}
EXPORT_SYMBOL(blk_queue_unplugged);
static int __init blk_settings_init(void)
{
blk_max_low_pfn = max_low_pfn - 1;

View File

@ -22,6 +22,7 @@ void blk_rq_timed_out_timer(unsigned long data);
void blk_delete_timer(struct request *);
void blk_add_timer(struct request *);
void __generic_unplug_device(struct request_queue *);
void blk_run_queue_async(struct request_queue *q);
/*
* Internal atomic flags for request handling

View File

@ -3368,7 +3368,7 @@ cfq_rq_enqueued(struct cfq_data *cfqd, struct cfq_queue *cfqq,
cfqd->busy_queues > 1) {
cfq_del_timer(cfqd, cfqq);
cfq_clear_cfqq_wait_request(cfqq);
__blk_run_queue(cfqd->queue, false);
__blk_run_queue(cfqd->queue);
} else {
cfq_blkiocg_update_idle_time_stats(
&cfqq->cfqg->blkg);
@ -3383,7 +3383,7 @@ cfq_rq_enqueued(struct cfq_data *cfqd, struct cfq_queue *cfqq,
* this new queue is RT and the current one is BE
*/
cfq_preempt_queue(cfqd, cfqq);
__blk_run_queue(cfqd->queue, false);
__blk_run_queue(cfqd->queue);
}
}
@ -3743,7 +3743,7 @@ static void cfq_kick_queue(struct work_struct *work)
struct request_queue *q = cfqd->queue;
spin_lock_irq(q->queue_lock);
__blk_run_queue(cfqd->queue, false);
__blk_run_queue(cfqd->queue);
spin_unlock_irq(q->queue_lock);
}

View File

@ -642,7 +642,7 @@ void elv_quiesce_start(struct request_queue *q)
*/
elv_drain_elevator(q);
while (q->rq.elvpriv) {
__blk_run_queue(q, false);
__blk_run_queue(q);
spin_unlock_irq(q->queue_lock);
msleep(10);
spin_lock_irq(q->queue_lock);
@ -695,7 +695,7 @@ void __elv_add_request(struct request_queue *q, struct request *rq, int where)
* with anything. There's no point in delaying queue
* processing.
*/
__blk_run_queue(q, false);
__blk_run_queue(q);
break;
case ELEVATOR_INSERT_SORT_MERGE:

View File

@ -390,13 +390,6 @@ static int raid_is_congested(struct dm_target_callbacks *cb, int bits)
return md_raid5_congested(&rs->md, bits);
}
static void raid_unplug(struct dm_target_callbacks *cb)
{
struct raid_set *rs = container_of(cb, struct raid_set, callbacks);
md_raid5_kick_device(rs->md.private);
}
/*
* Construct a RAID4/5/6 mapping:
* Args:
@ -487,7 +480,6 @@ static int raid_ctr(struct dm_target *ti, unsigned argc, char **argv)
}
rs->callbacks.congested_fn = raid_is_congested;
rs->callbacks.unplug_fn = raid_unplug;
dm_table_add_target_callbacks(ti->table, &rs->callbacks);
return 0;

View File

@ -447,48 +447,59 @@ EXPORT_SYMBOL(md_flush_request);
/* Support for plugging.
* This mirrors the plugging support in request_queue, but does not
* require having a whole queue
* require having a whole queue or request structures.
* We allocate an md_plug_cb for each md device and each thread it gets
* plugged on. This links tot the private plug_handle structure in the
* personality data where we keep a count of the number of outstanding
* plugs so other code can see if a plug is active.
*/
static void plugger_work(struct work_struct *work)
{
struct plug_handle *plug =
container_of(work, struct plug_handle, unplug_work);
plug->unplug_fn(plug);
}
static void plugger_timeout(unsigned long data)
{
struct plug_handle *plug = (void *)data;
kblockd_schedule_work(NULL, &plug->unplug_work);
}
void plugger_init(struct plug_handle *plug,
void (*unplug_fn)(struct plug_handle *))
{
plug->unplug_flag = 0;
plug->unplug_fn = unplug_fn;
init_timer(&plug->unplug_timer);
plug->unplug_timer.function = plugger_timeout;
plug->unplug_timer.data = (unsigned long)plug;
INIT_WORK(&plug->unplug_work, plugger_work);
}
EXPORT_SYMBOL_GPL(plugger_init);
struct md_plug_cb {
struct blk_plug_cb cb;
mddev_t *mddev;
};
void plugger_set_plug(struct plug_handle *plug)
static void plugger_unplug(struct blk_plug_cb *cb)
{
if (!test_and_set_bit(PLUGGED_FLAG, &plug->unplug_flag))
mod_timer(&plug->unplug_timer, jiffies + msecs_to_jiffies(3)+1);
struct md_plug_cb *mdcb = container_of(cb, struct md_plug_cb, cb);
if (atomic_dec_and_test(&mdcb->mddev->plug_cnt))
md_wakeup_thread(mdcb->mddev->thread);
kfree(mdcb);
}
EXPORT_SYMBOL_GPL(plugger_set_plug);
int plugger_remove_plug(struct plug_handle *plug)
/* Check that an unplug wakeup will come shortly.
* If not, wakeup the md thread immediately
*/
int mddev_check_plugged(mddev_t *mddev)
{
if (test_and_clear_bit(PLUGGED_FLAG, &plug->unplug_flag)) {
del_timer(&plug->unplug_timer);
return 1;
} else
struct blk_plug *plug = current->plug;
struct md_plug_cb *mdcb;
if (!plug)
return 0;
}
EXPORT_SYMBOL_GPL(plugger_remove_plug);
list_for_each_entry(mdcb, &plug->cb_list, cb.list) {
if (mdcb->cb.callback == plugger_unplug &&
mdcb->mddev == mddev) {
/* Already on the list, move to top */
if (mdcb != list_first_entry(&plug->cb_list,
struct md_plug_cb,
cb.list))
list_move(&mdcb->cb.list, &plug->cb_list);
return 1;
}
}
/* Not currently on the callback list */
mdcb = kmalloc(sizeof(*mdcb), GFP_ATOMIC);
if (!mdcb)
return 0;
mdcb->mddev = mddev;
mdcb->cb.callback = plugger_unplug;
atomic_inc(&mddev->plug_cnt);
list_add(&mdcb->cb.list, &plug->cb_list);
return 1;
}
EXPORT_SYMBOL_GPL(mddev_check_plugged);
static inline mddev_t *mddev_get(mddev_t *mddev)
{
@ -538,6 +549,7 @@ void mddev_init(mddev_t *mddev)
atomic_set(&mddev->active, 1);
atomic_set(&mddev->openers, 0);
atomic_set(&mddev->active_io, 0);
atomic_set(&mddev->plug_cnt, 0);
spin_lock_init(&mddev->write_lock);
atomic_set(&mddev->flush_pending, 0);
init_waitqueue_head(&mddev->sb_wait);
@ -4723,7 +4735,6 @@ static void md_clean(mddev_t *mddev)
mddev->bitmap_info.chunksize = 0;
mddev->bitmap_info.daemon_sleep = 0;
mddev->bitmap_info.max_write_behind = 0;
mddev->plug = NULL;
}
static void __md_stop_writes(mddev_t *mddev)
@ -6688,12 +6699,6 @@ int md_allow_write(mddev_t *mddev)
}
EXPORT_SYMBOL_GPL(md_allow_write);
void md_unplug(mddev_t *mddev)
{
if (mddev->plug)
mddev->plug->unplug_fn(mddev->plug);
}
#define SYNC_MARKS 10
#define SYNC_MARK_STEP (3*HZ)
void md_do_sync(mddev_t *mddev)

View File

@ -29,26 +29,6 @@
typedef struct mddev_s mddev_t;
typedef struct mdk_rdev_s mdk_rdev_t;
/* generic plugging support - like that provided with request_queue,
* but does not require a request_queue
*/
struct plug_handle {
void (*unplug_fn)(struct plug_handle *);
struct timer_list unplug_timer;
struct work_struct unplug_work;
unsigned long unplug_flag;
};
#define PLUGGED_FLAG 1
void plugger_init(struct plug_handle *plug,
void (*unplug_fn)(struct plug_handle *));
void plugger_set_plug(struct plug_handle *plug);
int plugger_remove_plug(struct plug_handle *plug);
static inline void plugger_flush(struct plug_handle *plug)
{
del_timer_sync(&plug->unplug_timer);
cancel_work_sync(&plug->unplug_work);
}
/*
* MD's 'extended' device
*/
@ -199,6 +179,9 @@ struct mddev_s
int delta_disks, new_level, new_layout;
int new_chunk_sectors;
atomic_t plug_cnt; /* If device is expecting
* more bios soon.
*/
struct mdk_thread_s *thread; /* management thread */
struct mdk_thread_s *sync_thread; /* doing resync or reconstruct */
sector_t curr_resync; /* last block scheduled */
@ -336,7 +319,6 @@ struct mddev_s
struct list_head all_mddevs;
struct attribute_group *to_remove;
struct plug_handle *plug; /* if used by personality */
struct bio_set *bio_set;
@ -516,7 +498,6 @@ extern int md_integrity_register(mddev_t *mddev);
extern void md_integrity_add_rdev(mdk_rdev_t *rdev, mddev_t *mddev);
extern int strict_strtoul_scaled(const char *cp, unsigned long *res, int scale);
extern void restore_bitmap_write_access(struct file *file);
extern void md_unplug(mddev_t *mddev);
extern void mddev_init(mddev_t *mddev);
extern int md_run(mddev_t *mddev);
@ -530,4 +511,5 @@ extern struct bio *bio_clone_mddev(struct bio *bio, gfp_t gfp_mask,
mddev_t *mddev);
extern struct bio *bio_alloc_mddev(gfp_t gfp_mask, int nr_iovecs,
mddev_t *mddev);
extern int mddev_check_plugged(mddev_t *mddev);
#endif /* _MD_MD_H */

View File

@ -565,12 +565,6 @@ static void flush_pending_writes(conf_t *conf)
spin_unlock_irq(&conf->device_lock);
}
static void md_kick_device(mddev_t *mddev)
{
blk_flush_plug(current);
md_wakeup_thread(mddev->thread);
}
/* Barriers....
* Sometimes we need to suspend IO while we do something else,
* either some resync/recovery, or reconfigure the array.
@ -600,7 +594,7 @@ static void raise_barrier(conf_t *conf)
/* Wait until no block IO is waiting */
wait_event_lock_irq(conf->wait_barrier, !conf->nr_waiting,
conf->resync_lock, md_kick_device(conf->mddev));
conf->resync_lock, );
/* block any new IO from starting */
conf->barrier++;
@ -608,7 +602,7 @@ static void raise_barrier(conf_t *conf)
/* Now wait for all pending IO to complete */
wait_event_lock_irq(conf->wait_barrier,
!conf->nr_pending && conf->barrier < RESYNC_DEPTH,
conf->resync_lock, md_kick_device(conf->mddev));
conf->resync_lock, );
spin_unlock_irq(&conf->resync_lock);
}
@ -630,7 +624,7 @@ static void wait_barrier(conf_t *conf)
conf->nr_waiting++;
wait_event_lock_irq(conf->wait_barrier, !conf->barrier,
conf->resync_lock,
md_kick_device(conf->mddev));
);
conf->nr_waiting--;
}
conf->nr_pending++;
@ -666,8 +660,7 @@ static void freeze_array(conf_t *conf)
wait_event_lock_irq(conf->wait_barrier,
conf->nr_pending == conf->nr_queued+1,
conf->resync_lock,
({ flush_pending_writes(conf);
md_kick_device(conf->mddev); }));
flush_pending_writes(conf));
spin_unlock_irq(&conf->resync_lock);
}
static void unfreeze_array(conf_t *conf)
@ -729,6 +722,7 @@ static int make_request(mddev_t *mddev, struct bio * bio)
const unsigned long do_sync = (bio->bi_rw & REQ_SYNC);
const unsigned long do_flush_fua = (bio->bi_rw & (REQ_FLUSH | REQ_FUA));
mdk_rdev_t *blocked_rdev;
int plugged;
/*
* Register the new request and wait if the reconstruction
@ -820,6 +814,8 @@ static int make_request(mddev_t *mddev, struct bio * bio)
* inc refcount on their rdev. Record them by setting
* bios[x] to bio
*/
plugged = mddev_check_plugged(mddev);
disks = conf->raid_disks;
retry_write:
blocked_rdev = NULL;
@ -925,7 +921,7 @@ static int make_request(mddev_t *mddev, struct bio * bio)
/* In case raid1d snuck in to freeze_array */
wake_up(&conf->wait_barrier);
if (do_sync || !bitmap)
if (do_sync || !bitmap || !plugged)
md_wakeup_thread(mddev->thread);
return 0;
@ -1516,13 +1512,16 @@ static void raid1d(mddev_t *mddev)
conf_t *conf = mddev->private;
struct list_head *head = &conf->retry_list;
mdk_rdev_t *rdev;
struct blk_plug plug;
md_check_recovery(mddev);
blk_start_plug(&plug);
for (;;) {
char b[BDEVNAME_SIZE];
flush_pending_writes(conf);
if (atomic_read(&mddev->plug_cnt) == 0)
flush_pending_writes(conf);
spin_lock_irqsave(&conf->device_lock, flags);
if (list_empty(head)) {
@ -1593,6 +1592,7 @@ static void raid1d(mddev_t *mddev)
}
cond_resched();
}
blk_finish_plug(&plug);
}
@ -2039,7 +2039,6 @@ static int stop(mddev_t *mddev)
md_unregister_thread(mddev->thread);
mddev->thread = NULL;
blk_sync_queue(mddev->queue); /* the unplug fn references 'conf'*/
if (conf->r1bio_pool)
mempool_destroy(conf->r1bio_pool);
kfree(conf->mirrors);

View File

@ -634,12 +634,6 @@ static void flush_pending_writes(conf_t *conf)
spin_unlock_irq(&conf->device_lock);
}
static void md_kick_device(mddev_t *mddev)
{
blk_flush_plug(current);
md_wakeup_thread(mddev->thread);
}
/* Barriers....
* Sometimes we need to suspend IO while we do something else,
* either some resync/recovery, or reconfigure the array.
@ -669,15 +663,15 @@ static void raise_barrier(conf_t *conf, int force)
/* Wait until no block IO is waiting (unless 'force') */
wait_event_lock_irq(conf->wait_barrier, force || !conf->nr_waiting,
conf->resync_lock, md_kick_device(conf->mddev));
conf->resync_lock, );
/* block any new IO from starting */
conf->barrier++;
/* No wait for all pending IO to complete */
/* Now wait for all pending IO to complete */
wait_event_lock_irq(conf->wait_barrier,
!conf->nr_pending && conf->barrier < RESYNC_DEPTH,
conf->resync_lock, md_kick_device(conf->mddev));
conf->resync_lock, );
spin_unlock_irq(&conf->resync_lock);
}
@ -698,7 +692,7 @@ static void wait_barrier(conf_t *conf)
conf->nr_waiting++;
wait_event_lock_irq(conf->wait_barrier, !conf->barrier,
conf->resync_lock,
md_kick_device(conf->mddev));
);
conf->nr_waiting--;
}
conf->nr_pending++;
@ -734,8 +728,8 @@ static void freeze_array(conf_t *conf)
wait_event_lock_irq(conf->wait_barrier,
conf->nr_pending == conf->nr_queued+1,
conf->resync_lock,
({ flush_pending_writes(conf);
md_kick_device(conf->mddev); }));
flush_pending_writes(conf));
spin_unlock_irq(&conf->resync_lock);
}
@ -762,6 +756,7 @@ static int make_request(mddev_t *mddev, struct bio * bio)
const unsigned long do_fua = (bio->bi_rw & REQ_FUA);
unsigned long flags;
mdk_rdev_t *blocked_rdev;
int plugged;
if (unlikely(bio->bi_rw & REQ_FLUSH)) {
md_flush_request(mddev, bio);
@ -870,6 +865,8 @@ static int make_request(mddev_t *mddev, struct bio * bio)
* inc refcount on their rdev. Record them by setting
* bios[x] to bio
*/
plugged = mddev_check_plugged(mddev);
raid10_find_phys(conf, r10_bio);
retry_write:
blocked_rdev = NULL;
@ -946,9 +943,8 @@ static int make_request(mddev_t *mddev, struct bio * bio)
/* In case raid10d snuck in to freeze_array */
wake_up(&conf->wait_barrier);
if (do_sync || !mddev->bitmap)
if (do_sync || !mddev->bitmap || !plugged)
md_wakeup_thread(mddev->thread);
return 0;
}
@ -1640,9 +1636,11 @@ static void raid10d(mddev_t *mddev)
conf_t *conf = mddev->private;
struct list_head *head = &conf->retry_list;
mdk_rdev_t *rdev;
struct blk_plug plug;
md_check_recovery(mddev);
blk_start_plug(&plug);
for (;;) {
char b[BDEVNAME_SIZE];
@ -1716,6 +1714,7 @@ static void raid10d(mddev_t *mddev)
}
cond_resched();
}
blk_finish_plug(&plug);
}

View File

@ -27,12 +27,12 @@
*
* We group bitmap updates into batches. Each batch has a number.
* We may write out several batches at once, but that isn't very important.
* conf->bm_write is the number of the last batch successfully written.
* conf->bm_flush is the number of the last batch that was closed to
* conf->seq_write is the number of the last batch successfully written.
* conf->seq_flush is the number of the last batch that was closed to
* new additions.
* When we discover that we will need to write to any block in a stripe
* (in add_stripe_bio) we update the in-memory bitmap and record in sh->bm_seq
* the number of the batch it will be in. This is bm_flush+1.
* the number of the batch it will be in. This is seq_flush+1.
* When we are ready to do a write, if that batch hasn't been written yet,
* we plug the array and queue the stripe for later.
* When an unplug happens, we increment bm_flush, thus closing the current
@ -199,14 +199,12 @@ static void __release_stripe(raid5_conf_t *conf, struct stripe_head *sh)
BUG_ON(!list_empty(&sh->lru));
BUG_ON(atomic_read(&conf->active_stripes)==0);
if (test_bit(STRIPE_HANDLE, &sh->state)) {
if (test_bit(STRIPE_DELAYED, &sh->state)) {
if (test_bit(STRIPE_DELAYED, &sh->state))
list_add_tail(&sh->lru, &conf->delayed_list);
plugger_set_plug(&conf->plug);
} else if (test_bit(STRIPE_BIT_DELAY, &sh->state) &&
sh->bm_seq - conf->seq_write > 0) {
else if (test_bit(STRIPE_BIT_DELAY, &sh->state) &&
sh->bm_seq - conf->seq_write > 0)
list_add_tail(&sh->lru, &conf->bitmap_list);
plugger_set_plug(&conf->plug);
} else {
else {
clear_bit(STRIPE_BIT_DELAY, &sh->state);
list_add_tail(&sh->lru, &conf->handle_list);
}
@ -461,7 +459,7 @@ get_active_stripe(raid5_conf_t *conf, sector_t sector,
< (conf->max_nr_stripes *3/4)
|| !conf->inactive_blocked),
conf->device_lock,
md_raid5_kick_device(conf));
);
conf->inactive_blocked = 0;
} else
init_stripe(sh, sector, previous);
@ -1470,7 +1468,7 @@ static int resize_stripes(raid5_conf_t *conf, int newsize)
wait_event_lock_irq(conf->wait_for_stripe,
!list_empty(&conf->inactive_list),
conf->device_lock,
blk_flush_plug(current));
);
osh = get_free_stripe(conf);
spin_unlock_irq(&conf->device_lock);
atomic_set(&nsh->count, 1);
@ -3623,8 +3621,7 @@ static void raid5_activate_delayed(raid5_conf_t *conf)
atomic_inc(&conf->preread_active_stripes);
list_add_tail(&sh->lru, &conf->hold_list);
}
} else
plugger_set_plug(&conf->plug);
}
}
static void activate_bit_delay(raid5_conf_t *conf)
@ -3641,21 +3638,6 @@ static void activate_bit_delay(raid5_conf_t *conf)
}
}
void md_raid5_kick_device(raid5_conf_t *conf)
{
blk_flush_plug(current);
raid5_activate_delayed(conf);
md_wakeup_thread(conf->mddev->thread);
}
EXPORT_SYMBOL_GPL(md_raid5_kick_device);
static void raid5_unplug(struct plug_handle *plug)
{
raid5_conf_t *conf = container_of(plug, raid5_conf_t, plug);
md_raid5_kick_device(conf);
}
int md_raid5_congested(mddev_t *mddev, int bits)
{
raid5_conf_t *conf = mddev->private;
@ -3945,6 +3927,7 @@ static int make_request(mddev_t *mddev, struct bio * bi)
struct stripe_head *sh;
const int rw = bio_data_dir(bi);
int remaining;
int plugged;
if (unlikely(bi->bi_rw & REQ_FLUSH)) {
md_flush_request(mddev, bi);
@ -3963,6 +3946,7 @@ static int make_request(mddev_t *mddev, struct bio * bi)
bi->bi_next = NULL;
bi->bi_phys_segments = 1; /* over-loaded to count active stripes */
plugged = mddev_check_plugged(mddev);
for (;logical_sector < last_sector; logical_sector += STRIPE_SECTORS) {
DEFINE_WAIT(w);
int disks, data_disks;
@ -4057,7 +4041,7 @@ static int make_request(mddev_t *mddev, struct bio * bi)
* add failed due to overlap. Flush everything
* and wait a while
*/
md_raid5_kick_device(conf);
md_wakeup_thread(mddev->thread);
release_stripe(sh);
schedule();
goto retry;
@ -4077,6 +4061,9 @@ static int make_request(mddev_t *mddev, struct bio * bi)
}
}
if (!plugged)
md_wakeup_thread(mddev->thread);
spin_lock_irq(&conf->device_lock);
remaining = raid5_dec_bi_phys_segments(bi);
spin_unlock_irq(&conf->device_lock);
@ -4478,24 +4465,30 @@ static void raid5d(mddev_t *mddev)
struct stripe_head *sh;
raid5_conf_t *conf = mddev->private;
int handled;
struct blk_plug plug;
pr_debug("+++ raid5d active\n");
md_check_recovery(mddev);
blk_start_plug(&plug);
handled = 0;
spin_lock_irq(&conf->device_lock);
while (1) {
struct bio *bio;
if (conf->seq_flush != conf->seq_write) {
int seq = conf->seq_flush;
if (atomic_read(&mddev->plug_cnt) == 0 &&
!list_empty(&conf->bitmap_list)) {
/* Now is a good time to flush some bitmap updates */
conf->seq_flush++;
spin_unlock_irq(&conf->device_lock);
bitmap_unplug(mddev->bitmap);
spin_lock_irq(&conf->device_lock);
conf->seq_write = seq;
conf->seq_write = conf->seq_flush;
activate_bit_delay(conf);
}
if (atomic_read(&mddev->plug_cnt) == 0)
raid5_activate_delayed(conf);
while ((bio = remove_bio_from_retry(conf))) {
int ok;
@ -4525,6 +4518,7 @@ static void raid5d(mddev_t *mddev)
spin_unlock_irq(&conf->device_lock);
async_tx_issue_pending_all();
blk_finish_plug(&plug);
pr_debug("--- raid5d inactive\n");
}
@ -5141,8 +5135,6 @@ static int run(mddev_t *mddev)
mdname(mddev));
md_set_array_sectors(mddev, raid5_size(mddev, 0, 0));
plugger_init(&conf->plug, raid5_unplug);
mddev->plug = &conf->plug;
if (mddev->queue) {
int chunk_size;
/* read-ahead size must cover two whole stripes, which
@ -5192,7 +5184,6 @@ static int stop(mddev_t *mddev)
mddev->thread = NULL;
if (mddev->queue)
mddev->queue->backing_dev_info.congested_fn = NULL;
plugger_flush(&conf->plug); /* the unplug fn references 'conf'*/
free_conf(conf);
mddev->private = NULL;
mddev->to_remove = &raid5_attrs_group;

View File

@ -400,8 +400,6 @@ struct raid5_private_data {
* Cleared when a sync completes.
*/
struct plug_handle plug;
/* per cpu variables */
struct raid5_percpu {
struct page *spare_page; /* Used when checking P/Q in raid6 */

View File

@ -443,7 +443,7 @@ static void scsi_run_queue(struct request_queue *q)
&sdev->request_queue->queue_flags);
if (flagset)
queue_flag_set(QUEUE_FLAG_REENTER, sdev->request_queue);
__blk_run_queue(sdev->request_queue, false);
__blk_run_queue(sdev->request_queue);
if (flagset)
queue_flag_clear(QUEUE_FLAG_REENTER, sdev->request_queue);
spin_unlock(sdev->request_queue->queue_lock);

View File

@ -3829,7 +3829,7 @@ fc_bsg_goose_queue(struct fc_rport *rport)
!test_bit(QUEUE_FLAG_REENTER, &rport->rqst_q->queue_flags);
if (flagset)
queue_flag_set(QUEUE_FLAG_REENTER, rport->rqst_q);
__blk_run_queue(rport->rqst_q, false);
__blk_run_queue(rport->rqst_q);
if (flagset)
queue_flag_clear(QUEUE_FLAG_REENTER, rport->rqst_q);
spin_unlock_irqrestore(rport->rqst_q->queue_lock, flags);

View File

@ -196,7 +196,6 @@ typedef void (request_fn_proc) (struct request_queue *q);
typedef int (make_request_fn) (struct request_queue *q, struct bio *bio);
typedef int (prep_rq_fn) (struct request_queue *, struct request *);
typedef void (unprep_rq_fn) (struct request_queue *, struct request *);
typedef void (unplugged_fn) (struct request_queue *);
struct bio_vec;
struct bvec_merge_data {
@ -284,7 +283,6 @@ struct request_queue
rq_timed_out_fn *rq_timed_out_fn;
dma_drain_needed_fn *dma_drain_needed;
lld_busy_fn *lld_busy_fn;
unplugged_fn *unplugged_fn;
/*
* Dispatch queue sorting
@ -699,7 +697,7 @@ extern void blk_start_queue(struct request_queue *q);
extern void blk_stop_queue(struct request_queue *q);
extern void blk_sync_queue(struct request_queue *q);
extern void __blk_stop_queue(struct request_queue *q);
extern void __blk_run_queue(struct request_queue *q, bool force_kblockd);
extern void __blk_run_queue(struct request_queue *q);
extern void blk_run_queue(struct request_queue *);
extern int blk_rq_map_user(struct request_queue *, struct request *,
struct rq_map_data *, void __user *, unsigned long,
@ -843,7 +841,6 @@ extern void blk_queue_dma_alignment(struct request_queue *, int);
extern void blk_queue_update_dma_alignment(struct request_queue *, int);
extern void blk_queue_softirq_done(struct request_queue *, softirq_done_fn *);
extern void blk_queue_rq_timed_out(struct request_queue *, rq_timed_out_fn *);
extern void blk_queue_unplugged(struct request_queue *, unplugged_fn *);
extern void blk_queue_rq_timeout(struct request_queue *, unsigned int);
extern void blk_queue_flush(struct request_queue *q, unsigned int flush);
extern struct backing_dev_info *blk_get_backing_dev_info(struct block_device *bdev);
@ -860,8 +857,13 @@ extern void blk_put_queue(struct request_queue *);
struct blk_plug {
unsigned long magic;
struct list_head list;
struct list_head cb_list;
unsigned int should_sort;
};
struct blk_plug_cb {
struct list_head list;
void (*callback)(struct blk_plug_cb *);
};
extern void blk_start_plug(struct blk_plug *);
extern void blk_finish_plug(struct blk_plug *);
@ -887,7 +889,7 @@ static inline bool blk_needs_flush_plug(struct task_struct *tsk)
{
struct blk_plug *plug = tsk->plug;
return plug && !list_empty(&plug->list);
return plug && (!list_empty(&plug->list) || !list_empty(&plug->cb_list));
}
/*

View File

@ -197,7 +197,6 @@ struct dm_target {
struct dm_target_callbacks {
struct list_head list;
int (*congested_fn) (struct dm_target_callbacks *, int);
void (*unplug_fn)(struct dm_target_callbacks *);
};
int dm_register_target(struct target_type *t);