blk-mq: first cut deadline scheduling mq-deadline
authorJens Axboe <axboe@fb.com>
Thu, 19 Feb 2015 22:26:43 +0000 (14:26 -0800)
committerJens Axboe <axboe@fb.com>
Thu, 19 Feb 2015 22:26:43 +0000 (14:26 -0800)
Option for per-hw queue or all hw queue scheduling. Some rough edges
around restart of queue(s) that need to be looked at some more.
Additionally, it probably makes sense to track which queues should be
restarted, so we don't just blindly restart all of them. This could
unify ATOM_RESTART and ATOM_RESTART_ALL to be the same operation,
since both would look at what queues to restart.

Current tracking is done utilizing the blk-mq tag iterators. Both
queued and started requests are accounted for. We don't dispatch a
request if older requests haven't been dispatched yet.

When queueing is invoked, we always dispatch (at least) one request.
This may violate deadlines slightly. Currently this is done to avoid
further complicating the restart logic. Right now we know that a queue
will dispatch at least one request, if restarted and has requests
pending.

sysfs attributes are currently per-hw queue, should probably tie them
together for some of the cases.

It does read vs write deadlines, not sync vs async. Could make that
an option.

Signed-off-by: Jens Axboe <axboe@fb.com>
block/blk-mq-sysfs.c
block/blk-mq-tag.c
block/blk-mq.c
block/blk.h
drivers/block/mtip32xx/mtip32xx.c
drivers/block/null_blk.c
drivers/block/nvme-core.c
drivers/scsi/scsi_lib.c
include/linux/blk-mq.h
include/linux/blkdev.h

index 1630a20d5dcfa550ebe9c8815927d51b70bd9d56..1c434ad9022966f40859a07298e7645b35320403 100644 (file)
@@ -235,6 +235,49 @@ static ssize_t blk_mq_hw_sysfs_cpus_show(struct blk_mq_hw_ctx *hctx, char *page)
        return ret;
 }
 
