summaryrefslogtreecommitdiffstats
path: root/block/blk-mq.c
diff options
context:
space:
mode:
Diffstat (limited to 'block/blk-mq.c')
-rw-r--r--block/blk-mq.c572
1 files changed, 262 insertions, 310 deletions
diff --git a/block/blk-mq.c b/block/blk-mq.c
index f2224ffd225d..6cef42f419a5 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -37,12 +37,8 @@
#include "blk-wbt.h"
#include "blk-mq-sched.h"
-static DEFINE_MUTEX(all_q_mutex);
-static LIST_HEAD(all_q_list);
-
static void blk_mq_poll_stats_start(struct request_queue *q);
static void blk_mq_poll_stats_fn(struct blk_stat_callback *cb);
-static void __blk_mq_stop_hw_queues(struct request_queue *q, bool sync);
static int blk_mq_poll_stats_bkt(const struct request *rq)
{
@@ -154,13 +150,28 @@ void blk_mq_unfreeze_queue(struct request_queue *q)
}
EXPORT_SYMBOL_GPL(blk_mq_unfreeze_queue);
+/*
+ * FIXME: replace the scsi_internal_device_*block_nowait() calls in the
+ * mpt3sas driver such that this function can be removed.
+ */
+void blk_mq_quiesce_queue_nowait(struct request_queue *q)
+{
+ unsigned long flags;
+
+ spin_lock_irqsave(q->queue_lock, flags);
+ 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 queue_rq calls have finished
+ * 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. Additionally, it is not prevented that
- * new queue_rq() calls occur unless the queue has been stopped first.
+ * 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)
{
@@ -168,11 +179,11 @@ void blk_mq_quiesce_queue(struct request_queue *q)
unsigned int i;
bool rcu = false;
- __blk_mq_stop_hw_queues(q, true);
+ blk_mq_quiesce_queue_nowait(q);
queue_for_each_hw_ctx(q, hctx, i) {
if (hctx->flags & BLK_MQ_F_BLOCKING)
- synchronize_srcu(&hctx->queue_rq_srcu);
+ synchronize_srcu(hctx->queue_rq_srcu);
else
rcu = true;
}
@@ -181,6 +192,26 @@ void blk_mq_quiesce_queue(struct request_queue *q)
}
EXPORT_SYMBOL_GPL(blk_mq_quiesce_queue);
+/*
+ * blk_mq_unquiesce_queue() - counterpart of blk_mq_quiesce_queue()
+ * @q: request queue.
+ *
+ * This function recovers queue into the state before quiescing
+ * which is done by blk_mq_quiesce_queue.
+ */
+void blk_mq_unquiesce_queue(struct request_queue *q)
+{
+ unsigned long flags;
+
+ spin_lock_irqsave(q->queue_lock, flags);
+ queue_flag_clear(QUEUE_FLAG_QUIESCED, q);
+ spin_unlock_irqrestore(q->queue_lock, flags);
+
+ /* dispatch requests which are inserted during quiescing */
+ blk_mq_run_hw_queues(q, true);
+}
+EXPORT_SYMBOL_GPL(blk_mq_unquiesce_queue);
+
void blk_mq_wake_waiters(struct request_queue *q)
{
struct blk_mq_hw_ctx *hctx;
@@ -204,15 +235,33 @@ bool blk_mq_can_queue(struct blk_mq_hw_ctx *hctx)
}
EXPORT_SYMBOL(blk_mq_can_queue);
-void blk_mq_rq_ctx_init(struct request_queue *q, struct blk_mq_ctx *ctx,
- struct request *rq, unsigned int op)
+static struct request *blk_mq_rq_ctx_init(struct blk_mq_alloc_data *data,
+ unsigned int tag, unsigned int op)
{
+ struct blk_mq_tags *tags = blk_mq_tags_from_data(data);
+ struct request *rq = tags->static_rqs[tag];
+
+ rq->rq_flags = 0;
+
+ 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;
+ data->hctx->tags->rqs[rq->tag] = rq;
+ }
+
INIT_LIST_HEAD(&rq->queuelist);
/* csd/requeue_work/fifo_time is initialized before use */
- rq->q = q;
- rq->mq_ctx = ctx;
+ rq->q = data->q;
+ rq->mq_ctx = data->ctx;
rq->cmd_flags = op;
- if (blk_queue_io_stat(q))
+ if (blk_queue_io_stat(data->q))
rq->rq_flags |= RQF_IO_STAT;
/* do not touch atomic flags, it needs atomic ops against the timer */
rq->cpu = -1;
@@ -241,44 +290,60 @@ void blk_mq_rq_ctx_init(struct request_queue *q, struct blk_mq_ctx *ctx,
rq->end_io_data = NULL;
rq->next_rq = NULL;
- ctx->rq_dispatched[op_is_sync(op)]++;
+ data->ctx->rq_dispatched[op_is_sync(op)]++;
+ return rq;
}
-EXPORT_SYMBOL_GPL(blk_mq_rq_ctx_init);
-struct request *__blk_mq_alloc_request(struct blk_mq_alloc_data *data,
- unsigned int op)
+static struct request *blk_mq_get_request(struct request_queue *q,
+ struct bio *bio, unsigned int op,
+ struct blk_mq_alloc_data *data)
{
+ struct elevator_queue *e = q->elevator;
struct request *rq;
unsigned int tag;
- tag = blk_mq_get_tag(data);
- if (tag != BLK_MQ_TAG_FAIL) {
- struct blk_mq_tags *tags = blk_mq_tags_from_data(data);
+ blk_queue_enter_live(q);
+ data->q = q;
+ if (likely(!data->ctx))
+ data->ctx = blk_mq_get_ctx(q);
+ if (likely(!data->hctx))
+ data->hctx = blk_mq_map_queue(q, data->ctx->cpu);
+ if (op & REQ_NOWAIT)
+ data->flags |= BLK_MQ_REQ_NOWAIT;
- rq = tags->static_rqs[tag];
+ if (e) {
+ data->flags |= BLK_MQ_REQ_INTERNAL;
- 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;
- data->hctx->tags->rqs[rq->tag] = rq;
- }
+ /*
+ * Flush requests are special and go directly to the
+ * dispatch list.
+ */
+ if (!op_is_flush(op) && e->type->ops.mq.limit_depth)
+ e->type->ops.mq.limit_depth(op, data);
+ }
- blk_mq_rq_ctx_init(data->q, data->ctx, rq, op);
- return rq;
+ tag = blk_mq_get_tag(data);
+ if (tag == BLK_MQ_TAG_FAIL) {
+ blk_queue_exit(q);
+ return NULL;
}
- return NULL;
+ rq = blk_mq_rq_ctx_init(data, tag, op);
+ if (!op_is_flush(op)) {
+ rq->elv.icq = NULL;
+ if (e && e->type->ops.mq.prepare_request) {
+ if (e->type->icq_cache && rq_ioc(bio))
+ blk_mq_sched_assign_ioc(rq, bio);
+
+ e->type->ops.mq.prepare_request(rq, bio);
+ rq->rq_flags |= RQF_ELVPRIV;
+ }
+ }
+ data->hctx->queued++;
+ return rq;
}
-EXPORT_SYMBOL_GPL(__blk_mq_alloc_request);
-struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
+struct request *blk_mq_alloc_request(struct request_queue *q, unsigned int op,
unsigned int flags)
{
struct blk_mq_alloc_data alloc_data = { .flags = flags };
@@ -289,7 +354,7 @@ struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
if (ret)
return ERR_PTR(ret);
- rq = blk_mq_sched_get_request(q, NULL, rw, &alloc_data);
+ rq = blk_mq_get_request(q, NULL, op, &alloc_data);
blk_mq_put_ctx(alloc_data.ctx);
blk_queue_exit(q);
@@ -304,8 +369,8 @@ struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
}
EXPORT_SYMBOL(blk_mq_alloc_request);
-struct request *blk_mq_alloc_request_hctx(struct request_queue *q, int rw,
- unsigned int flags, unsigned int hctx_idx)
+struct request *blk_mq_alloc_request_hctx(struct request_queue *q,
+ unsigned int op, unsigned int flags, unsigned int hctx_idx)
{
struct blk_mq_alloc_data alloc_data = { .flags = flags };
struct request *rq;
@@ -340,7 +405,7 @@ struct request *blk_mq_alloc_request_hctx(struct request_queue *q, int rw,
cpu = cpumask_first(alloc_data.hctx->cpumask);
alloc_data.ctx = __blk_mq_get_ctx(q, cpu);
- rq = blk_mq_sched_get_request(q, NULL, rw, &alloc_data);
+ rq = blk_mq_get_request(q, NULL, op, &alloc_data);
blk_queue_exit(q);
@@ -351,17 +416,28 @@ struct request *blk_mq_alloc_request_hctx(struct request_queue *q, int rw,
}
EXPORT_SYMBOL_GPL(blk_mq_alloc_request_hctx);
-void __blk_mq_finish_request(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
- struct request *rq)
+void blk_mq_free_request(struct request *rq)
{
- const int sched_tag = rq->internal_tag;
struct request_queue *q = rq->q;
+ struct elevator_queue *e = q->elevator;
+ struct blk_mq_ctx *ctx = rq->mq_ctx;
+ struct blk_mq_hw_ctx *hctx = blk_mq_map_queue(q, ctx->cpu);
+ const int sched_tag = rq->internal_tag;
+
+ if (rq->rq_flags & RQF_ELVPRIV) {
+ if (e && e->type->ops.mq.finish_request)
+ e->type->ops.mq.finish_request(rq);
+ if (rq->elv.icq) {
+ put_io_context(rq->elv.icq->ioc);
+ rq->elv.icq = NULL;
+ }
+ }
+ ctx->rq_completed[rq_is_sync(rq)]++;
if (rq->rq_flags & RQF_MQ_INFLIGHT)
atomic_dec(&hctx->nr_active);
wbt_done(q->rq_wb, &rq->issue_stat);
- rq->rq_flags = 0;
clear_bit(REQ_ATOM_STARTED, &rq->atomic_flags);
clear_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags);
@@ -372,29 +448,9 @@ void __blk_mq_finish_request(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
blk_mq_sched_restart(hctx);
blk_queue_exit(q);
}
-
-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_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_finish_request);
-
-void blk_mq_free_request(struct request *rq)
-{
- blk_mq_sched_put_request(rq);
-}
EXPORT_SYMBOL_GPL(blk_mq_free_request);
-inline void __blk_mq_end_request(struct request *rq, int error)
+inline void __blk_mq_end_request(struct request *rq, blk_status_t error)
{
blk_account_io_done(rq);
@@ -409,7 +465,7 @@ inline void __blk_mq_end_request(struct request *rq, int error)
}
EXPORT_SYMBOL(__blk_mq_end_request);
-void blk_mq_end_request(struct request *rq, int error)
+void blk_mq_end_request(struct request *rq, blk_status_t error)
{
if (blk_update_request(rq, error, blk_rq_bytes(rq)))
BUG();
@@ -753,50 +809,6 @@ static void blk_mq_timeout_work(struct work_struct *work)
blk_queue_exit(q);
}
-/*
- * Reverse check our software queue for entries that we could potentially
- * merge with. Currently includes a hand-wavy stop count of 8, to not spend
- * too much time checking for merges.
- */
-static bool blk_mq_attempt_merge(struct request_queue *q,
- struct blk_mq_ctx *ctx, struct bio *bio)
-{
- struct request *rq;
- int checked = 8;
-
- list_for_each_entry_reverse(rq, &ctx->rq_list, queuelist) {
- bool merged = false;
-
- if (!checked--)
- break;
-
- if (!blk_rq_merge_ok(rq, bio))
- continue;
-
- switch (blk_try_merge(rq, bio)) {
- case ELEVATOR_BACK_MERGE:
- if (blk_mq_sched_allow_merge(q, rq, bio))
- merged = bio_attempt_back_merge(q, rq, bio);
- break;
- case ELEVATOR_FRONT_MERGE:
- if (blk_mq_sched_allow_merge(q, rq, bio))
- merged = bio_attempt_front_merge(q, rq, bio);
- break;
- case ELEVATOR_DISCARD_MERGE:
- merged = bio_attempt_discard_merge(q, rq, bio);
- break;
- default:
- continue;
- }
-
- if (merged)
- ctx->rq_merged++;
- return merged;
- }
-
- return false;
-}
-
struct flush_busy_ctx_data {
struct blk_mq_hw_ctx *hctx;
struct list_head *list;
@@ -926,14 +938,14 @@ static bool reorder_tags_to_front(struct list_head *list)
return first != NULL;
}
-static int blk_mq_dispatch_wake(wait_queue_t *wait, unsigned mode, int flags,
+static int blk_mq_dispatch_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
void *key)
{
struct blk_mq_hw_ctx *hctx;
hctx = container_of(wait, struct blk_mq_hw_ctx, dispatch_wait);
- list_del(&wait->task_list);
+ list_del(&wait->entry);
clear_bit_unlock(BLK_MQ_S_TAG_WAITING, &hctx->state);
blk_mq_run_hw_queue(hctx, true);
return 1;
@@ -968,7 +980,7 @@ bool blk_mq_dispatch_rq_list(struct request_queue *q, struct list_head *list)
{
struct blk_mq_hw_ctx *hctx;
struct request *rq;
- int errors, queued, ret = BLK_MQ_RQ_QUEUE_OK;
+ int errors, queued;
if (list_empty(list))
return false;
@@ -979,6 +991,7 @@ bool blk_mq_dispatch_rq_list(struct request_queue *q, struct list_head *list)
errors = queued = 0;
do {
struct blk_mq_queue_data bd;
+ blk_status_t ret;
rq = list_first_entry(list, struct request, queuelist);
if (!blk_mq_get_driver_tag(rq, &hctx, false)) {
@@ -1019,25 +1032,20 @@ bool blk_mq_dispatch_rq_list(struct request_queue *q, struct list_head *list)
}
ret = q->mq_ops->queue_rq(hctx, &bd);
- switch (ret) {
- case BLK_MQ_RQ_QUEUE_OK:
- queued++;
- break;
- case BLK_MQ_RQ_QUEUE_BUSY:
+ if (ret == BLK_STS_RESOURCE) {
blk_mq_put_driver_tag_hctx(hctx, rq);
list_add(&rq->queuelist, list);
__blk_mq_requeue_request(rq);
break;
- default:
- pr_err("blk-mq: bad return on queue: %d\n", ret);
- case BLK_MQ_RQ_QUEUE_ERROR:
+ }
+
+ if (unlikely(ret != BLK_STS_OK)) {
errors++;
- blk_mq_end_request(rq, -EIO);
- break;
+ blk_mq_end_request(rq, BLK_STS_IOERR);
+ continue;
}
- if (ret == BLK_MQ_RQ_QUEUE_BUSY)
- break;
+ queued++;
} while (!list_empty(list));
hctx->dispatched[queued_to_index(queued)]++;
@@ -1075,7 +1083,7 @@ bool blk_mq_dispatch_rq_list(struct request_queue *q, struct list_head *list)
* - blk_mq_run_hw_queue() checks whether or not a queue has
* been stopped before rerunning a queue.
* - Some but not all block drivers stop a queue before
- * returning BLK_MQ_RQ_QUEUE_BUSY. Two exceptions are scsi-mq
+ * returning BLK_STS_RESOURCE. Two exceptions are scsi-mq
* and dm-rq.
*/
if (!blk_mq_sched_needs_restart(hctx) &&
@@ -1100,9 +1108,9 @@ static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
} else {
might_sleep();
- srcu_idx = srcu_read_lock(&hctx->queue_rq_srcu);
+ srcu_idx = srcu_read_lock(hctx->queue_rq_srcu);
blk_mq_sched_dispatch_requests(hctx);
- srcu_read_unlock(&hctx->queue_rq_srcu, srcu_idx);
+ srcu_read_unlock(hctx->queue_rq_srcu, srcu_idx);
}
}
@@ -1134,8 +1142,10 @@ static int blk_mq_hctx_next_cpu(struct blk_mq_hw_ctx *hctx)
static void __blk_mq_delay_run_hw_queue(struct blk_mq_hw_ctx *hctx, bool async,
unsigned long msecs)
{
- if (unlikely(blk_mq_hctx_stopped(hctx) ||
- !blk_mq_hw_queue_mapped(hctx)))
+ if (WARN_ON_ONCE(!blk_mq_hw_queue_mapped(hctx)))
+ return;
+
+ if (unlikely(blk_mq_hctx_stopped(hctx)))
return;
if (!async && !(hctx->flags & BLK_MQ_F_BLOCKING)) {
@@ -1201,34 +1211,39 @@ bool blk_mq_queue_stopped(struct request_queue *q)
}
EXPORT_SYMBOL(blk_mq_queue_stopped);
-static void __blk_mq_stop_hw_queue(struct blk_mq_hw_ctx *hctx, bool sync)
+/*
+ * This function is often used for pausing .queue_rq() by driver when
+ * there isn't enough resource or some conditions aren't satisfied, and
+ * BLK_MQ_RQ_QUEUE_BUSY is usually returned.
+ *
+ * We do not guarantee that dispatch can be drained or blocked
+ * after blk_mq_stop_hw_queue() returns. Please use
+ * blk_mq_quiesce_queue() for that requirement.
+ */
+void blk_mq_stop_hw_queue(struct blk_mq_hw_ctx *hctx)
{
- if (sync)
- cancel_delayed_work_sync(&hctx->run_work);
- else
- cancel_delayed_work(&hctx->run_work);
+ cancel_delayed_work(&hctx->run_work);
set_bit(BLK_MQ_S_STOPPED, &hctx->state);
}
-
-void blk_mq_stop_hw_queue(struct blk_mq_hw_ctx *hctx)
-{
- __blk_mq_stop_hw_queue(hctx, false);
-}
EXPORT_SYMBOL(blk_mq_stop_hw_queue);
-static void __blk_mq_stop_hw_queues(struct request_queue *q, bool sync)
+/*
+ * This function is often used for pausing .queue_rq() by driver when
+ * there isn't enough resource or some conditions aren't satisfied, and
+ * BLK_MQ_RQ_QUEUE_BUSY is usually returned.
+ *
+ * We do not guarantee that dispatch can be drained or blocked
+ * after blk_mq_stop_hw_queues() returns. Please use
+ * blk_mq_quiesce_queue() for that requirement.
+ */
+void blk_mq_stop_hw_queues(struct request_queue *q)
{
struct blk_mq_hw_ctx *hctx;
int i;
queue_for_each_hw_ctx(q, hctx, i)
- __blk_mq_stop_hw_queue(hctx, sync);
-}
-
-void blk_mq_stop_hw_queues(struct request_queue *q)
-{
- __blk_mq_stop_hw_queues(q, false);
+ blk_mq_stop_hw_queue(hctx);
}
EXPORT_SYMBOL(blk_mq_stop_hw_queues);
@@ -1295,7 +1310,7 @@ static void blk_mq_run_work_fn(struct work_struct *work)
void blk_mq_delay_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs)
{
- if (unlikely(!blk_mq_hw_queue_mapped(hctx)))
+ if (WARN_ON_ONCE(!blk_mq_hw_queue_mapped(hctx)))
return;
/*
@@ -1317,6 +1332,8 @@ static inline void __blk_mq_insert_req_list(struct blk_mq_hw_ctx *hctx,
{
struct blk_mq_ctx *ctx = rq->mq_ctx;
+ lockdep_assert_held(&ctx->lock);
+
trace_block_rq_insert(hctx->queue, rq);
if (at_head)
@@ -1330,6 +1347,8 @@ void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx, struct request *rq,
{
struct blk_mq_ctx *ctx = rq->mq_ctx;
+ lockdep_assert_held(&ctx->lock);
+
__blk_mq_insert_req_list(hctx, rq, at_head);
blk_mq_hctx_mark_pending(hctx, ctx);
}
@@ -1427,30 +1446,13 @@ static inline bool hctx_allow_merges(struct blk_mq_hw_ctx *hctx)
!blk_queue_nomerges(hctx->queue);
}
-static inline bool blk_mq_merge_queue_io(struct blk_mq_hw_ctx *hctx,
- struct blk_mq_ctx *ctx,
- struct request *rq, struct bio *bio)
+static inline void blk_mq_queue_io(struct blk_mq_hw_ctx *hctx,
+ struct blk_mq_ctx *ctx,
+ struct request *rq)
{
- if (!hctx_allow_merges(hctx) || !bio_mergeable(bio)) {
- blk_mq_bio_to_request(rq, bio);
- spin_lock(&ctx->lock);
-insert_rq:
- __blk_mq_insert_request(hctx, rq, false);
- spin_unlock(&ctx->lock);
- return false;
- } else {
- struct request_queue *q = hctx->queue;
-
- spin_lock(&ctx->lock);
- if (!blk_mq_attempt_merge(q, ctx, bio)) {
- blk_mq_bio_to_request(rq, bio);
- goto insert_rq;
- }
-
- spin_unlock(&ctx->lock);
- __blk_mq_finish_request(hctx, ctx, rq);
- return true;
- }
+ spin_lock(&ctx->lock);
+ __blk_mq_insert_request(hctx, rq, false);
+ spin_unlock(&ctx->lock);
}
static blk_qc_t request_to_qc_t(struct blk_mq_hw_ctx *hctx, struct request *rq)
@@ -1461,22 +1463,29 @@ static blk_qc_t request_to_qc_t(struct blk_mq_hw_ctx *hctx, struct request *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,
- bool may_sleep)
+static void __blk_mq_try_issue_directly(struct blk_mq_hw_ctx *hctx,
+ struct request *rq,
+ blk_qc_t *cookie, bool may_sleep)
{
struct request_queue *q = rq->q;
struct blk_mq_queue_data bd = {
.rq = rq,
.last = true,
};
- struct blk_mq_hw_ctx *hctx;
blk_qc_t new_cookie;
- int ret;
+ blk_status_t ret;
+ bool run_queue = true;
+
+ /* RCU or SRCU read lock is needed before checking quiesced flag */
+ if (blk_mq_hctx_stopped(hctx) || blk_queue_quiesced(q)) {
+ run_queue = false;
+ goto insert;
+ }
if (q->elevator)
goto insert;
- if (!blk_mq_get_driver_tag(rq, &hctx, false))
+ if (!blk_mq_get_driver_tag(rq, NULL, false))
goto insert;
new_cookie = request_to_qc_t(hctx, rq);
@@ -1487,20 +1496,21 @@ static void __blk_mq_try_issue_directly(struct request *rq, blk_qc_t *cookie,
* would have done
*/
ret = q->mq_ops->queue_rq(hctx, &bd);
- if (ret == BLK_MQ_RQ_QUEUE_OK) {
+ switch (ret) {
+ case BLK_STS_OK:
*cookie = new_cookie;
return;
- }
-
- if (ret == BLK_MQ_RQ_QUEUE_ERROR) {
+ case BLK_STS_RESOURCE:
+ __blk_mq_requeue_request(rq);
+ goto insert;
+ default:
*cookie = BLK_QC_T_NONE;
- blk_mq_end_request(rq, -EIO);
+ blk_mq_end_request(rq, ret);
return;
}
- __blk_mq_requeue_request(rq);
insert:
- blk_mq_sched_insert_request(rq, false, true, false, may_sleep);
+ blk_mq_sched_insert_request(rq, false, run_queue, false, may_sleep);
}
static void blk_mq_try_issue_directly(struct blk_mq_hw_ctx *hctx,
@@ -1508,16 +1518,16 @@ static void blk_mq_try_issue_directly(struct blk_mq_hw_ctx *hctx,
{
if (!(hctx->flags & BLK_MQ_F_BLOCKING)) {
rcu_read_lock();
- __blk_mq_try_issue_directly(rq, cookie, false);
+ __blk_mq_try_issue_directly(hctx, rq, cookie, false);
rcu_read_unlock();
} else {
unsigned int srcu_idx;
might_sleep();
- srcu_idx = srcu_read_lock(&hctx->queue_rq_srcu);
- __blk_mq_try_issue_directly(rq, cookie, true);
- srcu_read_unlock(&hctx->queue_rq_srcu, srcu_idx);
+ srcu_idx = srcu_read_lock(hctx->queue_rq_srcu);
+ __blk_mq_try_issue_directly(hctx, rq, cookie, true);
+ srcu_read_unlock(hctx->queue_rq_srcu, srcu_idx);
}
}
@@ -1535,7 +1545,7 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
blk_queue_bounce(q, &bio);
- blk_queue_split(q, &bio, q->bio_split);
+ blk_queue_split(q, &bio);
if (bio_integrity_enabled(bio) && bio_integrity_prep(bio)) {
bio_io_error(bio);
@@ -1553,9 +1563,11 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
trace_block_getrq(q, bio, bio->bi_opf);
- rq = blk_mq_sched_get_request(q, bio, bio->bi_opf, &data);
+ rq = blk_mq_get_request(q, bio, bio->bi_opf, &data);
if (unlikely(!rq)) {
__wbt_done(q->rq_wb, wb_acct);
+ if (bio->bi_opf & REQ_NOWAIT)
+ bio_wouldblock_error(bio);
return BLK_QC_T_NONE;
}
@@ -1619,9 +1631,12 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
blk_mq_put_ctx(data.ctx);
- if (same_queue_rq)
+ if (same_queue_rq) {
+ data.hctx = blk_mq_map_queue(q,
+ same_queue_rq->mq_ctx->cpu);
blk_mq_try_issue_directly(data.hctx, same_queue_rq,
&cookie);
+ }
} else if (q->nr_hw_queues > 1 && is_sync) {
blk_mq_put_ctx(data.ctx);
blk_mq_bio_to_request(rq, bio);
@@ -1630,11 +1645,12 @@ static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
blk_mq_put_ctx(data.ctx);
blk_mq_bio_to_request(rq, bio);
blk_mq_sched_insert_request(rq, false, true, true, true);
- } else if (!blk_mq_merge_queue_io(data.hctx, data.ctx, rq, bio)) {
+ } else {
blk_mq_put_ctx(data.ctx);
+ blk_mq_bio_to_request(rq, bio);
+ blk_mq_queue_io(data.hctx, data.ctx, rq);
blk_mq_run_hw_queue(data.hctx, true);
- } else
- blk_mq_put_ctx(data.ctx);
+ }
return cookie;
}
@@ -1857,7 +1873,7 @@ static void blk_mq_exit_hctx(struct request_queue *q,
set->ops->exit_hctx(hctx, hctx_idx);
if (hctx->flags & BLK_MQ_F_BLOCKING)
- cleanup_srcu_struct(&hctx->queue_rq_srcu);
+ cleanup_srcu_struct(hctx->queue_rq_srcu);
blk_mq_remove_cpuhp(hctx);
blk_free_flush_queue(hctx->fq);
@@ -1891,7 +1907,6 @@ static int blk_mq_init_hctx(struct request_queue *q,
spin_lock_init(&hctx->lock);
INIT_LIST_HEAD(&hctx->dispatch);
hctx->queue = q;
- hctx->queue_num = hctx_idx;
hctx->flags = set->flags & ~BLK_MQ_F_TAG_SHARED;
cpuhp_state_add_instance_nocalls(CPUHP_BLK_MQ_DEAD, &hctx->cpuhp_dead);
@@ -1930,7 +1945,7 @@ static int blk_mq_init_hctx(struct request_queue *q,
goto free_fq;
if (hctx->flags & BLK_MQ_F_BLOCKING)
- init_srcu_struct(&hctx->queue_rq_srcu);
+ init_srcu_struct(hctx->queue_rq_srcu);
blk_mq_debugfs_register_hctx(q, hctx);
@@ -1966,8 +1981,8 @@ static void blk_mq_init_cpu_queues(struct request_queue *q,
INIT_LIST_HEAD(&__ctx->rq_list);
__ctx->queue = q;
- /* If the cpu isn't online, the cpu is mapped to first hctx */
- if (!cpu_online(i))
+ /* If the cpu isn't present, the cpu is mapped to first hctx */
+ if (!cpu_present(i))
continue;
hctx = blk_mq_map_queue(q, i);
@@ -2010,8 +2025,7 @@ static void blk_mq_free_map_and_requests(struct blk_mq_tag_set *set,
}
}
-static void blk_mq_map_swqueue(struct request_queue *q,
- const struct cpumask *online_mask)
+static void blk_mq_map_swqueue(struct request_queue *q)
{
unsigned int i, hctx_idx;
struct blk_mq_hw_ctx *hctx;
@@ -2029,13 +2043,11 @@ static void blk_mq_map_swqueue(struct request_queue *q,
}
/*
- * Map software to hardware queues
+ * Map software to hardware queues.
+ *
+ * If the cpu isn't present, the cpu is mapped to first hctx.
*/
- for_each_possible_cpu(i) {
- /* If the cpu isn't online, the cpu is mapped to first hctx */
- if (!cpumask_test_cpu(i, online_mask))
- continue;
-
+ for_each_present_cpu(i) {
hctx_idx = q->mq_map[i];
/* unmapped hw queue can be remapped after CPU topo changed */
if (!set->tags[hctx_idx] &&
@@ -2094,20 +2106,30 @@ static void blk_mq_map_swqueue(struct request_queue *q,
}
}
+/*
+ * Caller needs to ensure that we're either frozen/quiesced, or that
+ * the queue isn't live yet.
+ */
static void queue_set_hctx_shared(struct request_queue *q, bool shared)
{
struct blk_mq_hw_ctx *hctx;
int i;
queue_for_each_hw_ctx(q, hctx, i) {
- if (shared)
+ if (shared) {
+ if (test_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state))
+ atomic_inc(&q->shared_hctx_restart);
hctx->flags |= BLK_MQ_F_TAG_SHARED;
- else
+ } else {
+ if (test_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state))
+ atomic_dec(&q->shared_hctx_restart);
hctx->flags &= ~BLK_MQ_F_TAG_SHARED;
+ }
}
}
-static void blk_mq_update_tag_set_depth(struct blk_mq_tag_set *set, bool shared)
+static void blk_mq_update_tag_set_depth(struct blk_mq_tag_set *set,
+ bool shared)
{
struct request_queue *q;
@@ -2205,6 +2227,20 @@ struct request_queue *blk_mq_init_queue(struct blk_mq_tag_set *set)
}
EXPORT_SYMBOL(blk_mq_init_queue);
+static int blk_mq_hw_ctx_size(struct blk_mq_tag_set *tag_set)
+{
+ int hw_ctx_size = sizeof(struct blk_mq_hw_ctx);
+
+ BUILD_BUG_ON(ALIGN(offsetof(struct blk_mq_hw_ctx, queue_rq_srcu),
+ __alignof__(struct blk_mq_hw_ctx)) !=
+ sizeof(struct blk_mq_hw_ctx));
+
+ if (tag_set->flags & BLK_MQ_F_BLOCKING)
+ hw_ctx_size += sizeof(struct srcu_struct);
+
+ return hw_ctx_size;
+}
+
static void blk_mq_realloc_hw_ctxs(struct blk_mq_tag_set *set,
struct request_queue *q)
{
@@ -2219,7 +2255,7 @@ static void blk_mq_realloc_hw_ctxs(struct blk_mq_tag_set *set,
continue;
node = blk_mq_hw_queue_to_node(q->mq_map, i);
- hctxs[i] = kzalloc_node(sizeof(struct blk_mq_hw_ctx),
+ hctxs[i] = kzalloc_node(blk_mq_hw_ctx_size(set),
GFP_KERNEL, node);
if (!hctxs[i])
break;
@@ -2321,16 +2357,8 @@ struct request_queue *blk_mq_init_allocated_queue(struct blk_mq_tag_set *set,
blk_queue_softirq_done(q, set->ops->complete);
blk_mq_init_cpu_queues(q, set->nr_hw_queues);
-
- get_online_cpus();
- mutex_lock(&all_q_mutex);
-
- list_add_tail(&q->all_q_node, &all_q_list);
blk_mq_add_queue_tag_set(set, q);
- blk_mq_map_swqueue(q, cpu_online_mask);
-
- mutex_unlock(&all_q_mutex);
- put_online_cpus();
+ blk_mq_map_swqueue(q);
if (!(set->flags & BLK_MQ_F_NO_SCHED)) {
int ret;
@@ -2356,18 +2384,12 @@ void blk_mq_free_queue(struct request_queue *q)
{
struct blk_mq_tag_set *set = q->tag_set;
- mutex_lock(&all_q_mutex);
- list_del_init(&q->all_q_node);
- mutex_unlock(&all_q_mutex);
-
blk_mq_del_queue_tag_set(q);
-
blk_mq_exit_hw_queues(q, set, set->nr_hw_queues);
}
/* Basically redo blk_mq_init_queue with queue frozen */
-static void blk_mq_queue_reinit(struct request_queue *q,
- const struct cpumask *online_mask)
+static void blk_mq_queue_reinit(struct request_queue *q)
{
WARN_ON_ONCE(!atomic_read(&q->mq_freeze_depth));
@@ -2380,76 +2402,12 @@ static void blk_mq_queue_reinit(struct request_queue *q,
* involves free and re-allocate memory, worthy doing?)
*/
- blk_mq_map_swqueue(q, online_mask);
+ blk_mq_map_swqueue(q);
blk_mq_sysfs_register(q);
blk_mq_debugfs_register_hctxs(q);
}
-/*
- * New online cpumask which is going to be set in this hotplug event.
- * Declare this cpumasks as global as cpu-hotplug operation is invoked
- * one-by-one and dynamically allocating this could result in a failure.
- */
-static struct cpumask cpuhp_online_new;
-
-static void blk_mq_queue_reinit_work(void)
-{
- struct request_queue *q;
-
- mutex_lock(&all_q_mutex);
- /*
- * We need to freeze and reinit all existing queues. Freezing
- * involves synchronous wait for an RCU grace period and doing it
- * one by one may take a long time. Start freezing all queues in
- * one swoop and then wait for the completions so that freezing can
- * take place in parallel.
- */
- list_for_each_entry(q, &all_q_list, all_q_node)
- blk_freeze_queue_start(q);
- list_for_each_entry(q, &all_q_list, all_q_node)
- blk_mq_freeze_queue_wait(q);
-
- list_for_each_entry(q, &all_q_list, all_q_node)
- blk_mq_queue_reinit(q, &cpuhp_online_new);
-
- list_for_each_entry(q, &all_q_list, all_q_node)
- blk_mq_unfreeze_queue(q);
-
- mutex_unlock(&all_q_mutex);
-}
-
-static int blk_mq_queue_reinit_dead(unsigned int cpu)
-{
- cpumask_copy(&cpuhp_online_new, cpu_online_mask);
- blk_mq_queue_reinit_work();
- return 0;
-}
-
-/*
- * Before hotadded cpu starts handling requests, new mappings must be
- * established. Otherwise, these requests in hw queue might never be
- * dispatched.
- *
- * For example, there is a single hw queue (hctx) and two CPU queues (ctx0
- * for CPU0, and ctx1 for CPU1).
- *
- * 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, 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)
-{
- cpumask_copy(&cpuhp_online_new, cpu_online_mask);
- cpumask_set_cpu(cpu, &cpuhp_online_new);
- blk_mq_queue_reinit_work();
- return 0;
-}
-
static int __blk_mq_alloc_rq_maps(struct blk_mq_tag_set *set)
{
int i;
@@ -2641,7 +2599,8 @@ int blk_mq_update_nr_requests(struct request_queue *q, unsigned int nr)
return ret;
}
-void blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set, int nr_hw_queues)
+static void __blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set,
+ int nr_hw_queues)
{
struct request_queue *q;
@@ -2659,12 +2618,19 @@ void blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set, int nr_hw_queues)
blk_mq_update_queue_map(set);
list_for_each_entry(q, &set->tag_list, tag_set_list) {
blk_mq_realloc_hw_ctxs(set, q);
- blk_mq_queue_reinit(q, cpu_online_mask);
+ blk_mq_queue_reinit(q);
}
list_for_each_entry(q, &set->tag_list, tag_set_list)
blk_mq_unfreeze_queue(q);
}
+
+void blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set, int nr_hw_queues)
+{
+ mutex_lock(&set->tag_list_lock);
+ __blk_mq_update_nr_hw_queues(set, nr_hw_queues);
+ mutex_unlock(&set->tag_list_lock);
+}
EXPORT_SYMBOL_GPL(blk_mq_update_nr_hw_queues);
/* Enable polling stats and return whether they were already enabled. */
@@ -2868,24 +2834,10 @@ bool blk_mq_poll(struct request_queue *q, blk_qc_t cookie)
}
EXPORT_SYMBOL_GPL(blk_mq_poll);
-void blk_mq_disable_hotplug(void)
-{
- mutex_lock(&all_q_mutex);
-}
-
-void blk_mq_enable_hotplug(void)
-{
- mutex_unlock(&all_q_mutex);
-}
-
static int __init blk_mq_init(void)
{
cpuhp_setup_state_multi(CPUHP_BLK_MQ_DEAD, "block/mq:dead", NULL,
blk_mq_hctx_notify_dead);
-
- cpuhp_setup_state_nocalls(CPUHP_BLK_MQ_PREPARE, "block/mq:prepare",
- blk_mq_queue_reinit_prepare,
- blk_mq_queue_reinit_dead);
return 0;
}
subsys_initcall(blk_mq_init);