summaryrefslogtreecommitdiff
path: root/block/blk-mq.c
diff options
context:
space:
mode:
Diffstat (limited to 'block/blk-mq.c')
-rw-r--r--block/blk-mq.c502
1 files changed, 317 insertions, 185 deletions
diff --git a/block/blk-mq.c b/block/blk-mq.c
index 9248edd8a7d3..629cf421417f 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -19,7 +19,6 @@
#include <linux/smp.h>
#include <linux/interrupt.h>
#include <linux/llist.h>
-#include <linux/list_sort.h>
#include <linux/cpu.h>
#include <linux/cache.h>
#include <linux/sched/sysctl.h>
@@ -242,27 +241,28 @@ EXPORT_SYMBOL_GPL(blk_mq_unfreeze_queue);
*/
void blk_mq_quiesce_queue_nowait(struct request_queue *q)
{
- blk_queue_flag_set(QUEUE_FLAG_QUIESCED, q);
+ unsigned long flags;
+
+ spin_lock_irqsave(&q->queue_lock, flags);
+ if (!q->quiesce_depth++)
+ blk_queue_flag_set(QUEUE_FLAG_QUIESCED, q);
+ spin_unlock_irqrestore(&q->queue_lock, flags);
}
EXPORT_SYMBOL_GPL(blk_mq_quiesce_queue_nowait);
/**
- * blk_mq_quiesce_queue() - wait until all ongoing dispatches have finished
+ * blk_mq_wait_quiesce_done() - wait until in-progress quiesce is done
* @q: request queue.
*
- * Note: this function does not prevent that the struct request end_io()
- * callback function is invoked. Once this function is returned, we make
- * sure no dispatch can happen until the queue is unquiesced via
- * blk_mq_unquiesce_queue().
+ * Note: it is driver's responsibility for making sure that quiesce has
+ * been started.
*/
-void blk_mq_quiesce_queue(struct request_queue *q)
+void blk_mq_wait_quiesce_done(struct request_queue *q)
{
struct blk_mq_hw_ctx *hctx;
unsigned int i;
bool rcu = false;
- blk_mq_quiesce_queue_nowait(q);
-
queue_for_each_hw_ctx(q, hctx, i) {
if (hctx->flags & BLK_MQ_F_BLOCKING)
synchronize_srcu(hctx->srcu);
@@ -272,6 +272,22 @@ void blk_mq_quiesce_queue(struct request_queue *q)
if (rcu)
synchronize_rcu();
}
+EXPORT_SYMBOL_GPL(blk_mq_wait_quiesce_done);
+
+/**
+ * blk_mq_quiesce_queue() - wait until all ongoing dispatches have finished
+ * @q: request queue.
+ *
+ * Note: this function does not prevent that the struct request end_io()
+ * callback function is invoked. Once this function is returned, we make
+ * sure no dispatch can happen until the queue is unquiesced via
+ * blk_mq_unquiesce_queue().
+ */
+void blk_mq_quiesce_queue(struct request_queue *q)
+{
+ blk_mq_quiesce_queue_nowait(q);
+ blk_mq_wait_quiesce_done(q);
+}
EXPORT_SYMBOL_GPL(blk_mq_quiesce_queue);
/*
@@ -283,10 +299,21 @@ EXPORT_SYMBOL_GPL(blk_mq_quiesce_queue);
*/
void blk_mq_unquiesce_queue(struct request_queue *q)
{
- blk_queue_flag_clear(QUEUE_FLAG_QUIESCED, q);
+ unsigned long flags;
+ bool run_queue = false;
+
+ spin_lock_irqsave(&q->queue_lock, flags);
+ if (WARN_ON_ONCE(q->quiesce_depth <= 0)) {
+ ;
+ } else if (!--q->quiesce_depth) {
+ blk_queue_flag_clear(QUEUE_FLAG_QUIESCED, q);
+ run_queue = true;
+ }
+ spin_unlock_irqrestore(&q->queue_lock, flags);
/* dispatch requests which are inserted during quiescing */
- blk_mq_run_hw_queues(q, true);
+ if (run_queue)
+ blk_mq_run_hw_queues(q, true);
}
EXPORT_SYMBOL_GPL(blk_mq_unquiesce_queue);
@@ -301,40 +328,37 @@ void blk_mq_wake_waiters(struct request_queue *q)
}
static struct request *blk_mq_rq_ctx_init(struct blk_mq_alloc_data *data,
- unsigned int tag, u64 alloc_time_ns)
+ struct blk_mq_tags *tags, unsigned int tag, u64 alloc_time_ns)
{
struct blk_mq_ctx *ctx = data->ctx;
struct blk_mq_hw_ctx *hctx = data->hctx;
struct request_queue *q = data->q;
- struct elevator_queue *e = q->elevator;
- struct blk_mq_tags *tags = blk_mq_tags_from_data(data);
struct request *rq = tags->static_rqs[tag];
- unsigned int rq_flags = 0;
- if (e) {
- rq_flags = RQF_ELV;
- rq->tag = BLK_MQ_NO_TAG;
- rq->internal_tag = tag;
- } else {
- rq->tag = tag;
- rq->internal_tag = BLK_MQ_NO_TAG;
- }
+ rq->q = q;
+ rq->mq_ctx = ctx;
+ rq->mq_hctx = hctx;
+ rq->cmd_flags = data->cmd_flags;
if (data->flags & BLK_MQ_REQ_PM)
- rq_flags |= RQF_PM;
+ data->rq_flags |= RQF_PM;
if (blk_queue_io_stat(q))
- rq_flags |= RQF_IO_STAT;
- rq->rq_flags = rq_flags;
+ data->rq_flags |= RQF_IO_STAT;
+ rq->rq_flags = data->rq_flags;
+
+ if (!(data->rq_flags & RQF_ELV)) {
+ rq->tag = tag;
+ rq->internal_tag = BLK_MQ_NO_TAG;
+ } else {
+ rq->tag = BLK_MQ_NO_TAG;
+ rq->internal_tag = tag;
+ }
+ rq->timeout = 0;
if (blk_mq_need_time_stamp(rq))
rq->start_time_ns = ktime_get_ns();
else
rq->start_time_ns = 0;
- /* csd/requeue_work/fifo_time is initialized before use */
- rq->q = q;
- rq->mq_ctx = ctx;
- rq->mq_hctx = hctx;
- rq->cmd_flags = data->cmd_flags;
rq->rq_disk = NULL;
rq->part = NULL;
#ifdef CONFIG_BLK_RQ_ALLOC_TIME
@@ -346,7 +370,6 @@ static struct request *blk_mq_rq_ctx_init(struct blk_mq_alloc_data *data,
#if defined(CONFIG_BLK_DEV_INTEGRITY)
rq->nr_integrity_segments = 0;
#endif
- rq->timeout = 0;
rq->end_io = NULL;
rq->end_io_data = NULL;
@@ -381,22 +404,28 @@ __blk_mq_alloc_requests_batch(struct blk_mq_alloc_data *data,
u64 alloc_time_ns)
{
unsigned int tag, tag_offset;
+ struct blk_mq_tags *tags;
struct request *rq;
- unsigned long tags;
+ unsigned long tag_mask;
int i, nr = 0;
- tags = blk_mq_get_tags(data, data->nr_tags, &tag_offset);
- if (unlikely(!tags))
+ tag_mask = blk_mq_get_tags(data, data->nr_tags, &tag_offset);
+ if (unlikely(!tag_mask))
return NULL;
- for (i = 0; tags; i++) {
- if (!(tags & (1UL << i)))
+ tags = blk_mq_tags_from_data(data);
+ for (i = 0; tag_mask; i++) {
+ if (!(tag_mask & (1UL << i)))
continue;
tag = tag_offset + i;
- tags &= ~(1UL << i);
- rq = blk_mq_rq_ctx_init(data, tag, alloc_time_ns);
+ prefetch(tags->static_rqs[tag]);
+ tag_mask &= ~(1UL << i);
+ rq = blk_mq_rq_ctx_init(data, tags, tag, alloc_time_ns);
rq_list_add(data->cached_rq, rq);
+ nr++;
}
+ /* caller already holds a reference, add for remainder */
+ percpu_ref_get_many(&data->q->q_usage_counter, nr - 1);
data->nr_tags -= nr;
return rq_list_pop(data->cached_rq);
@@ -405,7 +434,6 @@ __blk_mq_alloc_requests_batch(struct blk_mq_alloc_data *data,
static struct request *__blk_mq_alloc_requests(struct blk_mq_alloc_data *data)
{
struct request_queue *q = data->q;
- struct elevator_queue *e = q->elevator;
u64 alloc_time_ns = 0;
struct request *rq;
unsigned int tag;
@@ -417,7 +445,11 @@ static struct request *__blk_mq_alloc_requests(struct blk_mq_alloc_data *data)
if (data->cmd_flags & REQ_NOWAIT)
data->flags |= BLK_MQ_REQ_NOWAIT;
- if (e) {
+ if (q->elevator) {
+ struct elevator_queue *e = q->elevator;
+
+ data->rq_flags |= RQF_ELV;
+
/*
* Flush/passthrough requests are special and go directly to the
* dispatch list. Don't include reserved tags in the
@@ -433,7 +465,7 @@ static struct request *__blk_mq_alloc_requests(struct blk_mq_alloc_data *data)
retry:
data->ctx = blk_mq_get_ctx(q);
data->hctx = blk_mq_map_queue(q, data->cmd_flags, data->ctx);
- if (!e)
+ if (!(data->rq_flags & RQF_ELV))
blk_mq_tag_busy(data->hctx);
/*
@@ -465,7 +497,8 @@ retry:
goto retry;
}
- return blk_mq_rq_ctx_init(data, tag, alloc_time_ns);
+ return blk_mq_rq_ctx_init(data, blk_mq_tags_from_data(data), tag,
+ alloc_time_ns);
}
struct request *blk_mq_alloc_request(struct request_queue *q, unsigned int op,
@@ -544,12 +577,15 @@ struct request *blk_mq_alloc_request_hctx(struct request_queue *q,
if (!q->elevator)
blk_mq_tag_busy(data.hctx);
+ else
+ data.rq_flags |= RQF_ELV;
ret = -EWOULDBLOCK;
tag = blk_mq_get_tag(&data);
if (tag == BLK_MQ_NO_TAG)
goto out_queue_exit;
- return blk_mq_rq_ctx_init(&data, tag, alloc_time_ns);
+ return blk_mq_rq_ctx_init(&data, blk_mq_tags_from_data(&data), tag,
+ alloc_time_ns);
out_queue_exit:
blk_queue_exit(q);
@@ -580,7 +616,7 @@ void blk_mq_free_request(struct request *rq)
struct request_queue *q = rq->q;
struct blk_mq_hw_ctx *hctx = rq->mq_hctx;
- if (rq->rq_flags & (RQF_ELVPRIV | RQF_ELV)) {
+ if (rq->rq_flags & RQF_ELVPRIV) {
struct elevator_queue *e = q->elevator;
if (e->type->ops.finish_request)
@@ -609,34 +645,30 @@ void blk_mq_free_plug_rqs(struct blk_plug *plug)
{
struct request *rq;
- while ((rq = rq_list_pop(&plug->cached_rq)) != NULL) {
- percpu_ref_get(&rq->q->q_usage_counter);
+ while ((rq = rq_list_pop(&plug->cached_rq)) != NULL)
blk_mq_free_request(rq);
- }
}
static void req_bio_endio(struct request *rq, struct bio *bio,
unsigned int nbytes, blk_status_t error)
{
- if (error)
+ if (unlikely(error)) {
bio->bi_status = error;
-
- if (unlikely(rq->rq_flags & RQF_QUIET))
- bio_set_flag(bio, BIO_QUIET);
-
- bio_advance(bio, nbytes);
-
- if (req_op(rq) == REQ_OP_ZONE_APPEND && error == BLK_STS_OK) {
+ } else if (req_op(rq) == REQ_OP_ZONE_APPEND) {
/*
* Partial zone append completions cannot be supported as the
* BIO fragments may end up not being written sequentially.
*/
- if (bio->bi_iter.bi_size)
+ if (bio->bi_iter.bi_size != nbytes)
bio->bi_status = BLK_STS_IOERR;
else
bio->bi_iter.bi_sector = rq->__sector;
}
+ bio_advance(bio, nbytes);
+
+ if (unlikely(rq->rq_flags & RQF_QUIET))
+ bio_set_flag(bio, BIO_QUIET);
/* don't actually finish bio if it's part of flush sequence */
if (bio->bi_iter.bi_size == 0 && !(rq->rq_flags & RQF_FLUSH_SEQ))
bio_endio(bio);
@@ -680,7 +712,7 @@ bool blk_update_request(struct request *req, blk_status_t error,
{
int total_bytes;
- trace_block_rq_complete(req, blk_status_to_errno(error), nr_bytes);
+ trace_block_rq_complete(req, error, nr_bytes);
if (!req->bio)
return false;
@@ -799,6 +831,13 @@ static inline void blk_mq_flush_tag_batch(struct blk_mq_hw_ctx *hctx,
{
struct request_queue *q = hctx->queue;
+ /*
+ * All requests should have been marked as RQF_MQ_INFLIGHT, so
+ * update hctx->nr_active in batch
+ */
+ if (hctx->flags & BLK_MQ_F_TAG_QUEUE_SHARED)
+ __blk_mq_sub_active_requests(hctx, nr_tags);
+
blk_mq_put_tags(hctx->tags, tag_array, nr_tags);
percpu_ref_put_many(&q->q_usage_counter, nr_tags);
}
@@ -806,7 +845,7 @@ static inline void blk_mq_flush_tag_batch(struct blk_mq_hw_ctx *hctx,
void blk_mq_end_request_batch(struct io_comp_batch *iob)
{
int tags[TAG_COMP_BATCH], nr_tags = 0;
- struct blk_mq_hw_ctx *last_hctx = NULL;
+ struct blk_mq_hw_ctx *cur_hctx = NULL;
struct request *rq;
u64 now = 0;
@@ -829,17 +868,17 @@ void blk_mq_end_request_batch(struct io_comp_batch *iob)
blk_pm_mark_last_busy(rq);
rq_qos_done(rq->q, rq);
- if (nr_tags == TAG_COMP_BATCH ||
- (last_hctx && last_hctx != rq->mq_hctx)) {
- blk_mq_flush_tag_batch(last_hctx, tags, nr_tags);
+ if (nr_tags == TAG_COMP_BATCH || cur_hctx != rq->mq_hctx) {
+ if (cur_hctx)
+ blk_mq_flush_tag_batch(cur_hctx, tags, nr_tags);
nr_tags = 0;
+ cur_hctx = rq->mq_hctx;
}
tags[nr_tags++] = rq->tag;
- last_hctx = rq->mq_hctx;
}
if (nr_tags)
- blk_mq_flush_tag_batch(last_hctx, tags, nr_tags);
+ blk_mq_flush_tag_batch(cur_hctx, tags, nr_tags);
}
EXPORT_SYMBOL_GPL(blk_mq_end_request_batch);
@@ -1040,7 +1079,6 @@ void blk_mq_requeue_request(struct request *rq, bool kick_requeue_list)
/* this request will be re-inserted to io scheduler queue */
blk_mq_sched_requeue_request(rq);
- BUG_ON(!list_empty(&rq->queuelist));
blk_mq_add_to_requeue_list(rq, true, kick_requeue_list);
}
EXPORT_SYMBOL(blk_mq_requeue_request);
@@ -1121,17 +1159,6 @@ void blk_mq_delay_kick_requeue_list(struct request_queue *q,
}
EXPORT_SYMBOL(blk_mq_delay_kick_requeue_list);
-struct request *blk_mq_tag_to_rq(struct blk_mq_tags *tags, unsigned int tag)
-{
- if (tag < tags->nr_tags) {
- prefetch(tags->rqs[tag]);
- return tags->rqs[tag];
- }
-
- return NULL;
-}
-EXPORT_SYMBOL(blk_mq_tag_to_rq);
-
static bool blk_mq_rq_inflight(struct blk_mq_hw_ctx *hctx, struct request *rq,
void *priv, bool reserved)
{
@@ -1336,7 +1363,7 @@ struct request *blk_mq_dequeue_from_ctx(struct blk_mq_hw_ctx *hctx,
return data.rq;
}
-static bool __blk_mq_get_driver_tag(struct request *rq)
+static bool __blk_mq_alloc_driver_tag(struct request *rq)
{
struct sbitmap_queue *bt = &rq->mq_hctx->tags->bitmap_tags;
unsigned int tag_offset = rq->mq_hctx->tags->nr_reserved_tags;
@@ -1360,11 +1387,9 @@ static bool __blk_mq_get_driver_tag(struct request *rq)
return true;
}
-bool blk_mq_get_driver_tag(struct request *rq)
+bool __blk_mq_get_driver_tag(struct blk_mq_hw_ctx *hctx, struct request *rq)
{
- struct blk_mq_hw_ctx *hctx = rq->mq_hctx;
-
- if (rq->tag == BLK_MQ_NO_TAG && !__blk_mq_get_driver_tag(rq))
+ if (rq->tag == BLK_MQ_NO_TAG && !__blk_mq_alloc_driver_tag(rq))
return false;
if ((hctx->flags & BLK_MQ_F_TAG_QUEUE_SHARED) &&
@@ -1594,6 +1619,7 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list,
int errors, queued;
blk_status_t ret = BLK_STS_OK;
LIST_HEAD(zone_list);
+ bool needs_resource = false;
if (list_empty(list))
return false;
@@ -1639,6 +1665,8 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list,
queued++;
break;
case BLK_STS_RESOURCE:
+ needs_resource = true;
+ fallthrough;
case BLK_STS_DEV_RESOURCE:
blk_mq_handle_dev_resource(rq, list);
goto out;
@@ -1649,6 +1677,7 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list,
* accept.
*/
blk_mq_handle_zone_resource(rq, &zone_list);
+ needs_resource = true;
break;
default:
errors++;
@@ -1673,7 +1702,6 @@ out:
/* For non-shared tags, the RESTART check will suffice */
bool no_tag = prep == PREP_DISPATCH_NO_TAG &&
(hctx->flags & BLK_MQ_F_TAG_QUEUE_SHARED);
- bool no_budget_avail = prep == PREP_DISPATCH_NO_BUDGET;
if (nr_budgets)
blk_mq_release_budgets(q, list);
@@ -1714,14 +1742,16 @@ out:
* If driver returns BLK_STS_RESOURCE and SCHED_RESTART
* bit is set, run queue after a delay to avoid IO stalls
* that could otherwise occur if the queue is idle. We'll do
- * similar if we couldn't get budget and SCHED_RESTART is set.
+ * similar if we couldn't get budget or couldn't lock a zone
+ * and SCHED_RESTART is set.
*/
needs_restart = blk_mq_sched_needs_restart(hctx);
+ if (prep == PREP_DISPATCH_NO_BUDGET)
+ needs_resource = true;
if (!needs_restart ||
(no_tag && list_empty_careful(&hctx->dispatch_wait.entry)))
blk_mq_run_hw_queue(hctx, true);
- else if (needs_restart && (ret == BLK_STS_RESOURCE ||
- no_budget_avail))
+ else if (needs_restart && needs_resource)
blk_mq_delay_run_hw_queue(hctx, BLK_MQ_RESOURCE_DELAY);
blk_mq_update_dispatch_busy(hctx, true);
@@ -2161,54 +2191,106 @@ void blk_mq_insert_requests(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
spin_unlock(&ctx->lock);
}
-static int plug_rq_cmp(void *priv, const struct list_head *a,
- const struct list_head *b)
+static void blk_mq_commit_rqs(struct blk_mq_hw_ctx *hctx, int *queued,
+ bool from_schedule)
+{
+ if (hctx->queue->mq_ops->commit_rqs) {
+ trace_block_unplug(hctx->queue, *queued, !from_schedule);
+ hctx->queue->mq_ops->commit_rqs(hctx);
+ }
+ *queued = 0;
+}
+
+static void blk_mq_plug_issue_direct(struct blk_plug *plug, bool from_schedule)
{
- struct request *rqa = container_of(a, struct request, queuelist);
- struct request *rqb = container_of(b, struct request, queuelist);
+ struct blk_mq_hw_ctx *hctx = NULL;
+ struct request *rq;
+ int queued = 0;
+ int errors = 0;
- if (rqa->mq_ctx != rqb->mq_ctx)
- return rqa->mq_ctx > rqb->mq_ctx;
- if (rqa->mq_hctx != rqb->mq_hctx)
- return rqa->mq_hctx > rqb->mq_hctx;
+ while ((rq = rq_list_pop(&plug->mq_list))) {
+ bool last = rq_list_empty(plug->mq_list);
+ blk_status_t ret;
+
+ if (hctx != rq->mq_hctx) {
+ if (hctx)
+ blk_mq_commit_rqs(hctx, &queued, from_schedule);
+ hctx = rq->mq_hctx;
+ }
+
+ ret = blk_mq_request_issue_directly(rq, last);
+ switch (ret) {
+ case BLK_STS_OK:
+ queued++;
+ break;
+ case BLK_STS_RESOURCE:
+ case BLK_STS_DEV_RESOURCE:
+ blk_mq_request_bypass_insert(rq, false, last);
+ blk_mq_commit_rqs(hctx, &queued, from_schedule);
+ return;
+ default:
+ blk_mq_end_request(rq, ret);
+ errors++;
+ break;
+ }
+ }
- return blk_rq_pos(rqa) > blk_rq_pos(rqb);
+ /*
+ * If we didn't flush the entire list, we could have told the driver
+ * there was more coming, but that turned out to be a lie.
+ */
+ if (errors)
+ blk_mq_commit_rqs(hctx, &queued, from_schedule);
}
void blk_mq_flush_plug_list(struct blk_plug *plug, bool from_schedule)
{
+ struct blk_mq_hw_ctx *this_hctx;
+ struct blk_mq_ctx *this_ctx;
+ unsigned int depth;
LIST_HEAD(list);
- if (list_empty(&plug->mq_list))
+ if (rq_list_empty(plug->mq_list))
return;
- list_splice_init(&plug->mq_list, &list);
-
- if (plug->rq_count > 2 && plug->multiple_queues)
- list_sort(NULL, &list, plug_rq_cmp);
-
plug->rq_count = 0;
+ if (!plug->multiple_queues && !plug->has_elevator && !from_schedule) {
+ blk_mq_plug_issue_direct(plug, false);
+ if (rq_list_empty(plug->mq_list))
+ return;
+ }
+
+ this_hctx = NULL;
+ this_ctx = NULL;
+ depth = 0;
do {
- struct list_head rq_list;
- struct request *rq, *head_rq = list_entry_rq(list.next);
- struct list_head *pos = &head_rq->queuelist; /* skip first */
- struct blk_mq_hw_ctx *this_hctx = head_rq->mq_hctx;
- struct blk_mq_ctx *this_ctx = head_rq->mq_ctx;
- unsigned int depth = 1;
-
- list_for_each_continue(pos, &list) {
- rq = list_entry_rq(pos);
- BUG_ON(!rq->q);
- if (rq->mq_hctx != this_hctx || rq->mq_ctx != this_ctx)
- break;
- depth++;
+ struct request *rq;
+
+ rq = rq_list_pop(&plug->mq_list);
+
+ if (!this_hctx) {
+ this_hctx = rq->mq_hctx;
+ this_ctx = rq->mq_ctx;
+ } else if (this_hctx != rq->mq_hctx || this_ctx != rq->mq_ctx) {
+ trace_block_unplug(this_hctx->queue, depth,
+ !from_schedule);
+ blk_mq_sched_insert_requests(this_hctx, this_ctx,
+ &list, from_schedule);
+ depth = 0;
+ this_hctx = rq->mq_hctx;
+ this_ctx = rq->mq_ctx;
+
}
- list_cut_before(&rq_list, &list, pos);
- trace_block_unplug(head_rq->q, depth, !from_schedule);
- blk_mq_sched_insert_requests(this_hctx, this_ctx, &rq_list,
+ list_add(&rq->queuelist, &list);
+ depth++;
+ } while (!rq_list_empty(plug->mq_list));
+
+ if (!list_empty(&list)) {
+ trace_block_unplug(this_hctx->queue, depth, !from_schedule);
+ blk_mq_sched_insert_requests(this_hctx, this_ctx, &list,
from_schedule);
- } while(!list_empty(&list));
+ }
}
static void blk_mq_bio_to_request(struct request *rq, struct bio *bio,
@@ -2388,16 +2470,17 @@ void blk_mq_try_issue_list_directly(struct blk_mq_hw_ctx *hctx,
static void blk_add_rq_to_plug(struct blk_plug *plug, struct request *rq)
{
- list_add_tail(&rq->queuelist, &plug->mq_list);
- plug->rq_count++;
- if (!plug->multiple_queues && !list_is_singular(&plug->mq_list)) {
- struct request *tmp;
+ if (!plug->multiple_queues) {
+ struct request *nxt = rq_list_peek(&plug->mq_list);
- tmp = list_first_entry(&plug->mq_list, struct request,
- queuelist);
- if (tmp->q != rq->q)
+ if (nxt && nxt->q != rq->q)
plug->multiple_queues = true;
}
+ if (!plug->has_elevator && (rq->rq_flags & RQF_ELV))
+ plug->has_elevator = true;
+ rq->rq_next = NULL;
+ rq_list_add(&plug->mq_list, rq);
+ plug->rq_count++;
}
/*
@@ -2412,6 +2495,83 @@ static inline unsigned short blk_plug_max_rq_count(struct blk_plug *plug)
return BLK_MAX_REQUEST_COUNT;
}
+static bool blk_attempt_bio_merge(struct request_queue *q, struct bio *bio,
+ unsigned int nr_segs, bool *same_queue_rq)
+{
+ if (!blk_queue_nomerges(q) && bio_mergeable(bio)) {
+ if (blk_attempt_plug_merge(q, bio, nr_segs, same_queue_rq))
+ return true;
+ if (blk_mq_sched_bio_merge(q, bio, nr_segs))
+ return true;
+ }
+ return false;
+}
+
+static struct request *blk_mq_get_new_requests(struct request_queue *q,
+ struct blk_plug *plug,
+ struct bio *bio,
+ unsigned int nsegs,
+ bool *same_queue_rq)
+{
+ struct blk_mq_alloc_data data = {
+ .q = q,
+ .nr_tags = 1,
+ .cmd_flags = bio->bi_opf,
+ };
+ struct request *rq;
+
+ if (unlikely(bio_queue_enter(bio)))
+ return NULL;
+ if (unlikely(!submit_bio_checks(bio)))
+ goto put_exit;
+ if (blk_attempt_bio_merge(q, bio, nsegs, same_queue_rq))
+ goto put_exit;
+
+ rq_qos_throttle(q, bio);
+
+ if (plug) {
+ data.nr_tags = plug->nr_ios;
+ plug->nr_ios = 1;
+ data.cached_rq = &plug->cached_rq;
+ }
+
+ rq = __blk_mq_alloc_requests(&data);
+ if (rq)
+ return rq;
+
+ rq_qos_cleanup(q, bio);
+ if (bio->bi_opf & REQ_NOWAIT)
+ bio_wouldblock_error(bio);
+put_exit:
+ blk_queue_exit(q);
+ return NULL;
+}
+
+static inline struct request *blk_mq_get_request(struct request_queue *q,
+ struct blk_plug *plug,
+ struct bio *bio,
+ unsigned int nsegs,
+ bool *same_queue_rq)
+{
+ if (plug) {
+ struct request *rq;
+
+ rq = rq_list_peek(&plug->cached_rq);
+ if (rq && rq->q == q) {
+ if (unlikely(!submit_bio_checks(bio)))
+ return NULL;
+ if (blk_attempt_bio_merge(q, bio, nsegs, same_queue_rq))
+ return NULL;
+ plug->cached_rq = rq_list_next(rq);
+ INIT_LIST_HEAD(&rq->queuelist);
+ rq_qos_throttle(q, bio);
+ return rq;
+ }
+ }
+
+ return blk_mq_get_new_requests(q, plug, bio, nsegs, same_queue_rq);
+}
+
/**
* blk_mq_submit_bio - Create and send a request to block device.
* @bio: Bio pointer.
@@ -2429,53 +2589,26 @@ void blk_mq_submit_bio(struct bio *bio)
{
struct request_queue *q = bdev_get_queue(bio->bi_bdev);
const int is_sync = op_is_sync(bio->bi_opf);
- const int is_flush_fua = op_is_flush(bio->bi_opf);
struct request *rq;
struct blk_plug *plug;
- struct request *same_queue_rq = NULL;
+ bool same_queue_rq = false;
unsigned int nr_segs = 1;
blk_status_t ret;
+ if (unlikely(!blk_crypto_bio_prep(&bio)))
+ return;
+
blk_queue_bounce(q, &bio);
if (blk_may_split(q, bio))
__blk_queue_split(q, &bio, &nr_segs);
if (!bio_integrity_prep(bio))
- goto queue_exit;
-
- if (!is_flush_fua && !blk_queue_nomerges(q) &&
- blk_attempt_plug_merge(q, bio, nr_segs, &same_queue_rq))
- goto queue_exit;
-
- if (blk_mq_sched_bio_merge(q, bio, nr_segs))
- goto queue_exit;
-
- rq_qos_throttle(q, bio);
+ return;
plug = blk_mq_plug(q, bio);
- if (plug && plug->cached_rq) {
- rq = rq_list_pop(&plug->cached_rq);
- INIT_LIST_HEAD(&rq->queuelist);
- } else {
- struct blk_mq_alloc_data data = {
- .q = q,
- .nr_tags = 1,
- .cmd_flags = bio->bi_opf,
- };
-
- if (plug) {
- data.nr_tags = plug->nr_ios;
- plug->nr_ios = 1;
- data.cached_rq = &plug->cached_rq;
- }
- rq = __blk_mq_alloc_requests(&data);
- if (unlikely(!rq)) {
- rq_qos_cleanup(q, bio);
- if (bio->bi_opf & REQ_NOWAIT)
- bio_wouldblock_error(bio);
- goto queue_exit;
- }
- }
+ rq = blk_mq_get_request(q, plug, bio, nr_segs, &same_queue_rq);
+ if (unlikely(!rq))
+ return;
trace_block_getrq(bio);
@@ -2491,14 +2624,12 @@ void blk_mq_submit_bio(struct bio *bio)
return;
}
- if (unlikely(is_flush_fua)) {
- struct blk_mq_hw_ctx *hctx = rq->mq_hctx;
- /* Bypass scheduler for flush requests */
- blk_insert_flush(rq);
- blk_mq_run_hw_queue(hctx, true);
- } else if (plug && (q->nr_hw_queues == 1 ||
- blk_mq_is_shared_tags(rq->mq_hctx->flags) ||
- q->mq_ops->commit_rqs || !blk_queue_nonrot(q))) {
+ if (op_is_flush(bio->bi_opf) && blk_insert_flush(rq))
+ return;
+
+ if (plug && (q->nr_hw_queues == 1 ||
+ blk_mq_is_shared_tags(rq->mq_hctx->flags) ||
+ q->mq_ops->commit_rqs || !blk_queue_nonrot(q))) {
/*
* Use plugging if we have a ->commit_rqs() hook as well, as
* we know the driver uses bd->last in a smart fashion.
@@ -2509,14 +2640,16 @@ void blk_mq_submit_bio(struct bio *bio)
unsigned int request_count = plug->rq_count;
struct request *last = NULL;
- if (!request_count)
+ if (!request_count) {
trace_block_plug(q);
- else
- last = list_entry_rq(plug->mq_list.prev);
+ } else if (!blk_queue_nomerges(q)) {
+ last = rq_list_peek(&plug->mq_list);
+ if (blk_rq_bytes(last) < BLK_PLUG_FLUSH_SIZE)
+ last = NULL;
+ }
- if (request_count >= blk_plug_max_rq_count(plug) || (last &&
- blk_rq_bytes(last) >= BLK_PLUG_FLUSH_SIZE)) {
- blk_flush_plug_list(plug, false);
+ if (request_count >= blk_plug_max_rq_count(plug) || last) {
+ blk_mq_flush_plug_list(plug, false);
trace_block_plug(q);
}
@@ -2525,6 +2658,8 @@ void blk_mq_submit_bio(struct bio *bio)
/* Insert the request at the IO scheduler queue */
blk_mq_sched_insert_request(rq, false, true, true);
} else if (plug && !blk_queue_nomerges(q)) {
+ struct request *next_rq = NULL;
+
/*
* We do limited plugging. If the bio can be merged, do that.
* Otherwise the existing request in the plug list will be
@@ -2532,19 +2667,16 @@ void blk_mq_submit_bio(struct bio *bio)
* The plug list might get flushed before this. If that happens,
* the plug list is empty, and same_queue_rq is invalid.
*/
- if (list_empty(&plug->mq_list))
- same_queue_rq = NULL;
if (same_queue_rq) {
- list_del_init(&same_queue_rq->queuelist);
+ next_rq = rq_list_pop(&plug->mq_list);
plug->rq_count--;
}
blk_add_rq_to_plug(plug, rq);
trace_block_plug(q);
- if (same_queue_rq) {
+ if (next_rq) {
trace_block_unplug(q, 1, true);
- blk_mq_try_issue_directly(same_queue_rq->mq_hctx,
- same_queue_rq);
+ blk_mq_try_issue_directly(next_rq->mq_hctx, next_rq);
}
} else if ((q->nr_hw_queues > 1 && is_sync) ||
!rq->mq_hctx->dispatch_busy) {
@@ -2557,10 +2689,6 @@ void blk_mq_submit_bio(struct bio *bio)
/* Default case. */
blk_mq_sched_insert_request(rq, false, true, true);
}
-
- return;
-queue_exit:
- blk_queue_exit(q);
}
static size_t order_to_size(unsigned int order)
@@ -3546,7 +3674,6 @@ static void blk_mq_realloc_hw_ctxs(struct blk_mq_tag_set *set,
struct blk_mq_hw_ctx *hctx = hctxs[j];
if (hctx) {
- __blk_mq_free_map_and_rqs(set, j);
blk_mq_exit_hctx(q, set, hctx, j);
hctxs[j] = NULL;
}
@@ -4054,8 +4181,13 @@ fallback:
list_for_each_entry(q, &set->tag_list, tag_set_list) {
blk_mq_realloc_hw_ctxs(set, q);
if (q->nr_hw_queues != set->nr_hw_queues) {
+ int i = prev_nr_hw_queues;
+
pr_warn("Increasing nr_hw_queues to %d fails, fallback to %d\n",
nr_hw_queues, prev_nr_hw_queues);
+ for (; i < set->nr_hw_queues; i++)
+ __blk_mq_free_map_and_rqs(set, i);
+
set->nr_hw_queues = prev_nr_hw_queues;
blk_mq_map_queues(&set->map[HCTX_TYPE_DEFAULT]);
goto fallback;