+static ssize_t blk_mq_hw_sysfs_dead_show(u64 val, char *page)
+{
+       return sprintf(page, "%llu\n", (unsigned long long) val);
+}
+
+static ssize_t blk_mq_hw_sysfs_dead_store(u64 *val, const char *page,
+                                         size_t length)
+{
+       unsigned long long store;
+       int err;
+
+       err = kstrtoull(page, 10, &store);
+       if (err)
+               return -EINVAL;
+
+       *val = store;
+       return length;
+}
+
+static ssize_t blk_mq_hw_sysfs_rdead_show(struct blk_mq_hw_ctx *hctx,
+                                         char *page)
+{
+       return blk_mq_hw_sysfs_dead_show(hctx->fifo_usec[0], page);
+}
+
+static ssize_t blk_mq_hw_sysfs_rdead_store(struct blk_mq_hw_ctx *hctx,
+                                          const char *page, size_t length)
+{
+       return blk_mq_hw_sysfs_dead_store(&hctx->fifo_usec[0], page, length);
+}
+
+static ssize_t blk_mq_hw_sysfs_wdead_show(struct blk_mq_hw_ctx *hctx,
+                                         char *page)
+{
+       return blk_mq_hw_sysfs_dead_show(hctx->fifo_usec[1], page);
+}
+
+static ssize_t blk_mq_hw_sysfs_wdead_store(struct blk_mq_hw_ctx *hctx,
+                                          const char *page, size_t length)
+{
+       return blk_mq_hw_sysfs_dead_store(&hctx->fifo_usec[1], page, length);
+}
+
 static struct blk_mq_ctx_sysfs_entry blk_mq_sysfs_dispatched = {
        .attr = {.name = "dispatched", .mode = S_IRUGO },
        .show = blk_mq_sysfs_dispatched_show,
@@ -288,6 +331,16 @@ static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_cpus = {
        .attr = {.name = "cpu_list", .mode = S_IRUGO },
        .show = blk_mq_hw_sysfs_cpus_show,
 };
+static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_rdead = {
+       .attr = {.name = "read_deadline_us", .mode = S_IRUGO | S_IWUSR },
+       .show = blk_mq_hw_sysfs_rdead_show,
+       .store = blk_mq_hw_sysfs_rdead_store,
+};
+static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_wdead = {
+       .attr = {.name = "write_deadline_us", .mode = S_IRUGO | S_IWUSR },
+       .show = blk_mq_hw_sysfs_wdead_show,
+       .store = blk_mq_hw_sysfs_wdead_store,
+};
 
 static struct attribute *default_hw_ctx_attrs[] = {
        &blk_mq_hw_sysfs_queued.attr,
@@ -300,6 +353,12 @@ static struct attribute *default_hw_ctx_attrs[] = {
        NULL,
 };
 
+static struct attribute *sched_hw_attrs[] = {
+       &blk_mq_hw_sysfs_rdead.attr,
+       &blk_mq_hw_sysfs_wdead.attr,
+       NULL,
+};
+
 static const struct sysfs_ops blk_mq_sysfs_ops = {
        .show   = blk_mq_sysfs_show,
        .store  = blk_mq_sysfs_store,
@@ -360,6 +419,19 @@ static int blk_mq_register_hctx(struct blk_mq_hw_ctx *hctx)
                        break;
        }
 
+       if (ret)
+               return ret;
+
+       if (hctx->flags & BLK_MQ_F_DEADLINE) {
+               struct attribute *attr;
+
+               for (i = 0; (attr = sched_hw_attrs[i]) != NULL; i++) {
+                       ret = sysfs_create_file(&hctx->kobj, attr);
+                       if (ret)
+                               break;
+               }
+       }
+
        return ret;
 }
 
index d53a764b05eacde776a4454054795d1c0277d676..c10fceecf4d6050b8bade683b59f358cb3764476 100644 (file)
@@ -428,8 +428,10 @@ static void bt_for_each(struct blk_mq_hw_ctx *hctx,
                     bit < bm->depth;
                     bit = find_next_bit(&bm->word, bm->depth, bit + 1)) {
                        rq = blk_mq_tag_to_rq(hctx->tags, off + bit);
-                       if (rq->q == hctx->queue)
-                               fn(hctx, rq, data, reserved);
+                       if (rq->q != hctx->queue)
+                               continue;
+                       if (fn(hctx, rq, data, reserved))
+                               break;
                }
 
                off += (1 << bt->bits_per_word);
index 4f4bea21052e41068112ead8cbb4e0a42cb7a9d6..45e3a2b4e3205568d06eefee9903b6a554d307da 100644 (file)
@@ -33,7 +33,7 @@ static DEFINE_MUTEX(all_q_mutex);
 static LIST_HEAD(all_q_list);
 
 static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx);
