All of lore.kernel.org
 help / color / mirror / Atom feed
* block: loop: convert to blk-mq
@ 2014-08-30 16:08 Ming Lei
  2014-08-30 16:08 ` [PATCH v2 1/6] blk-mq: export blk_mq_freeze_queue and blk_mq_unfreeze_queue Ming Lei
                   ` (5 more replies)
  0 siblings, 6 replies; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov

Hi,

The following 6 patches convert current loop driver into blk-mq:

	- loop's scalability gets improved much
	- loop driver gets quite simplified, and the conversion can
	be throught as cleanup

The previous kernel AIO patches will be submitted in another patchset
for improving loop's performance.

V2:
	- replace work queue with kthread worker because wq may cause
	regression on some fast devices(such as loop over null_blk, tmpfs
	file, ...), in these cases, loop thread becomes CPU bound actually,
	so work queue will introduce lots of unnecessary context switch, and
	decrease throughput a lot one this cases
	- introduce prepare_flush_rq_fn callback and its pair in blk-mq ops
	for setting up flush rq's pdu correctly
	- move kernel aio patches into another patchset as suggested by
	Christoph
	- no change in the other 4 patches

V1:
	- improve failure path in aio_kernel_submit()

 block/blk-flush.c      |    4 +
 block/blk-mq.c         |   16 +-
 block/blk-mq.h         |    1 -
 drivers/block/loop.c   |  411 +++++++++++++++++++++++++-----------------------
 drivers/block/loop.h   |   20 ++-
 include/linux/blk-mq.h |   14 ++
 6 files changed, 259 insertions(+), 207 deletions(-)



Thanks,
--
Ming Lei


^ permalink raw reply	[flat|nested] 11+ messages in thread

* [PATCH v2 1/6] blk-mq: export blk_mq_freeze_queue and blk_mq_unfreeze_queue
  2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
@ 2014-08-30 16:08 ` Ming Lei
  2014-08-30 16:08 ` [PATCH v2 2/6] blk-mq: introduce prepare_flush_rq_fn callback and its pair Ming Lei
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Ming Lei

It is handy to use the two helpers for switching backend file
in loop driver, so export them.

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 block/blk-mq.c         |    4 +++-
 block/blk-mq.h         |    1 -
 include/linux/blk-mq.h |    2 ++
 3 files changed, 5 insertions(+), 2 deletions(-)

diff --git a/block/blk-mq.c b/block/blk-mq.c
index 4aac826..3ad7524 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -124,8 +124,9 @@ void blk_mq_freeze_queue(struct request_queue *q)
 	}
 	wait_event(q->mq_freeze_wq, percpu_ref_is_zero(&q->mq_usage_counter));
 }
+EXPORT_SYMBOL(blk_mq_freeze_queue);
 
-static void blk_mq_unfreeze_queue(struct request_queue *q)
+void blk_mq_unfreeze_queue(struct request_queue *q)
 {
 	bool wake;
 
@@ -138,6 +139,7 @@ static void blk_mq_unfreeze_queue(struct request_queue *q)
 		wake_up_all(&q->mq_freeze_wq);
 	}
 }
