Forum | Documentation | Website | Blog

Skip to content
Snippets Groups Projects
blk-mq.c 69.3 KiB
Newer Older
		if (ret == BLK_STS_RESOURCE) {
			blk_mq_put_driver_tag_hctx(hctx, rq);
			list_add(&rq->queuelist, list);
			__blk_mq_requeue_request(rq);
			blk_mq_end_request(rq, BLK_STS_IOERR);
	} while (!list_empty(list));
	hctx->dispatched[queued_to_index(queued)]++;

	/*
	 * Any items that need requeuing? Stuff them into hctx->dispatch,
	 * that is where we will continue on next queue run.
	 */
	if (!list_empty(list)) {
		 * If an I/O scheduler has been configured and we got a driver
		 * tag for the next request already, free it again.
		 */
		rq = list_first_entry(list, struct request, queuelist);
		blk_mq_put_driver_tag(rq);

		spin_lock(&hctx->lock);
		list_splice_init(list, &hctx->dispatch);
		spin_unlock(&hctx->lock);
		 * If SCHED_RESTART was set by the caller of this function and
		 * it is no longer set that means that it was cleared by another
		 * thread and hence that a queue rerun is needed.
		 * If TAG_WAITING is set that means that an I/O scheduler has
		 * been configured and another thread is waiting for a driver
		 * tag. To guarantee fairness, do not rerun this hardware queue
		 * but let the other thread grab the driver tag.
		 * If no I/O scheduler has been configured it is possible that
		 * the hardware queue got stopped and restarted before requests
		 * were pushed back onto the dispatch list. Rerun the queue to
		 * avoid starvation. Notes:
		 * - 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_STS_RESOURCE. Two exceptions are scsi-mq
		if (!blk_mq_sched_needs_restart(hctx) &&
		    !test_bit(BLK_MQ_S_TAG_WAITING, &hctx->state))
			blk_mq_run_hw_queue(hctx, true);
	return (queued + errors) != 0;
static void __blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx)
{
	int srcu_idx;

	WARN_ON(!cpumask_test_cpu(raw_smp_processor_id(), hctx->cpumask) &&
		cpu_online(hctx->next_cpu));

	if (!(hctx->flags & BLK_MQ_F_BLOCKING)) {
		rcu_read_lock();
		blk_mq_sched_dispatch_requests(hctx);
		rcu_read_unlock();
	} else {
		srcu_idx = srcu_read_lock(&hctx->queue_rq_srcu);
		blk_mq_sched_dispatch_requests(hctx);
		srcu_read_unlock(&hctx->queue_rq_srcu, srcu_idx);
	}
}

/*
 * It'd be great if the workqueue API had a way to pass
 * in a mask and had some smarts for more clever placement.
 * For now we just round-robin here, switching for every
 * BLK_MQ_CPU_WORK_BATCH queued items.
 */
static int blk_mq_hctx_next_cpu(struct blk_mq_hw_ctx *hctx)
{
	if (hctx->queue->nr_hw_queues == 1)
		return WORK_CPU_UNBOUND;

		next_cpu = cpumask_next(hctx->next_cpu, hctx->cpumask);
		if (next_cpu >= nr_cpu_ids)
			next_cpu = cpumask_first(hctx->cpumask);

		hctx->next_cpu = next_cpu;
		hctx->next_cpu_batch = BLK_MQ_CPU_WORK_BATCH;
	}

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 (!async && !(hctx->flags & BLK_MQ_F_BLOCKING)) {
		int cpu = get_cpu();
		if (cpumask_test_cpu(cpu, hctx->cpumask)) {
	kblockd_schedule_delayed_work_on(blk_mq_hctx_next_cpu(hctx),
					 &hctx->run_work,
					 msecs_to_jiffies(msecs));
}

void blk_mq_delay_run_hw_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs)
{
	__blk_mq_delay_run_hw_queue(hctx, true, msecs);
}
EXPORT_SYMBOL(blk_mq_delay_run_hw_queue);

void blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx, bool async)
{
	__blk_mq_delay_run_hw_queue(hctx, async, 0);
Omar Sandoval's avatar
Omar Sandoval committed
EXPORT_SYMBOL(blk_mq_run_hw_queue);
void blk_mq_run_hw_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) ||
		    blk_mq_hctx_stopped(hctx))
		blk_mq_run_hw_queue(hctx, async);
EXPORT_SYMBOL(blk_mq_run_hw_queues);
/**
 * blk_mq_queue_stopped() - check whether one or more hctxs have been stopped
 * @q: request queue.
 *
 * The caller is responsible for serializing this function against
 * blk_mq_{start,stop}_hw_queue().
 */
bool blk_mq_queue_stopped(struct request_queue *q)
{
	struct blk_mq_hw_ctx *hctx;
	int i;

	queue_for_each_hw_ctx(q, hctx, i)
		if (blk_mq_hctx_stopped(hctx))
			return true;

	return false;
}
EXPORT_SYMBOL(blk_mq_queue_stopped);

static void __blk_mq_stop_hw_queue(struct blk_mq_hw_ctx *hctx, bool sync)
	if (sync)
		cancel_delayed_work_sync(&hctx->run_work);
	else
		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)
{
	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);
}
EXPORT_SYMBOL(blk_mq_stop_hw_queues);

void blk_mq_start_hw_queue(struct blk_mq_hw_ctx *hctx)
{
	clear_bit(BLK_MQ_S_STOPPED, &hctx->state);
	blk_mq_run_hw_queue(hctx, false);
}
EXPORT_SYMBOL(blk_mq_start_hw_queue);

void blk_mq_start_hw_queues(struct request_queue *q)
{
	struct blk_mq_hw_ctx *hctx;
	int i;

	queue_for_each_hw_ctx(q, hctx, i)
		blk_mq_start_hw_queue(hctx);
}
EXPORT_SYMBOL(blk_mq_start_hw_queues);

void blk_mq_start_stopped_hw_queue(struct blk_mq_hw_ctx *hctx, bool async)
{
	if (!blk_mq_hctx_stopped(hctx))
		return;

	clear_bit(BLK_MQ_S_STOPPED, &hctx->state);
	blk_mq_run_hw_queue(hctx, async);
}
EXPORT_SYMBOL_GPL(blk_mq_start_stopped_hw_queue);

void blk_mq_start_stopped_hw_queues(struct request_queue *q, bool async)
{
	struct blk_mq_hw_ctx *hctx;
	int i;

	queue_for_each_hw_ctx(q, hctx, i)
		blk_mq_start_stopped_hw_queue(hctx, async);
}
EXPORT_SYMBOL(blk_mq_start_stopped_hw_queues);

static void blk_mq_run_work_fn(struct work_struct *work)
{
	struct blk_mq_hw_ctx *hctx;

	hctx = container_of(work, struct blk_mq_hw_ctx, run_work.work);
	/*
	 * If we are stopped, don't run the queue. The exception is if
	 * BLK_MQ_S_START_ON_RUN is set. For that case, we auto-clear
	 * the STOPPED bit and run it.
	 */
	if (test_bit(BLK_MQ_S_STOPPED, &hctx->state)) {
		if (!test_bit(BLK_MQ_S_START_ON_RUN, &hctx->state))
			return;
		clear_bit(BLK_MQ_S_START_ON_RUN, &hctx->state);
		clear_bit(BLK_MQ_S_STOPPED, &hctx->state);
	}

void blk_mq_delay_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs)
{
	if (unlikely(!blk_mq_hw_queue_mapped(hctx)))
		return;
	/*
	 * Stop the hw queue, then modify currently delayed work.
	 * This should prevent us from running the queue prematurely.
	 * Mark the queue as auto-clearing STOPPED when it runs.
	 */
	blk_mq_stop_hw_queue(hctx);
	set_bit(BLK_MQ_S_START_ON_RUN, &hctx->state);
	kblockd_mod_delayed_work_on(blk_mq_hctx_next_cpu(hctx),
					&hctx->run_work,
					msecs_to_jiffies(msecs));
}
EXPORT_SYMBOL(blk_mq_delay_queue);