-static void blk_mq_run_queues(struct request_queue *q);
+static void blk_mq_run_queues(struct request_queue *q, bool async);
 
 /*
  * Check if any of the ctx's have pending work in this hardware queue
@@ -42,6 +42,9 @@ static bool blk_mq_hctx_has_pending(struct blk_mq_hw_ctx *hctx)
 {
        unsigned int i;
 
+       if (!list_empty_careful(&hctx->dispatch))
+               return true;
+
        for (i = 0; i < hctx->ctx_map.map_size; i++)
                if (hctx->ctx_map.map[i].word)
                        return true;
@@ -118,7 +121,7 @@ void blk_mq_freeze_queue_start(struct request_queue *q)
 
        if (freeze) {
                percpu_ref_kill(&q->mq_usage_counter);
-               blk_mq_run_queues(q);
+               blk_mq_run_queues(q, false);
        }
 }
 EXPORT_SYMBOL_GPL(blk_mq_freeze_queue_start);
@@ -298,6 +301,7 @@ static void __blk_mq_free_request(struct blk_mq_hw_ctx *hctx,
        rq->cmd_flags = 0;
 
        clear_bit(REQ_ATOM_STARTED, &rq->atomic_flags);
+       clear_bit(REQ_ATOM_FIFO, &rq->atomic_flags);
        blk_mq_put_tag(hctx, tag, &ctx->last_tag);
        blk_mq_queue_exit(q);
 }
@@ -305,10 +309,28 @@ static void __blk_mq_free_request(struct blk_mq_hw_ctx *hctx,
 void blk_mq_free_hctx_request(struct blk_mq_hw_ctx *hctx, struct request *rq)
 {
        struct blk_mq_ctx *ctx = rq->mq_ctx;
+       bool need_start = false, need_start_all = false;
 
        ctx->rq_completed[rq_is_sync(rq)]++;
+
+       /*
+        * If RESTART is set, we originally stopped dispatching IO because
+        * of this request. Kick the the queue back into gear.
+        */
+       if (test_bit(REQ_ATOM_RESTART_ALL, &rq->atomic_flags)) {
+               clear_bit(REQ_ATOM_RESTART_ALL, &rq->atomic_flags);
+               need_start_all = true;
+       } else if (test_bit(REQ_ATOM_RESTART, &rq->atomic_flags)) {
+               clear_bit(REQ_ATOM_RESTART, &rq->atomic_flags);
+               need_start = true;
+       }
+
        __blk_mq_free_request(hctx, ctx, rq);
 
+       if (need_start_all)
+               blk_mq_run_queues(hctx->queue, true);
+       else if (need_start)
+               blk_mq_run_hw_queue(hctx, true);
 }
 EXPORT_SYMBOL_GPL(blk_mq_free_hctx_request);
 
@@ -622,8 +644,8 @@ void blk_mq_rq_timed_out(struct request *req, bool reserved)
        }
 }
 
-static void blk_mq_check_expired(struct blk_mq_hw_ctx *hctx,
-               struct request *rq, void *priv, bool reserved)
+static int blk_mq_check_expired(struct blk_mq_hw_ctx *hctx, struct request *rq,
+                               void *priv, bool reserved)
 {
        struct blk_mq_timeout_data *data = priv;
 
@@ -636,10 +658,10 @@ static void blk_mq_check_expired(struct blk_mq_hw_ctx *hctx,
                        rq->errors = -EIO;
                        blk_mq_complete_request(rq);
                }
-               return;
+               return 0;
        }
        if (rq->cmd_flags & REQ_NO_TIMEOUT)
-               return;
+               return 0;
 
        if (time_after_eq(jiffies, rq->deadline)) {
                if (!blk_mark_rq_complete(rq))
@@ -648,6 +670,8 @@ static void blk_mq_check_expired(struct blk_mq_hw_ctx *hctx,
                data->next = rq->deadline;
                data->next_set = 1;
        }
+
+       return 0;
 }
 
 static void blk_mq_rq_timer(unsigned long priv)
@@ -753,6 +777,128 @@ static void flush_busy_ctxs(struct blk_mq_hw_ctx *hctx, struct list_head *list)
        }
 }
 