+EXPORT_SYMBOL(blk_mq_unfreeze_queue);
 
 bool blk_mq_can_queue(struct blk_mq_hw_ctx *hctx)
 {
diff --git a/block/blk-mq.h b/block/blk-mq.h
index ca4964a..3800316 100644
--- a/block/blk-mq.h
+++ b/block/blk-mq.h
@@ -28,7 +28,6 @@ struct blk_mq_ctx {
 void __blk_mq_complete_request(struct request *rq);
 void blk_mq_run_hw_queue(struct blk_mq_hw_ctx *hctx, bool async);
 void blk_mq_init_flush(struct request_queue *q);
-void blk_mq_freeze_queue(struct request_queue *q);
 void blk_mq_free_queue(struct request_queue *q);
 void blk_mq_clone_flush_request(struct request *flush_rq,
 		struct request *orig_rq);
diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
index a1e31f2..c6b2bfa 100644
--- a/include/linux/blk-mq.h
+++ b/include/linux/blk-mq.h
@@ -174,6 +174,8 @@ void blk_mq_start_hw_queues(struct request_queue *q);
 void blk_mq_start_stopped_hw_queues(struct request_queue *q, bool async);
 void blk_mq_delay_queue(struct blk_mq_hw_ctx *hctx, unsigned long msecs);
 void blk_mq_tag_busy_iter(struct blk_mq_tags *tags, void (*fn)(void *data, unsigned long *), void *data);
+void blk_mq_freeze_queue(struct request_queue *q);
+void blk_mq_unfreeze_queue(struct request_queue *q);
 
 /*
  * Driver command data is immediately after the request. So subtract request
-- 
1.7.9.5


^ permalink raw reply related	[flat|nested] 11+ messages in thread

* [PATCH v2 2/6] blk-mq: introduce prepare_flush_rq_fn callback and its pair
  2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
  2014-08-30 16:08 ` [PATCH v2 1/6] blk-mq: export blk_mq_freeze_queue and blk_mq_unfreeze_queue Ming Lei
@ 2014-08-30 16:08 ` Ming Lei
  2014-08-30 16:08 ` [PATCH v2 3/6] block: loop: convert to blk-mq Ming Lei
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Ming Lei

Currently pdu of the flush rq is simlpy copied from another rq,
it isn't enough to initialize pointer field well, so introduce
these two callbacks for driver to handle the case easily.

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 block/blk-flush.c      |    4 ++++
 block/blk-mq.c         |   12 ++++++++++--
 include/linux/blk-mq.h |   12 ++++++++++++
 3 files changed, 26 insertions(+), 2 deletions(-)

diff --git a/block/blk-flush.c b/block/blk-flush.c
index 3cb5e9e..cd36596 100644
--- a/block/blk-flush.c
+++ b/block/blk-flush.c
@@ -225,6 +225,10 @@ static void flush_end_io(struct request *flush_rq, int error)
 
 	if (q->mq_ops) {
 		spin_lock_irqsave(&q->mq_flush_lock, flags);
+		if (q->mq_ops->unprepare_flush_rq)
+			q->mq_ops->unprepare_flush_rq(
+				q->tag_set->driver_data,
+				q, q->flush_rq);
 		q->flush_rq->tag = -1;
 	}
 
diff --git a/block/blk-mq.c b/block/blk-mq.c
index 3ad7524..4a610d2 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -291,11 +291,19 @@ void blk_mq_clone_flush_request(struct request *flush_rq,
 {
 	struct blk_mq_hw_ctx *hctx =
 		orig_rq->q->mq_ops->map_queue(orig_rq->q, orig_rq->mq_ctx->cpu);
+	int ret = 0;
 
 	flush_rq->mq_ctx = orig_rq->mq_ctx;
 	flush_rq->tag = orig_rq->tag;
-	memcpy(blk_mq_rq_to_pdu(flush_rq), blk_mq_rq_to_pdu(orig_rq),
-		hctx->cmd_size);
+
+	if (orig_rq->q->mq_ops->prepare_flush_rq)
+		ret = orig_rq->q->mq_ops->prepare_flush_rq(
+				orig_rq->q->tag_set->driver_data,
+				orig_rq->q, flush_rq, orig_rq);
+	else
+		memcpy(blk_mq_rq_to_pdu(flush_rq),
+		       blk_mq_rq_to_pdu(orig_rq), hctx->cmd_size);
+	WARN_ON(ret);
 }
 
 inline void __blk_mq_end_io(struct request *rq, int error)
diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
index c6b2bfa..8935f34 100644
--- a/include/linux/blk-mq.h
+++ b/include/linux/blk-mq.h
@@ -85,6 +85,10 @@ typedef int (init_request_fn)(void *, struct request *, unsigned int,
 		unsigned int, unsigned int);
 typedef void (exit_request_fn)(void *, struct request *, unsigned int,
 		unsigned int);
+typedef int (prepare_flush_rq_fn)(void *, struct request_queue *,
+		struct request *, const struct request *);
+typedef void (unprepare_flush_rq_fn)(void *, struct request_queue *,
+		struct request *);
 
 struct blk_mq_ops {
 	/*
@@ -119,6 +123,14 @@ struct blk_mq_ops {
 	 */
 	init_request_fn		*init_request;
 	exit_request_fn		*exit_request;
+
+	/*
+	 * Called after flush request cloned by the block layer to allow
+	 * the driver to set up driver specific data, since simple memcpy
+	 * may not initialize flush req well.
+	 */
+	prepare_flush_rq_fn	*prepare_flush_rq;
+	unprepare_flush_rq_fn	*unprepare_flush_rq;
 };
 
 enum {
-- 
1.7.9.5


^ permalink raw reply related	[flat|nested] 11+ messages in thread

* [PATCH v2 3/6] block: loop: convert to blk-mq
  2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
  2014-08-30 16:08 ` [PATCH v2 1/6] blk-mq: export blk_mq_freeze_queue and blk_mq_unfreeze_queue Ming Lei
  2014-08-30 16:08 ` [PATCH v2 2/6] blk-mq: introduce prepare_flush_rq_fn callback and its pair Ming Lei
@ 2014-08-30 16:08 ` Ming Lei
  2014-08-30 22:03   ` Elliott, Robert (Server Storage)
  2014-08-30 16:08 ` [PATCH v2 4/6] block: loop: say goodby to bio Ming Lei
                   ` (2 subsequent siblings)
  5 siblings, 1 reply; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Ming Lei

The conversion is a bit straightforward, and use per-hw_queue
kthread work queue to dispatch reqests of loop block, so
scalability gets improved a lot if nr_hw_queues is increased.

Another benefit is that loop driver code gets simplified
much, and the patch can be thought as cleanup too.

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c |  322 +++++++++++++++++++++++++++-----------------------
 drivers/block/loop.h |   20 +++-
 2 files changed, 187 insertions(+), 155 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 6cb1beb..b02122d 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -75,6 +75,7 @@
 #include <linux/sysfs.h>
 #include <linux/miscdevice.h>
 #include <linux/falloc.h>
+#include <linux/blk-mq.h>
 #include "loop.h"
 
 #include <asm/uaccess.h>
@@ -466,109 +467,37 @@ out:
 	return ret;
 }
 
-/*
- * Add bio to back of pending list
- */
-static void loop_add_bio(struct loop_device *lo, struct bio *bio)
-{
-	lo->lo_bio_count++;
-	bio_list_add(&lo->lo_bio_list, bio);
-}
-
-/*
- * Grab first pending buffer
- */
-static struct bio *loop_get_bio(struct loop_device *lo)
-{
-	lo->lo_bio_count--;
-	return bio_list_pop(&lo->lo_bio_list);
-}
-
-static void loop_make_request(struct request_queue *q, struct bio *old_bio)
-{
-	struct loop_device *lo = q->queuedata;
-	int rw = bio_rw(old_bio);
-
-	if (rw == READA)
-		rw = READ;
-
-	BUG_ON(!lo || (rw != READ && rw != WRITE));
-
-	spin_lock_irq(&lo->lo_lock);
-	if (lo->lo_state != Lo_bound)
-		goto out;
-	if (unlikely(rw == WRITE && (lo->lo_flags & LO_FLAGS_READ_ONLY)))
-		goto out;
-	if (lo->lo_bio_count >= q->nr_congestion_on)
-		wait_event_lock_irq(lo->lo_req_wait,
-				    lo->lo_bio_count < q->nr_congestion_off,
-				    lo->lo_lock);
-	loop_add_bio(lo, old_bio);
-	wake_up(&lo->lo_event);
-	spin_unlock_irq(&lo->lo_lock);
-	return;
-
-out:
-	spin_unlock_irq(&lo->lo_lock);
-	bio_io_error(old_bio);
-}
-
 struct switch_request {
 	struct file *file;
 	struct completion wait;
 };
 
-static void do_loop_switch(struct loop_device *, struct switch_request *);
-
-static inline void loop_handle_bio(struct loop_device *lo, struct bio *bio)
+static inline int loop_handle_bio(struct loop_device *lo, struct bio *bio)
 {
-	if (unlikely(!bio->bi_bdev)) {
-		do_loop_switch(lo, bio->bi_private);
-		bio_put(bio);
-	} else {
-		int ret = do_bio_filebacked(lo, bio);
-		bio_endio(bio, ret);
-	}
+	int ret = do_bio_filebacked(lo, bio);
+	return ret;
 }
 
 /*
- * worker thread that handles reads/writes to file backed loop devices,
- * to avoid blocking in our make_request_fn. it also does loop decrypting
- * on reads for block backed loop, as that is too heavy to do from
- * b_end_io context where irqs may be disabled.
- *
- * Loop explanation:  loop_clr_fd() sets lo_state to Lo_rundown before
- * calling kthread_stop().  Therefore once kthread_should_stop() is
- * true, make_request will not place any more requests.  Therefore
- * once kthread_should_stop() is true and lo_bio is NULL, we are
- * done with the loop.
+ * Do the actual switch; called from the BIO completion routine
  */
-static int loop_thread(void *data)
+static void do_loop_switch(struct loop_device *lo, struct switch_request *p)
 {
-	struct loop_device *lo = data;
-	struct bio *bio;
-
-	set_user_nice(current, MIN_NICE);
-
-	while (!kthread_should_stop() || !bio_list_empty(&lo->lo_bio_list)) {
-
-		wait_event_interruptible(lo->lo_event,
-				!bio_list_empty(&lo->lo_bio_list) ||
-				kthread_should_stop());
-
-		if (bio_list_empty(&lo->lo_bio_list))
-			continue;
-		spin_lock_irq(&lo->lo_lock);
-		bio = loop_get_bio(lo);
-		if (lo->lo_bio_count < lo->lo_queue->nr_congestion_off)
-			wake_up(&lo->lo_req_wait);
-		spin_unlock_irq(&lo->lo_lock);
+	struct file *file = p->file;
+	struct file *old_file = lo->lo_backing_file;
+	struct address_space *mapping;
 
-		BUG_ON(!bio);
-		loop_handle_bio(lo, bio);
-	}
+	/* if no new file, only flush of queued bios requested */
+	if (!file)
+		return;
 
-	return 0;
+	mapping = file->f_mapping;
+	mapping_set_gfp_mask(old_file->f_mapping, lo->old_gfp_mask);
+	lo->lo_backing_file = file;
+	lo->lo_blocksize = S_ISBLK(mapping->host->i_mode) ?
+		mapping->host->i_bdev->bd_block_size : PAGE_SIZE;
+	lo->old_gfp_mask = mapping_gfp_mask(mapping);
+	mapping_set_gfp_mask(mapping, lo->old_gfp_mask & ~(__GFP_IO|__GFP_FS));
 }
 
 /*
@@ -579,15 +508,18 @@ static int loop_thread(void *data)
 static int loop_switch(struct loop_device *lo, struct file *file)
 {
 	struct switch_request w;
-	struct bio *bio = bio_alloc(GFP_KERNEL, 0);
-	if (!bio)
-		return -ENOMEM;
-	init_completion(&w.wait);
+
 	w.file = file;
-	bio->bi_private = &w;
-	bio->bi_bdev = NULL;
-	loop_make_request(lo->lo_queue, bio);
-	wait_for_completion(&w.wait);
+
+	/* freeze queue and wait for completion of scheduled requests */
+	blk_mq_freeze_queue(lo->lo_queue);
+
+	/* do the switch action */
+	do_loop_switch(lo, &w);
+
+	/* unfreeze */
+	blk_mq_unfreeze_queue(lo->lo_queue);
+
 	return 0;
 }
 
