blk-mq: improve warning for running a queue on the wrong CPU
[linux-2.6-block.git] / block / blk-mq.c
index f9b9049b1284cc8adf65c1eed611e4c09d2d6584..13f5a6c1de76827c3aa2eaab28061581971e0c5c 100644 (file)
@@ -159,16 +159,17 @@ 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 rw_flags)
+                              struct request *rq, int op,
+                              unsigned int op_flags)
 {
        if (blk_queue_io_stat(q))
-               rw_flags |= REQ_IO_STAT;
+               op_flags |= REQ_IO_STAT;
 
        INIT_LIST_HEAD(&rq->queuelist);
        /* csd/requeue_work/fifo_time is initialized before use */
        rq->q = q;
        rq->mq_ctx = ctx;
-       rq->cmd_flags |= rw_flags;
+       req_set_op_attrs(rq, op, op_flags);
        /* do not touch atomic flags, it needs atomic ops against the timer */
        rq->cpu = -1;
        INIT_HLIST_NODE(&rq->hash);
@@ -203,11 +204,11 @@ static 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[rw_is_sync(rw_flags)]++;
+       ctx->rq_dispatched[rw_is_sync(op, op_flags)]++;
 }
 
 static struct request *
-__blk_mq_alloc_request(struct blk_mq_alloc_data *data, int rw)
+__blk_mq_alloc_request(struct blk_mq_alloc_data *data, int op, int op_flags)
 {
        struct request *rq;
        unsigned int tag;
@@ -222,7 +223,7 @@ __blk_mq_alloc_request(struct blk_mq_alloc_data *data, int rw)
                }
 
                rq->tag = tag;
-               blk_mq_rq_ctx_init(data->q, data->ctx, rq, rw);
+               blk_mq_rq_ctx_init(data->q, data->ctx, rq, op, op_flags);
                return rq;
        }
 
@@ -246,7 +247,7 @@ struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
        hctx = q->mq_ops->map_queue(q, ctx->cpu);
        blk_mq_set_alloc_data(&alloc_data, q, flags, ctx, hctx);
 
-       rq = __blk_mq_alloc_request(&alloc_data, rw);
+       rq = __blk_mq_alloc_request(&alloc_data, rw, 0);
        if (!rq && !(flags & BLK_MQ_REQ_NOWAIT)) {
                __blk_mq_run_hw_queue(hctx);
                blk_mq_put_ctx(ctx);
@@ -254,7 +255,7 @@ struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
                ctx = blk_mq_get_ctx(q);
                hctx = q->mq_ops->map_queue(q, ctx->cpu);
                blk_mq_set_alloc_data(&alloc_data, q, flags, ctx, hctx);
-               rq =  __blk_mq_alloc_request(&alloc_data, rw);
+               rq =  __blk_mq_alloc_request(&alloc_data, rw, 0);
                ctx = alloc_data.ctx;
        }
        blk_mq_put_ctx(ctx);
@@ -262,10 +263,53 @@ struct request *blk_mq_alloc_request(struct request_queue *q, int rw,
                blk_queue_exit(q);
                return ERR_PTR(-EWOULDBLOCK);
        }
+
+       rq->__data_len = 0;
+       rq->__sector = (sector_t) -1;
+       rq->bio = rq->biotail = NULL;
        return rq;
 }
 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 blk_mq_hw_ctx *hctx;
+       struct blk_mq_ctx *ctx;
+       struct request *rq;
+       struct blk_mq_alloc_data alloc_data;
+       int ret;
+
+       /*
+        * If the tag allocator sleeps we could get an allocation for a
+        * different hardware context.  No need to complicate the low level
+        * allocator for this for the rare use case of a command tied to
+        * a specific queue.
+        */
+       if (WARN_ON_ONCE(!(flags & BLK_MQ_REQ_NOWAIT)))
+               return ERR_PTR(-EINVAL);
+
+       if (hctx_idx >= q->nr_hw_queues)
+               return ERR_PTR(-EIO);
+
+       ret = blk_queue_enter(q, true);
+       if (ret)
+               return ERR_PTR(ret);
+
+       hctx = q->queue_hw_ctx[hctx_idx];
+       ctx = __blk_mq_get_ctx(q, cpumask_first(hctx->cpumask));
+
+       blk_mq_set_alloc_data(&alloc_data, q, flags, ctx, hctx);
+       rq = __blk_mq_alloc_request(&alloc_data, rw, 0);
+       if (!rq) {
+               blk_queue_exit(q);
+               return ERR_PTR(-EWOULDBLOCK);
+       }
+
+       return rq;
+}
+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)
 {
@@ -628,7 +672,20 @@ static void blk_mq_timeout_work(struct work_struct *work)
        };
        int i;
 