+static int rq_deadline_cmp(void *priv, struct list_head *a, struct list_head *b)
+{
+       struct request *rqa = container_of(a, struct request, queuelist);
+       struct request *rqb = container_of(b, struct request, queuelist);
+
+       if (rqa->fifo_usec < rqb->fifo_usec)
+               return -1;
+       else if (rqa->fifo_usec > rqb->fifo_usec)
+               return 1;
+
+       return 0;
+}
+
+struct deadline_data {
+       /*
+        * Input
+        */
+       struct request *rq;
+       bool local;
+
+       /*
+        * Output
+        */
+       bool may_dispatch;
+};
+
+static int deadline_iter_fn(struct blk_mq_hw_ctx *hctx, struct request *rq,
+                           void *data, bool reserved)
+{
+       struct deadline_data *d = data;
+       const int bit = d->local ? REQ_ATOM_RESTART : REQ_ATOM_RESTART_ALL;
+
+       /*
+        * Ignore reserved requests, that's mostly error handling. And ignore
+        * non filesystem requests.
+        */
+       if (reserved)
+               return 1;
+       if (rq->cmd_type != REQ_TYPE_FS)
+               return 0;
+
+       /*
+        * Complete, we don't care. We could potentially check for STARTED
+        * as well. Right now we ignore that bit, which means we look at all
+        * requests, both the ones that are queued up and the ones that have
+        * been issued to hw as well. If FIFO is set, the request is both
+        * allocated and has a valid ->fifo_usec set.
+        */
+       if (test_bit(REQ_ATOM_COMPLETE, &rq->atomic_flags) ||
+           !test_bit(REQ_ATOM_FIFO, &rq->atomic_flags))
+               return 0;
+
+       if (d->rq->fifo_usec <= rq->fifo_usec)
+               return 0;
+
+       /*
+        * We're newer than 'rq', so defer dispatch of this request. Mark
+        * 'rq' as restarting the queue when it completes.
+        */
+       set_bit(bit, &rq->atomic_flags);
+
+       /*
+        * If we didn't race on completion, stop here.
+        */
+       if (!test_bit(REQ_ATOM_COMPLETE, &rq->atomic_flags)) {
+               d->may_dispatch = false;
+               return 1;
+       }
+
+       /*
+        * Request completed meanwhile, continue our search.
+        */
+       clear_bit(bit, &rq->atomic_flags);
+       return 0;
+}
+
+static bool blk_mq_deadline_may_dispatch(struct blk_mq_hw_ctx *hctx,
+                                        struct request *rq)
+{
+       struct blk_mq_hw_ctx *__hctx;
+       struct deadline_data d;
+       int i;
+
+       if (!(hctx->flags & BLK_MQ_F_DEADLINE))
+               return true;
+       if (rq->cmd_type != REQ_TYPE_FS)
+               return true;
+
+       /*
+        * Iterate requests that are queued or inflight, disallow submission
+        * of current 'rq' if have requests older than ->fifo_usec pending.
+        */
+       d.rq = rq;
+       d.local = true;
+       d.may_dispatch = true;
+       blk_mq_tag_busy_iter(hctx, deadline_iter_fn, &d);
+
+       /*
+        * On disallow or for single queue cases, or for cases where we
+        * only deadline schedule per hardware queue, we are done.
+        */
+       if (!d.may_dispatch || hctx->queue->nr_hw_queues == 1 ||
+           !(hctx->flags & BLK_MQ_F_DEADLINE_ALL))
+               return d.may_dispatch;
+
+       /*
+        * We are allowed dispatch from our local hw queue. Check others
+        * and see if that's also the case.
+        */
+       d.local = false;
+       queue_for_each_hw_ctx(hctx->queue, __hctx, i) {
+               if (__hctx == hctx || !blk_mq_hw_queue_mapped(__hctx))
+                       continue;
+
+               blk_mq_tag_busy_iter(__hctx, deadline_iter_fn, &d);
+               if (!d.may_dispatch)
+                       break;
+       }
+
+       return d.may_dispatch;
+}
+
 /*
  * Run this hardware queue, pulling any software queues mapped to it in.
  * Note that this function currently has various problems around ordering
@@ -762,6 +908,7 @@ static void flush_busy_ctxs(struct blk_mq_hw_ctx *hctx, struct list_head *list)
 static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
 {
        struct request_queue *q = hctx->queue;
+       const int use_deadline = hctx->flags & BLK_MQ_F_DEADLINE;
        struct request *rq;
        LIST_HEAD(rq_list);
        LIST_HEAD(driver_list);
@@ -791,6 +938,9 @@ static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
                spin_unlock(&hctx->lock);
        }
 
+       if (use_deadline)
+               list_sort(NULL, &rq_list, rq_deadline_cmp);
+
        /*
         * Start off with dptr being NULL, so we start the first request
         * immediately, even if we have more pending.
@@ -806,11 +956,20 @@ static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
                int ret;
 
                rq = list_first_entry(&rq_list, struct request, queuelist);
-               list_del_init(&rq->queuelist);
 
+               if (queued && !blk_mq_deadline_may_dispatch(hctx, rq))
+                       break;
+
+               list_del_init(&rq->queuelist);
                bd.rq = rq;
                bd.list = dptr;
-               bd.last = list_empty(&rq_list);
+
+               /*
+                * If we're deadline scheduling, we don't want to hit the
+                * case of not having passed 'last == true' in case we
+                * decide to stop dispatching.
+                */
+               bd.last = list_empty(&rq_list) || (use_deadline && queued);
 
                ret = q->mq_ops->queue_rq(hctx, &bd);
                switch (ret) {
@@ -904,18 +1063,17 @@ void blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx, bool async)
                        &hctx->run_work, 0);
 }
 
