summaryrefslogtreecommitdiffstats
path: root/block
diff options
context:
space:
mode:
authorJens Axboe <axboe@fb.com>2017-01-17 06:03:22 -0700
committerJens Axboe <axboe@fb.com>2017-01-17 10:04:20 -0700
commitbd166ef183c263c5ced656d49ef19c7da4adc774 (patch)
tree449bbd3b4e671b370b96e3846b2281116e7089e9 /block
parent2af8cbe30531eca73c8f3ba277f155fc0020b01a (diff)
blk-mq-sched: add framework for MQ capable IO schedulers
This adds a set of hooks that intercepts the blk-mq path of allocating/inserting/issuing/completing requests, allowing us to develop a scheduler within that framework. We reuse the existing elevator scheduler API on the registration side, but augment that with the scheduler flagging support for the blk-mq interfce, and with a separate set of ops hooks for MQ devices. We split driver and scheduler tags, so we can run the scheduling independently of device queue depth. Signed-off-by: Jens Axboe <axboe@fb.com> Reviewed-by: Bart Van Assche <bart.vanassche@sandisk.com> Reviewed-by: Omar Sandoval <osandov@fb.com>
Diffstat (limited to 'block')
-rw-r--r--block/Makefile2
-rw-r--r--block/blk-cgroup.c24
-rw-r--r--block/blk-core.c4
-rw-r--r--block/blk-exec.c3
-rw-r--r--block/blk-flush.c12
-rw-r--r--block/blk-ioc.c8
-rw-r--r--block/blk-merge.c2
-rw-r--r--block/blk-mq-sched.c368
-rw-r--r--block/blk-mq-sched.h170
-rw-r--r--block/blk-mq-sysfs.c13
-rw-r--r--block/blk-mq.c318
-rw-r--r--block/blk-mq.h8
-rw-r--r--block/blk-tag.c1
-rw-r--r--block/elevator.c204
14 files changed, 945 insertions, 192 deletions
diff --git a/block/Makefile b/block/Makefile
index a827f988c4e6..2eee9e1bb6db 100644
--- a/block/Makefile
+++ b/block/Makefile
@@ -6,7 +6,7 @@ obj-$(CONFIG_BLOCK) := bio.o elevator.o blk-core.o blk-tag.o blk-sysfs.o \
blk-flush.o blk-settings.o blk-ioc.o blk-map.o \
blk-exec.o blk-merge.o blk-softirq.o blk-timeout.o \
blk-lib.o blk-mq.o blk-mq-tag.o blk-stat.o \
- blk-mq-sysfs.o blk-mq-cpumap.o ioctl.o \
+ blk-mq-sysfs.o blk-mq-cpumap.o blk-mq-sched.o ioctl.o \
genhd.o scsi_ioctl.o partition-generic.o ioprio.o \
badblocks.o partitions/
diff --git a/block/blk-cgroup.c b/block/blk-cgroup.c
index 8ba0af780e88..2630f64bed19 100644
--- a/block/blk-cgroup.c
+++ b/block/blk-cgroup.c
@@ -1223,7 +1223,11 @@ int blkcg_activate_policy(struct request_queue *q,
if (blkcg_policy_enabled(q, pol))
return 0;
- blk_queue_bypass_start(q);
+ if (q->mq_ops) {
+ blk_mq_freeze_queue(q);
+ blk_mq_quiesce_queue(q);
+ } else
+ blk_queue_bypass_start(q);
pd_prealloc:
if (!pd_prealloc) {
pd_prealloc = pol->pd_alloc_fn(GFP_KERNEL, q->node);
@@ -1261,7 +1265,10 @@ pd_prealloc:
spin_unlock_irq(q->queue_lock);
out_bypass_end:
- blk_queue_bypass_end(q);
+ if (q->mq_ops)
+ blk_mq_unfreeze_queue(q);
+ else
+ blk_queue_bypass_end(q);
if (pd_prealloc)
pol->pd_free_fn(pd_prealloc);
return ret;
@@ -1284,7 +1291,12 @@ void blkcg_deactivate_policy(struct request_queue *q,
if (!blkcg_policy_enabled(q, pol))
return;
- blk_queue_bypass_start(q);
+ if (q->mq_ops) {
+ blk_mq_freeze_queue(q);
+ blk_mq_quiesce_queue(q);
+ } else
+ blk_queue_bypass_start(q);
+
spin_lock_irq(q->queue_lock);
__clear_bit(pol->plid, q->blkcg_pols);
@@ -1304,7 +1316,11 @@ void blkcg_deactivate_policy(struct request_queue *q,
}
spin_unlock_irq(q->queue_lock);
- blk_queue_bypass_end(q);
+
+ if (q->mq_ops)
+ blk_mq_unfreeze_queue(q);
+ else
+ blk_queue_bypass_end(q);
}
EXPORT_SYMBOL_GPL(blkcg_deactivate_policy);
diff --git a/block/blk-core.c b/block/blk-core.c
index 92baea07acbc..a61f1407f4f6 100644
--- a/block/blk-core.c
+++ b/block/blk-core.c
@@ -39,6 +39,7 @@
#include "blk.h"
#include "blk-mq.h"
+#include "blk-mq-sched.h"
#include "blk-wbt.h"
EXPORT_TRACEPOINT_SYMBOL_GPL(block_bio_remap);
@@ -134,6 +135,7 @@ void blk_rq_init(struct request_queue *q, struct request *rq)
rq->cmd = rq->__cmd;
rq->cmd_len = BLK_MAX_CDB;
rq->tag = -1;
+ rq->internal_tag = -1;
rq->start_time = jiffies;
set_start_time_ns(rq);
rq->part = NULL;
@@ -2127,7 +2129,7 @@ int blk_insert_cloned_request(struct request_queue *q, struct request *rq)
if (q->mq_ops) {
if (blk_queue_io_stat(q))
blk_account_io_start(rq, true);
- blk_mq_insert_request(rq, false, true, false);
+ blk_mq_sched_insert_request(rq, false, true, false);
return 0;
}
diff --git a/block/blk-exec.c b/block/blk-exec.c
index 3ecb00a6cf45..86656fdfa637 100644
--- a/block/blk-exec.c
+++ b/block/blk-exec.c
@@ -9,6 +9,7 @@
#include <linux/sched/sysctl.h>
#include "blk.h"
+#include "blk-mq-sched.h"
/*
* for max sense size
@@ -65,7 +66,7 @@ void blk_execute_rq_nowait(struct request_queue *q, struct gendisk *bd_disk,
* be reused after dying flag is set
*/
if (q->mq_ops) {
- blk_mq_insert_request(rq, at_head, true, false);
+ blk_mq_sched_insert_request(rq, at_head, true, false);
return;
}
diff --git a/block/blk-flush.c b/block/blk-flush.c
index 20b7c7a02f1c..d7de34ee39c2 100644
--- a/block/blk-flush.c
+++ b/block/blk-flush.c
@@ -74,6 +74,7 @@
#include "blk.h"
#include "blk-mq.h"
#include "blk-mq-tag.h"
+#include "blk-mq-sched.h"
/* FLUSH/FUA sequences */
enum {
@@ -391,9 +392,10 @@ static void mq_flush_data_end_io(struct request *rq, int error)
* the comment in flush_end_io().
*/
spin_lock_irqsave(&fq->mq_flush_lock, flags);
- if (blk_flush_complete_seq(rq, fq, REQ_FSEQ_DATA, error))
- blk_mq_run_hw_queue(hctx, true);
+ blk_flush_complete_seq(rq, fq, REQ_FSEQ_DATA, error);
spin_unlock_irqrestore(&fq->mq_flush_lock, flags);
+
+ blk_mq_run_hw_queue(hctx, true);
}
/**
@@ -453,9 +455,9 @@ void blk_insert_flush(struct request *rq)
*/
if ((policy & REQ_FSEQ_DATA) &&
!(policy & (REQ_FSEQ_PREFLUSH | REQ_FSEQ_POSTFLUSH))) {
- if (q->mq_ops) {
- blk_mq_insert_request(rq, false, true, false);
- } else
+ if (q->mq_ops)
+ blk_mq_sched_insert_request(rq, false, true, false);
+ else
list_add_tail(&rq->queuelist, &q->queue_head);
return;
}
diff --git a/block/blk-ioc.c b/block/blk-ioc.c
index ab372092a57d..fe186a9eade9 100644
--- a/block/blk-ioc.c
+++ b/block/blk-ioc.c
@@ -43,7 +43,9 @@ static void ioc_exit_icq(struct io_cq *icq)
if (icq->flags & ICQ_EXITED)
return;
- if (et->ops.sq.elevator_exit_icq_fn)
+ if (et->uses_mq && et->ops.mq.exit_icq)
+ et->ops.mq.exit_icq(icq);
+ else if (!et->uses_mq && et->ops.sq.elevator_exit_icq_fn)
et->ops.sq.elevator_exit_icq_fn(icq);
icq->flags |= ICQ_EXITED;
@@ -383,7 +385,9 @@ struct io_cq *ioc_create_icq(struct io_context *ioc, struct request_queue *q,
if (likely(!radix_tree_insert(&ioc->icq_tree, q->id, icq))) {
hlist_add_head(&icq->ioc_node, &ioc->icq_list);
list_add(&icq->q_node, &q->icq_list);
- if (et->ops.sq.elevator_init_icq_fn)
+ if (et->uses_mq && et->ops.mq.init_icq)
+ et->ops.mq.init_icq(icq);
+ else if (!et->uses_mq && et->ops.sq.elevator_init_icq_fn)
et->ops.sq.elevator_init_icq_fn(icq);
} else {
kmem_cache_free(et->icq_cache, icq);
diff --git a/block/blk-merge.c b/block/blk-merge.c
index 480570b691dc..6aa43dec5af4 100644
--- a/block/blk-merge.c
+++ b/block/blk-merge.c
@@ -763,7 +763,7 @@ int blk_attempt_req_merge(struct request_queue *q, struct request *rq,
{
struct elevator_queue *e = q->elevator;
- if (e->type->ops.sq.elevator_allow_rq_merge_fn)
+ if (!e->uses_mq && e->type->ops.sq.elevator_allow_rq_merge_fn)
if (!e->type->ops.sq.elevator_allow_rq_merge_fn(q, rq, next))
return 0;
diff --git a/block/blk-mq-sched.c b/block/blk-mq-sched.c
new file mode 100644
index 000000000000..26759798a0b3
--- /dev/null
+++ b/block/blk-mq-sched.c
@@ -0,0 +1,368 @@
+/*
+ * blk-mq scheduling framework
+ *
+ * Copyright (C) 2016 Jens Axboe
+ */
+#include <linux/kernel.h>
+#include <linux/module.h>
+#include <linux/blk-mq.h>
+
+#include <trace/events/block.h>
+
+#include "blk.h"
+#include "blk-mq.h"
+#include "blk-mq-sched.h"
+#include "blk-mq-tag.h"
+#include "blk-wbt.h"
+
+void blk_mq_sched_free_hctx_data(struct request_queue *q,
+ void (*exit)(struct blk_mq_hw_ctx *))
+{
+ struct blk_mq_hw_ctx *hctx;
+ int i;
+
+ queue_for_each_hw_ctx(q, hctx, i) {
+ if (exit && hctx->sched_data)
+ exit(hctx);
+ kfree(hctx->sched_data);
+ hctx->sched_data = NULL;
+ }
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_free_hctx_data);
+
+int blk_mq_sched_init_hctx_data(struct request_queue *q, size_t size,
+ int (*init)(struct blk_mq_hw_ctx *),
+ void (*exit)(struct blk_mq_hw_ctx *))
+{
+ struct blk_mq_hw_ctx *hctx;
+ int ret;
+ int i;
+
+ queue_for_each_hw_ctx(q, hctx, i) {
+ hctx->sched_data = kmalloc_node(size, GFP_KERNEL, hctx->numa_node);
+ if (!hctx->sched_data) {
+ ret = -ENOMEM;
+ goto error;
+ }
+
+ if (init) {
+ ret = init(hctx);
+ if (ret) {
+ /*
+ * We don't want to give exit() a partially
+ * initialized sched_data. init() must clean up
+ * if it fails.
+ */
+ kfree(hctx->sched_data);
+ hctx->sched_data = NULL;
+ goto error;
+ }
+ }
+ }
+
+ return 0;
+error:
+ blk_mq_sched_free_hctx_data(q, exit);
+ return ret;
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_init_hctx_data);
+
+static void __blk_mq_sched_assign_ioc(struct request_queue *q,
+ struct request *rq, struct io_context *ioc)
+{
+ struct io_cq *icq;
+
+ spin_lock_irq(q->queue_lock);
+ icq = ioc_lookup_icq(ioc, q);
+ spin_unlock_irq(q->queue_lock);
+
+ if (!icq) {
+ icq = ioc_create_icq(ioc, q, GFP_ATOMIC);
+ if (!icq)
+ return;
+ }
+
+ rq->elv.icq = icq;
+ if (!blk_mq_sched_get_rq_priv(q, rq)) {
+ rq->rq_flags |= RQF_ELVPRIV;
+ get_io_context(icq->ioc);
+ return;
+ }
+
+ rq->elv.icq = NULL;
+}
+
+static void blk_mq_sched_assign_ioc(struct request_queue *q,
+ struct request *rq, struct bio *bio)
+{
+ struct io_context *ioc;
+
+ ioc = rq_ioc(bio);
+ if (ioc)
+ __blk_mq_sched_assign_ioc(q, rq, ioc);
+}
+
+struct request *blk_mq_sched_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 blk_mq_hw_ctx *hctx;
+ struct blk_mq_ctx *ctx;
+ struct request *rq;
+ const bool is_flush = op & (REQ_PREFLUSH | REQ_FUA);
+
+ blk_queue_enter_live(q);
+ ctx = blk_mq_get_ctx(q);
+ hctx = blk_mq_map_queue(q, ctx->cpu);
+
+ blk_mq_set_alloc_data(data, q, 0, ctx, hctx);
+
+ if (e) {
+ data->flags |= BLK_MQ_REQ_INTERNAL;
+
+ /*
+ * Flush requests are special and go directly to the
+ * dispatch list.
+ */
+ if (!is_flush && e->type->ops.mq.get_request) {
+ rq = e->type->ops.mq.get_request(q, op, data);
+ if (rq)
+ rq->rq_flags |= RQF_QUEUED;
+ } else
+ rq = __blk_mq_alloc_request(data, op);
+ } else {
+ rq = __blk_mq_alloc_request(data, op);
+ data->hctx->tags->rqs[rq->tag] = rq;
+ }
+
+ if (rq) {
+ if (!is_flush) {
+ rq->elv.icq = NULL;
+ if (e && e->type->icq_cache)
+ blk_mq_sched_assign_ioc(q, rq, bio);
+ }
+ data->hctx->queued++;
+ return rq;
+ }
+
+ blk_queue_exit(q);
+ return NULL;
+}
+
+void blk_mq_sched_put_request(struct request *rq)
+{
+ struct request_queue *q = rq->q;
+ struct elevator_queue *e = q->elevator;
+
+ if (rq->rq_flags & RQF_ELVPRIV) {
+ blk_mq_sched_put_rq_priv(rq->q, rq);
+ if (rq->elv.icq) {
+ put_io_context(rq->elv.icq->ioc);
+ rq->elv.icq = NULL;
+ }
+ }
+
+ if ((rq->rq_flags & RQF_QUEUED) && e && e->type->ops.mq.put_request)
+ e->type->ops.mq.put_request(rq);
+ else
+ blk_mq_finish_request(rq);
+}
+
+void blk_mq_sched_dispatch_requests(struct blk_mq_hw_ctx *hctx)
+{
+ struct elevator_queue *e = hctx->queue->elevator;
+ LIST_HEAD(rq_list);
+
+ if (unlikely(blk_mq_hctx_stopped(hctx)))
+ return;
+
+ hctx->run++;
+
+ /*
+ * If we have previous entries on our dispatch list, grab them first 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);
+ }
+
+ /*
+ * Only ask the scheduler for requests, if we didn't have residual
+ * requests from the dispatch list. This is to avoid the case where
+ * we only ever dispatch a fraction of the requests available because
+ * of low device queue depth. Once we pull requests out of the IO
+ * scheduler, we can no longer merge or sort them. So it's best to
+ * leave them there for as long as we can. Mark the hw queue as
+ * needing a restart in that case.
+ */
+ if (list_empty(&rq_list)) {
+ if (e && e->type->ops.mq.dispatch_requests)
+ e->type->ops.mq.dispatch_requests(hctx, &rq_list);
+ else
+ blk_mq_flush_busy_ctxs(hctx, &rq_list);
+ } else
+ blk_mq_sched_mark_restart(hctx);
+
+ blk_mq_dispatch_rq_list(hctx, &rq_list);
+}
+
+void blk_mq_sched_move_to_dispatch(struct blk_mq_hw_ctx *hctx,
+ struct list_head *rq_list,
+ struct request *(*get_rq)(struct blk_mq_hw_ctx *))
+{
+ do {
+ struct request *rq;
+
+ rq = get_rq(hctx);
+ if (!rq)
+ break;
+
+ list_add_tail(&rq->queuelist, rq_list);
+ } while (1);
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_move_to_dispatch);
+
+bool blk_mq_sched_try_merge(struct request_queue *q, struct bio *bio)
+{
+ struct request *rq;
+ int ret;
+
+ ret = elv_merge(q, &rq, bio);
+ if (ret == ELEVATOR_BACK_MERGE) {
+ if (!blk_mq_sched_allow_merge(q, rq, bio))
+ return false;
+ if (bio_attempt_back_merge(q, rq, bio)) {
+ if (!attempt_back_merge(q, rq))
+ elv_merged_request(q, rq, ret);
+ return true;
+ }
+ } else if (ret == ELEVATOR_FRONT_MERGE) {
+ if (!blk_mq_sched_allow_merge(q, rq, bio))
+ return false;
+ if (bio_attempt_front_merge(q, rq, bio)) {
+ if (!attempt_front_merge(q, rq))
+ elv_merged_request(q, rq, ret);
+ return true;
+ }
+ }
+
+ return false;
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_try_merge);
+
+bool __blk_mq_sched_bio_merge(struct request_queue *q, struct bio *bio)
+{
+ struct elevator_queue *e = q->elevator;
+
+ if (e->type->ops.mq.bio_merge) {
+ struct blk_mq_ctx *ctx = blk_mq_get_ctx(q);
+ struct blk_mq_hw_ctx *hctx = blk_mq_map_queue(q, ctx->cpu);
+
+ blk_mq_put_ctx(ctx);
+ return e->type->ops.mq.bio_merge(hctx, bio);
+ }
+
+ return false;
+}
+
+bool blk_mq_sched_try_insert_merge(struct request_queue *q, struct request *rq)
+{
+ return rq_mergeable(rq) && elv_attempt_insert_merge(q, rq);
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_try_insert_merge);
+
+void blk_mq_sched_request_inserted(struct request *rq)
+{
+ trace_block_rq_insert(rq->q, rq);
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_request_inserted);
+
+bool blk_mq_sched_bypass_insert(struct blk_mq_hw_ctx *hctx, struct request *rq)
+{
+ if (rq->tag == -1) {
+ rq->rq_flags |= RQF_SORTED;
+ return false;
+ }
+
+ /*
+ * If we already have a real request tag, send directly to
+ * the dispatch list.
+ */
+ spin_lock(&hctx->lock);
+ list_add(&rq->queuelist, &hctx->dispatch);
+ spin_unlock(&hctx->lock);
+ return true;
+}
+EXPORT_SYMBOL_GPL(blk_mq_sched_bypass_insert);
+
+static void blk_mq_sched_free_tags(struct blk_mq_tag_set *set,
+ struct blk_mq_hw_ctx *hctx,
+ unsigned int hctx_idx)
+{
+ if (hctx->sched_tags) {
+ blk_mq_free_rqs(set, hctx->sched_tags, hctx_idx);
+ blk_mq_free_rq_map(hctx->sched_tags);
+ hctx->sched_tags = NULL;
+ }
+}
+
+int blk_mq_sched_setup(struct request_queue *q)
+{
+ struct blk_mq_tag_set *set = q->tag_set;
+ struct blk_mq_hw_ctx *hctx;
+ int ret, i;
+
+ /*
+ * Default to 256, since we don't split into sync/async like the
+ * old code did. Additionally, this is a per-hw queue depth.
+ */
+ q->nr_requests = 2 * BLKDEV_MAX_RQ;
+
+ /*
+ * We're switching to using an IO scheduler, so setup the hctx
+ * scheduler tags and switch the request map from the regular
+ * tags to scheduler tags. First allocate what we need, so we
+ * can safely fail and fallback, if needed.
+ */
+ ret = 0;
+ queue_for_each_hw_ctx(q, hctx, i) {
+ hctx->sched_tags = blk_mq_alloc_rq_map(set, i, q->nr_requests, 0);
+ if (!hctx->sched_tags) {
+ ret = -ENOMEM;
+ break;
+ }
+ ret = blk_mq_alloc_rqs(set, hctx->sched_tags, i, q->nr_requests);
+ if (ret)
+ break;
+ }
+
+ /*
+ * If we failed, free what we did allocate
+ */
+ if (ret) {
+ queue_for_each_hw_ctx(q, hctx, i) {
+ if (!hctx->sched_tags)
+ continue;
+ blk_mq_sched_free_tags(set, hctx, i);
+ }
+
+ return ret;
+ }
+
+ return 0;
+}
+
+void blk_mq_sched_teardown(struct request_queue *q)
+{
+ struct blk_mq_tag_set *set = q->tag_set;
+ struct blk_mq_hw_ctx *hctx;
+ int i;
+
+ queue_for_each_hw_ctx(q, hctx, i)
+ blk_mq_sched_free_tags(set, hctx, i);
+}
diff --git a/block/blk-mq-sched.h b/block/blk-mq-sched.h
new file mode 100644
index 000000000000..35c49e2e008a
--- /dev/null
+++ b/block/blk-mq-sched.h
@@ -0,0 +1,170 @@
+#ifndef BLK_MQ_SCHED_H
+#define BLK_MQ_SCHED_H
+
+#include "blk-mq.h"
+#include "blk-mq-tag.h"
+
+int blk_mq_sched_init_hctx_data(struct request_queue *q, size_t size,
+ int (*init)(struct blk_mq_hw_ctx *),
+ void (*exit)(struct blk_mq_hw_ctx *));
+
+void blk_mq_sched_free_hctx_data(struct request_queue *q,
+ void (*exit)(struct blk_mq_hw_ctx *));
+
+struct request *blk_mq_sched_get_request(struct request_queue *q, struct bio *bio, unsigned int op, struct blk_mq_alloc_data *data);
+void blk_mq_sched_put_request(struct request *rq);
+
+void blk_mq_sched_request_inserted(struct request *rq);
+bool blk_mq_sched_bypass_insert(struct blk_mq_hw_ctx *hctx, struct request *rq);
+bool blk_mq_sched_try_merge(struct request_queue *q, struct bio *bio);
+bool __blk_mq_sched_bio_merge(struct request_queue *q, struct bio *bio);
+bool blk_mq_sched_try_insert_merge(struct request_queue *q, struct request *rq);
+
+void blk_mq_sched_dispatch_requests(struct blk_mq_hw_ctx *hctx);
+void blk_mq_sched_move_to_dispatch(struct blk_mq_hw_ctx *hctx,
+ struct list_head *rq_list,
+ struct request *(*get_rq)(struct blk_mq_hw_ctx *));
+
+int blk_mq_sched_setup(struct request_queue *q);
+void blk_mq_sched_teardown(struct request_queue *q);
+
+static inline bool
+blk_mq_sched_bio_merge(struct request_queue *q, struct bio *bio)
+{
+ struct elevator_queue *e = q->elevator;
+
+ if (!e || blk_queue_nomerges(q) || !bio_mergeable(bio))
+ return false;
+
+ return __blk_mq_sched_bio_merge(q, bio);
+}
+
+static inline int blk_mq_sched_get_rq_priv(struct request_queue *q,
+ struct request *rq)
+{
+ struct elevator_queue *e = q->elevator;
+
+ if (e && e->type->ops.mq.get_rq_priv)
+ return e->type->ops.mq.get_rq_priv(q, rq);
+
+ return 0;
+}
+
+static inline void blk_mq_sched_put_rq_priv(struct request_queue *q,
+ struct request *rq)
+{
+ struct elevator_queue *e = q->elevator;
+
+ if (e && e->type->ops.mq.put_rq_priv)
+ e->type->ops.mq.put_rq_priv(q, rq);
+}
+
+static inline void
+blk_mq_sched_insert_request(struct request *rq, bool at_head, bool run_queue,
+ bool async)
+{
+ 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);
+
+ if (e && e->type->ops.mq.insert_requests) {
+ LIST_HEAD(list);
+
+ list_add(&rq->queuelist, &list);
+ e->type->ops.mq.insert_requests(hctx, &list, at_head);
+ } else {
+ 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 inline void
+blk_mq_sched_insert_requests(struct request_queue *q, struct blk_mq_ctx *ctx,
+ struct list_head *list, bool run_queue_async)
+{
+ struct blk_mq_hw_ctx *hctx = blk_mq_map_queue(q, ctx->cpu);
+ struct elevator_queue *e = hctx->queue->elevator;
+
+ if (e && e->type->ops.mq.insert_requests)
+ e->type->ops.mq.insert_requests(hctx, list, false);
+ else
+ blk_mq_insert_requests(hctx, ctx, list);
+
+ blk_mq_run_hw_queue(hctx, run_queue_async);
+}
+
+static inline bool
+blk_mq_sched_allow_merge(struct request_queue *q, struct request *rq,
+ struct bio *bio)
+{
+ struct elevator_queue *e = q->elevator;
+
+ if (e && e->type->ops.mq.allow_merge)
+ return e->type->ops.mq.allow_merge(q, rq, bio);
+
+ return true;
+}
+
+static inline void
+blk_mq_sched_completed_request(struct blk_mq_hw_ctx *hctx, struct request *rq)
+{
+ struct elevator_queue *e = hctx->queue->elevator;
+
+ if (e && e->type->ops.mq.completed_request)
+ e->type->ops.mq.completed_request(hctx, rq);
+
+ BUG_ON(rq->internal_tag == -1);
+
+ blk_mq_put_tag(hctx, hctx->sched_tags, rq->mq_ctx, rq->internal_tag);
+
+ if (test_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state)) {
+ clear_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state);
+ blk_mq_run_hw_queue(hctx, true);
+ }
+}
+
+static inline void blk_mq_sched_started_request(struct request *rq)
+{
+ struct request_queue *q = rq->q;
+ struct elevator_queue *e = q->elevator;
+
+ if (e && e->type->ops.mq.started_request)
+ e->type->ops.mq.started_request(rq);
+}
+
+static inline void blk_mq_sched_requeue_request(struct request *rq)
+{
+ struct request_queue *q = rq->q;
+ struct elevator_queue *e = q->elevator;
+
+ if (e && e->type->ops.mq.requeue_request)
+ e->type->ops.mq.requeue_request(rq);
+}
+
+static inline bool blk_mq_sched_has_work(struct blk_mq_hw_ctx *hctx)
+{
+ struct elevator_queue *e = hctx->queue->elevator;
+
+ if (e && e->type->ops.mq.has_work)
+ return e->type->ops.mq.has_work(hctx);
+
+ return false;
+}
+
+static inline void blk_mq_sched_mark_restart(struct blk_mq_hw_ctx *hctx)
+{
+ if (!test_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state))
+ set_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state);
+}
+
+static inline bool blk_mq_sched_needs_restart(struct blk_mq_hw_ctx *hctx)
+{
+ return test_bit(BLK_MQ_S_SCHED_RESTART, &hctx->state);
+}
+
+#endif
diff --git a/block/blk-mq-sysfs.c b/block/blk-mq-sysfs.c
index eacd3af72099..2caecaa98e40 100644
--- a/block/blk-mq-sysfs.c
+++ b/block/blk-mq-sysfs.c
@@ -231,6 +231,14 @@ static ssize_t blk_mq_hw_sysfs_rq_list_show(struct blk_mq_hw_ctx *hctx,
return ret;
}
+static ssize_t blk_mq_hw_sysfs_sched_tags_show(struct blk_mq_hw_ctx *hctx, char *page)
+{
+ if (hctx->sched_tags)
+ return blk_mq_tag_sysfs_show(hctx->sched_tags, page);
+
+ return 0;
+}
+
static ssize_t blk_mq_hw_sysfs_tags_show(struct blk_mq_hw_ctx *hctx, char *page)
{
return blk_mq_tag_sysfs_show(hctx->tags, page);
@@ -345,6 +353,10 @@ static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_pending = {
.attr = {.name = "pending", .mode = S_IRUGO },
.show = blk_mq_hw_sysfs_rq_list_show,
};
+static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_sched_tags = {
+ .attr = {.name = "sched_tags", .mode = S_IRUGO },
+ .show = blk_mq_hw_sysfs_sched_tags_show,
+};
static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_tags = {
.attr = {.name = "tags", .mode = S_IRUGO },
.show = blk_mq_hw_sysfs_tags_show,
@@ -370,6 +382,7 @@ static struct attribute *default_hw_ctx_attrs[] = {
&blk_mq_hw_sysfs_dispatched.attr,
&blk_mq_hw_sysfs_pending.attr,
&blk_mq_hw_sysfs_tags.attr,
+ &blk_mq_hw_sysfs_sched_tags.attr,
&blk_mq_hw_sysfs_cpus.attr,
&blk_mq_hw_sysfs_active.attr,
&blk_mq_hw_sysfs_poll.attr,
diff --git a/block/blk-mq.c b/block/blk-mq.c
index 89b81254201b..45e1707a9f86 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -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);
@@ -41,7 +42,9 @@ static LIST_HEAD(all_q_list);
*/
static 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);
}
/*
@@ -223,15 +226,23 @@ struct request *__blk_mq_alloc_request(struct blk_mq_alloc_data *data,
tag = blk_mq_get_tag(data);
if (tag != BLK_MQ_TAG_FAIL) {
- rq = data->hctx->tags->static_rqs[tag];
+ struct blk_mq_tags *tags = blk_mq_tags_from_data(data);
+
+ rq = tags->static_rqs[tag];
if (blk_mq_tag_busy(data->hctx)) {
rq->rq_flags = RQF_MQ_INFLIGHT;
atomic_inc(&data->hctx->nr_active);
}
- rq->tag = tag;
- data->hctx->tags->rqs[tag] = rq;
+ if (data->flags & BLK_MQ_REQ_INTERNAL) {
+ rq->tag = -1;
+ rq->internal_tag = tag;
+ } else {
+ rq->tag = tag;
+ rq->internal_tag = -1;
+ }
+
blk_mq_rq_ctx_init(data->q, data->ctx, rq, op);
return rq;
}
@@ -243,26 +254,21 @@ 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 request *rq;
struct blk_mq_alloc_data alloc_data;
+ struct request *rq;
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;
@@ -322,10 +328,10 @@ out_queue_exit:
}
EXPORT_SYMBOL_GPL(blk_mq_alloc_request_hctx);
-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)
@@ -336,22 +342,30 @@ void __blk_mq_free_request(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
clear_bit(REQ_ATOM_STARTED, &rq->atomic_flags);
clear_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags);
- blk_mq_put_tag(hctx, hctx->tags, 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_queue_exit(q);
}
-static void blk_mq_free_hctx_request(struct blk_mq_hw_ctx *hctx,
+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);
}
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);
@@ -469,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);
@@ -517,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))
@@ -551,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);
}
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);
}
blk_mq_run_hw_queues(q, false);
@@ -765,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++;
@@ -824,6 +847,59 @@ static inline unsigned int queued_to_index(unsigned int queued)
return min(BLK_MQ_MAX_DISPATCH_ORDER - 1, ilog2(queued) + 1);
}
+static 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,
+ .ctx = rq->mq_ctx,
+ .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) {
+ data.hctx->tags->rqs[rq->tag] = rq;
+ goto done;
+ }
+
+ return false;
+}
+
+/*
+ * 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;
@@ -846,6 +922,12 @@ 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;
+ blk_mq_sched_mark_restart(hctx);
+ break;
+ }
list_del_init(&rq->queuelist);