-       if (blk_queue_enter(q, true))
+       /* A deadlock might occur if a request is stuck requiring a
+        * timeout at the same time a queue freeze is waiting
+        * completion, since the timeout code would not be able to
+        * acquire the queue reference here.
+        *
+        * That's why we don't use blk_queue_enter here; instead, we use
+        * percpu_ref_tryget directly, because we need to be able to
+        * obtain a reference even in the short window between the queue
+        * starting to freeze, by dropping the first reference in
+        * blk_mq_freeze_queue_start, and the moment the last request is
+        * consumed, marked by the instant q_usage_counter reaches
+        * zero.
+        */
+       if (!percpu_ref_tryget(&q->q_usage_counter))
                return;
 
        blk_mq_queue_tag_busy_iter(q, blk_mq_check_expired, &data);
@@ -736,11 +793,12 @@ static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
        struct list_head *dptr;
        int queued;
 
-       WARN_ON(!cpumask_test_cpu(raw_smp_processor_id(), hctx->cpumask));
-
        if (unlikely(test_bit(BLK_MQ_S_STOPPED, &hctx->state)))
                return;
 
+       WARN_ON(!cpumask_test_cpu(raw_smp_processor_id(), hctx->cpumask) &&
+               cpu_online(hctx->next_cpu));
+
        hctx->run++;
 
        /*
@@ -784,7 +842,7 @@ static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
                switch (ret) {
                case BLK_MQ_RQ_QUEUE_OK:
                        queued++;
-                       continue;
+                       break;
                case BLK_MQ_RQ_QUEUE_BUSY:
                        list_add(&rq->queuelist, &rq_list);
                        __blk_mq_requeue_request(rq);
@@ -979,10 +1037,11 @@ void blk_mq_delay_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs)
 EXPORT_SYMBOL(blk_mq_delay_queue);
 
 static inline void __blk_mq_insert_req_list(struct blk_mq_hw_ctx *hctx,
-                                           struct blk_mq_ctx *ctx,
                                            struct request *rq,
                                            bool at_head)
 {
+       struct blk_mq_ctx *ctx = rq->mq_ctx;
+
        trace_block_rq_insert(hctx->queue, rq);
 
        if (at_head)
@@ -996,20 +1055,16 @@ static void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx,
 {
        struct blk_mq_ctx *ctx = rq->mq_ctx;
 
-       __blk_mq_insert_req_list(hctx, ctx, rq, at_head);
+       __blk_mq_insert_req_list(hctx, rq, at_head);
        blk_mq_hctx_mark_pending(hctx, ctx);
 }
 
 void blk_mq_insert_request(struct request *rq, bool at_head, bool run_queue,
-               bool async)
+                          bool async)
 {
+       struct blk_mq_ctx *ctx = rq->mq_ctx;
        struct request_queue *q = rq->q;
        struct blk_mq_hw_ctx *hctx;
-       struct blk_mq_ctx *ctx = rq->mq_ctx, *current_ctx;
-
-       current_ctx = blk_mq_get_ctx(q);
-       if (!cpu_online(ctx->cpu))
-               rq->mq_ctx = ctx = current_ctx;
 
        hctx = q->mq_ops->map_queue(q, ctx->cpu);
 
@@ -1019,8 +1074,6 @@ void blk_mq_insert_request(struct request *rq, bool at_head, bool run_queue,
 
        if (run_queue)
                blk_mq_run_hw_queue(hctx, async);
-
-       blk_mq_put_ctx(current_ctx);
 }
 
 static void blk_mq_insert_requests(struct request_queue *q,
@@ -1031,14 +1084,9 @@ static void blk_mq_insert_requests(struct request_queue *q,
 
 {
        struct blk_mq_hw_ctx *hctx;
-       struct blk_mq_ctx *current_ctx;
 
        trace_block_unplug(q, depth, !from_schedule);
 
-       current_ctx = blk_mq_get_ctx(q);
-
-       if (!cpu_online(ctx->cpu))
-               ctx = current_ctx;
        hctx = q->mq_ops->map_queue(q, ctx->cpu);
 
        /*
@@ -1050,15 +1098,14 @@ static void blk_mq_insert_requests(struct request_queue *q,
                struct request *rq;
 
                rq = list_first_entry(list, struct request, queuelist);
+               BUG_ON(rq->mq_ctx != ctx);
                list_del_init(&rq->queuelist);
-               rq->mq_ctx = ctx;
-               __blk_mq_insert_req_list(hctx, ctx, rq, false);
+               __blk_mq_insert_req_list(hctx, rq, false);
        }
        blk_mq_hctx_mark_pending(hctx, ctx);
        spin_unlock(&ctx->lock);
 
        blk_mq_run_hw_queue(hctx, from_schedule);
-       blk_mq_put_ctx(current_ctx);
 }
 
 static int plug_ctx_cmp(void *priv, struct list_head *a, struct list_head *b)
@@ -1169,28 +1216,29 @@ static struct request *blk_mq_map_request(struct request_queue *q,
        struct blk_mq_hw_ctx *hctx;
        struct blk_mq_ctx *ctx;
        struct request *rq;
-       int rw = bio_data_dir(bio);
+       int op = bio_data_dir(bio);
+       int op_flags = 0;
        struct blk_mq_alloc_data alloc_data;
 
        blk_queue_enter_live(q);
        ctx = blk_mq_get_ctx(q);
        hctx = q->mq_ops->map_queue(q, ctx->cpu);
 
-       if (rw_is_sync(bio->bi_rw))
-               rw |= REQ_SYNC;
+       if (rw_is_sync(bio_op(bio), bio->bi_opf))
+               op_flags |= REQ_SYNC;
 
-       trace_block_getrq(q, bio, rw);
+       trace_block_getrq(q, bio, op);
        blk_mq_set_alloc_data(&alloc_data, q, BLK_MQ_REQ_NOWAIT, ctx, hctx);
-       rq = __blk_mq_alloc_request(&alloc_data, rw);
+       rq = __blk_mq_alloc_request(&alloc_data, op, op_flags);
        if (unlikely(!rq)) {
                __blk_mq_run_hw_queue(hctx);
                blk_mq_put_ctx(ctx);
-               trace_block_sleeprq(q, bio, rw);
+               trace_block_sleeprq(q, bio, op);
 
                ctx = blk_mq_get_ctx(q);
                hctx = q->mq_ops->map_queue(q, ctx->cpu);
                blk_mq_set_alloc_data(&alloc_data, q, 0, ctx, hctx);
-               rq = __blk_mq_alloc_request(&alloc_data, rw);
+               rq = __blk_mq_alloc_request(&alloc_data, op, op_flags);
                ctx = alloc_data.ctx;
                hctx = alloc_data.hctx;
        }
@@ -1244,8 +1292,8 @@ static int blk_mq_direct_issue_request(struct request *rq, blk_qc_t *cookie)
  */
 static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
 {
-       const int is_sync = rw_is_sync(bio->bi_rw);
-       const int is_flush_fua = bio->bi_rw & (REQ_FLUSH | REQ_FUA);
+       const int is_sync = rw_is_sync(bio_op(bio), bio->bi_opf);
+       const int is_flush_fua = bio->bi_opf & (REQ_PREFLUSH | REQ_FUA);
        struct blk_map_ctx data;
        struct request *rq;
        unsigned int request_count = 0;
@@ -1338,8 +1386,8 @@ done:
  */
 static blk_qc_t blk_sq_make_request(struct request_queue *q, struct bio *bio)
 {
-       const int is_sync = rw_is_sync(bio->bi_rw);
-       const int is_flush_fua = bio->bi_rw & (REQ_FLUSH | REQ_FUA);
+       const int is_sync = rw_is_sync(bio_op(bio), bio->bi_opf);
+       const int is_flush_fua = bio->bi_opf & (REQ_PREFLUSH | REQ_FUA);
        struct blk_plug *plug;
        unsigned int request_count = 0;
        struct blk_map_ctx data;
@@ -1572,16 +1620,17 @@ static int blk_mq_alloc_bitmap(struct blk_mq_ctxmap *bitmap, int node)
        return 0;
 }
 
+/*
+ * 'cpu' is going away. splice any existing rq_list entries from this
+ * software queue to the hw queue dispatch list, and ensure that it
+ * gets run.
+ */
 static int blk_mq_hctx_cpu_offline(struct blk_mq_hw_ctx *hctx, int cpu)
 {
-       struct request_queue *q = hctx->queue;
        struct blk_mq_ctx *ctx;
        LIST_HEAD(tmp);
 
-       /*
-        * Move ctx entries to new CPU, if this one is going away.
-        */
-       ctx = __blk_mq_get_ctx(q, cpu);
+       ctx = __blk_mq_get_ctx(hctx->queue, cpu);
 
        spin_lock(&ctx->lock);
        if (!list_empty(&ctx->rq_list)) {
@@ -1593,24 +1642,11 @@ static int blk_mq_hctx_cpu_offline(struct blk_mq_hw_ctx *hctx, int cpu)
        if (list_empty(&tmp))
                return NOTIFY_OK;
 
-       ctx = blk_mq_get_ctx(q);
-       spin_lock(&ctx->lock);
-
-       while (!list_empty(&tmp)) {
-               struct request *rq;
-
-               rq = list_first_entry(&tmp, struct request, queuelist);
-               rq->mq_ctx = ctx;
-               list_move_tail(&rq->queuelist, &ctx->rq_list);
-       }
-
-       hctx = q->mq_ops->map_queue(q, ctx->cpu);
-       blk_mq_hctx_mark_pending(hctx, ctx);
-
-       spin_unlock(&ctx->lock);
+       spin_lock(&hctx->lock);
+       list_splice_tail_init(&tmp, &hctx->dispatch);
+       spin_unlock(&hctx->lock);
 
        blk_mq_run_hw_queue(hctx, true);
-       blk_mq_put_ctx(ctx);
        return NOTIFY_OK;
 }