-static void blk_mq_run_queues(struct request_queue *q)
+static void blk_mq_run_queues(struct request_queue *q, bool async)
 {
        struct blk_mq_hw_ctx *hctx;
        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) ||
                    test_bit(BLK_MQ_S_STOPPED, &hctx->state))
                        continue;
 
-               blk_mq_run_hw_queue(hctx, false);
+               blk_mq_run_hw_queue(hctx, async);
        }
 }
 
@@ -999,6 +1157,23 @@ void blk_mq_delay_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs)
 }
 EXPORT_SYMBOL(blk_mq_delay_queue);
 
+static void blk_mq_set_fifo_time(struct blk_mq_hw_ctx *hctx,
+                                struct request *rq, bool at_head)
+{
+       if (!(hctx->flags & BLK_MQ_F_DEADLINE))
+               return;
+
+       /*
+        * TODO: at_head is now not at head, if we have requests older
+        * than this pending.
+        */
+       rq->fifo_usec = ktime_to_us(ktime_get());
+       if (!at_head)
+               rq->fifo_usec += hctx->fifo_usec[rq_data_dir(rq)];
+
+       set_bit(REQ_ATOM_FIFO, &rq->atomic_flags);
+}
+
 static void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx,
                                    struct request *rq, bool at_head)
 {
@@ -1011,6 +1186,7 @@ static void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx,
        else
                list_add_tail(&rq->queuelist, &ctx->rq_list);
 
+       blk_mq_set_fifo_time(hctx, rq, at_head);
        blk_mq_hctx_mark_pending(hctx, ctx);
 }
 
@@ -1251,11 +1427,13 @@ static void blk_mq_make_request(struct request_queue *q, struct bio *bio)
        }
 
        /*
-        * If the driver supports defer issued based on 'last', then
-        * queue it up like normal since we can potentially save some
-        * CPU this way.
+        * If the driver supports defer issued based on 'last' or has
+        * deadline scheduling enabled, then queue it up like normal. This is
+        * needed for deadline to work, and for deferred issue we can
+        * potentially save some CPU this way.
         */
