summaryrefslogtreecommitdiff
path: root/block/blk-core.c
diff options
context:
space:
mode:
Diffstat (limited to 'block/blk-core.c')
-rw-r--r--block/blk-core.c737
1 files changed, 449 insertions, 288 deletions
diff --git a/block/blk-core.c b/block/blk-core.c
index a63336d49f30..a2e58eeb3549 100644
--- a/block/blk-core.c
+++ b/block/blk-core.c
@@ -27,6 +27,7 @@
#include <linux/writeback.h>
#include <linux/task_io_accounting_ops.h>
#include <linux/fault-inject.h>
+#include <linux/list_sort.h>
#define CREATE_TRACE_POINTS
#include <trace/events/block.h>
@@ -149,39 +150,29 @@ EXPORT_SYMBOL(blk_rq_init);
static void req_bio_endio(struct request *rq, struct bio *bio,
unsigned int nbytes, int error)
{
- struct request_queue *q = rq->q;
-
- if (&q->flush_rq != rq) {
- if (error)
- clear_bit(BIO_UPTODATE, &bio->bi_flags);
- else if (!test_bit(BIO_UPTODATE, &bio->bi_flags))
- error = -EIO;
+ if (error)
+ clear_bit(BIO_UPTODATE, &bio->bi_flags);
+ else if (!test_bit(BIO_UPTODATE, &bio->bi_flags))
+ error = -EIO;
- if (unlikely(nbytes > bio->bi_size)) {
- printk(KERN_ERR "%s: want %u bytes done, %u left\n",
- __func__, nbytes, bio->bi_size);
- nbytes = bio->bi_size;
- }
+ if (unlikely(nbytes > bio->bi_size)) {
+ printk(KERN_ERR "%s: want %u bytes done, %u left\n",
+ __func__, nbytes, bio->bi_size);
+ nbytes = bio->bi_size;
+ }
- if (unlikely(rq->cmd_flags & REQ_QUIET))
- set_bit(BIO_QUIET, &bio->bi_flags);
+ if (unlikely(rq->cmd_flags & REQ_QUIET))
+ set_bit(BIO_QUIET, &bio->bi_flags);
- bio->bi_size -= nbytes;
- bio->bi_sector += (nbytes >> 9);
+ bio->bi_size -= nbytes;
+ bio->bi_sector += (nbytes >> 9);
- if (bio_integrity(bio))
- bio_integrity_advance(bio, nbytes);
+ if (bio_integrity(bio))
+ bio_integrity_advance(bio, nbytes);
- if (bio->bi_size == 0)
- bio_endio(bio, error);
- } else {
- /*
- * Okay, this is the sequenced flush request in
- * progress, just record the error;
- */
- if (error && !q->flush_err)
- q->flush_err = error;
- }
+ /* don't actually finish bio if it's part of flush sequence */
+ if (bio->bi_size == 0 && !(rq->cmd_flags & REQ_FLUSH_SEQ))
+ bio_endio(bio, error);
}
void blk_dump_rq_flags(struct request *rq, char *msg)
@@ -207,136 +198,32 @@ void blk_dump_rq_flags(struct request *rq, char *msg)
}
EXPORT_SYMBOL(blk_dump_rq_flags);
-/*
- * "plug" the device if there are no outstanding requests: this will
- * force the transfer to start only after we have put all the requests
- * on the list.
- *
- * This is called with interrupts off and no requests on the queue and
- * with the queue lock held.
- */
-void blk_plug_device(struct request_queue *q)
+static void blk_delay_work(struct work_struct *work)
{
- WARN_ON(!irqs_disabled());
-
- /*
- * don't plug a stopped queue, it must be paired with blk_start_queue()
- * which will restart the queueing
- */
- if (blk_queue_stopped(q))
- return;
+ struct request_queue *q;
- if (!queue_flag_test_and_set(QUEUE_FLAG_PLUGGED, q)) {
- mod_timer(&q->unplug_timer, jiffies + q->unplug_delay);
- trace_block_plug(q);
- }
+ q = container_of(work, struct request_queue, delay_work.work);
+ spin_lock_irq(q->queue_lock);
+ __blk_run_queue(q);
+ spin_unlock_irq(q->queue_lock);
}
-EXPORT_SYMBOL(blk_plug_device);
/**
- * blk_plug_device_unlocked - plug a device without queue lock held
- * @q: The &struct request_queue to plug
+ * blk_delay_queue - restart queueing after defined interval
+ * @q: The &struct request_queue in question
+ * @msecs: Delay in msecs
*
* Description:
- * Like @blk_plug_device(), but grabs the queue lock and disables
- * interrupts.
- **/
-void blk_plug_device_unlocked(struct request_queue *q)
-{
- unsigned long flags;
-
- spin_lock_irqsave(q->queue_lock, flags);
- blk_plug_device(q);
- spin_unlock_irqrestore(q->queue_lock, flags);
-}
-EXPORT_SYMBOL(blk_plug_device_unlocked);
-
-/*
- * remove the queue from the plugged list, if present. called with
- * queue lock held and interrupts disabled.
- */
-int blk_remove_plug(struct request_queue *q)
-{
- WARN_ON(!irqs_disabled());
-
- if (!queue_flag_test_and_clear(QUEUE_FLAG_PLUGGED, q))
- return 0;
-
- del_timer(&q->unplug_timer);
- return 1;
-}
-EXPORT_SYMBOL(blk_remove_plug);
-
-/*
- * remove the plug and let it rip..
+ * Sometimes queueing needs to be postponed for a little while, to allow
+ * resources to come back. This function will make sure that queueing is
+ * restarted around the specified time.
*/
-void __generic_unplug_device(struct request_queue *q)
+void blk_delay_queue(struct request_queue *q, unsigned long msecs)
{
- if (unlikely(blk_queue_stopped(q)))
- return;
- if (!blk_remove_plug(q) && !blk_queue_nonrot(q))
- return;
-
- q->request_fn(q);
+ queue_delayed_work(kblockd_workqueue, &q->delay_work,
+ msecs_to_jiffies(msecs));
}
-
-/**
- * generic_unplug_device - fire a request queue
- * @q: The &struct request_queue in question
- *
- * Description:
- * Linux uses plugging to build bigger requests queues before letting
- * the device have at them. If a queue is plugged, the I/O scheduler
- * is still adding and merging requests on the queue. Once the queue
- * gets unplugged, the request_fn defined for the queue is invoked and
- * transfers started.
- **/
-void generic_unplug_device(struct request_queue *q)
-{
- if (blk_queue_plugged(q)) {
- spin_lock_irq(q->queue_lock);
- __generic_unplug_device(q);
- spin_unlock_irq(q->queue_lock);
- }
-}
-EXPORT_SYMBOL(generic_unplug_device);
-
-static void blk_backing_dev_unplug(struct backing_dev_info *bdi,
- struct page *page)
-{
- struct request_queue *q = bdi->unplug_io_data;
-
- blk_unplug(q);
-}
-
-void blk_unplug_work(struct work_struct *work)
-{
- struct request_queue *q =
- container_of(work, struct request_queue, unplug_work);
-
- trace_block_unplug_io(q);
- q->unplug_fn(q);
-}
-
-void blk_unplug_timeout(unsigned long data)
-{
- struct request_queue *q = (struct request_queue *)data;
-
- trace_block_unplug_timer(q);
- kblockd_schedule_work(q, &q->unplug_work);
-}
-
-void blk_unplug(struct request_queue *q)
-{
- /*
- * devices don't necessarily have an ->unplug_fn defined
- */
- if (q->unplug_fn) {
- trace_block_unplug_io(q);
- q->unplug_fn(q);
- }
-}
-EXPORT_SYMBOL(blk_unplug);
+EXPORT_SYMBOL(blk_delay_queue);
/**
* blk_start_queue - restart a previously stopped queue
@@ -352,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);
@@ -372,7 +259,7 @@ EXPORT_SYMBOL(blk_start_queue);
**/
void blk_stop_queue(struct request_queue *q)
{
- blk_remove_plug(q);
+ __cancel_delayed_work(&q->delay_work);
queue_flag_set(QUEUE_FLAG_STOPPED, q);
}
EXPORT_SYMBOL(blk_stop_queue);
@@ -390,51 +277,51 @@ EXPORT_SYMBOL(blk_stop_queue);
* that its ->make_request_fn will not re-add plugging prior to calling
* this function.
*
+ * This function does not cancel any asynchronous activity arising
+ * out of elevator or throttling code. That would require elevaotor_exit()
+ * and blk_throtl_exit() to be called with queue lock initialized.
+ *
*/
void blk_sync_queue(struct request_queue *q)
{
- del_timer_sync(&q->unplug_timer);
del_timer_sync(&q->timeout);
- cancel_work_sync(&q->unplug_work);
- throtl_shutdown_timer_wq(q);
+ cancel_delayed_work_sync(&q->delay_work);
}
EXPORT_SYMBOL(blk_sync_queue);
/**
* __blk_run_queue - run a single device queue
* @q: The queue to run
- * @force_kblockd: Don't run @q->request_fn directly. Use kblockd.
*
* 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)
{
- blk_remove_plug(q);
-
if (unlikely(blk_queue_stopped(q)))
return;
- if (elv_queue_empty(q))
- return;
-
- /*
- * 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)) {
- q->request_fn(q);
- queue_flag_clear(QUEUE_FLAG_REENTER, q);
- } else {
- queue_flag_set(QUEUE_FLAG_PLUGGED, q);
- kblockd_schedule_work(q, &q->unplug_work);
- }
+ q->request_fn(q);
}
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);
+}
+EXPORT_SYMBOL(blk_run_queue_async);
+
+/**
* blk_run_queue - run a single device queue
* @q: The queue to run
*
@@ -447,7 +334,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);
@@ -457,6 +344,11 @@ void blk_put_queue(struct request_queue *q)
kobject_put(&q->kobj);
}
+/*
+ * Note: If a driver supplied the queue lock, it should not zap that lock
+ * unexpectedly as some queue cleanup components like elevator_exit() and
+ * blk_throtl_exit() need queue lock.
+ */
void blk_cleanup_queue(struct request_queue *q)
{
/*
@@ -475,6 +367,8 @@ void blk_cleanup_queue(struct request_queue *q)
if (q->elevator)
elevator_exit(q->elevator);
+ blk_throtl_exit(q);
+
blk_put_queue(q);
}
EXPORT_SYMBOL(blk_cleanup_queue);
@@ -517,8 +411,6 @@ struct request_queue *blk_alloc_queue_node(gfp_t gfp_mask, int node_id)
if (!q)
return NULL;
- q->backing_dev_info.unplug_io_fn = blk_backing_dev_unplug;
- q->backing_dev_info.unplug_io_data = q;
q->backing_dev_info.ra_pages =
(VM_MAX_READAHEAD * 1024) / PAGE_CACHE_SIZE;
q->backing_dev_info.state = 0;
@@ -538,17 +430,24 @@ struct request_queue *blk_alloc_queue_node(gfp_t gfp_mask, int node_id)
setup_timer(&q->backing_dev_info.laptop_mode_wb_timer,
laptop_mode_timer_fn, (unsigned long) q);
- init_timer(&q->unplug_timer);
setup_timer(&q->timeout, blk_rq_timed_out_timer, (unsigned long) q);
INIT_LIST_HEAD(&q->timeout_list);
- INIT_LIST_HEAD(&q->pending_flushes);
- INIT_WORK(&q->unplug_work, blk_unplug_work);
+ INIT_LIST_HEAD(&q->flush_queue[0]);
+ INIT_LIST_HEAD(&q->flush_queue[1]);
+ INIT_LIST_HEAD(&q->flush_data_in_flight);
+ INIT_DELAYED_WORK(&q->delay_work, blk_delay_work);
kobject_init(&q->kobj, &blk_queue_ktype);
mutex_init(&q->sysfs_lock);
spin_lock_init(&q->__queue_lock);
+ /*
+ * By default initialize queue_lock to internal lock and driver can
+ * override it later if need be.
+ */
+ q->queue_lock = &q->__queue_lock;
+
return q;
}
EXPORT_SYMBOL(blk_alloc_queue_node);
@@ -631,9 +530,11 @@ blk_init_allocated_queue_node(struct request_queue *q, request_fn_proc *rfn,
q->request_fn = rfn;
q->prep_rq_fn = NULL;
q->unprep_rq_fn = NULL;
- q->unplug_fn = generic_unplug_device;
q->queue_flags = QUEUE_FLAG_DEFAULT;
- q->queue_lock = lock;
+
+ /* Override internal queue lock with supplied lock pointer */
+ if (lock)
+ q->queue_lock = lock;
/*
* This also sets hw/phys segments, boundary and size
@@ -666,6 +567,8 @@ int blk_get_queue(struct request_queue *q)
static inline void blk_free_request(struct request_queue *q, struct request *rq)
{
+ BUG_ON(rq->cmd_flags & REQ_ON_PLUG);
+
if (rq->cmd_flags & REQ_ELVPRIV)
elv_put_request(q, rq);
mempool_free(rq, q->rq.rq_pool);
@@ -762,6 +665,25 @@ static void freed_request(struct request_queue *q, int sync, int priv)
}
/*
+ * Determine if elevator data should be initialized when allocating the
+ * request associated with @bio.
+ */
+static bool blk_rq_should_init_elevator(struct bio *bio)
+{
+ if (!bio)
+ return true;
+
+ /*
+ * Flush requests do not use the elevator so skip initialization.
+ * This allows a request to share the flush and elevator data.
+ */
+ if (bio->bi_rw & (REQ_FLUSH | REQ_FUA))
+ return false;
+
+ return true;
+}
+
+/*
* Get a free request, queue_lock must be held.
* Returns NULL on failure, with queue_lock held.
* Returns !NULL on success, with queue_lock *not held*.
@@ -773,7 +695,7 @@ static struct request *get_request(struct request_queue *q, int rw_flags,
struct request_list *rl = &q->rq;
struct io_context *ioc = NULL;
const bool is_sync = rw_is_sync(rw_flags) != 0;
- int may_queue, priv;
+ int may_queue, priv = 0;
may_queue = elv_may_queue(q, rw_flags);
if (may_queue == ELV_MQUEUE_NO)
@@ -817,9 +739,11 @@ static struct request *get_request(struct request_queue *q, int rw_flags,
rl->count[is_sync]++;
rl->starved[is_sync] = 0;
- priv = !test_bit(QUEUE_FLAG_ELVSWITCH, &q->queue_flags);
- if (priv)
- rl->elvpriv++;
+ if (blk_rq_should_init_elevator(bio)) {
+ priv = !test_bit(QUEUE_FLAG_ELVSWITCH, &q->queue_flags);
+ if (priv)
+ rl->elvpriv++;
+ }
if (blk_queue_io_stat(q))
rw_flags |= REQ_IO_STAT;
@@ -866,8 +790,8 @@ out:
}
/*
- * No available requests for this queue, unplug the device and wait for some
- * requests to become available.
+ * No available requests for this queue, wait for some requests to become
+ * available.
*
* Called with q->queue_lock held, and returns with it unlocked.
*/
@@ -888,7 +812,6 @@ static struct request *get_request_wait(struct request_queue *q, int rw_flags,
trace_block_sleeprq(q, bio, rw_flags & 1);
- __generic_unplug_device(q);
spin_unlock_irq(q->queue_lock);
io_schedule();
@@ -1010,6 +933,13 @@ void blk_requeue_request(struct request_queue *q, struct request *rq)
}
EXPORT_SYMBOL(blk_requeue_request);
+static void add_acct_request(struct request_queue *q, struct request *rq,
+ int where)
+{
+ drive_stat_acct(rq, 1);
+ __elv_add_request(q, rq, where);
+}
+
/**
* blk_insert_request - insert a special request into a request queue
* @q: request queue where request should be inserted
@@ -1052,9 +982,8 @@ void blk_insert_request(struct request_queue *q, struct request *rq,
if (blk_rq_tagged(rq))
blk_queue_end_tag(q, rq);
- drive_stat_acct(rq, 1);
- __elv_add_request(q, rq, where, 0);
- __blk_run_queue(q, false);
+ add_acct_request(q, rq, where);
+ __blk_run_queue(q);
spin_unlock_irqrestore(q->queue_lock, flags);
}
EXPORT_SYMBOL(blk_insert_request);
@@ -1174,6 +1103,113 @@ void blk_add_request_payload(struct request *rq, struct page *page,
}
EXPORT_SYMBOL_GPL(blk_add_request_payload);
+static bool bio_attempt_back_merge(struct request_queue *q, struct request *req,
+ struct bio *bio)
+{
+ const int ff = bio->bi_rw & REQ_FAILFAST_MASK;
+
+ /*
+ * Debug stuff, kill later
+ */
+ if (!rq_mergeable(req)) {
+ blk_dump_rq_flags(req, "back");
+ return false;
+ }
+
+ if (!ll_back_merge_fn(q, req, bio))
+ return false;
+
+ trace_block_bio_backmerge(q, bio);
+
+ if ((req->cmd_flags & REQ_FAILFAST_MASK) != ff)
+ blk_rq_set_mixed_merge(req);
+
+ req->biotail->bi_next = bio;
+ req->biotail = bio;
+ req->__data_len += bio->bi_size;
+ req->ioprio = ioprio_best(req->ioprio, bio_prio(bio));
+
+ drive_stat_acct(req, 0);
+ return true;
+}
+
+static bool bio_attempt_front_merge(struct request_queue *q,
+ struct request *req, struct bio *bio)
+{
+ const int ff = bio->bi_rw & REQ_FAILFAST_MASK;
+ sector_t sector;
+
+ /*
+ * Debug stuff, kill later
+ */
+ if (!rq_mergeable(req)) {
+ blk_dump_rq_flags(req, "front");
+ return false;
+ }
+
+ if (!ll_front_merge_fn(q, req, bio))
+ return false;
+
+ trace_block_bio_frontmerge(q, bio);
+
+ if ((req->cmd_flags & REQ_FAILFAST_MASK) != ff)
+ blk_rq_set_mixed_merge(req);
+
+ sector = bio->bi_sector;
+
+ bio->bi_next = req->bio;
+ req->bio = bio;
+
+ /*
+ * may not be valid. if the low level driver said
+ * it didn't need a bounce buffer then it better
+ * not touch req->buffer either...
+ */
+ req->buffer = bio_data(bio);
+ req->__sector = bio->bi_sector;
+ req->__data_len += bio->bi_size;
+ req->ioprio = ioprio_best(req->ioprio, bio_prio(bio));
+
+ drive_stat_acct(req, 0);
+ return true;
+}
+
+/*
+ * Attempts to merge with the plugged list in the current process. Returns
+ * true if merge was successful, otherwise false.
+ */
+static bool attempt_plug_merge(struct task_struct *tsk, struct request_queue *q,
+ struct bio *bio)
+{
+ struct blk_plug *plug;
+ struct request *rq;
+ bool ret = false;
+
+ plug = tsk->plug;
+ if (!plug)
+ goto out;
+
+ list_for_each_entry_reverse(rq, &plug->list, queuelist) {
+ int el_ret;
+
+ if (rq->q != q)
+ continue;
+
+ el_ret = elv_try_merge(rq, bio);
+ if (el_ret == ELEVATOR_BACK_MERGE) {
+ ret = bio_attempt_back_merge(q, rq, bio);
+ if (ret)
+ break;
+ } else if (el_ret == ELEVATOR_FRONT_MERGE) {
+ ret = bio_attempt_front_merge(q, rq, bio);
+ if (ret)
+ break;
+ }
+ }
+out:
+ return ret;
+}
+
void init_request_from_bio(struct request *req, struct bio *bio)
{
req->cpu = bio->bi_comp_cpu;
@@ -1189,26 +1225,12 @@ void init_request_from_bio(struct request *req, struct bio *bio)
blk_rq_bio_prep(req->q, req, bio);
}
-/*
- * Only disabling plugging for non-rotational devices if it does tagging
- * as well, otherwise we do need the proper merging
- */
-static inline bool queue_should_plug(struct request_queue *q)
-{
- return !(blk_queue_nonrot(q) && blk_queue_tagged(q));
-}
-
static int __make_request(struct request_queue *q, struct bio *bio)
{
- struct request *req;
- int el_ret;
- unsigned int bytes = bio->bi_size;
- const unsigned short prio = bio_prio(bio);
const bool sync = !!(bio->bi_rw & REQ_SYNC);
- const bool unplug = !!(bio->bi_rw & REQ_UNPLUG);
- const unsigned long ff = bio->bi_rw & REQ_FAILFAST_MASK;
- int where = ELEVATOR_INSERT_SORT;
- int rw_flags;
+ struct blk_plug *plug;
+ int el_ret, rw_flags, where = ELEVATOR_INSERT_SORT;
+ struct request *req;
/*
* low level driver can indicate that it wants pages above a
@@ -1217,78 +1239,36 @@ static int __make_request(struct request_queue *q, struct bio *bio)
*/
blk_queue_bounce(q, &bio);
- spin_lock_irq(q->queue_lock);
-
if (bio->bi_rw & (REQ_FLUSH | REQ_FUA)) {
- where = ELEVATOR_INSERT_FRONT;
+ spin_lock_irq(q->queue_lock);
+ where = ELEVATOR_INSERT_FLUSH;
goto get_rq;
}
- if (elv_queue_empty(q))
- goto get_rq;
-
- el_ret = elv_merge(q, &req, bio);
- switch (el_ret) {
- case ELEVATOR_BACK_MERGE:
- BUG_ON(!rq_mergeable(req));
-
- if (!ll_back_merge_fn(q, req, bio))
- break;
-
- trace_block_bio_backmerge(q, bio);
-
- if ((req->cmd_flags & REQ_FAILFAST_MASK) != ff)
- blk_rq_set_mixed_merge(req);
-
- req->biotail->bi_next = bio;
- req->biotail = bio;
- req->__data_len += bytes;
- req->ioprio = ioprio_best(req->ioprio, prio);
- if (!blk_rq_cpu_valid(req))
- req->cpu = bio->bi_comp_cpu;
- drive_stat_acct(req, 0);
- elv_bio_merged(q, req, bio);
- if (!attempt_back_merge(q, req))
- elv_merged_request(q, req, el_ret);
+ /*
+ * Check if we can merge with the plugged list before grabbing
+ * any locks.
+ */
+ if (attempt_plug_merge(current, q, bio))
goto out;
- case ELEVATOR_FRONT_MERGE:
- BUG_ON(!rq_mergeable(req));
-
- if (!ll_front_merge_fn(q, req, bio))
- break;
-
- trace_block_bio_frontmerge(q, bio);
+ spin_lock_irq(q->queue_lock);
- if ((req->cmd_flags & REQ_FAILFAST_MASK) != ff) {
- blk_rq_set_mixed_merge(req);
- req->cmd_flags &= ~REQ_FAILFAST_MASK;
- req->cmd_flags |= ff;
+ el_ret = elv_merge(q, &req, bio);
+ if (el_ret == ELEVATOR_BACK_MERGE) {
+ BUG_ON(req->cmd_flags & REQ_ON_PLUG);
+ if (bio_attempt_back_merge(q, req, bio)) {
+ if (!attempt_back_merge(q, req))
+ elv_merged_request(q, req, el_ret);
+ goto out_unlock;
+ }
+ } else if (el_ret == ELEVATOR_FRONT_MERGE) {
+ BUG_ON(req->cmd_flags & REQ_ON_PLUG);
+ if (bio_attempt_front_merge(q, req, bio)) {
+ if (!attempt_front_merge(q, req))
+ elv_merged_request(q, req, el_ret);
+ goto out_unlock;
}
-
- bio->bi_next = req->bio;
- req->bio = bio;
-
- /*
- * may not be valid. if the low level driver said
- * it didn't need a bounce buffer then it better
- * not touch req->buffer either...
- */
- req->buffer = bio_data(bio);
- req->__sector = bio->bi_sector;
- req->__data_len += bytes;
- req->ioprio = ioprio_best(req->ioprio, prio);
- if (!blk_rq_cpu_valid(req))
- req->cpu = bio->bi_comp_cpu;
- drive_stat_acct(req, 0);
- elv_bio_merged(q, req, bio);
- if (!attempt_front_merge(q, req))
- elv_merged_request(q, req, el_ret);
- goto out;
-
- /* ELV_NO_MERGE: elevator says don't/can't merge. */
- default:
- ;
}
get_rq:
@@ -1315,20 +1295,43 @@ get_rq:
*/
init_request_from_bio(req, bio);
- spin_lock_irq(q->queue_lock);
if (test_bit(QUEUE_FLAG_SAME_COMP, &q->queue_flags) ||
- bio_flagged(bio, BIO_CPU_AFFINE))
- req->cpu = blk_cpu_to_group(smp_processor_id());
- if (queue_should_plug(q) && elv_queue_empty(q))
- blk_plug_device(q);
-
- /* insert the request into the elevator */
- drive_stat_acct(req, 1);
- __elv_add_request(q, req, where, 0);
+ bio_flagged(bio, BIO_CPU_AFFINE)) {
+ req->cpu = blk_cpu_to_group(get_cpu());
+ put_cpu();
+ }
+
+ plug = current->plug;
+ if (plug) {
+ /*
+ * If this is the first request added after a plug, fire
+ * of a plug trace. If others have been added before, check
+ * if we have multiple devices in this plug. If so, make a
+ * note to sort the list before dispatch.
+ */
+ if (list_empty(&plug->list))
+ trace_block_plug(q);
+ else if (!plug->should_sort) {
+ struct request *__rq;
+
+ __rq = list_entry_rq(plug->list.prev);
+ if (__rq->q != q)
+ plug->should_sort = 1;
+ }
+ /*
+ * Debug flag, kill later
+ */
+ req->cmd_flags |= REQ_ON_PLUG;
+ list_add_tail(&req->queuelist, &plug->list);
+ drive_stat_acct(req, 1);
+ } else {
+ spin_lock_irq(q->queue_lock);
+ add_acct_request(q, req, where);
+ __blk_run_queue(q);
+out_unlock:
+ spin_unlock_irq(q->queue_lock);
+ }
out:
- if (unplug || !queue_should_plug(q))
- __generic_unplug_device(q);
- spin_unlock_irq(q->queue_lock);
return 0;
}
@@ -1731,9 +1734,7 @@ int blk_insert_cloned_request(struct request_queue *q, struct request *rq)
*/
BUG_ON(blk_queued_rq(rq));
- drive_stat_acct(rq, 1);
- __elv_add_request(q, rq, ELEVATOR_INSERT_BACK, 0);
-
+ add_acct_request(q, rq, ELEVATOR_INSERT_BACK);
spin_unlock_irqrestore(q->queue_lock, flags);
return 0;
@@ -1805,7 +1806,7 @@ static void blk_account_io_done(struct request *req)
* normal IO on queueing nor completion. Accounting the
* containing request is enough.
*/
- if (blk_do_io_stat(req) && req != &req->q->flush_rq) {
+ if (blk_do_io_stat(req) && !(req->cmd_flags & REQ_FLUSH_SEQ)) {
unsigned long duration = jiffies - req->start_time;
const int rw = rq_data_dir(req);
struct hd_struct *part;
@@ -2162,7 +2163,7 @@ bool blk_update_request(struct request *req, int error, unsigned int nr_bytes)
* size, something has gone terribly wrong.
*/
if (blk_rq_bytes(req) < blk_rq_cur_bytes(req)) {
- printk(KERN_ERR "blk: request botched\n");
+ blk_dump_rq_flags(req, "request botched");
req->__data_len = blk_rq_cur_bytes(req);
}
@@ -2628,6 +2629,166 @@ int kblockd_schedule_work(struct request_queue *q, struct work_struct *work)
}
EXPORT_SYMBOL(kblockd_schedule_work);
+int kblockd_schedule_delayed_work(struct request_queue *q,
+ struct delayed_work *dwork, unsigned long delay)
+{
+ return queue_delayed_work(kblockd_workqueue, dwork, delay);
+}
+EXPORT_SYMBOL(kblockd_schedule_delayed_work);
+
+#define PLUG_MAGIC 0x91827364
+
+void blk_start_plug(struct blk_plug *plug)
+{
+ struct task_struct *tsk = current;
+
+ plug->magic = PLUG_MAGIC;
+ INIT_LIST_HEAD(&plug->list);
+ INIT_LIST_HEAD(&plug->cb_list);
+ plug->should_sort = 0;
+
+ /*
+ * If this is a nested plug, don't actually assign it. It will be
+ * flushed on its own.
+ */
+ if (!tsk->plug) {
+ /*
+ * Store ordering should not be needed here, since a potential
+ * preempt will imply a full memory barrier
+ */
+ tsk->plug = plug;
+ }
+}
+EXPORT_SYMBOL(blk_start_plug);
+
+static int plug_rq_cmp(void *priv, struct list_head *a, struct list_head *b)
+{
+ struct request *rqa = container_of(a, struct request, queuelist);
+ struct request *rqb = container_of(b, struct request, queuelist);
+
+ return !(rqa->q <= rqb->q);
+}
+
+/*
+ * If 'from_schedule' is true, then postpone the dispatch of requests
+ * until a safe kblockd context. We due this to avoid accidental big
+ * additional stack usage in driver dispatch, in places where the originally
+ * plugger did not intend it.
+ */
+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);
+
+ /*
+ * 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)
+{
+ struct request_queue *q;
+ unsigned long flags;
+ struct request *rq;
+ LIST_HEAD(list);
+ unsigned int depth;
+
+ BUG_ON(plug->magic != PLUG_MAGIC);
+
+ flush_plug_callbacks(plug);
+ if (list_empty(&plug->list))
+ return;
+
+ list_splice_init(&plug->list, &list);
+
+ if (plug->should_sort) {
+ list_sort(NULL, &list, plug_rq_cmp);
+ plug->should_sort = 0;
+ }
+
+ q = NULL;
+ depth = 0;
+
+ /*
+ * Save and disable interrupts here, to avoid doing it for every
+ * queue lock we have to take.
+ */
+ local_irq_save(flags);
+ while (!list_empty(&list)) {
+ rq = list_entry_rq(list.next);
+ list_del_init(&rq->queuelist);
+ BUG_ON(!(rq->cmd_flags & REQ_ON_PLUG));
+ BUG_ON(!rq->q);
+ if (rq->q != q) {
+ /*
+ * This drops the queue lock
+ */
+ if (q)
+ queue_unplugged(q, depth, from_schedule);
+ q = rq->q;
+ depth = 0;
+ spin_lock(q->queue_lock);
+ }
+ rq->cmd_flags &= ~REQ_ON_PLUG;
+
+ /*
+ * rq is already accounted, so use raw insert
+ */
+ if (rq->cmd_flags & (REQ_FLUSH | REQ_FUA))
+ __elv_add_request(q, rq, ELEVATOR_INSERT_FLUSH);
+ else
+ __elv_add_request(q, rq, ELEVATOR_INSERT_SORT_MERGE);
+
+ depth++;
+ }
+
+ /*
+ * This drops the queue lock
+ */
+ if (q)
+ queue_unplugged(q, depth, from_schedule);
+
+ local_irq_restore(flags);
+}
+
+void blk_finish_plug(struct blk_plug *plug)
+{
+ blk_flush_plug_list(plug, false);
+
+ if (plug == current->plug)
+ current->plug = NULL;
+}
+EXPORT_SYMBOL(blk_finish_plug);
+
int __init blk_dev_init(void)
{
BUILD_BUG_ON(__REQ_NR_BITS > 8 *