|
@@ -32,6 +32,7 @@
|
|
|
#include "blk-mq-tag.h"
|
|
|
#include "blk-stat.h"
|
|
|
#include "blk-wbt.h"
|
|
|
+#include "blk-mq-sched.h"
|
|
|
|
|
|
static DEFINE_MUTEX(all_q_mutex);
|
|
|
static LIST_HEAD(all_q_list);
|
|
@@ -39,9 +40,11 @@ static LIST_HEAD(all_q_list);
|
|
|
/*
|
|
|
* Check if any of the ctx's have pending work in this hardware queue
|
|
|
*/
|
|
|
-static bool blk_mq_hctx_has_pending(struct blk_mq_hw_ctx *hctx)
|
|
|
+bool blk_mq_hctx_has_pending(struct blk_mq_hw_ctx *hctx)
|
|
|
{
|
|
|
- return sbitmap_any_bit_set(&hctx->ctx_map);
|
|
|
+ return sbitmap_any_bit_set(&hctx->ctx_map) ||
|
|
|
+ !list_empty_careful(&hctx->dispatch) ||
|
|
|
+ blk_mq_sched_has_work(hctx);
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -167,8 +170,8 @@ bool blk_mq_can_queue(struct blk_mq_hw_ctx *hctx)
|
|
|
}
|
|
|
EXPORT_SYMBOL(blk_mq_can_queue);
|
|
|
|
|
|
-static void blk_mq_rq_ctx_init(struct request_queue *q, struct blk_mq_ctx *ctx,
|
|
|
- struct request *rq, unsigned int op)
|
|
|
+void blk_mq_rq_ctx_init(struct request_queue *q, struct blk_mq_ctx *ctx,
|
|
|
+ struct request *rq, unsigned int op)
|
|
|
{
|
|
|
INIT_LIST_HEAD(&rq->queuelist);
|
|
|
/* csd/requeue_work/fifo_time is initialized before use */
|
|
@@ -213,53 +216,58 @@ static void blk_mq_rq_ctx_init(struct request_queue *q, struct blk_mq_ctx *ctx,
|
|
|
|
|
|
ctx->rq_dispatched[op_is_sync(op)]++;
|
|
|
}
|
|
|
+EXPORT_SYMBOL_GPL(blk_mq_rq_ctx_init);
|
|
|
|
|
|
-static struct request *
|
|
|
-__blk_mq_alloc_request(struct blk_mq_alloc_data *data, unsigned int op)
|
|
|
+struct request *__blk_mq_alloc_request(struct blk_mq_alloc_data *data,
|
|
|
+ unsigned int op)
|
|
|
{
|
|
|
struct request *rq;
|
|
|
unsigned int tag;
|
|
|
|
|
|
tag = blk_mq_get_tag(data);
|
|
|
if (tag != BLK_MQ_TAG_FAIL) {
|
|
|
- rq = data->hctx->tags->rqs[tag];
|
|
|
+ struct blk_mq_tags *tags = blk_mq_tags_from_data(data);
|
|
|
|
|
|
- if (blk_mq_tag_busy(data->hctx)) {
|
|
|
- rq->rq_flags = RQF_MQ_INFLIGHT;
|
|
|
- atomic_inc(&data->hctx->nr_active);
|
|
|
+ rq = tags->static_rqs[tag];
|
|
|
+
|
|
|
+ if (data->flags & BLK_MQ_REQ_INTERNAL) {
|
|
|
+ rq->tag = -1;
|
|
|
+ rq->internal_tag = tag;
|
|
|
+ } else {
|
|
|
+ if (blk_mq_tag_busy(data->hctx)) {
|
|
|
+ rq->rq_flags = RQF_MQ_INFLIGHT;
|
|
|
+ atomic_inc(&data->hctx->nr_active);
|
|
|
+ }
|
|
|
+ rq->tag = tag;
|
|
|
+ rq->internal_tag = -1;
|
|
|
}
|
|
|
|
|
|
- rq->tag = tag;
|
|
|
blk_mq_rq_ctx_init(data->q, data->ctx, rq, op);
|
|
|
return rq;
|
|
|
}
|
|
|
|
|
|
return NULL;
|
|
|
}
|
|
|
+EXPORT_SYMBOL_GPL(__blk_mq_alloc_request);
|
|
|
|
|
|
struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
|
|
|
unsigned int flags)
|
|
|
{
|
|
|
- struct blk_mq_ctx *ctx;
|
|
|
- struct blk_mq_hw_ctx *hctx;
|
|
|
+ struct blk_mq_alloc_data alloc_data = { .flags = flags };
|
|
|
struct request *rq;
|
|
|
- struct blk_mq_alloc_data alloc_data;
|
|
|
int ret;
|
|
|
|
|
|
ret = blk_queue_enter(q, flags & BLK_MQ_REQ_NOWAIT);
|
|
|
if (ret)
|
|
|
return ERR_PTR(ret);
|
|
|
|
|
|
- ctx = blk_mq_get_ctx(q);
|
|
|
- hctx = blk_mq_map_queue(q, ctx->cpu);
|
|
|
- blk_mq_set_alloc_data(&alloc_data, q, flags, ctx, hctx);
|
|
|
- rq = __blk_mq_alloc_request(&alloc_data, rw);
|
|
|
- blk_mq_put_ctx(ctx);
|
|
|
+ rq = blk_mq_sched_get_request(q, NULL, rw, &alloc_data);
|
|
|
|
|
|
- if (!rq) {
|
|
|
- blk_queue_exit(q);
|
|
|
+ blk_mq_put_ctx(alloc_data.ctx);
|
|
|
+ blk_queue_exit(q);
|
|
|
+
|
|
|
+ if (!rq)
|
|
|
return ERR_PTR(-EWOULDBLOCK);
|
|
|
- }
|
|
|
|
|
|
rq->__data_len = 0;
|
|
|
rq->__sector = (sector_t) -1;
|
|
@@ -319,10 +327,10 @@ out_queue_exit:
|
|
|
}
|
|
|
EXPORT_SYMBOL_GPL(blk_mq_alloc_request_hctx);
|
|
|
|
|
|
-static void __blk_mq_free_request(struct blk_mq_hw_ctx *hctx,
|
|
|
- struct blk_mq_ctx *ctx, struct request *rq)
|
|
|
+void __blk_mq_finish_request(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
|
|
|
+ struct request *rq)
|
|
|
{
|
|
|
- const int tag = rq->tag;
|
|
|
+ const int sched_tag = rq->internal_tag;
|
|
|
struct request_queue *q = rq->q;
|
|
|
|
|
|
if (rq->rq_flags & RQF_MQ_INFLIGHT)
|
|
@@ -333,23 +341,31 @@ static void __blk_mq_free_request(struct blk_mq_hw_ctx *hctx,
|
|
|
|
|
|
clear_bit(REQ_ATOM_STARTED, &rq->atomic_flags);
|
|
|
clear_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags);
|
|
|
- blk_mq_put_tag(hctx, ctx, tag);
|
|
|
+ if (rq->tag != -1)
|
|
|
+ blk_mq_put_tag(hctx, hctx->tags, ctx, rq->tag);
|
|
|
+ if (sched_tag != -1)
|
|
|
+ blk_mq_sched_completed_request(hctx, rq);
|
|
|
+ blk_mq_sched_restart_queues(hctx);
|
|
|
blk_queue_exit(q);
|
|
|
}
|
|
|
|
|
|
-void blk_mq_free_hctx_request(struct blk_mq_hw_ctx *hctx, struct request *rq)
|
|
|
+static void blk_mq_finish_hctx_request(struct blk_mq_hw_ctx *hctx,
|
|
|
+ struct request *rq)
|
|
|
{
|
|
|
struct blk_mq_ctx *ctx = rq->mq_ctx;
|
|
|
|
|
|
ctx->rq_completed[rq_is_sync(rq)]++;
|
|
|
- __blk_mq_free_request(hctx, ctx, rq);
|
|
|
+ __blk_mq_finish_request(hctx, ctx, rq);
|
|
|
+}
|
|
|
|
|
|
+void blk_mq_finish_request(struct request *rq)
|
|
|
+{
|
|
|
+ blk_mq_finish_hctx_request(blk_mq_map_queue(rq->q, rq->mq_ctx->cpu), rq);
|
|
|
}
|
|
|
-EXPORT_SYMBOL_GPL(blk_mq_free_hctx_request);
|
|
|
|
|
|
void blk_mq_free_request(struct request *rq)
|
|
|
{
|
|
|
- blk_mq_free_hctx_request(blk_mq_map_queue(rq->q, rq->mq_ctx->cpu), rq);
|
|
|
+ blk_mq_sched_put_request(rq);
|
|
|
}
|
|
|
EXPORT_SYMBOL_GPL(blk_mq_free_request);
|
|
|
|
|
@@ -467,6 +483,8 @@ void blk_mq_start_request(struct request *rq)
|
|
|
{
|
|
|
struct request_queue *q = rq->q;
|
|
|
|
|
|
+ blk_mq_sched_started_request(rq);
|
|
|
+
|
|
|
trace_block_rq_issue(q, rq);
|
|
|
|
|
|
rq->resid_len = blk_rq_bytes(rq);
|
|
@@ -515,6 +533,7 @@ static void __blk_mq_requeue_request(struct request *rq)
|
|
|
|
|
|
trace_block_rq_requeue(q, rq);
|
|
|
wbt_requeue(q->rq_wb, &rq->issue_stat);
|
|
|
+ blk_mq_sched_requeue_request(rq);
|
|
|
|
|
|
if (test_and_clear_bit(REQ_ATOM_STARTED, &rq->atomic_flags)) {
|
|
|
if (q->dma_drain_size && blk_rq_bytes(rq))
|
|
@@ -549,13 +568,13 @@ static void blk_mq_requeue_work(struct work_struct *work)
|
|
|
|
|
|
rq->rq_flags &= ~RQF_SOFTBARRIER;
|
|
|
list_del_init(&rq->queuelist);
|
|
|
- blk_mq_insert_request(rq, true, false, false);
|
|
|
+ blk_mq_sched_insert_request(rq, true, false, false, true);
|
|
|
}
|
|
|
|
|
|
while (!list_empty(&rq_list)) {
|
|
|
rq = list_entry(rq_list.next, struct request, queuelist);
|
|
|
list_del_init(&rq->queuelist);
|
|
|
- blk_mq_insert_request(rq, false, false, false);
|
|
|
+ blk_mq_sched_insert_request(rq, false, false, false, true);
|
|
|
}
|
|
|
|
|
|
blk_mq_run_hw_queues(q, false);
|
|
@@ -639,7 +658,7 @@ struct blk_mq_timeout_data {
|
|
|
|
|
|
void blk_mq_rq_timed_out(struct request *req, bool reserved)
|
|
|
{
|
|
|
- struct blk_mq_ops *ops = req->q->mq_ops;
|
|
|
+ const struct blk_mq_ops *ops = req->q->mq_ops;
|
|
|
enum blk_eh_timer_return ret = BLK_EH_RESET_TIMER;
|
|
|
|
|
|
/*
|
|
@@ -763,6 +782,12 @@ static bool blk_mq_attempt_merge(struct request_queue *q,
|
|
|
continue;
|
|
|
|
|
|
el_ret = blk_try_merge(rq, bio);
|
|
|
+ if (el_ret == ELEVATOR_NO_MERGE)
|
|
|
+ continue;
|
|
|
+
|
|
|
+ if (!blk_mq_sched_allow_merge(q, rq, bio))
|
|
|
+ break;
|
|
|
+
|
|
|
if (el_ret == ELEVATOR_BACK_MERGE) {
|
|
|
if (bio_attempt_back_merge(q, rq, bio)) {
|
|
|
ctx->rq_merged++;
|
|
@@ -803,7 +828,7 @@ static bool flush_busy_ctx(struct sbitmap *sb, unsigned int bitnr, void *data)
|
|
|
* Process software queues that have been marked busy, splicing them
|
|
|
* to the for-dispatch
|
|
|
*/
|
|
|
-static void flush_busy_ctxs(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
+void blk_mq_flush_busy_ctxs(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
{
|
|
|
struct flush_busy_ctx_data data = {
|
|
|
.hctx = hctx,
|
|
@@ -812,6 +837,7 @@ static void flush_busy_ctxs(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
|
|
|
sbitmap_for_each_set(&hctx->ctx_map, flush_busy_ctx, &data);
|
|
|
}
|
|
|
+EXPORT_SYMBOL_GPL(blk_mq_flush_busy_ctxs);
|
|
|
|
|
|
static inline unsigned int queued_to_index(unsigned int queued)
|
|
|
{
|
|
@@ -821,6 +847,77 @@ static inline unsigned int queued_to_index(unsigned int queued)
|
|
|
return min(BLK_MQ_MAX_DISPATCH_ORDER - 1, ilog2(queued) + 1);
|
|
|
}
|
|
|
|
|
|
+bool blk_mq_get_driver_tag(struct request *rq, struct blk_mq_hw_ctx **hctx,
|
|
|
+ bool wait)
|
|
|
+{
|
|
|
+ struct blk_mq_alloc_data data = {
|
|
|
+ .q = rq->q,
|
|
|
+ .hctx = blk_mq_map_queue(rq->q, rq->mq_ctx->cpu),
|
|
|
+ .flags = wait ? 0 : BLK_MQ_REQ_NOWAIT,
|
|
|
+ };
|
|
|
+
|
|
|
+ if (blk_mq_hctx_stopped(data.hctx))
|
|
|
+ return false;
|
|
|
+
|
|
|
+ if (rq->tag != -1) {
|
|
|
+done:
|
|
|
+ if (hctx)
|
|
|
+ *hctx = data.hctx;
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ rq->tag = blk_mq_get_tag(&data);
|
|
|
+ if (rq->tag >= 0) {
|
|
|
+ if (blk_mq_tag_busy(data.hctx)) {
|
|
|
+ rq->rq_flags |= RQF_MQ_INFLIGHT;
|
|
|
+ atomic_inc(&data.hctx->nr_active);
|
|
|
+ }
|
|
|
+ data.hctx->tags->rqs[rq->tag] = rq;
|
|
|
+ goto done;
|
|
|
+ }
|
|
|
+
|
|
|
+ return false;
|
|
|
+}
|
|
|
+
|
|
|
+static void blk_mq_put_driver_tag(struct blk_mq_hw_ctx *hctx,
|
|
|
+ struct request *rq)
|
|
|
+{
|
|
|
+ if (rq->tag == -1 || rq->internal_tag == -1)
|
|
|
+ return;
|
|
|
+
|
|
|
+ blk_mq_put_tag(hctx, hctx->tags, rq->mq_ctx, rq->tag);
|
|
|
+ rq->tag = -1;
|
|
|
+
|
|
|
+ if (rq->rq_flags & RQF_MQ_INFLIGHT) {
|
|
|
+ rq->rq_flags &= ~RQF_MQ_INFLIGHT;
|
|
|
+ atomic_dec(&hctx->nr_active);
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
+/*
|
|
|
+ * If we fail getting a driver tag because all the driver tags are already
|
|
|
+ * assigned and on the dispatch list, BUT the first entry does not have a
|
|
|
+ * tag, then we could deadlock. For that case, move entries with assigned
|
|
|
+ * driver tags to the front, leaving the set of tagged requests in the
|
|
|
+ * same order, and the untagged set in the same order.
|
|
|
+ */
|
|
|
+static bool reorder_tags_to_front(struct list_head *list)
|
|
|
+{
|
|
|
+ struct request *rq, *tmp, *first = NULL;
|
|
|
+
|
|
|
+ list_for_each_entry_safe_reverse(rq, tmp, list, queuelist) {
|
|
|
+ if (rq == first)
|
|
|
+ break;
|
|
|
+ if (rq->tag != -1) {
|
|
|
+ list_move(&rq->queuelist, list);
|
|
|
+ if (!first)
|
|
|
+ first = rq;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ return first != NULL;
|
|
|
+}
|
|
|
+
|
|
|
bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
{
|
|
|
struct request_queue *q = hctx->queue;
|
|
@@ -843,6 +940,20 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
struct blk_mq_queue_data bd;
|
|
|
|
|
|
rq = list_first_entry(list, struct request, queuelist);
|
|
|
+ if (!blk_mq_get_driver_tag(rq, &hctx, false)) {
|
|
|
+ if (!queued && reorder_tags_to_front(list))
|
|
|
+ continue;
|
|
|
+
|
|
|
+ /*
|
|
|
+ * We failed getting a driver tag. Mark the queue(s)
|
|
|
+ * as needing a restart. Retry getting a tag again,
|
|
|
+ * in case the needed IO completed right before we
|
|
|
+ * marked the queue as needing a restart.
|
|
|
+ */
|
|
|
+ blk_mq_sched_mark_restart(hctx);
|
|
|
+ if (!blk_mq_get_driver_tag(rq, &hctx, false))
|
|
|
+ break;
|
|
|
+ }
|
|
|
list_del_init(&rq->queuelist);
|
|
|
|
|
|
bd.rq = rq;
|
|
@@ -855,6 +966,7 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
queued++;
|
|
|
break;
|
|
|
case BLK_MQ_RQ_QUEUE_BUSY:
|
|
|
+ blk_mq_put_driver_tag(hctx, rq);
|
|
|
list_add(&rq->queuelist, list);
|
|
|
__blk_mq_requeue_request(rq);
|
|
|
break;
|
|
@@ -885,7 +997,7 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
*/
|
|
|
if (!list_empty(list)) {
|
|
|
spin_lock(&hctx->lock);
|
|
|
- list_splice(list, &hctx->dispatch);
|
|
|
+ list_splice_init(list, &hctx->dispatch);
|
|
|
spin_unlock(&hctx->lock);
|
|
|
|
|
|
/*
|
|
@@ -896,47 +1008,17 @@ bool blk_mq_dispatch_rq_list(struct blk_mq_hw_ctx *hctx, struct list_head *list)
|
|
|
* the requests in rq_list might get lost.
|
|
|
*
|
|
|
* blk_mq_run_hw_queue() already checks the STOPPED bit
|
|
|
- **/
|
|
|
- blk_mq_run_hw_queue(hctx, true);
|
|
|
+ *
|
|
|
+ * If RESTART is set, then let completion restart the queue
|
|
|
+ * instead of potentially looping here.
|
|
|
+ */
|
|
|
+ if (!blk_mq_sched_needs_restart(hctx))
|
|
|
+ blk_mq_run_hw_queue(hctx, true);
|
|
|
}
|
|
|
|
|
|
return ret != BLK_MQ_RQ_QUEUE_BUSY;
|
|
|
}
|
|
|
|
|
|
-/*
|
|
|
- * Run this hardware queue, pulling any software queues mapped to it in.
|
|
|
- * Note that this function currently has various problems around ordering
|
|
|
- * of IO. In particular, we'd like FIFO behaviour on handling existing
|
|
|
- * items on the hctx->dispatch list. Ignore that for now.
|
|
|
- */
|
|
|
-static void blk_mq_process_rq_list(struct blk_mq_hw_ctx *hctx)
|
|
|
-{
|
|
|
- LIST_HEAD(rq_list);
|
|
|
-
|
|
|
- if (unlikely(blk_mq_hctx_stopped(hctx)))
|
|
|
- return;
|
|
|
-
|
|
|
- hctx->run++;
|
|
|
-
|
|
|
- /*
|
|
|
- * Touch any software queue that has pending entries.
|
|
|
- */
|
|
|
- flush_busy_ctxs(hctx, &rq_list);
|
|
|
-
|
|
|
- /*
|
|
|
- * If we have previous entries on our dispatch list, grab them
|
|
|
- * and stuff them at the front for more fair dispatch.
|
|
|
- */
|
|
|
- if (!list_empty_careful(&hctx->dispatch)) {
|
|
|
- spin_lock(&hctx->lock);
|
|
|
- if (!list_empty(&hctx->dispatch))
|
|
|
- list_splice_init(&hctx->dispatch, &rq_list);
|
|
|
- spin_unlock(&hctx->lock);
|
|
|
- }
|
|
|
-
|
|
|
- blk_mq_dispatch_rq_list(hctx, &rq_list);
|
|
|
-}
|
|
|
-
|
|
|
static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
|
|
|
{
|
|
|
int srcu_idx;
|
|
@@ -946,11 +1028,11 @@ static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
|
|
|
|
|
|
if (!(hctx->flags & BLK_MQ_F_BLOCKING)) {
|
|
|
rcu_read_lock();
|
|
|
- blk_mq_process_rq_list(hctx);
|
|
|
+ blk_mq_sched_dispatch_requests(hctx);
|
|
|
rcu_read_unlock();
|
|
|
} else {
|
|
|
srcu_idx = srcu_read_lock(&hctx->queue_rq_srcu);
|
|
|
- blk_mq_process_rq_list(hctx);
|
|
|
+ blk_mq_sched_dispatch_requests(hctx);
|
|
|
srcu_read_unlock(&hctx->queue_rq_srcu, srcu_idx);
|
|
|
}
|
|
|
}
|
|
@@ -1006,8 +1088,7 @@ void blk_mq_run_hw_queues(struct request_queue *q, bool async)
|
|
|
int i;
|
|
|
|
|
|
queue_for_each_hw_ctx(q, hctx, i) {
|
|
|
- if ((!blk_mq_hctx_has_pending(hctx) &&
|
|
|
- list_empty_careful(&hctx->dispatch)) ||
|
|
|
+ if (!blk_mq_hctx_has_pending(hctx) ||
|
|
|
blk_mq_hctx_stopped(hctx))
|
|
|
continue;
|
|
|
|
|
@@ -1116,6 +1197,7 @@ void blk_mq_delay_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs)
|
|
|
if (unlikely(!blk_mq_hw_queue_mapped(hctx)))
|
|
|
return;
|
|
|
|
|
|
+ blk_mq_stop_hw_queue(hctx);
|
|
|
kblockd_schedule_delayed_work_on(blk_mq_hctx_next_cpu(hctx),
|
|
|
&hctx->delay_work, msecs_to_jiffies(msecs));
|
|
|
}
|
|
@@ -1135,8 +1217,8 @@ static inline void __blk_mq_insert_req_list(struct blk_mq_hw_ctx *hctx,
|
|
|
list_add_tail(&rq->queuelist, &ctx->rq_list);
|
|
|
}
|
|
|
|
|
|
-static void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx,
|
|
|
- struct request *rq, bool at_head)
|
|
|
+void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx, struct request *rq,
|
|
|
+ bool at_head)
|
|
|
{
|
|
|
struct blk_mq_ctx *ctx = rq->mq_ctx;
|
|
|
|
|
@@ -1144,32 +1226,10 @@ static void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx,
|
|
|
blk_mq_hctx_mark_pending(hctx, ctx);
|
|
|
}
|
|
|
|
|
|
-void blk_mq_insert_request(struct request *rq, bool at_head, bool run_queue,
|
|
|
- bool async)
|
|
|
-{
|
|
|
- struct blk_mq_ctx *ctx = rq->mq_ctx;
|
|
|
- struct request_queue *q = rq->q;
|
|
|
- struct blk_mq_hw_ctx *hctx = blk_mq_map_queue(q, ctx->cpu);
|
|
|
-
|
|
|
- spin_lock(&ctx->lock);
|
|
|
- __blk_mq_insert_request(hctx, rq, at_head);
|
|
|
- spin_unlock(&ctx->lock);
|
|
|
-
|
|
|
- if (run_queue)
|
|
|
- blk_mq_run_hw_queue(hctx, async);
|
|
|
-}
|
|
|
-
|
|
|
-static void blk_mq_insert_requests(struct request_queue *q,
|
|
|
- struct blk_mq_ctx *ctx,
|
|
|
- struct list_head *list,
|
|
|
- int depth,
|
|
|
- bool from_schedule)
|
|
|
+void blk_mq_insert_requests(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
|
|
|
+ struct list_head *list)
|
|
|
|
|
|
{
|
|
|
- struct blk_mq_hw_ctx *hctx = blk_mq_map_queue(q, ctx->cpu);
|
|
|
-
|
|
|
- trace_block_unplug(q, depth, !from_schedule);
|
|
|
-
|
|
|
/*
|
|
|
* preemption doesn't flush plug list, so it's possible ctx->cpu is
|
|
|
* offline now
|
|
@@ -1185,8 +1245,6 @@ static void blk_mq_insert_requests(struct request_queue *q,
|
|
|
}
|
|
|
blk_mq_hctx_mark_pending(hctx, ctx);
|
|
|
spin_unlock(&ctx->lock);
|
|
|
-
|
|
|
- blk_mq_run_hw_queue(hctx, from_schedule);
|
|
|
}
|
|
|
|
|
|
static int plug_ctx_cmp(void *priv, struct list_head *a, struct list_head *b)
|
|
@@ -1222,9 +1280,10 @@ void blk_mq_flush_plug_list(struct blk_plug *plug, bool from_schedule)
|
|
|
BUG_ON(!rq->q);
|
|
|
if (rq->mq_ctx != this_ctx) {
|
|
|
if (this_ctx) {
|
|
|
- blk_mq_insert_requests(this_q, this_ctx,
|
|
|
- &ctx_list, depth,
|
|
|
- from_schedule);
|
|
|
+ trace_block_unplug(this_q, depth, from_schedule);
|
|
|
+ blk_mq_sched_insert_requests(this_q, this_ctx,
|
|
|
+ &ctx_list,
|
|
|
+ from_schedule);
|
|
|
}
|
|
|
|
|
|
this_ctx = rq->mq_ctx;
|
|
@@ -1241,8 +1300,9 @@ void blk_mq_flush_plug_list(struct blk_plug *plug, bool from_schedule)
|
|
|
* on 'ctx_list'. Do those.
|
|
|
*/
|
|
|
if (this_ctx) {
|
|
|
- blk_mq_insert_requests(this_q, this_ctx, &ctx_list, depth,
|
|
|
- from_schedule);
|
|
|
+ trace_block_unplug(this_q, depth, from_schedule);
|
|
|
+ blk_mq_sched_insert_requests(this_q, this_ctx, &ctx_list,
|
|
|
+ from_schedule);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1280,46 +1340,39 @@ insert_rq:
|
|
|
}
|
|
|
|
|
|
spin_unlock(&ctx->lock);
|
|
|
- __blk_mq_free_request(hctx, ctx, rq);
|
|
|
+ __blk_mq_finish_request(hctx, ctx, rq);
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-static struct request *blk_mq_map_request(struct request_queue *q,
|
|
|
- struct bio *bio,
|
|
|
- struct blk_mq_alloc_data *data)
|
|
|
+static blk_qc_t request_to_qc_t(struct blk_mq_hw_ctx *hctx, struct request *rq)
|
|
|
{
|
|
|
- struct blk_mq_hw_ctx *hctx;
|
|
|
- struct blk_mq_ctx *ctx;
|
|
|
- struct request *rq;
|
|
|
-
|
|
|
- blk_queue_enter_live(q);
|
|
|
- ctx = blk_mq_get_ctx(q);
|
|
|
- hctx = blk_mq_map_queue(q, ctx->cpu);
|
|
|
-
|
|
|
- trace_block_getrq(q, bio, bio->bi_opf);
|
|
|
- blk_mq_set_alloc_data(data, q, 0, ctx, hctx);
|
|
|
- rq = __blk_mq_alloc_request(data, bio->bi_opf);
|
|
|
+ if (rq->tag != -1)
|
|
|
+ return blk_tag_to_qc_t(rq->tag, hctx->queue_num, false);
|
|
|
|
|
|
- data->hctx->queued++;
|
|
|
- return rq;
|
|
|
+ return blk_tag_to_qc_t(rq->internal_tag, hctx->queue_num, true);
|
|
|
}
|
|
|
|
|
|
static void blk_mq_try_issue_directly(struct request *rq, blk_qc_t *cookie)
|
|
|
{
|
|
|
- int ret;
|
|
|
struct request_queue *q = rq->q;
|
|
|
- struct blk_mq_hw_ctx *hctx = blk_mq_map_queue(q, rq->mq_ctx->cpu);
|
|
|
struct blk_mq_queue_data bd = {
|
|
|
.rq = rq,
|
|
|
.list = NULL,
|
|
|
.last = 1
|
|
|
};
|
|
|
- blk_qc_t new_cookie = blk_tag_to_qc_t(rq->tag, hctx->queue_num);
|
|
|
+ struct blk_mq_hw_ctx *hctx;
|
|
|
+ blk_qc_t new_cookie;
|
|
|
+ int ret;
|
|
|
|
|
|
- if (blk_mq_hctx_stopped(hctx))
|
|
|
+ if (q->elevator)
|
|
|
goto insert;
|
|
|
|
|
|
+ if (!blk_mq_get_driver_tag(rq, &hctx, false))
|
|
|
+ goto insert;
|
|
|
+
|
|
|
+ new_cookie = request_to_qc_t(hctx, rq);
|
|
|
+
|
|
|
/*
|
|
|
* For OK queue, we are done. For error, kill it. Any other
|
|
|
* error (busy), just add it to our list as we previously
|
|
@@ -1341,7 +1394,7 @@ static void blk_mq_try_issue_directly(struct request *rq, blk_qc_t *cookie)
|
|
|
}
|
|
|
|
|
|
insert:
|
|
|
- blk_mq_insert_request(rq, false, true, true);
|
|
|
+ blk_mq_sched_insert_request(rq, false, true, true, false);
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -1352,8 +1405,8 @@ insert:
|
|
|
static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
{
|
|
|
const int is_sync = op_is_sync(bio->bi_opf);
|
|
|
- const int is_flush_fua = bio->bi_opf & (REQ_PREFLUSH | REQ_FUA);
|
|
|
- struct blk_mq_alloc_data data;
|
|
|
+ const int is_flush_fua = op_is_flush(bio->bi_opf);
|
|
|
+ struct blk_mq_alloc_data data = { .flags = 0 };
|
|
|
struct request *rq;
|
|
|
unsigned int request_count = 0, srcu_idx;
|
|
|
struct blk_plug *plug;
|
|
@@ -1374,9 +1427,14 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
blk_attempt_plug_merge(q, bio, &request_count, &same_queue_rq))
|
|
|
return BLK_QC_T_NONE;
|
|
|
|
|
|
+ if (blk_mq_sched_bio_merge(q, bio))
|
|
|
+ return BLK_QC_T_NONE;
|
|
|
+
|
|
|
wb_acct = wbt_wait(q->rq_wb, bio, NULL);
|
|
|
|
|
|
- rq = blk_mq_map_request(q, bio, &data);
|
|
|
+ trace_block_getrq(q, bio, bio->bi_opf);
|
|
|
+
|
|
|
+ rq = blk_mq_sched_get_request(q, bio, bio->bi_opf, &data);
|
|
|
if (unlikely(!rq)) {
|
|
|
__wbt_done(q->rq_wb, wb_acct);
|
|
|
return BLK_QC_T_NONE;
|
|
@@ -1384,12 +1442,15 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
|
|
|
wbt_track(&rq->issue_stat, wb_acct);
|
|
|
|
|
|
- cookie = blk_tag_to_qc_t(rq->tag, data.hctx->queue_num);
|
|
|
+ cookie = request_to_qc_t(data.hctx, rq);
|
|
|
|
|
|
if (unlikely(is_flush_fua)) {
|
|
|
+ blk_mq_put_ctx(data.ctx);
|
|
|
blk_mq_bio_to_request(rq, bio);
|
|
|
+ blk_mq_get_driver_tag(rq, NULL, true);
|
|
|
blk_insert_flush(rq);
|
|
|
- goto run_queue;
|
|
|
+ blk_mq_run_hw_queue(data.hctx, true);
|
|
|
+ goto done;
|
|
|
}
|
|
|
|
|
|
plug = current->plug;
|
|
@@ -1438,6 +1499,13 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
goto done;
|
|
|
}
|
|
|
|
|
|
+ if (q->elevator) {
|
|
|
+ blk_mq_put_ctx(data.ctx);
|
|
|
+ blk_mq_bio_to_request(rq, bio);
|
|
|
+ blk_mq_sched_insert_request(rq, false, true,
|
|
|
+ !is_sync || is_flush_fua, true);
|
|
|
+ goto done;
|
|
|
+ }
|
|
|
if (!blk_mq_merge_queue_io(data.hctx, data.ctx, rq, bio)) {
|
|
|
/*
|
|
|
* For a SYNC request, send it to the hardware immediately. For
|
|
@@ -1445,7 +1513,6 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
* latter allows for merging opportunities and more efficient
|
|
|
* dispatching.
|
|
|
*/
|
|
|
-run_queue:
|
|
|
blk_mq_run_hw_queue(data.hctx, !is_sync || is_flush_fua);
|
|
|
}
|
|
|
blk_mq_put_ctx(data.ctx);
|
|
@@ -1460,10 +1527,10 @@ done:
|
|
|
static blk_qc_t blk_sq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
{
|
|
|
const int is_sync = op_is_sync(bio->bi_opf);
|
|
|
- const int is_flush_fua = bio->bi_opf & (REQ_PREFLUSH | REQ_FUA);
|
|
|
+ const int is_flush_fua = op_is_flush(bio->bi_opf);
|
|
|
struct blk_plug *plug;
|
|
|
unsigned int request_count = 0;
|
|
|
- struct blk_mq_alloc_data data;
|
|
|
+ struct blk_mq_alloc_data data = { .flags = 0 };
|
|
|
struct request *rq;
|
|
|
blk_qc_t cookie;
|
|
|
unsigned int wb_acct;
|
|
@@ -1483,9 +1550,14 @@ static blk_qc_t blk_sq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
} else
|
|
|
request_count = blk_plug_queued_count(q);
|
|
|
|
|
|
+ if (blk_mq_sched_bio_merge(q, bio))
|
|
|
+ return BLK_QC_T_NONE;
|
|
|
+
|
|
|
wb_acct = wbt_wait(q->rq_wb, bio, NULL);
|
|
|
|
|
|
- rq = blk_mq_map_request(q, bio, &data);
|
|
|
+ trace_block_getrq(q, bio, bio->bi_opf);
|
|
|
+
|
|
|
+ rq = blk_mq_sched_get_request(q, bio, bio->bi_opf, &data);
|
|
|
if (unlikely(!rq)) {
|
|
|
__wbt_done(q->rq_wb, wb_acct);
|
|
|
return BLK_QC_T_NONE;
|
|
@@ -1493,12 +1565,15 @@ static blk_qc_t blk_sq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
|
|
|
wbt_track(&rq->issue_stat, wb_acct);
|
|
|
|
|
|
- cookie = blk_tag_to_qc_t(rq->tag, data.hctx->queue_num);
|
|
|
+ cookie = request_to_qc_t(data.hctx, rq);
|
|
|
|
|
|
if (unlikely(is_flush_fua)) {
|
|
|
+ blk_mq_put_ctx(data.ctx);
|
|
|
blk_mq_bio_to_request(rq, bio);
|
|
|
+ blk_mq_get_driver_tag(rq, NULL, true);
|
|
|
blk_insert_flush(rq);
|
|
|
- goto run_queue;
|
|
|
+ blk_mq_run_hw_queue(data.hctx, true);
|
|
|
+ goto done;
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -1535,6 +1610,13 @@ static blk_qc_t blk_sq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
return cookie;
|
|
|
}
|
|
|
|
|
|
+ if (q->elevator) {
|
|
|
+ blk_mq_put_ctx(data.ctx);
|
|
|
+ blk_mq_bio_to_request(rq, bio);
|
|
|
+ blk_mq_sched_insert_request(rq, false, true,
|
|
|
+ !is_sync || is_flush_fua, true);
|
|
|
+ goto done;
|
|
|
+ }
|
|
|
if (!blk_mq_merge_queue_io(data.hctx, data.ctx, rq, bio)) {
|
|
|
/*
|
|
|
* For a SYNC request, send it to the hardware immediately. For
|
|
@@ -1542,16 +1624,16 @@ static blk_qc_t blk_sq_make_request(struct request_queue *q, struct bio *bio)
|
|
|
* latter allows for merging opportunities and more efficient
|
|
|
* dispatching.
|
|
|
*/
|
|
|
-run_queue:
|
|
|
blk_mq_run_hw_queue(data.hctx, !is_sync || is_flush_fua);
|
|
|
}
|
|
|
|
|
|
blk_mq_put_ctx(data.ctx);
|
|
|
+done:
|
|
|
return cookie;
|
|
|
}
|
|
|
|
|
|
-static void blk_mq_free_rq_map(struct blk_mq_tag_set *set,
|
|
|
- struct blk_mq_tags *tags, unsigned int hctx_idx)
|
|
|
+void blk_mq_free_rqs(struct blk_mq_tag_set *set, struct blk_mq_tags *tags,
|
|
|
+ unsigned int hctx_idx)
|
|
|
{
|
|
|
struct page *page;
|
|
|
|
|
@@ -1559,11 +1641,13 @@ static void blk_mq_free_rq_map(struct blk_mq_tag_set *set,
|
|
|
int i;
|
|
|
|
|
|
for (i = 0; i < tags->nr_tags; i++) {
|
|
|
- if (!tags->rqs[i])
|
|
|
+ struct request *rq = tags->static_rqs[i];
|
|
|
+
|
|
|
+ if (!rq)
|
|
|
continue;
|
|
|
- set->ops->exit_request(set->driver_data, tags->rqs[i],
|
|
|
+ set->ops->exit_request(set->driver_data, rq,
|
|
|
hctx_idx, i);
|
|
|
- tags->rqs[i] = NULL;
|
|
|
+ tags->static_rqs[i] = NULL;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1577,33 +1661,32 @@ static void blk_mq_free_rq_map(struct blk_mq_tag_set *set,
|
|
|
kmemleak_free(page_address(page));
|
|
|
__free_pages(page, page->private);
|
|
|
}
|
|
|
+}
|
|
|
|
|
|
+void blk_mq_free_rq_map(struct blk_mq_tags *tags)
|
|
|
+{
|
|
|
kfree(tags->rqs);
|
|
|
+ tags->rqs = NULL;
|
|
|
+ kfree(tags->static_rqs);
|
|
|
+ tags->static_rqs = NULL;
|
|
|
|
|
|
blk_mq_free_tags(tags);
|
|
|
}
|
|
|
|
|
|
-static size_t order_to_size(unsigned int order)
|
|
|
-{
|
|
|
- return (size_t)PAGE_SIZE << order;
|
|
|
-}
|
|
|
-
|
|
|
-static struct blk_mq_tags *blk_mq_init_rq_map(struct blk_mq_tag_set *set,
|
|
|
- unsigned int hctx_idx)
|
|
|
+struct blk_mq_tags *blk_mq_alloc_rq_map(struct blk_mq_tag_set *set,
|
|
|
+ unsigned int hctx_idx,
|
|
|
+ unsigned int nr_tags,
|
|
|
+ unsigned int reserved_tags)
|
|
|
{
|
|
|
struct blk_mq_tags *tags;
|
|
|
- unsigned int i, j, entries_per_page, max_order = 4;
|
|
|
- size_t rq_size, left;
|
|
|
|
|
|
- tags = blk_mq_init_tags(set->queue_depth, set->reserved_tags,
|
|
|
+ tags = blk_mq_init_tags(nr_tags, reserved_tags,
|
|
|
set->numa_node,
|
|
|
BLK_MQ_FLAG_TO_ALLOC_POLICY(set->flags));
|
|
|
if (!tags)
|
|
|
return NULL;
|
|
|
|
|
|
- INIT_LIST_HEAD(&tags->page_list);
|
|
|
-
|
|
|
- tags->rqs = kzalloc_node(set->queue_depth * sizeof(struct request *),
|
|
|
+ tags->rqs = kzalloc_node(nr_tags * sizeof(struct request *),
|
|
|
GFP_NOIO | __GFP_NOWARN | __GFP_NORETRY,
|
|
|
set->numa_node);
|
|
|
if (!tags->rqs) {
|
|
@@ -1611,15 +1694,40 @@ static struct blk_mq_tags *blk_mq_init_rq_map(struct blk_mq_tag_set *set,
|
|
|
return NULL;
|
|
|
}
|
|
|
|
|
|
+ tags->static_rqs = kzalloc_node(nr_tags * sizeof(struct request *),
|
|
|
+ GFP_NOIO | __GFP_NOWARN | __GFP_NORETRY,
|
|
|
+ set->numa_node);
|
|
|
+ if (!tags->static_rqs) {
|
|
|
+ kfree(tags->rqs);
|
|
|
+ blk_mq_free_tags(tags);
|
|
|
+ return NULL;
|
|
|
+ }
|
|
|
+
|
|
|
+ return tags;
|
|
|
+}
|
|
|
+
|
|
|
+static size_t order_to_size(unsigned int order)
|
|
|
+{
|
|
|
+ return (size_t)PAGE_SIZE << order;
|
|
|
+}
|
|
|
+
|
|
|
+int blk_mq_alloc_rqs(struct blk_mq_tag_set *set, struct blk_mq_tags *tags,
|
|
|
+ unsigned int hctx_idx, unsigned int depth)
|
|
|
+{
|
|
|
+ unsigned int i, j, entries_per_page, max_order = 4;
|
|
|
+ size_t rq_size, left;
|
|
|
+
|
|
|
+ INIT_LIST_HEAD(&tags->page_list);
|
|
|
+
|
|
|
/*
|
|
|
* rq_size is the size of the request plus driver payload, rounded
|
|
|
* to the cacheline size
|
|
|
*/
|
|
|
rq_size = round_up(sizeof(struct request) + set->cmd_size,
|
|
|
cache_line_size());
|
|
|
- left = rq_size * set->queue_depth;
|
|
|
+ left = rq_size * depth;
|
|
|
|
|
|
- for (i = 0; i < set->queue_depth; ) {
|
|
|
+ for (i = 0; i < depth; ) {
|
|
|
int this_order = max_order;
|
|
|
struct page *page;
|
|
|
int to_do;
|
|
@@ -1653,15 +1761,17 @@ static struct blk_mq_tags *blk_mq_init_rq_map(struct blk_mq_tag_set *set,
|
|
|
*/
|
|
|
kmemleak_alloc(p, order_to_size(this_order), 1, GFP_NOIO);
|
|
|
entries_per_page = order_to_size(this_order) / rq_size;
|
|
|
- to_do = min(entries_per_page, set->queue_depth - i);
|
|
|
+ to_do = min(entries_per_page, depth - i);
|
|
|
left -= to_do * rq_size;
|
|
|
for (j = 0; j < to_do; j++) {
|
|
|
- tags->rqs[i] = p;
|
|
|
+ struct request *rq = p;
|
|
|
+
|
|
|
+ tags->static_rqs[i] = rq;
|
|
|
if (set->ops->init_request) {
|
|
|
if (set->ops->init_request(set->driver_data,
|
|
|
- tags->rqs[i], hctx_idx, i,
|
|
|
+ rq, hctx_idx, i,
|
|
|
set->numa_node)) {
|
|
|
- tags->rqs[i] = NULL;
|
|
|
+ tags->static_rqs[i] = NULL;
|
|
|
goto fail;
|
|
|
}
|
|
|
}
|
|
@@ -1670,11 +1780,11 @@ static struct blk_mq_tags *blk_mq_init_rq_map(struct blk_mq_tag_set *set,
|
|
|
i++;
|
|
|
}
|
|
|
}
|
|
|
- return tags;
|
|
|
+ return 0;
|
|
|
|
|
|
fail:
|
|
|
- blk_mq_free_rq_map(set, tags, hctx_idx);
|
|
|
- return NULL;
|
|
|
+ blk_mq_free_rqs(set, tags, hctx_idx);
|
|
|
+ return -ENOMEM;
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -1866,6 +1976,35 @@ static void blk_mq_init_cpu_queues(struct request_queue *q,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+static bool __blk_mq_alloc_rq_map(struct blk_mq_tag_set *set, int hctx_idx)
|
|
|
+{
|
|
|
+ int ret = 0;
|
|
|
+
|
|
|
+ set->tags[hctx_idx] = blk_mq_alloc_rq_map(set, hctx_idx,
|
|
|
+ set->queue_depth, set->reserved_tags);
|
|
|
+ if (!set->tags[hctx_idx])
|
|
|
+ return false;
|
|
|
+
|
|
|
+ ret = blk_mq_alloc_rqs(set, set->tags[hctx_idx], hctx_idx,
|
|
|
+ set->queue_depth);
|
|
|
+ if (!ret)
|
|
|
+ return true;
|
|
|
+
|
|
|
+ blk_mq_free_rq_map(set->tags[hctx_idx]);
|
|
|
+ set->tags[hctx_idx] = NULL;
|
|
|
+ return false;
|
|
|
+}
|
|
|
+
|
|
|
+static void blk_mq_free_map_and_requests(struct blk_mq_tag_set *set,
|
|
|
+ unsigned int hctx_idx)
|
|
|
+{
|
|
|
+ if (set->tags[hctx_idx]) {
|
|
|
+ blk_mq_free_rqs(set, set->tags[hctx_idx], hctx_idx);
|
|
|
+ blk_mq_free_rq_map(set->tags[hctx_idx]);
|
|
|
+ set->tags[hctx_idx] = NULL;
|
|
|
+ }
|
|
|
+}
|
|
|
+
|
|
|
static void blk_mq_map_swqueue(struct request_queue *q,
|
|
|
const struct cpumask *online_mask)
|
|
|
{
|
|
@@ -1894,17 +2033,15 @@ static void blk_mq_map_swqueue(struct request_queue *q,
|
|
|
|
|
|
hctx_idx = q->mq_map[i];
|
|
|
/* unmapped hw queue can be remapped after CPU topo changed */
|
|
|
- if (!set->tags[hctx_idx]) {
|
|
|
- set->tags[hctx_idx] = blk_mq_init_rq_map(set, hctx_idx);
|
|
|
-
|
|
|
+ if (!set->tags[hctx_idx] &&
|
|
|
+ !__blk_mq_alloc_rq_map(set, hctx_idx)) {
|
|
|
/*
|
|
|
* If tags initialization fail for some hctx,
|
|
|
* that hctx won't be brought online. In this
|
|
|
* case, remap the current ctx to hctx[0] which
|
|
|
* is guaranteed to always have tags allocated
|
|
|
*/
|
|
|
- if (!set->tags[hctx_idx])
|
|
|
- q->mq_map[i] = 0;
|
|
|
+ q->mq_map[i] = 0;
|
|
|
}
|
|
|
|
|
|
ctx = per_cpu_ptr(q->queue_ctx, i);
|
|
@@ -1927,10 +2064,9 @@ static void blk_mq_map_swqueue(struct request_queue *q,
|
|
|
* fallback in case of a new remap fails
|
|
|
* allocation
|
|
|
*/
|
|
|
- if (i && set->tags[i]) {
|
|
|
- blk_mq_free_rq_map(set, set->tags[i], i);
|
|
|
- set->tags[i] = NULL;
|
|
|
- }
|
|
|
+ if (i && set->tags[i])
|
|
|
+ blk_mq_free_map_and_requests(set, i);
|
|
|
+
|
|
|
hctx->tags = NULL;
|
|
|
continue;
|
|
|
}
|
|
@@ -2023,6 +2159,8 @@ void blk_mq_release(struct request_queue *q)
|
|
|
struct blk_mq_hw_ctx *hctx;
|
|
|
unsigned int i;
|
|
|
|
|
|
+ blk_mq_sched_teardown(q);
|
|
|
+
|
|
|
/* hctx kobj stays in hctx */
|
|
|
queue_for_each_hw_ctx(q, hctx, i) {
|
|
|
if (!hctx)
|
|
@@ -2097,10 +2235,8 @@ static void blk_mq_realloc_hw_ctxs(struct blk_mq_tag_set *set,
|
|
|
struct blk_mq_hw_ctx *hctx = hctxs[j];
|
|
|
|
|
|
if (hctx) {
|
|
|
- if (hctx->tags) {
|
|
|
- blk_mq_free_rq_map(set, hctx->tags, j);
|
|
|
- set->tags[j] = NULL;
|
|
|
- }
|
|
|
+ if (hctx->tags)
|
|
|
+ blk_mq_free_map_and_requests(set, j);
|
|
|
blk_mq_exit_hctx(q, set, hctx, j);
|
|
|
free_cpumask_var(hctx->cpumask);
|
|
|
kobject_put(&hctx->kobj);
|
|
@@ -2181,6 +2317,14 @@ struct request_queue *blk_mq_init_allocated_queue(struct blk_mq_tag_set *set,
|
|
|
mutex_unlock(&all_q_mutex);
|
|
|
put_online_cpus();
|
|
|
|
|
|
+ if (!(set->flags & BLK_MQ_F_NO_SCHED)) {
|
|
|
+ int ret;
|
|
|
+
|
|
|
+ ret = blk_mq_sched_init(q);
|
|
|
+ if (ret)
|
|
|
+ return ERR_PTR(ret);
|
|
|
+ }
|
|
|
+
|
|
|
return q;
|
|
|
|
|
|
err_hctxs:
|
|
@@ -2279,10 +2423,10 @@ static int blk_mq_queue_reinit_dead(unsigned int cpu)
|
|
|
* Now CPU1 is just onlined and a request is inserted into ctx1->rq_list
|
|
|
* and set bit0 in pending bitmap as ctx1->index_hw is still zero.
|
|
|
*
|
|
|
- * And then while running hw queue, flush_busy_ctxs() finds bit0 is set in
|
|
|
- * pending bitmap and tries to retrieve requests in hctx->ctxs[0]->rq_list.
|
|
|
- * But htx->ctxs[0] is a pointer to ctx0, so the request in ctx1->rq_list
|
|
|
- * is ignored.
|
|
|
+ * And then while running hw queue, blk_mq_flush_busy_ctxs() finds bit0 is set
|
|
|
+ * in pending bitmap and tries to retrieve requests in hctx->ctxs[0]->rq_list.
|
|
|
+ * But htx->ctxs[0] is a pointer to ctx0, so the request in ctx1->rq_list is
|
|
|
+ * ignored.
|
|
|
*/
|
|
|
static int blk_mq_queue_reinit_prepare(unsigned int cpu)
|
|
|
{
|
|
@@ -2296,17 +2440,15 @@ static int __blk_mq_alloc_rq_maps(struct blk_mq_tag_set *set)
|
|
|
{
|
|
|
int i;
|
|
|
|
|
|
- for (i = 0; i < set->nr_hw_queues; i++) {
|
|
|
- set->tags[i] = blk_mq_init_rq_map(set, i);
|
|
|
- if (!set->tags[i])
|
|
|
+ for (i = 0; i < set->nr_hw_queues; i++)
|
|
|
+ if (!__blk_mq_alloc_rq_map(set, i))
|
|
|
goto out_unwind;
|
|
|
- }
|
|
|
|
|
|
return 0;
|
|
|
|
|
|
out_unwind:
|
|
|
while (--i >= 0)
|
|
|
- blk_mq_free_rq_map(set, set->tags[i], i);
|
|
|
+ blk_mq_free_rq_map(set->tags[i]);
|
|
|
|
|
|
return -ENOMEM;
|
|
|
}
|
|
@@ -2430,10 +2572,8 @@ void blk_mq_free_tag_set(struct blk_mq_tag_set *set)
|
|
|
{
|
|
|
int i;
|
|
|
|
|
|
- for (i = 0; i < nr_cpu_ids; i++) {
|
|
|
- if (set->tags[i])
|
|
|
- blk_mq_free_rq_map(set, set->tags[i], i);
|
|
|
- }
|
|
|
+ for (i = 0; i < nr_cpu_ids; i++)
|
|
|
+ blk_mq_free_map_and_requests(set, i);
|
|
|
|
|
|
kfree(set->mq_map);
|
|
|
set->mq_map = NULL;
|
|
@@ -2449,14 +2589,28 @@ int blk_mq_update_nr_requests(struct request_queue *q, unsigned int nr)
|
|
|
struct blk_mq_hw_ctx *hctx;
|
|
|
int i, ret;
|
|
|
|
|
|
- if (!set || nr > set->queue_depth)
|
|
|
+ if (!set)
|
|
|
return -EINVAL;
|
|
|
|
|
|
+ blk_mq_freeze_queue(q);
|
|
|
+ blk_mq_quiesce_queue(q);
|
|
|
+
|
|
|
ret = 0;
|
|
|
queue_for_each_hw_ctx(q, hctx, i) {
|
|
|
if (!hctx->tags)
|
|
|
continue;
|
|
|
- ret = blk_mq_tag_update_depth(hctx->tags, nr);
|
|
|
+ /*
|
|
|
+ * If we're using an MQ scheduler, just update the scheduler
|
|
|
+ * queue depth. This is similar to what the old code would do.
|
|
|
+ */
|
|
|
+ if (!hctx->sched_tags) {
|
|
|
+ ret = blk_mq_tag_update_depth(hctx, &hctx->tags,
|
|
|
+ min(nr, set->queue_depth),
|
|
|
+ false);
|
|
|
+ } else {
|
|
|
+ ret = blk_mq_tag_update_depth(hctx, &hctx->sched_tags,
|
|
|
+ nr, true);
|
|
|
+ }
|
|
|
if (ret)
|
|
|
break;
|
|
|
}
|
|
@@ -2464,6 +2618,9 @@ int blk_mq_update_nr_requests(struct request_queue *q, unsigned int nr)
|
|
|
if (!ret)
|
|
|
q->nr_requests = nr;
|
|
|
|
|
|
+ blk_mq_unfreeze_queue(q);
|
|
|
+ blk_mq_start_stopped_hw_queues(q, true);
|
|
|
+
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
@@ -2649,7 +2806,10 @@ bool blk_mq_poll(struct request_queue *q, blk_qc_t cookie)
|
|
|
blk_flush_plug_list(plug, false);
|
|
|
|
|
|
hctx = q->queue_hw_ctx[blk_qc_t_to_queue_num(cookie)];
|
|
|
- rq = blk_mq_tag_to_rq(hctx->tags, blk_qc_t_to_tag(cookie));
|
|
|
+ if (!blk_qc_t_is_internal(cookie))
|
|
|
+ rq = blk_mq_tag_to_rq(hctx->tags, blk_qc_t_to_tag(cookie));
|
|
|
+ else
|
|
|
+ rq = blk_mq_tag_to_rq(hctx->sched_tags, blk_qc_t_to_tag(cookie));
|
|
|
|
|
|
return __blk_mq_poll(hctx, rq);
|
|
|
}
|
|
@@ -2667,6 +2827,8 @@ void blk_mq_enable_hotplug(void)
|
|
|
|
|
|
static int __init blk_mq_init(void)
|
|
|
{
|
|
|
+ blk_mq_debugfs_init();
|
|
|
+
|
|
|
cpuhp_setup_state_multi(CPUHP_BLK_MQ_DEAD, "block/mq:dead", NULL,
|
|
|
blk_mq_hctx_notify_dead);
|
|
|
|