-       if (is_sync && !(data.hctx->flags & BLK_MQ_F_DEFER_ISSUE)) {
+       if (is_sync &&
+           !(data.hctx->flags & (BLK_MQ_F_DEFER_ISSUE | BLK_MQ_F_DEADLINE))) {
                struct blk_mq_queue_data bd = {
                        .rq = rq,
                        .list = NULL,
@@ -1665,6 +1843,11 @@ static int blk_mq_init_hctx(struct request_queue *q,
        hctx->queue_num = hctx_idx;
        hctx->flags = set->flags;
 
+       if (hctx->flags & BLK_MQ_F_DEADLINE) {
+               hctx->fifo_usec[0] = 100;
+               hctx->fifo_usec[1] = 1000;
+       }
+
        blk_mq_init_cpu_notifier(&hctx->cpu_notifier,
                                        blk_mq_hctx_notify, hctx);
        blk_mq_register_cpu_notifier(&hctx->cpu_notifier);
index 43b036185712c25f66d04a3c9e49565297c94ccd..ce2c7a7d50f53aed08a4d4a4a86d154cc2e14c01 100644 (file)
@@ -90,6 +90,9 @@ void blk_account_io_done(struct request *req);
 enum rq_atomic_flags {
        REQ_ATOM_COMPLETE = 0,
        REQ_ATOM_STARTED,
+       REQ_ATOM_FIFO,
+       REQ_ATOM_RESTART,
+       REQ_ATOM_RESTART_ALL,
 };
 
 /*
index 3bd7ca9853a8411aaf1401c5482bea8da974ac60..72bc5394f9ea7050d2e09830988369b90d93dc50 100644 (file)
 #include <linux/prefetch.h>
 #include "mtip32xx.h"
 
+static bool use_deadline = false;
+module_param(use_deadline, bool, S_IRUGO);
+MODULE_PARM_DESC(use_deadline, "Tell blk-mq to deadline schedule. Default: false");
+
 #define HW_CMD_SLOT_SZ         (MTIP_MAX_COMMAND_SLOTS * 32)
 
 /* DMA region containing RX Fis, Identify, RLE10, and SMART buffers */
@@ -3916,6 +3920,8 @@ skip_create_disk:
        dd->tags.cmd_size = sizeof(struct mtip_cmd);
        dd->tags.numa_node = dd->numa_node;
        dd->tags.flags = BLK_MQ_F_SHOULD_MERGE;
+       if (use_deadline)
+               dd->tags.flags |= BLK_MQ_F_DEADLINE;
        dd->tags.driver_data = dd;
 
        rv = blk_mq_alloc_tag_set(&dd->tags);
index 65cd61a4145ed2049944621c50b374cf742041ca..16c6e6704739c6d1e0810b807c6bf28accec1143 100644 (file)
@@ -147,6 +147,10 @@ static bool use_per_node_hctx = false;
 module_param(use_per_node_hctx, bool, S_IRUGO);
 MODULE_PARM_DESC(use_per_node_hctx, "Use per-node allocation for hardware context queues. Default: false");
 
+static bool use_deadline = false;
+module_param(use_deadline, bool, S_IRUGO);
+MODULE_PARM_DESC(use_deadline, "Tell blk-mq to deadline schedule. Default: false");
+
 static void put_tag(struct nullb_queue *nq, unsigned int tag)
 {
        clear_bit_unlock(tag, nq->tag_map);
@@ -523,6 +527,8 @@ static int null_add_dev(void)
                nullb->tag_set.numa_node = home_node;
                nullb->tag_set.cmd_size = sizeof(struct nullb_cmd);
                nullb->tag_set.flags = BLK_MQ_F_SHOULD_MERGE;
+               if (use_deadline)
+                       nullb->tag_set.flags |= BLK_MQ_F_DEADLINE;
                nullb->tag_set.driver_data = nullb;
 
                rv = blk_mq_alloc_tag_set(&nullb->tag_set);
index cbdfbbf983927e85a4a83d94d20047f2fadf6357..4b7c720db06f32ea8fccabbbf5ff9cfcc20cc467 100644 (file)
@@ -1124,8 +1124,8 @@ static void nvme_abort_req(struct request *req)
        }
 }
 
-static void nvme_cancel_queue_ios(struct blk_mq_hw_ctx *hctx,
-                               struct request *req, void *data, bool reserved)
+static int nvme_cancel_queue_ios(struct blk_mq_hw_ctx *hctx,
+                                struct request *req, void *data, bool reserved)
 {
        struct nvme_queue *nvmeq = data;
        void *ctx;
@@ -1134,12 +1134,12 @@ static void nvme_cancel_queue_ios(struct blk_mq_hw_ctx *hctx,
        struct nvme_completion cqe;
 
        if (!blk_mq_request_started(req))
-               return;
+               return 0;
 
        cmd = blk_mq_rq_to_pdu(req);
 
        if (cmd->ctx == CMD_CTX_CANCELLED)
-               return;
+               return 0;
 
        if (blk_queue_dying(req->q))
                cqe.status = cpu_to_le16((NVME_SC_ABORT_REQ | NVME_SC_DNR) << 1);
@@ -1151,6 +1151,7 @@ static void nvme_cancel_queue_ios(struct blk_mq_hw_ctx *hctx,
                                                req->tag, nvmeq->qid);
        ctx = cancel_cmd_info(cmd, &fn);
        fn(nvmeq, ctx, &cqe);
+       return 0;
 }
 
 static enum blk_eh_timer_return nvme_timeout(struct request *req, bool reserved)
index 54d7a6cbb98a48d06b932cce40c488488760da08..7fb6379c57a4766efe20f4a9d670605742afb6ce 100644 (file)
@@ -2196,7 +2196,7 @@ int scsi_mq_setup_tags(struct Scsi_Host *shost)
        shost->tag_set.queue_depth = shost->can_queue;
        shost->tag_set.cmd_size = cmd_size;
        shost->tag_set.numa_node = NUMA_NO_NODE;
-       shost->tag_set.flags = BLK_MQ_F_SHOULD_MERGE | BLK_MQ_F_SG_MERGE;
+       shost->tag_set.flags = BLK_MQ_F_SHOULD_MERGE | BLK_MQ_F_SG_MERGE | BLK_MQ_F_DEADLINE_ALL;
        shost->tag_set.flags |=
                BLK_ALLOC_POLICY_TO_MQ_FLAG(shost->hostt->tag_alloc_policy);
        shost->tag_set.driver_data = shost;
index 7aec861273357d71371b3064845f61904059ebf0..677690bf0e1e5d04bed028191b418495c803ddcc 100644 (file)
@@ -31,6 +31,8 @@ struct blk_mq_hw_ctx {
        int                     next_cpu;
        int                     next_cpu_batch;
 
+       u64                     fifo_usec[2];
+
        unsigned long           flags;          /* BLK_MQ_F_* flags */
 
        struct request_queue    *queue;
@@ -94,7 +96,7 @@ typedef int (init_request_fn)(void *, struct request *, unsigned int,
 typedef void (exit_request_fn)(void *, struct request *, unsigned int,
                unsigned int);
 
-typedef void (busy_iter_fn)(struct blk_mq_hw_ctx *, struct request *, void *,
+typedef int (busy_iter_fn)(struct blk_mq_hw_ctx *, struct request *, void *,
                bool);
 
 struct blk_mq_ops {
@@ -146,6 +148,9 @@ enum {
        BLK_MQ_F_SG_MERGE       = 1 << 2,
        BLK_MQ_F_SYSFS_UP       = 1 << 3,
        BLK_MQ_F_DEFER_ISSUE    = 1 << 4,
+       BLK_MQ_F_DEADLINE       = 1 << 5,
+       BLK_MQ_F_DEADLINE_ALL   = (1 << 6) | BLK_MQ_F_DEADLINE,
+
        BLK_MQ_F_ALLOC_POLICY_START_BIT = 8,
        BLK_MQ_F_ALLOC_POLICY_BITS = 1,
 
index 7f9a516f24dec57182f51cff3580db3904208a84..b30276e65b5fb17a8811c08529200804d4b92f50 100644 (file)
@@ -102,6 +102,7 @@ struct request {
        union {
                struct call_single_data csd;
                unsigned long fifo_time;
+               u64 fifo_usec;
        };
 
        struct request_queue *q;