static inline void __blk_mq_insert_req_list(struct blk_mq_hw_ctx *hctx,
					    struct request *rq,
					    bool at_head)
	struct blk_mq_ctx *ctx = rq->mq_ctx;

	trace_block_rq_insert(hctx->queue, rq);

	if (at_head)
		list_add(&rq->queuelist, &ctx->rq_list);
	else
		list_add_tail(&rq->queuelist, &ctx->rq_list);
void __blk_mq_insert_request(struct blk_mq_hw_ctx *hctx, struct request *rq,
			     bool at_head)
{
	struct blk_mq_ctx *ctx = rq->mq_ctx;

	__blk_mq_insert_req_list(hctx, rq, at_head);
	blk_mq_hctx_mark_pending(hctx, ctx);
}

void blk_mq_insert_requests(struct blk_mq_hw_ctx *hctx, struct blk_mq_ctx *ctx,
			    struct list_head *list)

{
	/*
	 * preemption doesn't flush plug list, so it's possible ctx->cpu is
	 * offline now
	 */
	spin_lock(&ctx->lock);
	while (!list_empty(list)) {
		struct request *rq;

		rq = list_first_entry(list, struct request, queuelist);
		BUG_ON(rq->mq_ctx != ctx);
		list_del_init(&rq->queuelist);
		__blk_mq_insert_req_list(hctx, rq, false);
	blk_mq_hctx_mark_pending(hctx, ctx);
	spin_unlock(&ctx->lock);
}

static int plug_ctx_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);

	return !(rqa->mq_ctx < rqb->mq_ctx ||
		 (rqa->mq_ctx == rqb->mq_ctx &&
		  blk_rq_pos(rqa) < blk_rq_pos(rqb)));
}

void blk_mq_flush_plug_list(struct blk_plug *plug, bool from_schedule)
{
	struct blk_mq_ctx *this_ctx;
	struct request_queue *this_q;
	struct request *rq;
	LIST_HEAD(list);
	LIST_HEAD(ctx_list);
	unsigned int depth;

	list_splice_init(&plug->mq_list, &list);

	list_sort(NULL, &list, plug_ctx_cmp);

	this_q = NULL;
	this_ctx = NULL;
	depth = 0;

	while (!list_empty(&list)) {
		rq = list_entry_rq(list.next);
		list_del_init(&rq->queuelist);
		BUG_ON(!rq->q);
		if (rq->mq_ctx != this_ctx) {
			if (this_ctx) {
				trace_block_unplug(this_q, depth, from_schedule);
				blk_mq_sched_insert_requests(this_q, this_ctx,
								&ctx_list,
								from_schedule);
			}

			this_ctx = rq->mq_ctx;
			this_q = rq->q;
			depth = 0;
		}

		depth++;
		list_add_tail(&rq->queuelist, &ctx_list);
	}

	/*
	 * If 'this_ctx' is set, we know we have entries to complete
	 * on 'ctx_list'. Do those.
	 */
	if (this_ctx) {
		trace_block_unplug(this_q, depth, from_schedule);
		blk_mq_sched_insert_requests(this_q, this_ctx, &ctx_list,
						from_schedule);
	}
}