@@ -596,39 +528,10 @@ static int loop_switch(struct loop_device *lo, struct file *file)
  */
 static int loop_flush(struct loop_device *lo)
 {
-	/* loop not yet configured, no running thread, nothing to flush */
-	if (!lo->lo_thread)
-		return 0;
-
 	return loop_switch(lo, NULL);
 }
 
 /*
- * Do the actual switch; called from the BIO completion routine
- */
-static void do_loop_switch(struct loop_device *lo, struct switch_request *p)
-{
-	struct file *file = p->file;
-	struct file *old_file = lo->lo_backing_file;
-	struct address_space *mapping;
-
-	/* if no new file, only flush of queued bios requested */
-	if (!file)
-		goto out;
-
-	mapping = file->f_mapping;
-	mapping_set_gfp_mask(old_file->f_mapping, lo->old_gfp_mask);
-	lo->lo_backing_file = file;
-	lo->lo_blocksize = S_ISBLK(mapping->host->i_mode) ?
-		mapping->host->i_bdev->bd_block_size : PAGE_SIZE;
-	lo->old_gfp_mask = mapping_gfp_mask(mapping);
-	mapping_set_gfp_mask(mapping, lo->old_gfp_mask & ~(__GFP_IO|__GFP_FS));
-out:
-	complete(&p->wait);
-}
-
-
-/*
  * loop_change_fd switched the backing store of a loopback device to
  * a new file. This is useful for operating system installers to free up
  * the original file and in High Availability environments to switch to
@@ -820,6 +723,48 @@ static void loop_config_discard(struct loop_device *lo)
 	queue_flag_set_unlocked(QUEUE_FLAG_DISCARD, q);
 }
 
+static void loop_unprepare_hctxs(struct loop_device *lo, unsigned int to)
+{
+	struct blk_mq_hw_ctx *hctx;
+	struct loop_hctx_data *data;
+	unsigned int i;
+
+	queue_for_each_hw_ctx(lo->lo_queue, hctx, i) {
+		if (i == to)
+			break;
+
+		data = hctx->driver_data;
+		flush_kthread_worker(&data->worker);
+		kthread_stop(data->worker_task);
+	}
+}
+
+static int loop_prepare_hctxs(struct loop_device *lo)
+{
+	struct request_queue *q = lo->lo_queue;
+	struct blk_mq_hw_ctx *hctx;
+	struct loop_hctx_data *data;
+	unsigned int i;
+
+	queue_for_each_hw_ctx(q, hctx, i) {
+		BUG_ON(i >= lo->tag_set.nr_hw_queues);
+		data = hctx->driver_data;
+
+		data->lo = lo;
+		init_kthread_worker(&data->worker);
+		data->worker_task = kthread_run(kthread_worker_fn,
+				&data->worker, "loop%d-%d",
+				lo->lo_number, i);
+		if (IS_ERR(data->worker_task)) {
+			loop_unprepare_hctxs(lo, i);
+			return -ENOMEM;
+		}
+		set_user_nice(data->worker_task, MIN_NICE);
+		sched_getaffinity(data->worker_task->pid, hctx->cpumask);
+	}
+	return 0;
+}
+
 static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 		       struct block_device *bdev, unsigned int arg)
 {
@@ -889,12 +834,9 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 	lo->transfer = transfer_none;
 	lo->ioctl = NULL;
 	lo->lo_sizelimit = 0;
-	lo->lo_bio_count = 0;
 	lo->old_gfp_mask = mapping_gfp_mask(mapping);
 	mapping_set_gfp_mask(mapping, lo->old_gfp_mask & ~(__GFP_IO|__GFP_FS));
 
-	bio_list_init(&lo->lo_bio_list);
-
 	if (!(lo_flags & LO_FLAGS_READ_ONLY) && file->f_op->fsync)
 		blk_queue_flush(lo->lo_queue, REQ_FLUSH);
 
@@ -906,14 +848,10 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 
 	set_blocksize(bdev, lo_blocksize);
 
-	lo->lo_thread = kthread_create(loop_thread, lo, "loop%d",
-						lo->lo_number);
-	if (IS_ERR(lo->lo_thread)) {
-		error = PTR_ERR(lo->lo_thread);
+	if ((error = loop_prepare_hctxs(lo)) != 0)
 		goto out_clr;
-	}
+
 	lo->lo_state = Lo_bound;
-	wake_up_process(lo->lo_thread);
 	if (part_shift)
 		lo->lo_flags |= LO_FLAGS_PARTSCAN;
 	if (lo->lo_flags & LO_FLAGS_PARTSCAN)
@@ -927,7 +865,6 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 
 out_clr:
 	loop_sysfs_exit(lo);
-	lo->lo_thread = NULL;
 	lo->lo_device = NULL;
 	lo->lo_backing_file = NULL;
 	lo->lo_flags = 0;
@@ -1014,7 +951,7 @@ static int loop_clr_fd(struct loop_device *lo)
 	lo->lo_state = Lo_rundown;
 	spin_unlock_irq(&lo->lo_lock);
 
-	kthread_stop(lo->lo_thread);
+	loop_unprepare_hctxs(lo, lo->tag_set.nr_hw_queues);
 
 	spin_lock_irq(&lo->lo_lock);
 	lo->lo_backing_file = NULL;
@@ -1028,7 +965,6 @@ static int loop_clr_fd(struct loop_device *lo)
 	lo->lo_offset = 0;
 	lo->lo_sizelimit = 0;
 	lo->lo_encrypt_key_size = 0;
-	lo->lo_thread = NULL;
 	memset(lo->lo_encrypt_key, 0, LO_KEY_SIZE);
 	memset(lo->lo_crypt_name, 0, LO_NAME_SIZE);
 	memset(lo->lo_file_name, 0, LO_NAME_SIZE);
@@ -1560,6 +1496,9 @@ module_param(max_loop, int, S_IRUGO);
 MODULE_PARM_DESC(max_loop, "Maximum number of loop devices");
 module_param(max_part, int, S_IRUGO);
 MODULE_PARM_DESC(max_part, "Maximum number of partitions per loop device");
+static int nr_queues = 1;
+module_param(nr_queues, int, S_IRUGO);
+MODULE_PARM_DESC(nr_queues, "Number of hw queues per loop device, default: 1");
 MODULE_LICENSE("GPL");
 MODULE_ALIAS_BLOCKDEV_MAJOR(LOOP_MAJOR);
 
@@ -1601,6 +1540,86 @@ int loop_unregister_transfer(int number)
 EXPORT_SYMBOL(loop_register_transfer);
 EXPORT_SYMBOL(loop_unregister_transfer);
 
+static int loop_queue_rq(struct blk_mq_hw_ctx *hctx, struct request *rq)
+{
+	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+	struct loop_hctx_data *data = hctx->driver_data;
+
+	cmd->hctx_data = data;
+	queue_kthread_work(&data->worker, &cmd->work);
+	return BLK_MQ_RQ_QUEUE_OK;
+}
+
+static void loop_queue_work(struct kthread_work *work)
+{
+	struct loop_cmd *cmd =
+		container_of(work, struct loop_cmd, work);
+	const bool write = cmd->rq->cmd_flags & REQ_WRITE;
+	struct loop_device *lo = cmd->hctx_data->lo;
+	int ret = -EIO;
+	struct bio *bio;
+
+	if (lo->lo_state != Lo_bound)
+		goto failed;
+
+	if (write && (lo->lo_flags & LO_FLAGS_READ_ONLY))
+		goto failed;
+
+	ret = 0;
+	__rq_for_each_bio(bio, cmd->rq)
+		ret |= loop_handle_bio(lo, bio);
+
+ failed:
+	if (ret)
+		cmd->rq->errors = -EIO;
+	blk_mq_complete_request(cmd->rq);
+}
+
+static int loop_init_request(void *data, struct request *rq,
+		unsigned int hctx_idx, unsigned int request_idx,
+		unsigned int numa_node)
+{
+	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+
+	cmd->rq = rq;
+	init_kthread_work(&cmd->work, loop_queue_work);
+
+	return 0;
+}
+
+static int loop_prepare_flush_rq(void *data, struct request_queue *q,
+		struct request *flush_rq,
+		const struct request *src_rq)
+{
+	/* borrow initialization helper for common rq */
+	loop_init_request(data, flush_rq, 0, -1, NUMA_NO_NODE);
+	return 0;
+}
+
+static int loop_init_hctx(struct blk_mq_hw_ctx *hctx, void *data,
+		unsigned int index)
+{
+	hctx->driver_data = kmalloc(sizeof(struct loop_hctx_data),
+			GFP_KERNEL);
+	if (!hctx->driver_data)
+		return -ENOMEM;
+	return 0;
+}
+
+static void loop_exit_hctx(struct blk_mq_hw_ctx *hctx, unsigned int index)
+{
+	kfree(hctx->driver_data);
+}
+
+static struct blk_mq_ops loop_mq_ops = {
+	.queue_rq       = loop_queue_rq,
+	.map_queue      = blk_mq_map_queue,
+	.init_request	= loop_init_request,
+	.init_hctx	= loop_init_hctx,
+	.exit_hctx	= loop_exit_hctx,
+	.prepare_flush_rq  = loop_prepare_flush_rq,
+};
+
 static int loop_add(struct loop_device **l, int i)
 {
 	struct loop_device *lo;
@@ -1627,15 +1646,20 @@ static int loop_add(struct loop_device **l, int i)
 	i = err;
 
 	err = -ENOMEM;
-	lo->lo_queue = blk_alloc_queue(GFP_KERNEL);
-	if (!lo->lo_queue)
+	lo->tag_set.ops = &loop_mq_ops;
+	lo->tag_set.nr_hw_queues = nr_queues;
+	lo->tag_set.queue_depth = 128;
+	lo->tag_set.numa_node = NUMA_NO_NODE;
+	lo->tag_set.cmd_size = sizeof(struct loop_cmd);
+	lo->tag_set.flags = BLK_MQ_F_SHOULD_MERGE;
+	lo->tag_set.driver_data = lo;
+
+	if (blk_mq_alloc_tag_set(&lo->tag_set))
 		goto out_free_idr;
 
-	/*
-	 * set queue make_request_fn
-	 */
-	blk_queue_make_request(lo->lo_queue, loop_make_request);
-	lo->lo_queue->queuedata = lo;
+	lo->lo_queue = blk_mq_init_queue(&lo->tag_set);
+	if (!lo->lo_queue)
+		goto out_cleanup_tags;
 
 	disk = lo->lo_disk = alloc_disk(1 << part_shift);
 	if (!disk)
@@ -1664,9 +1688,6 @@ static int loop_add(struct loop_device **l, int i)
 	disk->flags |= GENHD_FL_EXT_DEVT;
 	mutex_init(&lo->lo_ctl_mutex);
 	lo->lo_number		= i;
-	lo->lo_thread		= NULL;
-	init_waitqueue_head(&lo->lo_event);
-	init_waitqueue_head(&lo->lo_req_wait);
 	spin_lock_init(&lo->lo_lock);
 	disk->major		= LOOP_MAJOR;
 	disk->first_minor	= i << part_shift;
@@ -1680,6 +1701,8 @@ static int loop_add(struct loop_device **l, int i)
 
 out_free_queue:
 	blk_cleanup_queue(lo->lo_queue);
+out_cleanup_tags:
+	blk_mq_free_tag_set(&lo->tag_set);
 out_free_idr:
 	idr_remove(&loop_index_idr, i);
 out_free_dev:
@@ -1692,6 +1715,7 @@ static void loop_remove(struct loop_device *lo)
 {
 	del_gendisk(lo->lo_disk);
 	blk_cleanup_queue(lo->lo_queue);
+	blk_mq_free_tag_set(&lo->tag_set);
 	put_disk(lo->lo_disk);
 	kfree(lo);
 }
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index 90df5d6..adfcf4a 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -13,6 +13,7 @@
 #include <linux/blkdev.h>
 #include <linux/spinlock.h>
 #include <linux/mutex.h>
+#include <linux/workqueue.h>
 #include <uapi/linux/loop.h>
 
 /* Possible states of device */
@@ -52,19 +53,26 @@ struct loop_device {
 	gfp_t		old_gfp_mask;
 
 	spinlock_t		lo_lock;
-	struct bio_list		lo_bio_list;
-	unsigned int		lo_bio_count;
 	int			lo_state;
 	struct mutex		lo_ctl_mutex;
-	struct task_struct	*lo_thread;
-	wait_queue_head_t	lo_event;
-	/* wait queue for incoming requests */
-	wait_queue_head_t	lo_req_wait;
 
 	struct request_queue	*lo_queue;
+	struct blk_mq_tag_set	tag_set;
 	struct gendisk		*lo_disk;
 };
 
+struct loop_hctx_data {
+	struct kthread_worker worker;
+	struct task_struct *worker_task;
+	struct loop_device *lo;
+};
+
+struct loop_cmd {
+	struct kthread_work work;
+	struct request *rq;
+	struct loop_hctx_data *hctx_data;
+};
+
 /* Support for loadable transfer modules */
 struct loop_func_table {
 	int number;	/* filter type */ 
-- 
1.7.9.5


^ permalink raw reply related	[flat|nested] 11+ messages in thread

* [PATCH v2 4/6] block: loop: say goodby to bio
  2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
                   ` (2 preceding siblings ...)
  2014-08-30 16:08 ` [PATCH v2 3/6] block: loop: convert to blk-mq Ming Lei
@ 2014-08-30 16:08 ` Ming Lei
  2014-08-30 22:14   ` Elliott, Robert (Server Storage)
  2014-08-30 16:08 ` [PATCH v2 5/6] block: loop: introduce lo_discard() and lo_req_flush() Ming Lei
  2014-08-30 16:08 ` [PATCH v2 6/6] block: loop: don't handle REQ_FUA explicitly Ming Lei
  5 siblings, 1 reply; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Ming Lei

Switch to block request completely.

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c |   46 ++++++++++++++++++++--------------------------
 1 file changed, 20 insertions(+), 26 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index b02122d..b1181b0 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -285,12 +285,12 @@ static int do_lo_send_write(struct loop_device *lo, struct bio_vec *bvec,
 	return ret;
 }
 
-static int lo_send(struct loop_device *lo, struct bio *bio, loff_t pos)
+static int lo_send(struct loop_device *lo, struct request *rq, loff_t pos)
 {
 	int (*do_lo_send)(struct loop_device *, struct bio_vec *, loff_t,
 			struct page *page);
 	struct bio_vec bvec;
-	struct bvec_iter iter;
+	struct req_iterator iter;
 	struct page *page = NULL;
 	int ret = 0;
 
@@ -304,7 +304,7 @@ static int lo_send(struct loop_device *lo, struct bio *bio, loff_t pos)
 		do_lo_send = do_lo_send_direct_write;
 	}
 
-	bio_for_each_segment(bvec, bio, iter) {
+	rq_for_each_segment(bvec, rq, iter) {
 		ret = do_lo_send(lo, &bvec, pos, page);
 		if (ret < 0)
 			break;
@@ -392,19 +392,22 @@ do_lo_receive(struct loop_device *lo,
 }
 
 static int
-lo_receive(struct loop_device *lo, struct bio *bio, int bsize, loff_t pos)
+lo_receive(struct loop_device *lo, struct request *rq, int bsize, loff_t pos)
 {
 	struct bio_vec bvec;
-	struct bvec_iter iter;
+	struct req_iterator iter;
 	ssize_t s;
 
-	bio_for_each_segment(bvec, bio, iter) {
+	rq_for_each_segment(bvec, rq, iter) {
 		s = do_lo_receive(lo, &bvec, bsize, pos);
 		if (s < 0)
 			return s;
 
 		if (s != bvec.bv_len) {
-			zero_fill_bio(bio);
+			struct bio *bio;
+
+			__rq_for_each_bio(bio, rq)
+				zero_fill_bio(bio);
 			break;
 		}
 		pos += bvec.bv_len;
@@ -412,17 +415,17 @@ lo_receive(struct loop_device *lo, struct bio *bio, int bsize, loff_t pos)
 	return 0;
 }
 
-static int do_bio_filebacked(struct loop_device *lo, struct bio *bio)
+static int do_req_filebacked(struct loop_device *lo, struct request *rq)
 {
 	loff_t pos;
 	int ret;
 
-	pos = ((loff_t) bio->bi_iter.bi_sector << 9) + lo->lo_offset;
+	pos = ((loff_t) blk_rq_pos(rq) << 9) + lo->lo_offset;
 
-	if (bio_rw(bio) == WRITE) {
+	if (rq->cmd_flags & REQ_WRITE) {
 		struct file *file = lo->lo_backing_file;
 
-		if (bio->bi_rw & REQ_FLUSH) {
+		if (rq->cmd_flags & REQ_FLUSH) {
 			ret = vfs_fsync(file, 0);
 			if (unlikely(ret && ret != -EINVAL)) {
 				ret = -EIO;
@@ -436,7 +439,7 @@ static int do_bio_filebacked(struct loop_device *lo, struct bio *bio)
 		 * encryption is enabled, because it may give an attacker
 		 * useful information.
 		 */
-		if (bio->bi_rw & REQ_DISCARD) {
+		if (rq->cmd_flags & REQ_DISCARD) {
 			struct file *file = lo->lo_backing_file;
 			int mode = FALLOC_FL_PUNCH_HOLE | FALLOC_FL_KEEP_SIZE;
 
@@ -446,22 +449,22 @@ static int do_bio_filebacked(struct loop_device *lo, struct bio *bio)
 				goto out;
 			}
 			ret = file->f_op->fallocate(file, mode, pos,
-						    bio->bi_iter.bi_size);
+						    blk_rq_bytes(rq));
 			if (unlikely(ret && ret != -EINVAL &&
 				     ret != -EOPNOTSUPP))
 				ret = -EIO;
 			goto out;
 		}
 
-		ret = lo_send(lo, bio, pos);
+		ret = lo_send(lo, rq, pos);
 
-		if ((bio->bi_rw & REQ_FUA) && !ret) {
+		if ((rq->cmd_flags & REQ_FUA) && !ret) {
 			ret = vfs_fsync(file, 0);
 			if (unlikely(ret && ret != -EINVAL))
 				ret = -EIO;
 		}
 	} else
-		ret = lo_receive(lo, bio, lo->lo_blocksize, pos);
+		ret = lo_receive(lo, rq, lo->lo_blocksize, pos);
 
 out:
 	return ret;
@@ -472,12 +475,6 @@ struct switch_request {
 	struct completion wait;
 };
 
-static inline int loop_handle_bio(struct loop_device *lo, struct bio *bio)
-{
-	int ret = do_bio_filebacked(lo, bio);
-	return ret;
-}
-
 /*
  * Do the actual switch; called from the BIO completion routine
  */
@@ -1557,7 +1554,6 @@ static void loop_queue_work(struct kthread_work *work)
 	const bool write = cmd->rq->cmd_flags & REQ_WRITE;
 	struct loop_device *lo = cmd->hctx_data->lo;
 	int ret = -EIO;
-	struct bio *bio;
 
 	if (lo->lo_state != Lo_bound)
 		goto failed;
@@ -1565,9 +1561,7 @@ static void loop_queue_work(struct kthread_work *work)
 	if (write && (lo->lo_flags & LO_FLAGS_READ_ONLY))
 		goto failed;
 
-	ret = 0;
-	__rq_for_each_bio(bio, cmd->rq)
-		ret |= loop_handle_bio(lo, bio);
+	ret = do_req_filebacked(lo, cmd->rq);
 
  failed:
 	if (ret)
-- 
1.7.9.5


^ permalink raw reply related	[flat|nested] 11+ messages in thread

* [PATCH v2 5/6] block: loop: introduce lo_discard() and lo_req_flush()
  2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
                   ` (3 preceding siblings ...)
  2014-08-30 16:08 ` [PATCH v2 4/6] block: loop: say goodby to bio Ming Lei
@ 2014-08-30 16:08 ` Ming Lei
  2014-08-30 16:08 ` [PATCH v2 6/6] block: loop: don't handle REQ_FUA explicitly Ming Lei
  5 siblings, 0 replies; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Ming Lei

No behaviour change, just move the handling for REQ_DISCARD
and REQ_FLUSH in these two functions.

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c |   73 +++++++++++++++++++++++++++-----------------------
 1 file changed, 40 insertions(+), 33 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index b1181b0..d7cdb60 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -415,6 +415,40 @@ lo_receive(struct loop_device *lo, struct request *rq, int bsize, loff_t pos)
 	return 0;
 }
 
+static int lo_discard(struct loop_device *lo, struct request *rq, loff_t pos)
+{
+	/*
+	 * We use punch hole to reclaim the free space used by the
+	 * image a.k.a. discard. However we do not support discard if
+	 * encryption is enabled, because it may give an attacker
+	 * useful information.
+	 */
+	struct file *file = lo->lo_backing_file;
+	int mode = FALLOC_FL_PUNCH_HOLE | FALLOC_FL_KEEP_SIZE;
+	int ret;
+
+	if ((!file->f_op->fallocate) || lo->lo_encrypt_key_size) {
+		ret = -EOPNOTSUPP;
+		goto out;
+	}
+
+	ret = file->f_op->fallocate(file, mode, pos, blk_rq_bytes(rq));
+	if (unlikely(ret && ret != -EINVAL && ret != -EOPNOTSUPP))
+		ret = -EIO;
+ out:
+	return ret;
+}
+
+static int lo_req_flush(struct loop_device *lo, struct request *rq)
+{
+	struct file *file = lo->lo_backing_file;
+	int ret = vfs_fsync(file, 0);
+	if (unlikely(ret && ret != -EINVAL))
+		ret = -EIO;
+
+	return ret;
+}
+
 static int do_req_filebacked(struct loop_device *lo, struct request *rq)
 {
 	loff_t pos;
@@ -423,46 +457,19 @@ static int do_req_filebacked(struct loop_device *lo, struct request *rq)
 	pos = ((loff_t) blk_rq_pos(rq) << 9) + lo->lo_offset;
 
 	if (rq->cmd_flags & REQ_WRITE) {
-		struct file *file = lo->lo_backing_file;
-
-		if (rq->cmd_flags & REQ_FLUSH) {
-			ret = vfs_fsync(file, 0);
-			if (unlikely(ret && ret != -EINVAL)) {
-				ret = -EIO;
-				goto out;
-			}
-		}
 
-		/*
-		 * We use punch hole to reclaim the free space used by the
-		 * image a.k.a. discard. However we do not support discard if
-		 * encryption is enabled, because it may give an attacker
-		 * useful information.
-		 */
+		if (rq->cmd_flags & REQ_FLUSH)
+			ret = lo_req_flush(lo, rq);
+
 		if (rq->cmd_flags & REQ_DISCARD) {
-			struct file *file = lo->lo_backing_file;
-			int mode = FALLOC_FL_PUNCH_HOLE | FALLOC_FL_KEEP_SIZE;
-
-			if ((!file->f_op->fallocate) ||
-			    lo->lo_encrypt_key_size) {
-				ret = -EOPNOTSUPP;
-				goto out;
-			}
-			ret = file->f_op->fallocate(file, mode, pos,
-						    blk_rq_bytes(rq));
-			if (unlikely(ret && ret != -EINVAL &&
-				     ret != -EOPNOTSUPP))
-				ret = -EIO;
+			ret = lo_discard(lo, rq, pos);
 			goto out;
 		}
 
 		ret = lo_send(lo, rq, pos);
 
-		if ((rq->cmd_flags & REQ_FUA) && !ret) {
-			ret = vfs_fsync(file, 0);
-			if (unlikely(ret && ret != -EINVAL))
-				ret = -EIO;
-		}
+		if ((rq->cmd_flags & REQ_FUA) && !ret)
+			ret = lo_req_flush(lo, rq);
 	} else
 		ret = lo_receive(lo, rq, lo->lo_blocksize, pos);
 
-- 
1.7.9.5


^ permalink raw reply related	[flat|nested] 11+ messages in thread

* [PATCH v2 6/6] block: loop: don't handle REQ_FUA explicitly
  2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
                   ` (4 preceding siblings ...)
  2014-08-30 16:08 ` [PATCH v2 5/6] block: loop: introduce lo_discard() and lo_req_flush() Ming Lei
@ 2014-08-30 16:08 ` Ming Lei
  5 siblings, 0 replies; 11+ messages in thread
From: Ming Lei @ 2014-08-30 16:08 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Ming Lei

block core handles REQ_FUA by its flush state machine, so
won't do it in loop explicitly.

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c |   14 +++-----------
 1 file changed, 3 insertions(+), 11 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index d7cdb60..74ad96b 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -457,23 +457,15 @@ static int do_req_filebacked(struct loop_device *lo, struct request *rq)
 	pos = ((loff_t) blk_rq_pos(rq) << 9) + lo->lo_offset;
 
 	if (rq->cmd_flags & REQ_WRITE) {
-
 		if (rq->cmd_flags & REQ_FLUSH)
 			ret = lo_req_flush(lo, rq);
-
-		if (rq->cmd_flags & REQ_DISCARD) {
+		else if (rq->cmd_flags & REQ_DISCARD)
 			ret = lo_discard(lo, rq, pos);
-			goto out;
-		}
-
-		ret = lo_send(lo, rq, pos);
-
-		if ((rq->cmd_flags & REQ_FUA) && !ret)
-			ret = lo_req_flush(lo, rq);
+		else
+			ret = lo_send(lo, rq, pos);
 	} else
 		ret = lo_receive(lo, rq, lo->lo_blocksize, pos);
 
-out:
 	return ret;
 }
 
-- 
1.7.9.5


^ permalink raw reply related	[flat|nested] 11+ messages in thread

* RE: [PATCH v2 3/6] block: loop: convert to blk-mq
  2014-08-30 16:08 ` [PATCH v2 3/6] block: loop: convert to blk-mq Ming Lei
@ 2014-08-30 22:03   ` Elliott, Robert (Server Storage)
  2014-09-01  0:47     ` Ming Lei
  0 siblings, 1 reply; 11+ messages in thread
From: Elliott, Robert (Server Storage) @ 2014-08-30 22:03 UTC (permalink / raw)
  To: Ming Lei, Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov



> -----Original Message-----
> From: linux-kernel-owner@vger.kernel.org [mailto:linux-kernel-
> owner@vger.kernel.org] On Behalf Of Ming Lei
> Sent: Saturday, 30 August, 2014 11:08 AM
> To: Jens Axboe; linux-kernel@vger.kernel.org; Dave Kleikamp
> Cc: Zach Brown; Christoph Hellwig; Maxim Patlasov; Ming Lei
> Subject: [PATCH v2 3/6] block: loop: convert to blk-mq
> 
...
> -static inline void loop_handle_bio(struct loop_device *lo, struct
> bio *bio)
> +static inline int loop_handle_bio(struct loop_device *lo, struct bio
> *bio)
>  {
> -	if (unlikely(!bio->bi_bdev)) {
> -		do_loop_switch(lo, bio->bi_private);
> -		bio_put(bio);
> -	} else {
> -		int ret = do_bio_filebacked(lo, bio);
> -		bio_endio(bio, ret);
> -	}
> +	int ret = do_bio_filebacked(lo, bio);
> +	return ret;

No need for the temporary variable; just return the function
call.

...
> +static int loop_prepare_hctxs(struct loop_device *lo)
> +{
> +	struct request_queue *q = lo->lo_queue;
> +	struct blk_mq_hw_ctx *hctx;
> +	struct loop_hctx_data *data;
> +	unsigned int i;
> +
> +	queue_for_each_hw_ctx(q, hctx, i) {
> +		BUG_ON(i >= lo->tag_set.nr_hw_queues);

If something goes bad in the loop driver like that, is it
necessary to crash the whole kernel?  

> +		data = hctx->driver_data;
> +
> +		data->lo = lo;
> +		init_kthread_worker(&data->worker);
> +		data->worker_task = kthread_run(kthread_worker_fn,
> +				&data->worker, "loop%d-%d",
> +				lo->lo_number, i);
> +		if (IS_ERR(data->worker_task)) {
> +			loop_unprepare_hctxs(lo, i);
> +			return -ENOMEM;
> +		}
> +		set_user_nice(data->worker_task, MIN_NICE);
> +		sched_getaffinity(data->worker_task->pid, hctx->cpumask);

Is that supposed to be sched_setaffinity?  It looks like
getaffinity does have a side-effect of updating the CPU mask
based on the current active cpus, but there won't be a CPU mask
to update unless setaffinity was called.

...
> @@ -906,14 +848,10 @@ static int loop_set_fd(struct loop_device *lo,
> fmode_t mode,
> 
>  	set_blocksize(bdev, lo_blocksize);
> 
> -	lo->lo_thread = kthread_create(loop_thread, lo, "loop%d",
> -						lo->lo_number);
> -	if (IS_ERR(lo->lo_thread)) {
> -		error = PTR_ERR(lo->lo_thread);
> +	if ((error = loop_prepare_hctxs(lo)) != 0)
>  		goto out_clr;

I've been told that linux kernel style is:
	error = x();
	if (error)

...
> @@ -1014,7 +951,7 @@ static int loop_clr_fd(struct loop_device *lo)
>  	lo->lo_state = Lo_rundown;
>  	spin_unlock_irq(&lo->lo_lock);
> 
> -	kthread_stop(lo->lo_thread);
> +	loop_unprepare_hctxs(lo, lo->tag_set.nr_hw_queues);
> 
>  	spin_lock_irq(&lo->lo_lock);
>  	lo->lo_backing_file = NULL;
...
> +
> +static int loop_prepare_flush_rq(void *data, struct request_queue
> *q,
> +		struct request *flush_rq,
> +		const struct request *src_rq)
> +{
> +	/* borrow initialization helper for common rq */
> +	loop_init_request(data, flush_rq, 0, -1, NUMA_NO_NODE);
> +	return 0;
> +}

In patch 2/6 that function is called with:
	if (orig_rq->q->mq_ops->prepare_flush_rq)
		ret = orig_rq->q->mq_ops->prepare_flush_rq(
				orig_rq->q->tag_set->driver_data,
				orig_rq->q, flush_rq, orig_rq);


The src_rq argument is not used in this new function.
Do you anticipate it might be necessary in other drivers?

Is this new function allowed to modify *data, *flush_rq,
or *src_rq?  If not, const might be good to add.

If orig_rq is always passed, then this function could 
decode orig_rq->q and orig_rq->q->tag_set_>driver_data
on its own, eliminating the need for the first two arguments.

Similarly, in the unprepare_flush_rq function in patch 2,
which is not provided by the loop driver in this patch:

+		if (q->mq_ops->unprepare_flush_rq)
+			q->mq_ops->unprepare_flush_rq(
+				q->tag_set->driver_data,
+				q, q->flush_rq);

if q is always passed, then that function could calculate
q->tag_set_driver_data and q->flush_rq itself, eliminating
two arguments.

> +
> +static int loop_init_hctx(struct blk_mq_hw_ctx *hctx, void *data,
> +		unsigned int index)
> +{
> +	hctx->driver_data = kmalloc(sizeof(struct loop_hctx_data),
> +			GFP_KERNEL);

hctx->numa_node has been set before this; how about passing it 
along to kmalloc_node in case it has a useful value?

blk_mq_init_hw_queues sets it before calling this function:
		node = hctx->numa_node;
		if (node == NUMA_NO_NODE)
			node = hctx->numa_node = set->numa_node;
...
               if (set->ops->init_hctx &&
                    set->ops->init_hctx(hctx, set->driver_data, i))
                        break;

loop_add (down lower) just sets set->numa_node to NUMA_NO_NODE
now, but it could hold a more interesting value in the future.

> +	if (!hctx->driver_data)
> +		return -ENOMEM;
> +	return 0;
> +}
> +
> +static void loop_exit_hctx(struct blk_mq_hw_ctx *hctx, unsigned int
> index)
> +{
> +	kfree(hctx->driver_data);
> +}

Setting hctx->driver_data to NULL after kfree would reduce the risk
of the stale pointer ever being used.

> +
> +static struct blk_mq_ops loop_mq_ops = {
> +	.queue_rq       = loop_queue_rq,
> +	.map_queue      = blk_mq_map_queue,
> +	.init_request	= loop_init_request,
> +	.init_hctx	= loop_init_hctx,
> +	.exit_hctx	= loop_exit_hctx,
> +	.prepare_flush_rq  = loop_prepare_flush_rq,
> +};
> +
>  static int loop_add(struct loop_device **l, int i)
>  {
>  	struct loop_device *lo;
> @@ -1627,15 +1646,20 @@ static int loop_add(struct loop_device **l,
> int i)
>  	i = err;
> 
>  	err = -ENOMEM;
> -	lo->lo_queue = blk_alloc_queue(GFP_KERNEL);
> -	if (!lo->lo_queue)
> +	lo->tag_set.ops = &loop_mq_ops;
> +	lo->tag_set.nr_hw_queues = nr_queues;
> +	lo->tag_set.queue_depth = 128;
> +	lo->tag_set.numa_node = NUMA_NO_NODE;

scsi-mq also uses NUMA_NO_NODE right now.  Is there a better
choice?

> +	lo->tag_set.cmd_size = sizeof(struct loop_cmd);
> +	lo->tag_set.flags = BLK_MQ_F_SHOULD_MERGE;

scsi-mq includes BLK_MQ_F_SG_MERGE.  Should this driver?


> +	lo->tag_set.driver_data = lo;
> +
> +	if (blk_mq_alloc_tag_set(&lo->tag_set))
>  		goto out_free_idr;

Use:
	err = blk_mq_alloc_tag_set(&lo->tag_set);
	if (err)
so the return value is propagated out of this function
(the function ends with "return err;")

> 
> -	/*
> -	 * set queue make_request_fn
> -	 */
> -	blk_queue_make_request(lo->lo_queue, loop_make_request);
> -	lo->lo_queue->queuedata = lo;
> +	lo->lo_queue = blk_mq_init_queue(&lo->tag_set);
> +	if (!lo->lo_queue)
> +		goto out_cleanup_tags;

That needs to be IS_ERR(lo->lo_queue) because blk_mq_init_queue
returns ERR_PTR(-ENOMEM) on some errors.  scsi_mq_alloc_queue
does that.

...
> @@ -1680,6 +1701,8 @@ static int loop_add(struct loop_device **l, int
> i)
> 
>  out_free_queue:
>  	blk_cleanup_queue(lo->lo_queue);
> +out_cleanup_tags:
> +	blk_mq_free_tag_set(&lo->tag_set);

Although lo is freed a few lines below, setting lo->tag_set to
NULL would reduce the window in which a stale pointer could be used.

>  out_free_idr:
>  	idr_remove(&loop_index_idr, i);
>  out_free_dev:
> @@ -1692,6 +1715,7 @@ static void loop_remove(struct loop_device *lo)
>  {
>  	del_gendisk(lo->lo_disk);
>  	blk_cleanup_queue(lo->lo_queue);
> +	blk_mq_free_tag_set(&lo->tag_set);

Although lo is freed a few lines below, setting lo->tag_set to
NULL would reduce the window in which a stale pointer could be used.

>  	put_disk(lo->lo_disk);
>  	kfree(lo);
>  }


---
Rob Elliott    HP Server Storage




^ permalink raw reply	[flat|nested] 11+ messages in thread

* RE: [PATCH v2 4/6] block: loop: say goodby to bio
  2014-08-30 16:08 ` [PATCH v2 4/6] block: loop: say goodby to bio Ming Lei
@ 2014-08-30 22:14   ` Elliott, Robert (Server Storage)
  2014-09-01  4:18     ` Ming Lei
  0 siblings, 1 reply; 11+ messages in thread
From: Elliott, Robert (Server Storage) @ 2014-08-30 22:14 UTC (permalink / raw)
  To: Ming Lei, Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov



> -----Original Message-----
> From: linux-kernel-owner@vger.kernel.org [mailto:linux-kernel-
> owner@vger.kernel.org] On Behalf Of Ming Lei
> Sent: Saturday, 30 August, 2014 11:08 AM
> To: Jens Axboe; linux-kernel@vger.kernel.org; Dave Kleikamp
> Cc: Zach Brown; Christoph Hellwig; Maxim Patlasov; Ming Lei
> Subject: [PATCH v2 4/6] block: loop: say goodby to bio
...
> -static int do_bio_filebacked(struct loop_device *lo, struct bio
> *bio)
> +static int do_req_filebacked(struct loop_device *lo, struct request
> *rq)
>  {
>  	loff_t pos;
>  	int ret;
> 
> -	pos = ((loff_t) bio->bi_iter.bi_sector << 9) + lo->lo_offset;
> +	pos = ((loff_t) blk_rq_pos(rq) << 9) + lo->lo_offset;
> 
> -	if (bio_rw(bio) == WRITE) {
> +	if (rq->cmd_flags & REQ_WRITE) {
>  		struct file *file = lo->lo_backing_file;
> 
> -		if (bio->bi_rw & REQ_FLUSH) {
> +		if (rq->cmd_flags & REQ_FLUSH) {
>  			ret = vfs_fsync(file, 0);
>  			if (unlikely(ret && ret != -EINVAL)) {
>  				ret = -EIO;
> @@ -436,7 +439,7 @@ static int do_bio_filebacked(struct loop_device
> *lo, struct bio *bio)
>  		 * encryption is enabled, because it may give an attacker
>  		 * useful information.
>  		 */
> -		if (bio->bi_rw & REQ_DISCARD) {
> +		if (rq->cmd_flags & REQ_DISCARD) {
>  			struct file *file = lo->lo_backing_file;
>  			int mode = FALLOC_FL_PUNCH_HOLE |
> FALLOC_FL_KEEP_SIZE;

Can a REQ_WRITE_SAME make it through to here?


---
Rob Elliott    HP Server Storage




^ permalink raw reply	[flat|nested] 11+ messages in thread

* Re: [PATCH v2 3/6] block: loop: convert to blk-mq
  2014-08-30 22:03   ` Elliott, Robert (Server Storage)
@ 2014-09-01  0:47     ` Ming Lei
  0 siblings, 0 replies; 11+ messages in thread
From: Ming Lei @ 2014-09-01  0:47 UTC (permalink / raw)
  To: Elliott, Robert (Server Storage)
  Cc: Jens Axboe, linux-kernel, Dave Kleikamp, Zach Brown,
	Christoph Hellwig, Maxim Patlasov

Hi Robert,

Great thanks for your so detailed review.

On Sun, Aug 31, 2014 at 6:03 AM, Elliott, Robert (Server Storage)
<Elliott@hp.com> wrote:
>
>
>> -----Original Message-----
>> From: linux-kernel-owner@vger.kernel.org [mailto:linux-kernel-
>> owner@vger.kernel.org] On Behalf Of Ming Lei
>> Sent: Saturday, 30 August, 2014 11:08 AM
>> To: Jens Axboe; linux-kernel@vger.kernel.org; Dave Kleikamp
>> Cc: Zach Brown; Christoph Hellwig; Maxim Patlasov; Ming Lei
>> Subject: [PATCH v2 3/6] block: loop: convert to blk-mq
>>
> ...
>> -static inline void loop_handle_bio(struct loop_device *lo, struct
>> bio *bio)
>> +static inline int loop_handle_bio(struct loop_device *lo, struct bio
>> *bio)
>>  {
>> -     if (unlikely(!bio->bi_bdev)) {
>> -             do_loop_switch(lo, bio->bi_private);
>> -             bio_put(bio);
>> -     } else {
>> -             int ret = do_bio_filebacked(lo, bio);
>> -             bio_endio(bio, ret);
>> -     }
>> +     int ret = do_bio_filebacked(lo, bio);
>> +     return ret;
>
> No need for the temporary variable; just return the function
> call.
>
> ...
>> +static int loop_prepare_hctxs(struct loop_device *lo)
>> +{
>> +     struct request_queue *q = lo->lo_queue;
>> +     struct blk_mq_hw_ctx *hctx;
>> +     struct loop_hctx_data *data;
>> +     unsigned int i;
>> +
>> +     queue_for_each_hw_ctx(q, hctx, i) {
>> +             BUG_ON(i >= lo->tag_set.nr_hw_queues);
>
> If something goes bad in the loop driver like that, is it
> necessary to crash the whole kernel?

Actually, the BUG_ON() shouldn't have been added, will remove it.

>
>> +             data = hctx->driver_data;
>> +
>> +             data->lo = lo;
>> +             init_kthread_worker(&data->worker);
>> +             data->worker_task = kthread_run(kthread_worker_fn,
>> +                             &data->worker, "loop%d-%d",
>> +                             lo->lo_number, i);
>> +             if (IS_ERR(data->worker_task)) {
>> +                     loop_unprepare_hctxs(lo, i);
>> +                     return -ENOMEM;
>> +             }
>> +             set_user_nice(data->worker_task, MIN_NICE);
>> +             sched_getaffinity(data->worker_task->pid, hctx->cpumask);
>
> Is that supposed to be sched_setaffinity?  It looks like
> getaffinity does have a side-effect of updating the CPU mask
> based on the current active cpus, but there won't be a CPU mask
> to update unless setaffinity was called.

Hmm, it is a typo, and I meant sched_setaffinity(), but it isn't exported,
so set_cpus_allowed_ptr() should be used.

>
> ...
>> @@ -906,14 +848,10 @@ static int loop_set_fd(struct loop_device *lo,
>> fmode_t mode,
>>
>>       set_blocksize(bdev, lo_blocksize);
>>
>> -     lo->lo_thread = kthread_create(loop_thread, lo, "loop%d",
>> -                                             lo->lo_number);
>> -     if (IS_ERR(lo->lo_thread)) {
>> -             error = PTR_ERR(lo->lo_thread);
>> +     if ((error = loop_prepare_hctxs(lo)) != 0)
>>               goto out_clr;
>
> I've been told that linux kernel style is:
>         error = x();
>         if (error)
>
> ...
>> @@ -1014,7 +951,7 @@ static int loop_clr_fd(struct loop_device *lo)
>>       lo->lo_state = Lo_rundown;
>>       spin_unlock_irq(&lo->lo_lock);
>>
>> -     kthread_stop(lo->lo_thread);
>> +     loop_unprepare_hctxs(lo, lo->tag_set.nr_hw_queues);
>>
>>       spin_lock_irq(&lo->lo_lock);
>>       lo->lo_backing_file = NULL;
> ...
>> +
>> +static int loop_prepare_flush_rq(void *data, struct request_queue
>> *q,
>> +             struct request *flush_rq,
>> +             const struct request *src_rq)
>> +{
>> +     /* borrow initialization helper for common rq */
>> +     loop_init_request(data, flush_rq, 0, -1, NUMA_NO_NODE);
>> +     return 0;
>> +}
>
> In patch 2/6 that function is called with:
>         if (orig_rq->q->mq_ops->prepare_flush_rq)
>                 ret = orig_rq->q->mq_ops->prepare_flush_rq(
>                                 orig_rq->q->tag_set->driver_data,
>                                 orig_rq->q, flush_rq, orig_rq);
>
>
> The src_rq argument is not used in this new function.
> Do you anticipate it might be necessary in other drivers?

Yes, sooner or later the problem will be triggered in blk-mq
conversion for current drivers, and current default behaviour is to
copy pdu of src_rq to q->flush_rq, that is why the src_rq is passed.

> Is this new function allowed to modify *data, *flush_rq,
> or *src_rq?  If not, const might be good to add.

Instance pointed by data and src_rq shouldn't be modified.

>
> If orig_rq is always passed, then this function could
> decode orig_rq->q and orig_rq->q->tag_set_>driver_data
> on its own, eliminating the need for the first two arguments.

Looks a good idea.

>
> Similarly, in the unprepare_flush_rq function in patch 2,
> which is not provided by the loop driver in this patch:
>
> +               if (q->mq_ops->unprepare_flush_rq)
> +                       q->mq_ops->unprepare_flush_rq(
> +                               q->tag_set->driver_data,
> +                               q, q->flush_rq);
>
> if q is always passed, then that function could calculate
> q->tag_set_driver_data and q->flush_rq itself, eliminating
> two arguments.

I suggest to keep 'q' and 'q->flush_rq' because the callback
is closely related with flush req.

>
>> +
>> +static int loop_init_hctx(struct blk_mq_hw_ctx *hctx, void *data,
>> +             unsigned int index)
>> +{
>> +     hctx->driver_data = kmalloc(sizeof(struct loop_hctx_data),
>> +                     GFP_KERNEL);
>
> hctx->numa_node has been set before this; how about passing it
> along to kmalloc_node in case it has a useful value?

Yes, it is a good point.

> blk_mq_init_hw_queues sets it before calling this function:
>                 node = hctx->numa_node;
>                 if (node == NUMA_NO_NODE)
>                         node = hctx->numa_node = set->numa_node;
> ...
>                if (set->ops->init_hctx &&
>                     set->ops->init_hctx(hctx, set->driver_data, i))
>                         break;
>
> loop_add (down lower) just sets set->numa_node to NUMA_NO_NODE
> now, but it could hold a more interesting value in the future.

If nr_hw_queues is more than one, it has been set a more useful
value in blk_mq_init_cpu_queues().

>
>> +     if (!hctx->driver_data)
>> +             return -ENOMEM;
>> +     return 0;
>> +}
>> +
>> +static void loop_exit_hctx(struct blk_mq_hw_ctx *hctx, unsigned int
>> index)
>> +{
>> +     kfree(hctx->driver_data);
>> +}
>
> Setting hctx->driver_data to NULL after kfree would reduce the risk
> of the stale pointer ever being used.

If it is true, I suggest to do that in blk-mq.c instead of drivers.

>
>> +
>> +static struct blk_mq_ops loop_mq_ops = {
>> +     .queue_rq       = loop_queue_rq,
>> +     .map_queue      = blk_mq_map_queue,
>> +     .init_request   = loop_init_request,
>> +     .init_hctx      = loop_init_hctx,
>> +     .exit_hctx      = loop_exit_hctx,
>> +     .prepare_flush_rq  = loop_prepare_flush_rq,
>> +};
>> +
>>  static int loop_add(struct loop_device **l, int i)
>>  {
>>       struct loop_device *lo;
>> @@ -1627,15 +1646,20 @@ static int loop_add(struct loop_device **l,
>> int i)
>>       i = err;
>>
>>       err = -ENOMEM;
>> -     lo->lo_queue = blk_alloc_queue(GFP_KERNEL);
>> -     if (!lo->lo_queue)
>> +     lo->tag_set.ops = &loop_mq_ops;
>> +     lo->tag_set.nr_hw_queues = nr_queues;
>> +     lo->tag_set.queue_depth = 128;
>> +     lo->tag_set.numa_node = NUMA_NO_NODE;
>
> scsi-mq also uses NUMA_NO_NODE right now.  Is there a better
> choice?

They should be in same situation wrt. this problem.

>
>> +     lo->tag_set.cmd_size = sizeof(struct loop_cmd);
>> +     lo->tag_set.flags = BLK_MQ_F_SHOULD_MERGE;
>
> scsi-mq includes BLK_MQ_F_SG_MERGE.  Should this driver?

That is an interesting question, and maybe it is better to not do it
in loop and just depend on back file's.

>
>
>> +     lo->tag_set.driver_data = lo;
>> +
>> +     if (blk_mq_alloc_tag_set(&lo->tag_set))
>>               goto out_free_idr;
>
> Use:
>         err = blk_mq_alloc_tag_set(&lo->tag_set);
>         if (err)
> so the return value is propagated out of this function
> (the function ends with "return err;")

Right.

>>
>> -     /*
>> -      * set queue make_request_fn
>> -      */
>> -     blk_queue_make_request(lo->lo_queue, loop_make_request);
>> -     lo->lo_queue->queuedata = lo;
>> +     lo->lo_queue = blk_mq_init_queue(&lo->tag_set);
>> +     if (!lo->lo_queue)
>> +             goto out_cleanup_tags;
>
> That needs to be IS_ERR(lo->lo_queue) because blk_mq_init_queue
> returns ERR_PTR(-ENOMEM) on some errors.  scsi_mq_alloc_queue
> does that.

Good catch.

>
> ...
>> @@ -1680,6 +1701,8 @@ static int loop_add(struct loop_device **l, int
>> i)
>>
>>  out_free_queue:
>>       blk_cleanup_queue(lo->lo_queue);
>> +out_cleanup_tags:
>> +     blk_mq_free_tag_set(&lo->tag_set);
>
> Although lo is freed a few lines below, setting lo->tag_set to
> NULL would reduce the window in which a stale pointer could be used.

No, lo->tag_set isn't a pointer and the case needn't to worry about since
the queue has been cleaned up already.

>
>>  out_free_idr:
>>       idr_remove(&loop_index_idr, i);
>>  out_free_dev:
>> @@ -1692,6 +1715,7 @@ static void loop_remove(struct loop_device *lo)
>>  {
>>       del_gendisk(lo->lo_disk);
>>       blk_cleanup_queue(lo->lo_queue);
>> +     blk_mq_free_tag_set(&lo->tag_set);
>
> Although lo is freed a few lines below, setting lo->tag_set to
> NULL would reduce the window in which a stale pointer could be used.

Same with above.

>
>>       put_disk(lo->lo_disk);
>>       kfree(lo);
>>  }


Thanks,
--
Ming Lei

^ permalink raw reply	[flat|nested] 11+ messages in thread

* Re: [PATCH v2 4/6] block: loop: say goodby to bio
  2014-08-30 22:14   ` Elliott, Robert (Server Storage)
@ 2014-09-01  4:18     ` Ming Lei
  0 siblings, 0 replies; 11+ messages in thread
From: Ming Lei @ 2014-09-01  4:18 UTC (permalink / raw)
  To: Elliott, Robert (Server Storage)
  Cc: Jens Axboe, linux-kernel, Dave Kleikamp, Zach Brown,
	Christoph Hellwig, Maxim Patlasov

On Sun, Aug 31, 2014 at 6:14 AM, Elliott, Robert (Server Storage)
<Elliott@hp.com> wrote:
>
>
>> -----Original Message-----
>> From: linux-kernel-owner@vger.kernel.org [mailto:linux-kernel-
>> owner@vger.kernel.org] On Behalf Of Ming Lei
>> Sent: Saturday, 30 August, 2014 11:08 AM
>> To: Jens Axboe; linux-kernel@vger.kernel.org; Dave Kleikamp
>> Cc: Zach Brown; Christoph Hellwig; Maxim Patlasov; Ming Lei
>> Subject: [PATCH v2 4/6] block: loop: say goodby to bio
> ...
>> -static int do_bio_filebacked(struct loop_device *lo, struct bio
>> *bio)
>> +static int do_req_filebacked(struct loop_device *lo, struct request
>> *rq)
>>  {
>>       loff_t pos;
>>       int ret;
>>
>> -     pos = ((loff_t) bio->bi_iter.bi_sector << 9) + lo->lo_offset;
>> +     pos = ((loff_t) blk_rq_pos(rq) << 9) + lo->lo_offset;
>>
>> -     if (bio_rw(bio) == WRITE) {
>> +     if (rq->cmd_flags & REQ_WRITE) {
>>               struct file *file = lo->lo_backing_file;
>>
>> -             if (bio->bi_rw & REQ_FLUSH) {
>> +             if (rq->cmd_flags & REQ_FLUSH) {
>>                       ret = vfs_fsync(file, 0);
>>                       if (unlikely(ret && ret != -EINVAL)) {
>>                               ret = -EIO;
>> @@ -436,7 +439,7 @@ static int do_bio_filebacked(struct loop_device
>> *lo, struct bio *bio)
>>                * encryption is enabled, because it may give an attacker
>>                * useful information.
>>                */
>> -             if (bio->bi_rw & REQ_DISCARD) {
>> +             if (rq->cmd_flags & REQ_DISCARD) {
>>                       struct file *file = lo->lo_backing_file;
>>                       int mode = FALLOC_FL_PUNCH_HOLE |
>> FALLOC_FL_KEEP_SIZE;
>
> Can a REQ_WRITE_SAME make it through to here?

It can't because loop doesn't support WRITE_SAME.


Thanks,
--
Ming Lei

^ permalink raw reply	[flat|nested] 11+ messages in thread

end of thread, other threads:[~2014-09-01  4:18 UTC | newest]

Thread overview: 11+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2014-08-30 16:08 block: loop: convert to blk-mq Ming Lei
2014-08-30 16:08 ` [PATCH v2 1/6] blk-mq: export blk_mq_freeze_queue and blk_mq_unfreeze_queue Ming Lei
2014-08-30 16:08 ` [PATCH v2 2/6] blk-mq: introduce prepare_flush_rq_fn callback and its pair Ming Lei
2014-08-30 16:08 ` [PATCH v2 3/6] block: loop: convert to blk-mq Ming Lei
2014-08-30 22:03   ` Elliott, Robert (Server Storage)
2014-09-01  0:47     ` Ming Lei
2014-08-30 16:08 ` [PATCH v2 4/6] block: loop: say goodby to bio Ming Lei
2014-08-30 22:14   ` Elliott, Robert (Server Storage)
2014-09-01  4:18     ` Ming Lei
2014-08-30 16:08 ` [PATCH v2 5/6] block: loop: introduce lo_discard() and lo_req_flush() Ming Lei
2014-08-30 16:08 ` [PATCH v2 6/6] block: loop: don't handle REQ_FUA explicitly Ming Lei

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.