static void blk_mq_bio_to_request(struct request *rq, struct bio *bio)
{
	blk_init_request_from_bio(rq, bio);
	blk_account_io_start(rq, true);
static inline bool hctx_allow_merges(struct blk_mq_hw_ctx *hctx)
{
	return (hctx->flags & BLK_MQ_F_SHOULD_MERGE) &&
		!blk_queue_nomerges(hctx->queue);
}

static inline void blk_mq_queue_io(struct blk_mq_hw_ctx *hctx,
				   struct blk_mq_ctx *ctx,
				   struct request *rq)
{
	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)
{
	if (rq->tag != -1)
		return blk_tag_to_qc_t(rq->tag, hctx->queue_num, false);

	return blk_tag_to_qc_t(rq->internal_tag, hctx->queue_num, true);
Ming Lei's avatar
Ming Lei committed
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,
Ming Lei's avatar
Ming Lei committed
	bool run_queue = true;

	if (blk_mq_hctx_stopped(hctx)) {
		run_queue = false;
		goto insert;
	}
Ming Lei's avatar
Ming Lei committed
	if (!blk_mq_get_driver_tag(rq, NULL, false))
		goto insert;

	new_cookie = request_to_qc_t(hctx, rq);

	/*
	 * For OK queue, we are done. For error, kill it. Any other
	 * error (busy), just add it to our list as we previously
	 * would have done
	 */
	ret = q->mq_ops->queue_rq(hctx, &bd);
	case BLK_STS_RESOURCE:
		__blk_mq_requeue_request(rq);
		goto insert;
	default:
		blk_mq_end_request(rq, ret);
Ming Lei's avatar
Ming Lei committed
	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,
		struct request *rq, blk_qc_t *cookie)
{
	if (!(hctx->flags & BLK_MQ_F_BLOCKING)) {
		rcu_read_lock();
Ming Lei's avatar
Ming Lei committed
		__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);
Ming Lei's avatar
Ming Lei committed
		__blk_mq_try_issue_directly(hctx, rq, cookie, true);
		srcu_read_unlock(&hctx->queue_rq_srcu, srcu_idx);
	}
}

static blk_qc_t blk_mq_make_request(struct request_queue *q, struct bio *bio)
	const int is_sync = op_is_sync(bio->bi_opf);
	const int is_flush_fua = op_is_flush(bio->bi_opf);
	struct blk_mq_alloc_data data = { .flags = 0 };
	unsigned int request_count = 0;
	struct blk_plug *plug;
	struct request *same_queue_rq = NULL;
	unsigned int wb_acct;
	blk_queue_split(q, &bio);
	if (bio_integrity_enabled(bio) && bio_integrity_prep(bio)) {
		bio_io_error(bio);
	if (!is_flush_fua && !blk_queue_nomerges(q) &&
	    blk_attempt_plug_merge(q, bio, &request_count, &same_queue_rq))
		return BLK_QC_T_NONE;
	if (blk_mq_sched_bio_merge(q, bio))
		return BLK_QC_T_NONE;

	wb_acct = wbt_wait(q->rq_wb, bio, NULL);

	trace_block_getrq(q, bio, bio->bi_opf);

	rq = blk_mq_get_request(q, bio, bio->bi_opf, &data);
	if (unlikely(!rq)) {
		__wbt_done(q->rq_wb, wb_acct);
	}

	wbt_track(&rq->issue_stat, wb_acct);
	cookie = request_to_qc_t(data.hctx, rq);
	if (unlikely(is_flush_fua)) {
		blk_mq_put_ctx(data.ctx);
		blk_mq_bio_to_request(rq, bio);
		if (q->elevator) {
			blk_mq_sched_insert_request(rq, false, true, true,
					true);
			blk_insert_flush(rq);
			blk_mq_run_hw_queue(data.hctx, true);
	} else if (plug && q->nr_hw_queues == 1) {
		struct request *last = NULL;

		blk_mq_put_ctx(data.ctx);
		blk_mq_bio_to_request(rq, bio);

		/*
		 * @request_count may become stale because of schedule
		 * out, so check the list again.
		 */
		if (list_empty(&plug->mq_list))
			request_count = 0;
		else if (blk_queue_nomerges(q))
			request_count = blk_plug_queued_count(q);

		if (!request_count)
			trace_block_plug(q);
		else
			last = list_entry_rq(plug->mq_list.prev);
		if (request_count >= BLK_MAX_REQUEST_COUNT || (last &&
		    blk_rq_bytes(last) >= BLK_PLUG_FLUSH_SIZE)) {
			blk_flush_plug_list(plug, false);
			trace_block_plug(q);
		list_add_tail(&rq->queuelist, &plug->mq_list);
	} else if (plug && !blk_queue_nomerges(q)) {
		blk_mq_bio_to_request(rq, bio);
		 * We do limited plugging. If the bio can be merged, do that.
		 * Otherwise the existing request in the plug list will be
		 * issued. So the plug list will have one request at most
		 * The plug list might get flushed before this. If that happens,
		 * the plug list is empty, and same_queue_rq is invalid.
		if (list_empty(&plug->mq_list))
			same_queue_rq = NULL;
		if (same_queue_rq)
			list_del_init(&same_queue_rq->queuelist);
		list_add_tail(&rq->queuelist, &plug->mq_list);

		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);
		blk_mq_try_issue_directly(data.hctx, rq, &cookie);
	} else if (q->elevator) {
		blk_mq_put_ctx(data.ctx);
		blk_mq_bio_to_request(rq, bio);
		blk_mq_sched_insert_request(rq, false, true, true, true);
		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);
void blk_mq_free_rqs(struct blk_mq_tag_set *set, struct blk_mq_tags *tags,
		     unsigned int hctx_idx)
	if (tags->rqs && set->ops->exit_request) {
		for (i = 0; i < tags->nr_tags; i++) {
			struct request *rq = tags->static_rqs[i];

			if (!rq)
			set->ops->exit_request(set, rq, hctx_idx);
			tags->static_rqs[i] = NULL;
	while (!list_empty(&tags->page_list)) {
		page = list_first_entry(&tags->page_list, struct page, lru);
		list_del_init(&page->lru);
		/*
		 * Remove kmemleak object previously allocated in
		 * blk_mq_init_rq_map().
		 */
		kmemleak_free(page_address(page));
		__free_pages(page, page->private);
	}
void blk_mq_free_rq_map(struct blk_mq_tags *tags)
{
	kfree(tags->static_rqs);
	tags->static_rqs = NULL;
struct blk_mq_tags *blk_mq_alloc_rq_map(struct blk_mq_tag_set *set,
					unsigned int hctx_idx,
					unsigned int nr_tags,
					unsigned int reserved_tags)
	node = blk_mq_hw_queue_to_node(set->mq_map, hctx_idx);
	if (node == NUMA_NO_NODE)
		node = set->numa_node;

	tags = blk_mq_init_tags(nr_tags, reserved_tags, node,
				BLK_MQ_FLAG_TO_ALLOC_POLICY(set->flags));
	tags->rqs = kzalloc_node(nr_tags * sizeof(struct request *),
				 GFP_NOIO | __GFP_NOWARN | __GFP_NORETRY,
	if (!tags->rqs) {
		blk_mq_free_tags(tags);
		return NULL;
	}
	tags->static_rqs = kzalloc_node(nr_tags * sizeof(struct request *),
				 GFP_NOIO | __GFP_NOWARN | __GFP_NORETRY,
	if (!tags->static_rqs) {
		kfree(tags->rqs);
		blk_mq_free_tags(tags);
		return NULL;
	}

	return tags;
}

static size_t order_to_size(unsigned int order)
{
	return (size_t)PAGE_SIZE << order;
}

int blk_mq_alloc_rqs(struct blk_mq_tag_set *set, struct blk_mq_tags *tags,
		     unsigned int hctx_idx, unsigned int depth)
{
	unsigned int i, j, entries_per_page, max_order = 4;
	size_t rq_size, left;
	int node;

	node = blk_mq_hw_queue_to_node(set->mq_map, hctx_idx);
	if (node == NUMA_NO_NODE)
		node = set->numa_node;
	/*
	 * rq_size is the size of the request plus driver payload, rounded
	 * to the cacheline size
	 */
	rq_size = round_up(sizeof(struct request) + set->cmd_size,
				cache_line_size());
	for (i = 0; i < depth; ) {
		int this_order = max_order;
		struct page *page;
		int to_do;
		void *p;

		while (this_order && left < order_to_size(this_order - 1))
			page = alloc_pages_node(node,
				GFP_NOIO | __GFP_NOWARN | __GFP_NORETRY | __GFP_ZERO,
			if (page)
				break;
			if (!this_order--)
				break;
			if (order_to_size(this_order) < rq_size)
				break;
		} while (1);

		if (!page)

		page->private = this_order;
		list_add_tail(&page->lru, &tags->page_list);
		/*
		 * Allow kmemleak to scan these pages as they contain pointers
		 * to additional allocations like via ops->init_request().
		 */
		kmemleak_alloc(p, order_to_size(this_order), 1, GFP_NOIO);
		entries_per_page = order_to_size(this_order) / rq_size;
		to_do = min(entries_per_page, depth - i);
		left -= to_do * rq_size;
		for (j = 0; j < to_do; j++) {
			struct request *rq = p;

			tags->static_rqs[i] = rq;
			if (set->ops->init_request) {
				if (set->ops->init_request(set, rq, hctx_idx,
					tags->static_rqs[i] = NULL;
	blk_mq_free_rqs(set, tags, hctx_idx);
	return -ENOMEM;
/*
 * '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_notify_dead(unsigned int cpu, struct hlist_node *node)
	struct blk_mq_hw_ctx *hctx;
	struct blk_mq_ctx *ctx;
	LIST_HEAD(tmp);

	hctx = hlist_entry_safe(node, struct blk_mq_hw_ctx, cpuhp_dead);
	ctx = __blk_mq_get_ctx(hctx->queue, cpu);

	spin_lock(&ctx->lock);
	if (!list_empty(&ctx->rq_list)) {
		list_splice_init(&ctx->rq_list, &tmp);
		blk_mq_hctx_clear_pending(hctx, ctx);
	}
	spin_unlock(&ctx->lock);

	if (list_empty(&tmp))
	spin_lock(&hctx->lock);
	list_splice_tail_init(&tmp, &hctx->dispatch);
	spin_unlock(&hctx->lock);
static void blk_mq_remove_cpuhp(struct blk_mq_hw_ctx *hctx)
	cpuhp_state_remove_instance_nocalls(CPUHP_BLK_MQ_DEAD,
					    &hctx->cpuhp_dead);
/* hctx->ctxs will be freed in queue's release handler */
static void blk_mq_exit_hctx(struct request_queue *q,
		struct blk_mq_tag_set *set,
		struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
{
	blk_mq_debugfs_unregister_hctx(hctx);

	blk_mq_tag_idle(hctx);

	if (set->ops->exit_request)
		set->ops->exit_request(set, hctx->fq->flush_rq, hctx_idx);
	blk_mq_sched_exit_hctx(q, hctx, hctx_idx);

	if (set->ops->exit_hctx)
		set->ops->exit_hctx(hctx, hctx_idx);

	if (hctx->flags & BLK_MQ_F_BLOCKING)
		cleanup_srcu_struct(&hctx->queue_rq_srcu);

	blk_mq_remove_cpuhp(hctx);
	blk_free_flush_queue(hctx->fq);
	sbitmap_free(&hctx->ctx_map);
Ming Lei's avatar
Ming Lei committed
static void blk_mq_exit_hw_queues(struct request_queue *q,
		struct blk_mq_tag_set *set, int nr_queue)
{
	struct blk_mq_hw_ctx *hctx;
	unsigned int i;

	queue_for_each_hw_ctx(q, hctx, i) {
		if (i == nr_queue)
			break;
		blk_mq_exit_hctx(q, set, hctx, i);
static int blk_mq_init_hctx(struct request_queue *q,
		struct blk_mq_tag_set *set,
		struct blk_mq_hw_ctx *hctx, unsigned hctx_idx)
	int node;

	node = hctx->numa_node;
	if (node == NUMA_NO_NODE)
		node = hctx->numa_node = set->numa_node;

	INIT_DELAYED_WORK(&hctx->run_work, blk_mq_run_work_fn);
	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);

	hctx->tags = set->tags[hctx_idx];
	 * Allocate space for all possible cpus to avoid allocation at
	 * runtime
	hctx->ctxs = kmalloc_node(nr_cpu_ids * sizeof(void *),
					GFP_KERNEL, node);
	if (!hctx->ctxs)
		goto unregister_cpu_notifier;
	if (sbitmap_init_node(&hctx->ctx_map, nr_cpu_ids, ilog2(8), GFP_KERNEL,
			      node))
	hctx->nr_ctx = 0;
	if (set->ops->init_hctx &&
	    set->ops->init_hctx(hctx, set->driver_data, hctx_idx))
		goto free_bitmap;
	if (blk_mq_sched_init_hctx(q, hctx, hctx_idx))
		goto exit_hctx;

	hctx->fq = blk_alloc_flush_queue(q, hctx->numa_node, set->cmd_size);
	if (!hctx->fq)
	if (set->ops->init_request &&
	    set->ops->init_request(set, hctx->fq->flush_rq, hctx_idx,
				   node))
	if (hctx->flags & BLK_MQ_F_BLOCKING)
		init_srcu_struct(&hctx->queue_rq_srcu);

	blk_mq_debugfs_register_hctx(q, hctx);

 free_fq:
	kfree(hctx->fq);
 sched_exit_hctx:
	blk_mq_sched_exit_hctx(q, hctx, hctx_idx);
 exit_hctx:
	if (set->ops->exit_hctx)
		set->ops->exit_hctx(hctx, hctx_idx);
	sbitmap_free(&hctx->ctx_map);
 free_ctxs:
	kfree(hctx->ctxs);
 unregister_cpu_notifier:
	blk_mq_remove_cpuhp(hctx);

static void blk_mq_init_cpu_queues(struct request_queue *q,
				   unsigned int nr_hw_queues)
{
	unsigned int i;

	for_each_possible_cpu(i) {
		struct blk_mq_ctx *__ctx = per_cpu_ptr(q->queue_ctx, i);
		struct blk_mq_hw_ctx *hctx;

		__ctx->cpu = i;
		spin_lock_init(&__ctx->lock);
		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))
			continue;

		hctx = blk_mq_map_queue(q, i);
		/*
		 * Set local node, IFF we have more than one hw queue. If
		 * not, we remain on the home node of the device
		 */
		if (nr_hw_queues > 1 && hctx->numa_node == NUMA_NO_NODE)
			hctx->numa_node = local_memory_node(cpu_to_node(i));
static bool __blk_mq_alloc_rq_map(struct blk_mq_tag_set *set, int hctx_idx)
{
	int ret = 0;

	set->tags[hctx_idx] = blk_mq_alloc_rq_map(set, hctx_idx,
					set->queue_depth, set->reserved_tags);
	if (!set->tags[hctx_idx])
		return false;

	ret = blk_mq_alloc_rqs(set, set->tags[hctx_idx], hctx_idx,
				set->queue_depth);
	if (!ret)
		return true;

	blk_mq_free_rq_map(set->tags[hctx_idx]);
	set->tags[hctx_idx] = NULL;
	return false;
}

static void blk_mq_free_map_and_requests(struct blk_mq_tag_set *set,
					 unsigned int hctx_idx)
{
	if (set->tags[hctx_idx]) {
		blk_mq_free_rqs(set, set->tags[hctx_idx], hctx_idx);
		blk_mq_free_rq_map(set->tags[hctx_idx]);
		set->tags[hctx_idx] = NULL;
	}
static void blk_mq_map_swqueue(struct request_queue *q,
			       const struct cpumask *online_mask)
	struct blk_mq_hw_ctx *hctx;
	struct blk_mq_ctx *ctx;
	struct blk_mq_tag_set *set = q->tag_set;
	/*
	 * Avoid others reading imcomplete hctx->cpumask through sysfs
	 */
	mutex_lock(&q->sysfs_lock);

	queue_for_each_hw_ctx(q, hctx, i) {
		hctx->nr_ctx = 0;
	}

	/*
	 * Map software to hardware queues