linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v8 0/6] block: loop: improve loop with AIO
@ 2015-07-30 11:36 Ming Lei
  2015-07-30 11:36 ` [PATCH v8 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
                   ` (5 more replies)
  0 siblings, 6 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner

Hi Guys,

There are about 3 advantages to use direct I/O and AIO on
read/write loop's backing file:

1) double cache can be avoided, then memory usage gets
decreased a lot

2) not like user space direct I/O, there isn't cost of
pinning pages

3) avoid context switch for obtaining good throughput
- in buffered file read, random I/O throughput is often obtained
only if they are submitted concurrently from lots of tasks; but for
sequential I/O, most of times they can be hit from page cache, so
concurrent submissions often introduce unnecessary context switch
and can't improve throughput much. There was such discussion[1]
to use non-blocking I/O to improve the problem for application.
- with direct I/O and AIO, concurrent submissions can be
avoided and random read throughput can't be affected meantime

So this patchset trys to improve loop via AIO, and about 45% memory
usage can be decreased, see detailed data in commit log of patch4,
also IO throughput isn't affected too.

Following losetup changes for supporting loop dio/aio:
	- losetup git tree:
		http://kernel.ubuntu.com/git/ming/util-linux.git/log/?h=loop-dio-v7
	- how to set direct io
		losetup --direct-io 1 /dev/loopN
	- how to clear direct io
		losetup --direct-io 0 /dev/loopN
	- how to show if direct io is used for accessing backing file
		losetup -l

V8:
        - only 4/6 and 5/6 updated
        - support direct io when backing device's sector size is
          512, 1024, 2048 and 4096(only 512-byte sector was supported
          in V7)
V7:
        - only 4/6 and 5/6 updated
        - update direct io after lo->offset is changed(4/6)
        - fix updating LO_FLAGS_DIRECT_IO in __loop_update_dio()(4/6)
        - introduce ioctl cmd of LOOP_SET_DIRECT_IO(5/6) for case of
          'mount -o loop'
V6:
        - only patch 4 and patch 5 get updated
        - check lo->lo_offset to decide if direct IO can be supported(4/5)
        - introduce one flag for userspace(losetup) to keep updated
        if using direct I/O to access backing file(4/5)
        - implement patches for util-linux(losetup) so that losetup can
        enable direct I/O feature:(4/5)

           http://kernel.ubuntu.com/git/ming/util-linux.git/log/?h=losetup-dio

        - remove the direct IO control interface from sysfs(4/5)
        - handle partial read in case of direct read (5/5)
        - add more comments for direct IO (5/5)
V5:
        - don't introduce IOCB_DONT_DIRTY_PAGE and bypass dirtying
        for ITER_KVEC and ITER_BVEC direct IO(read), as required by
        Christoph

V4:
        - add detailed commit log for 'use kthread_work'
        - allow userspace(sysfs, losetup) to decide if dio/aio is
        used as suggested by Christoph and Dave Chinner
        - only use dio if the backing block device's min io size
        is 512 as pointed by Dave Chinner & Christoph
V3:
        - based on Al's iov_iter work and Christoph's kiocb changes
        - use kthread_work
        - introduce IOCB_DONT_DIRTY_PAGE flag
        - set QUEUE_FLAG_NOMERGES for loop's request queue
V2:
        - remove 'extra' parameter to aio_kernel_alloc()
        - try to avoid memory allcation inside queue req callback
        - introduce 'use_mq' sysfs file for enabling kernel aio or disabling it
V1:
        - link:
                http://marc.info/?t=140803157700004&r=1&w=2
        - improve failure path in aio_kernel_submit()

 drivers/block/loop.c      | 283 ++++++++++++++++++++++++++++++++++++++++++++++++-------------
 drivers/block/loop.h      |  15 ++--
 fs/direct-io.c            |   9 +-
 include/uapi/linux/loop.h |   2 +
 4 files changed, 243 insertions(+), 66 deletions(-)


Thanks,
Ming


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

* [PATCH v8 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read
  2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
@ 2015-07-30 11:36 ` Ming Lei
  2015-07-30 11:36 ` [PATCH v8 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop Ming Lei
                   ` (4 subsequent siblings)
  5 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, Ming Lei

When direct read IO is submitted from kernel, it is often
unnecessary to dirty pages, for example of loop, dirtying pages
have been considered in the upper filesystem(over loop) side
already, and they don't need to be dirtied again.

So this patch doesn't dirtying pages for ITER_BVEC/ITER_KVEC
direct read, and loop should be the 1st case to use ITER_BVEC/ITER_KVEC
for direct read I/O.

The patch is based on previous Dave's patch.

Reviewed-by: Dave Kleikamp <dave.kleikamp@oracle.com>
Reviewed-by: Christoph Hellwig <hch@lst.de>
Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 fs/direct-io.c | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)

diff --git a/fs/direct-io.c b/fs/direct-io.c
index 745d234..07e4b28 100644
--- a/fs/direct-io.c
+++ b/fs/direct-io.c
@@ -120,6 +120,7 @@ struct dio {
 	int page_errors;		/* errno from get_user_pages() */
 	int is_async;			/* is IO async ? */
 	bool defer_completion;		/* defer AIO completion to workqueue? */
+	bool should_dirty;		/* if pages should be dirtied */
 	int io_error;			/* IO error in completion path */
 	unsigned long refcount;		/* direct_io_worker() and bios */
 	struct bio *bio_list;		/* singly linked via bi_private */
@@ -393,7 +394,7 @@ static inline void dio_bio_submit(struct dio *dio, struct dio_submit *sdio)
 	dio->refcount++;
 	spin_unlock_irqrestore(&dio->bio_lock, flags);
 
-	if (dio->is_async && dio->rw == READ)
+	if (dio->is_async && dio->rw == READ && dio->should_dirty)
 		bio_set_pages_dirty(bio);
 
 	if (sdio->submit_io)
@@ -464,13 +465,14 @@ static int dio_bio_complete(struct dio *dio, struct bio *bio)
 	if (!uptodate)
 		dio->io_error = -EIO;
 
-	if (dio->is_async && dio->rw == READ) {
+	if (dio->is_async && dio->rw == READ && dio->should_dirty) {
 		bio_check_pages_dirty(bio);	/* transfers ownership */
 	} else {
 		bio_for_each_segment_all(bvec, bio, i) {
 			struct page *page = bvec->bv_page;
 
-			if (dio->rw == READ && !PageCompound(page))
+			if (dio->rw == READ && !PageCompound(page) &&
+					dio->should_dirty)
 				set_page_dirty_lock(page);
 			page_cache_release(page);
 		}
@@ -1217,6 +1219,7 @@ do_blockdev_direct_IO(struct kiocb *iocb, struct inode *inode,
 	spin_lock_init(&dio->bio_lock);
 	dio->refcount = 1;
 
+	dio->should_dirty = (iter->type == ITER_IOVEC);
 	sdio.iter = iter;
 	sdio.final_block_in_request =
 		(offset + iov_iter_count(iter)) >> blkbits;
-- 
1.9.1


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

* [PATCH v8 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop
  2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
  2015-07-30 11:36 ` [PATCH v8 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
@ 2015-07-30 11:36 ` Ming Lei
  2015-07-30 11:36 ` [PATCH v8 3/6] block: loop: use kthread_work Ming Lei
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, Ming Lei

It doesn't make sense to enable merge because the I/O
submitted to backing file is handled page by page.

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

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index f9889b6..eee7510 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -1595,6 +1595,12 @@ static int loop_add(struct loop_device **l, int i)
 	}
 	lo->lo_queue->queuedata = lo;
 
+	/*
+	 * It doesn't make sense to enable merge because the I/O
+	 * submitted to backing file is handled page by page.
+	 */
+	queue_flag_set_unlocked(QUEUE_FLAG_NOMERGES, lo->lo_queue);
+
 	INIT_LIST_HEAD(&lo->write_cmd_head);
 	INIT_WORK(&lo->write_work, loop_queue_write_work);
 
-- 
1.9.1


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

* [PATCH v8 3/6] block: loop: use kthread_work
  2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
  2015-07-30 11:36 ` [PATCH v8 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
  2015-07-30 11:36 ` [PATCH v8 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop Ming Lei
@ 2015-07-30 11:36 ` Ming Lei
  2015-07-30 11:36 ` [PATCH v8 4/6] block: loop: prepare for supporing direct IO Ming Lei
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, Ming Lei

The following patch will use dio/aio to submit IO to backing file,
then it needn't to schedule IO concurrently from work, so
use kthread_work for decreasing context switch cost a lot.

For non-AIO case, single thread has been used for long long time,
and it was just converted to work in v4.0, which has caused performance
regression for fedora live booting already. In discussion[1], even
though submitting I/O via work concurrently can improve random read IO
throughput, meantime it might hurt sequential read IO performance, so
better to restore to single thread behaviour.

For the following AIO support, it is better to use multi hw-queue
with per-hwq kthread than current work approach suppose there is so
high performance requirement for loop.

[1] http://marc.info/?t=143082678400002&r=1&w=2

Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c | 79 ++++++++++++++++------------------------------------
 drivers/block/loop.h | 10 +++----
 2 files changed, 28 insertions(+), 61 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index eee7510..1875aad 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -688,6 +688,23 @@ static void loop_config_discard(struct loop_device *lo)
 	queue_flag_set_unlocked(QUEUE_FLAG_DISCARD, q);
 }
 
+static void loop_unprepare_queue(struct loop_device *lo)
+{
+	flush_kthread_worker(&lo->worker);
+	kthread_stop(lo->worker_task);
+}
+
+static int loop_prepare_queue(struct loop_device *lo)
+{
+	init_kthread_worker(&lo->worker);
+	lo->worker_task = kthread_run(kthread_worker_fn,
+			&lo->worker, "loop%d", lo->lo_number);
+	if (IS_ERR(lo->worker_task))
+		return -ENOMEM;
+	set_user_nice(lo->worker_task, MIN_NICE);
+	return 0;
+}
+
 static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 		       struct block_device *bdev, unsigned int arg)
 {
@@ -745,11 +762,8 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 	size = get_loop_size(lo, file);
 	if ((loff_t)(sector_t)size != size)
 		goto out_putf;
-	error = -ENOMEM;
-	lo->wq = alloc_workqueue("kloopd%d",
-			WQ_MEM_RECLAIM | WQ_HIGHPRI | WQ_UNBOUND, 16,
-			lo->lo_number);
-	if (!lo->wq)
+	error = loop_prepare_queue(lo);
+	if (error)
 		goto out_putf;
 
 	error = 0;
@@ -903,8 +917,7 @@ static int loop_clr_fd(struct loop_device *lo)
 	lo->lo_flags = 0;
 	if (!part_shift)
 		lo->lo_disk->flags |= GENHD_FL_NO_PART_SCAN;
-	destroy_workqueue(lo->wq);
-	lo->wq = NULL;
+	loop_unprepare_queue(lo);
 	mutex_unlock(&lo->lo_ctl_mutex);
 	/*
 	 * Need not hold lo_ctl_mutex to fput backing file.
@@ -1461,23 +1474,7 @@ static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
 	if (lo->lo_state != Lo_bound)
 		return -EIO;
 
-	if (cmd->rq->cmd_flags & REQ_WRITE) {
-		struct loop_device *lo = cmd->rq->q->queuedata;
-		bool need_sched = true;
-
-		spin_lock_irq(&lo->lo_lock);
-		if (lo->write_started)
-			need_sched = false;
-		else
-			lo->write_started = true;
-		list_add_tail(&cmd->list, &lo->write_cmd_head);
-		spin_unlock_irq(&lo->lo_lock);
-
-		if (need_sched)
-			queue_work(lo->wq, &lo->write_work);
-	} else {
-		queue_work(lo->wq, &cmd->read_work);
-	}
+	queue_kthread_work(&lo->worker, &cmd->work);
 
 	return BLK_MQ_RQ_QUEUE_OK;
 }
@@ -1499,35 +1496,10 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 	blk_mq_complete_request(cmd->rq);
 }
 
-static void loop_queue_write_work(struct work_struct *work)
-{
-	struct loop_device *lo =
-		container_of(work, struct loop_device, write_work);
-	LIST_HEAD(cmd_list);
-
-	spin_lock_irq(&lo->lo_lock);
- repeat:
-	list_splice_init(&lo->write_cmd_head, &cmd_list);
-	spin_unlock_irq(&lo->lo_lock);
-
-	while (!list_empty(&cmd_list)) {
-		struct loop_cmd *cmd = list_first_entry(&cmd_list,
-				struct loop_cmd, list);
-		list_del_init(&cmd->list);
-		loop_handle_cmd(cmd);
-	}
-
-	spin_lock_irq(&lo->lo_lock);
-	if (!list_empty(&lo->write_cmd_head))
-		goto repeat;
-	lo->write_started = false;
-	spin_unlock_irq(&lo->lo_lock);
-}
-
-static void loop_queue_read_work(struct work_struct *work)
+static void loop_queue_work(struct kthread_work *work)
 {
 	struct loop_cmd *cmd =
-		container_of(work, struct loop_cmd, read_work);
+		container_of(work, struct loop_cmd, work);
 
 	loop_handle_cmd(cmd);
 }
@@ -1539,7 +1511,7 @@ static int loop_init_request(void *data, struct request *rq,
 	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
 
 	cmd->rq = rq;
-	INIT_WORK(&cmd->read_work, loop_queue_read_work);
+	init_kthread_work(&cmd->work, loop_queue_work);
 
 	return 0;
 }
@@ -1601,9 +1573,6 @@ static int loop_add(struct loop_device **l, int i)
 	 */
 	queue_flag_set_unlocked(QUEUE_FLAG_NOMERGES, lo->lo_queue);
 
-	INIT_LIST_HEAD(&lo->write_cmd_head);
-	INIT_WORK(&lo->write_work, loop_queue_write_work);
-
 	disk = lo->lo_disk = alloc_disk(1 << part_shift);
 	if (!disk)
 		goto out_free_queue;
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index 25e8997..b6c7d21 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -14,7 +14,7 @@
 #include <linux/blk-mq.h>
 #include <linux/spinlock.h>
 #include <linux/mutex.h>
-#include <linux/workqueue.h>
+#include <linux/kthread.h>
 #include <uapi/linux/loop.h>
 
 /* Possible states of device */
@@ -54,12 +54,10 @@ struct loop_device {
 	gfp_t		old_gfp_mask;
 
 	spinlock_t		lo_lock;
-	struct workqueue_struct *wq;
-	struct list_head	write_cmd_head;
-	struct work_struct	write_work;
-	bool			write_started;
 	int			lo_state;
 	struct mutex		lo_ctl_mutex;
+	struct kthread_worker	worker;
+	struct task_struct	*worker_task;
 
 	struct request_queue	*lo_queue;
 	struct blk_mq_tag_set	tag_set;
@@ -67,7 +65,7 @@ struct loop_device {
 };
 
 struct loop_cmd {
-	struct work_struct read_work;
+	struct kthread_work work;
 	struct request *rq;
 	struct list_head list;
 };
-- 
1.9.1


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

* [PATCH v8 4/6] block: loop: prepare for supporing direct IO
  2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (2 preceding siblings ...)
  2015-07-30 11:36 ` [PATCH v8 3/6] block: loop: use kthread_work Ming Lei
@ 2015-07-30 11:36 ` Ming Lei
  2015-07-30 15:09   ` Christoph Hellwig
  2015-07-30 15:30   ` Dave Kleikamp
  2015-07-30 11:36 ` [PATCH v8 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO Ming Lei
  2015-07-30 11:36 ` [PATCH v8 6/6] block: loop: support DIO & AIO Ming Lei
  5 siblings, 2 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, Ming Lei, linux-api

This patches provides one interface for enabling direct IO
from user space:

	- userspace(such as losetup) can pass 'file' which is
	opened/fcntl as O_DIRECT

Also __loop_update_dio() is introduced to check if direct I/O
can be used on current loop setting.

The last big change is to introduce LO_FLAGS_DIRECT_IO flag
for userspace to know if direct IO is used to access backing
file.

Cc: linux-api@vger.kernel.org
Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c      | 63 ++++++++++++++++++++++++++++++++++++++++++++++-
 drivers/block/loop.h      |  2 ++
 include/uapi/linux/loop.h |  1 +
 3 files changed, 65 insertions(+), 1 deletion(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 1875aad..799cc23 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -164,6 +164,47 @@ static loff_t get_loop_size(struct loop_device *lo, struct file *file)
 	return get_size(lo->lo_offset, lo->lo_sizelimit, file);
 }
 
+static void __loop_update_dio(struct loop_device *lo, bool dio)
+{
+	struct file *file = lo->lo_backing_file;
+	struct inode *inode = file->f_mapping->host;
+	bool use_dio;
+	unsigned dio_align = inode->i_sb->s_bdev ?
+		(bdev_io_min(inode->i_sb->s_bdev) - 1) : 0;
+
+	/*
+	 * We support direct I/O only if lo_offset is aligned
+	 * with the min I/O size of backing device.
+	 *
+	 * Request's offset and size will be checked in I/O path.
+	 */
+	if (dio) {
+		if (!dio_align || (lo->lo_offset & dio_align))
+			use_dio = false;
+		else
+			use_dio = true;
+	} else {
+		use_dio = false;
+	}
+
+	/* flush dirty pages before changing direct IO */
+	vfs_fsync(file, 0);
+
+	/*
+	 * The flag of LO_FLAGS_DIRECT_IO is handled similarly with
+	 * LO_FLAGS_READ_ONLY, both are set from kernel, and losetup
+	 * will get updated by ioctl(LOOP_GET_STATUS)
+	 */
+	blk_mq_freeze_queue(lo->lo_queue);
+	lo->use_dio = use_dio;
+	lo->dio_align = dio_align;
+	if (use_dio)
+		lo->lo_flags |= LO_FLAGS_DIRECT_IO;
+	else
+		lo->lo_flags &= ~LO_FLAGS_DIRECT_IO;
+	blk_mq_unfreeze_queue(lo->lo_queue);
+}
+
 static int
 figure_loop_size(struct loop_device *lo, loff_t offset, loff_t sizelimit)
 {
@@ -173,8 +214,12 @@ figure_loop_size(struct loop_device *lo, loff_t offset, loff_t sizelimit)
 
 	if (unlikely((loff_t)x != size))
 		return -EFBIG;
-	if (lo->lo_offset != offset)
+	if (lo->lo_offset != offset) {
 		lo->lo_offset = offset;
+
+		/* update dio if lo_offset is changed*/
+		__loop_update_dio(lo, lo->use_dio);
+	}
 	if (lo->lo_sizelimit != sizelimit)
 		lo->lo_sizelimit = sizelimit;
 	set_capacity(lo->lo_disk, x);
@@ -421,6 +466,11 @@ struct switch_request {
 	struct completion wait;
 };
 
+static inline void loop_update_dio(struct loop_device *lo)
+{
+	__loop_update_dio(lo, io_is_direct(lo->lo_backing_file));
+}
+
 /*
  * Do the actual switch; called from the BIO completion routine
  */
@@ -441,6 +491,7 @@ static void do_loop_switch(struct loop_device *lo, struct switch_request *p)
 		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));
+	loop_update_dio(lo);
 }
 
 /*
@@ -627,11 +678,19 @@ static ssize_t loop_attr_partscan_show(struct loop_device *lo, char *buf)
 	return sprintf(buf, "%s\n", partscan ? "1" : "0");
 }
 
+static ssize_t loop_attr_dio_show(struct loop_device *lo, char *buf)
+{
+	int dio = (lo->lo_flags & LO_FLAGS_DIRECT_IO);
+
+	return sprintf(buf, "%s\n", dio ? "1" : "0");
+}
+
 LOOP_ATTR_RO(backing_file);
 LOOP_ATTR_RO(offset);
 LOOP_ATTR_RO(sizelimit);
 LOOP_ATTR_RO(autoclear);
 LOOP_ATTR_RO(partscan);
+LOOP_ATTR_RO(dio);
 
 static struct attribute *loop_attrs[] = {
 	&loop_attr_backing_file.attr,
@@ -639,6 +698,7 @@ static struct attribute *loop_attrs[] = {
 	&loop_attr_sizelimit.attr,
 	&loop_attr_autoclear.attr,
 	&loop_attr_partscan.attr,
+	&loop_attr_dio.attr,
 	NULL,
 };
 
@@ -783,6 +843,7 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
 	if (!(lo_flags & LO_FLAGS_READ_ONLY) && file->f_op->fsync)
 		blk_queue_flush(lo->lo_queue, REQ_FLUSH);
 
+	loop_update_dio(lo);
 	set_capacity(lo->lo_disk, size);
 	bd_set_size(bdev, size << 9);
 	loop_sysfs_init(lo);
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index b6c7d21..63f8e14 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -58,6 +58,8 @@ struct loop_device {
 	struct mutex		lo_ctl_mutex;
 	struct kthread_worker	worker;
 	struct task_struct	*worker_task;
+	unsigned		dio_align;
+	bool			use_dio;
 
 	struct request_queue	*lo_queue;
 	struct blk_mq_tag_set	tag_set;
diff --git a/include/uapi/linux/loop.h b/include/uapi/linux/loop.h
index e0cecd2..949851c 100644
--- a/include/uapi/linux/loop.h
+++ b/include/uapi/linux/loop.h
@@ -21,6 +21,7 @@ enum {
 	LO_FLAGS_READ_ONLY	= 1,
 	LO_FLAGS_AUTOCLEAR	= 4,
 	LO_FLAGS_PARTSCAN	= 8,
+	LO_FLAGS_DIRECT_IO	= 16,
 };
 
 #include <asm/posix_types.h>	/* for __kernel_old_dev_t */
-- 
1.9.1


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

* [PATCH v8 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO
  2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (3 preceding siblings ...)
  2015-07-30 11:36 ` [PATCH v8 4/6] block: loop: prepare for supporing direct IO Ming Lei
@ 2015-07-30 11:36 ` Ming Lei
  2015-07-30 11:36 ` [PATCH v8 6/6] block: loop: support DIO & AIO Ming Lei
  5 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, Ming Lei, linux-api

If loop block is mounted via 'mount -o loop', it isn't easy
to pass file descriptor opened as O_DIRECT, so this patch
introduces a new command to support direct IO for this case.

Cc: linux-api@vger.kernel.org
Signed-off-by: Ming Lei <ming.lei@canonical.com>
---
 drivers/block/loop.c      | 19 +++++++++++++++++++
 include/uapi/linux/loop.h |  1 +
 2 files changed, 20 insertions(+)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 799cc23..133e4c7 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -1212,6 +1212,20 @@ static int loop_set_capacity(struct loop_device *lo, struct block_device *bdev)
 	return figure_loop_size(lo, lo->lo_offset, lo->lo_sizelimit);
 }
 
+static int loop_set_dio(struct loop_device *lo, unsigned long arg)
+{
+	int error = -ENXIO;
+	if (lo->lo_state != Lo_bound)
+		goto out;
+
+	__loop_update_dio(lo, !!arg);
+	if (lo->use_dio == !!arg)
+		return 0;
+	error = -EINVAL;
+ out:
+	return error;
+}
+
 static int lo_ioctl(struct block_device *bdev, fmode_t mode,
 	unsigned int cmd, unsigned long arg)
 {
@@ -1255,6 +1269,11 @@ static int lo_ioctl(struct block_device *bdev, fmode_t mode,
 		if ((mode & FMODE_WRITE) || capable(CAP_SYS_ADMIN))
 			err = loop_set_capacity(lo, bdev);
 		break;
+	case LOOP_SET_DIRECT_IO:
+		err = -EPERM;
+		if ((mode & FMODE_WRITE) || capable(CAP_SYS_ADMIN))
+			err = loop_set_dio(lo, arg);
+		break;
 	default:
 		err = lo->ioctl ? lo->ioctl(lo, cmd, arg) : -EINVAL;
 	}
diff --git a/include/uapi/linux/loop.h b/include/uapi/linux/loop.h
index 949851c..c8125ec 100644
--- a/include/uapi/linux/loop.h
+++ b/include/uapi/linux/loop.h
@@ -87,6 +87,7 @@ struct loop_info64 {
 #define LOOP_GET_STATUS64	0x4C05
 #define LOOP_CHANGE_FD		0x4C06
 #define LOOP_SET_CAPACITY	0x4C07
+#define LOOP_SET_DIRECT_IO	0x4C08
 
 /* /dev/loop-control interface */
 #define LOOP_CTL_ADD		0x4C80
-- 
1.9.1


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

* [PATCH v8 6/6] block: loop: support DIO & AIO
  2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (4 preceding siblings ...)
  2015-07-30 11:36 ` [PATCH v8 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO Ming Lei
@ 2015-07-30 11:36 ` Ming Lei
  2015-07-30 16:42   ` Christoph Hellwig
  5 siblings, 1 reply; 13+ messages in thread
From: Ming Lei @ 2015-07-30 11:36 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel, Dave Kleikamp
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, Ming Lei

There are about 3 advantages to use direct I/O and AIO on
read/write loop's backing file:

1) double cache can be avoided, then memory usage gets
decreased a lot

2) not like user space direct I/O, there isn't cost of
pinning pages

3) avoid context switch for obtaining good throughput
- in buffered file read, random I/O top throughput is often obtained
only if they are submitted concurrently from lots of tasks; but for
sequential I/O, most of times they can be hit from page cache, so
concurrent submissions often introduce unnecessary context switch
and can't improve throughput much. There was such discussion[1]
to use non-blocking I/O to improve the problem for application.
- with direct I/O and AIO, concurrent submissions can be
avoided and random read throughput can't be affected meantime

Follows my fio test result:

1. 16 jobs fio test inside ext4 file system over loop block
1) How to run
	- linux kernel: 4.1.0-rc2-next-20150506 with the patchset
	- the loop block is over one image on HDD.
	- linux psync, 16 jobs, size 400M, ext4 over loop block
	- test result: IOPS from fio output

2) Throughput result:
        -------------------------------------------------------------
        test cases          |randread   |read   |randwrite  |write  |
        -------------------------------------------------------------
        base                |240        |8705   |3763       |20914
        -------------------------------------------------------------
        base+loop aio       |242        |9258   |4577       |21451
        -------------------------------------------------------------

3) context switch
        - context switch decreased by ~16% with loop aio for randread,
	and decreased by ~33% for read

4) memory usage
	- After these four tests with loop aio: ~10% memory becomes used
	- After these four tests without loop aio: more than 55% memory
	becomes used

2. single job fio test inside ext4 file system over loop block(for Maxim Patlasov)
1) How to run
	- linux kernel: 4.1.0-rc2-next-20150506 with the patchset
	- the loop block is over one image on HDD.
	- linux psync, 1 job, size 4000M, ext4 over loop block
	- test result: IOPS from fio output

2) Throughput result:
        -------------------------------------------------------------
        test cases          |randread   |read   |randwrite  |write  |
        -------------------------------------------------------------
        base                |109        |21180  |4192       |22782
        -------------------------------------------------------------
        base+loop aio       |114        |21018  |5404       |22670
        -------------------------------------------------------------

3) context switch
        - context switch decreased by ~10% with loop aio for randread,
	and decreased by ~50% for read

4) memory usage
	- After these four tests with loop aio: ~10% memory becomes used
	- After these four tests without loop aio: more than 55% memory
	becomes used

Both 'context switch' and 'memory usage' data are got from sar.

[1] https://lwn.net/Articles/612483/
[2] sar graph when running fio over loop without the patchset
http://kernel.ubuntu.com/~ming/block/loop-aio/v3/lo-nonaio.pdf

[3] sar graph when running fio over loop with the patchset
http://kernel.ubuntu.com/~ming/block/loop-aio/v3/lo-aio.pdf

[4] sar graph when running fio over loop without the patchset
http://kernel.ubuntu.com/~ming/block/loop-aio/v3/lo-nonaio-1job.pdf

[5] sar graph when running fio over loop with the patchset
http://kernel.ubuntu.com/~ming/block/loop-aio/v3/lo-aio-1job.pdf

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

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 133e4c7..36f930a 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -198,6 +198,7 @@ static void __loop_update_dio(struct loop_device *lo, bool dio)
 	blk_mq_freeze_queue(lo->lo_queue);
 	lo->use_dio = use_dio;
 	lo->dio_align = dio_align;
+	lo->last_use_dio = use_dio;
 	if (use_dio)
 		lo->lo_flags |= LO_FLAGS_DIRECT_IO;
 	else
@@ -434,6 +435,102 @@ static int lo_req_flush(struct loop_device *lo, struct request *rq)
 	return ret;
 }
 
+static inline void handle_partial_read(struct loop_cmd *cmd, long bytes)
+{
+	if (bytes < 0 || (cmd->rq->cmd_flags & REQ_WRITE))
+		return;
+
+	if (unlikely(bytes < blk_rq_bytes(cmd->rq))) {
+		struct bio *bio = cmd->rq->bio;
+
+		bio_advance(bio, bytes);
+		zero_fill_bio(bio);
+	}
+}
+
+static void lo_rw_aio_complete(struct kiocb *iocb, long ret, long ret2)
+{
+	struct loop_cmd *cmd = container_of(iocb, struct loop_cmd, iocb);
+	struct request *rq = cmd->rq;
+
+	handle_partial_read(cmd, ret);
+
+	if (ret > 0)
+		ret = 0;
+	else if (ret < 0)
+		ret = -EIO;
+
+	rq->errors = ret;
+	blk_mq_complete_request(rq);
+}
+
+static int lo_rw_aio(struct loop_device *lo, struct loop_cmd *cmd,
+		     loff_t pos, bool rw)
+{
+	struct iov_iter iter;
+	struct bio_vec *bvec;
+	struct bio *bio = cmd->rq->bio;
+	struct file *file = lo->lo_backing_file;
+	int ret;
+
+	/* nomerge for loop request queue */
+	WARN_ON(cmd->rq->bio != cmd->rq->biotail);
+
+	bvec = __bvec_iter_bvec(bio->bi_io_vec, bio->bi_iter);
+	iov_iter_bvec(&iter, ITER_BVEC | rw, bvec,
+		      bio_segments(bio), blk_rq_bytes(cmd->rq));
+
+	cmd->iocb.ki_pos = pos;
+	cmd->iocb.ki_filp = file;
+	cmd->iocb.ki_complete = lo_rw_aio_complete;
+	cmd->iocb.ki_flags = IOCB_DIRECT;
+
+	if (rw == WRITE)
+		ret = file->f_op->write_iter(&cmd->iocb, &iter);
+	else
+		ret = file->f_op->read_iter(&cmd->iocb, &iter);
+
+	if (ret != -EIOCBQUEUED)
+		cmd->iocb.ki_complete(&cmd->iocb, ret, 0);
+	return 0;
+}
+
+
+static inline int lo_rw_simple(struct loop_device *lo,
+		struct request *rq, loff_t pos, bool rw)
+{
+	struct loop_cmd *cmd = blk_mq_rq_to_pdu(rq);
+
+	/*
+	 * When working at direct I/O, under very unusual cases,
+	 * such as unaligned direct I/O from application and
+	 * access to loop block device with 'unaligned' offset & size,
+	 * we have to fallback to non-dio mode.
+	 *
+	 * During the switch between dio and non-dio, page cache
+	 * has to be flushed to the backing file.
+	 */
+	if (unlikely(lo->use_dio && lo->last_use_dio != cmd->use_aio))
+		vfs_fsync(lo->lo_backing_file, 0);
+
+	if (cmd->use_aio)
+		return lo_rw_aio(lo, cmd, pos, rw);
+
+	/*
+	 * lo_write_simple and lo_read_simple should have been covered
+	 * by io submit style function like lo_rw_aio(), one blocker
+	 * is that lo_read_simple() need to call flush_dcache_page after
+	 * the page is written from kernel, and it isn't easy to handle
+	 * this in io submit style function which submits all segments
+	 * of the req at one time. And direct read IO doesn't need to
+	 * run flush_dcache_page().
+	 */
+	if (rw == WRITE)
+		return lo_write_simple(lo, rq, pos);
+	else
+		return lo_read_simple(lo, rq, pos);
+}
+
 static int do_req_filebacked(struct loop_device *lo, struct request *rq)
 {
 	loff_t pos;
@@ -449,13 +546,13 @@ static int do_req_filebacked(struct loop_device *lo, struct request *rq)
 		else if (lo->transfer)
 			ret = lo_write_transfer(lo, rq, pos);
 		else
-			ret = lo_write_simple(lo, rq, pos);
+			ret = lo_rw_simple(lo, rq, pos, WRITE);
 
 	} else {
 		if (lo->transfer)
 			ret = lo_read_transfer(lo, rq, pos);
 		else
-			ret = lo_read_simple(lo, rq, pos);
+			ret = lo_rw_simple(lo, rq, pos, READ);
 	}
 
 	return ret;
@@ -1543,6 +1640,13 @@ int loop_unregister_transfer(int number)
 EXPORT_SYMBOL(loop_register_transfer);
 EXPORT_SYMBOL(loop_unregister_transfer);
 
+static inline bool req_dio_aligned(struct loop_device *lo,
+		const struct request *rq)
+{
+	return !((blk_rq_pos(rq) << 9) & lo->dio_align) &&
+		!(blk_rq_bytes(rq) & lo->dio_align);
+}
+
 static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
 		const struct blk_mq_queue_data *bd)
 {
@@ -1554,6 +1658,13 @@ static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
 	if (lo->lo_state != Lo_bound)
 		return -EIO;
 
+	if (lo->use_dio && !lo->transfer &&
+			req_dio_aligned(lo, bd->rq) &&
+			!(cmd->rq->cmd_flags & (REQ_FLUSH | REQ_DISCARD)))
+		cmd->use_aio = true;
+	else
+		cmd->use_aio = false;
+
 	queue_kthread_work(&lo->worker, &cmd->work);
 
 	return BLK_MQ_RQ_QUEUE_OK;
@@ -1569,11 +1680,14 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
 		goto failed;
 
 	ret = do_req_filebacked(lo, cmd->rq);
+	lo->last_use_dio = cmd->use_aio;
 
  failed:
 	if (ret)
 		cmd->rq->errors = -EIO;
-	blk_mq_complete_request(cmd->rq);
+	/* complete non-aio request */
+	if (!cmd->use_aio || ret)
+		blk_mq_complete_request(cmd->rq);
 }
 
 static void loop_queue_work(struct kthread_work *work)
diff --git a/drivers/block/loop.h b/drivers/block/loop.h
index 63f8e14..f0ba26d 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -60,6 +60,7 @@ struct loop_device {
 	struct task_struct	*worker_task;
 	unsigned		dio_align;
 	bool			use_dio;
+	bool			last_use_dio;
 
 	struct request_queue	*lo_queue;
 	struct blk_mq_tag_set	tag_set;
@@ -70,6 +71,8 @@ struct loop_cmd {
 	struct kthread_work work;
 	struct request *rq;
 	struct list_head list;
+	bool use_aio;           /* use AIO interface to handle I/O */
+	struct kiocb iocb;
 };
 
 /* Support for loadable transfer modules */
-- 
1.9.1


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

* Re: [PATCH v8 4/6] block: loop: prepare for supporing direct IO
  2015-07-30 11:36 ` [PATCH v8 4/6] block: loop: prepare for supporing direct IO Ming Lei
@ 2015-07-30 15:09   ` Christoph Hellwig
  2015-07-30 15:21     ` Ming Lei
  2015-07-30 15:30   ` Dave Kleikamp
  1 sibling, 1 reply; 13+ messages in thread
From: Christoph Hellwig @ 2015-07-30 15:09 UTC (permalink / raw)
  To: Ming Lei
  Cc: Jens Axboe, linux-kernel, Dave Kleikamp, Zach Brown,
	Christoph Hellwig, Maxim Patlasov, Andrew Morton, Alexander Viro,
	Tejun Heo, Dave Chinner, linux-api

On Thu, Jul 30, 2015 at 07:36:22AM -0400, Ming Lei wrote:
> This patches provides one interface for enabling direct IO
> from user space:
> 
> 	- userspace(such as losetup) can pass 'file' which is
> 	opened/fcntl as O_DIRECT
> 
> Also __loop_update_dio() is introduced to check if direct I/O
> can be used on current loop setting.
> 
> The last big change is to introduce LO_FLAGS_DIRECT_IO flag
> for userspace to know if direct IO is used to access backing
> file.

Maybe i'm missing something because I was too busy to follow the
current discussion, but this still doesn't check that the loop
device sector size is aligned to the sector size of the underlying
filesystem.

E.g. with this you could create a loop device with a 512 byte
sector size on a filesystem with 4k sector size, and it would attempt
to use direct I/O and fail.

> +	unsigned dio_align = inode->i_sb->s_bdev ?
> +		(bdev_io_min(inode->i_sb->s_bdev) - 1) : 0;
> +
> +	/*
> +	 * We support direct I/O only if lo_offset is aligned
> +	 * with the min I/O size of backing device.
> +	 *
> +	 * Request's offset and size will be checked in I/O path.
> +	 */
> +	if (dio) {
> +		if (!dio_align || (lo->lo_offset & dio_align))
> +			use_dio = false;

Also this means you'll never use direct I/O on network filesystems,
which really would benefit from it.

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

* Re: [PATCH v8 4/6] block: loop: prepare for supporing direct IO
  2015-07-30 15:09   ` Christoph Hellwig
@ 2015-07-30 15:21     ` Ming Lei
  0 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 15:21 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Jens Axboe, Linux Kernel Mailing List, Dave Kleikamp, Zach Brown,
	Maxim Patlasov, Andrew Morton, Alexander Viro, Tejun Heo,
	Dave Chinner, linux-api

On Thu, Jul 30, 2015 at 11:09 AM, Christoph Hellwig <hch@infradead.org> wrote:
> On Thu, Jul 30, 2015 at 07:36:22AM -0400, Ming Lei wrote:
>> This patches provides one interface for enabling direct IO
>> from user space:
>>
>>       - userspace(such as losetup) can pass 'file' which is
>>       opened/fcntl as O_DIRECT
>>
>> Also __loop_update_dio() is introduced to check if direct I/O
>> can be used on current loop setting.
>>
>> The last big change is to introduce LO_FLAGS_DIRECT_IO flag
>> for userspace to know if direct IO is used to access backing
>> file.
>
> Maybe i'm missing something because I was too busy to follow the
> current discussion, but this still doesn't check that the loop
> device sector size is aligned to the sector size of the underlying
> filesystem.
>
> E.g. with this you could create a loop device with a 512 byte
> sector size on a filesystem with 4k sector size, and it would attempt
> to use direct I/O and fail.

The I/O size & offset has to be checked in I/O path, see patch 6 please,
could you comment on that patch?

>
>> +     unsigned dio_align = inode->i_sb->s_bdev ?
>> +             (bdev_io_min(inode->i_sb->s_bdev) - 1) : 0;
>> +
>> +     /*
>> +      * We support direct I/O only if lo_offset is aligned
>> +      * with the min I/O size of backing device.
>> +      *
>> +      * Request's offset and size will be checked in I/O path.
>> +      */
>> +     if (dio) {
>> +             if (!dio_align || (lo->lo_offset & dio_align))
>> +                     use_dio = false;
>
> Also this means you'll never use direct I/O on network filesystems,
> which really would benefit from it.

OK, that can be done by removing !dio_align in above check.

Thanks,

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

* Re: [PATCH v8 4/6] block: loop: prepare for supporing direct IO
  2015-07-30 11:36 ` [PATCH v8 4/6] block: loop: prepare for supporing direct IO Ming Lei
  2015-07-30 15:09   ` Christoph Hellwig
@ 2015-07-30 15:30   ` Dave Kleikamp
  2015-07-30 15:45     ` Ming Lei
  1 sibling, 1 reply; 13+ messages in thread
From: Dave Kleikamp @ 2015-07-30 15:30 UTC (permalink / raw)
  To: Ming Lei, Jens Axboe, linux-kernel
  Cc: Zach Brown, Christoph Hellwig, Maxim Patlasov, Andrew Morton,
	Alexander Viro, Tejun Heo, Dave Chinner, linux-api

On 07/30/2015 06:36 AM, Ming Lei wrote:
> This patches provides one interface for enabling direct IO
> from user space:
> 
> 	- userspace(such as losetup) can pass 'file' which is
> 	opened/fcntl as O_DIRECT
> 
> Also __loop_update_dio() is introduced to check if direct I/O
> can be used on current loop setting.
> 
> The last big change is to introduce LO_FLAGS_DIRECT_IO flag
> for userspace to know if direct IO is used to access backing
> file.

lo->use_dio and LO_FLAGS_DIRECT_IO seem redundant. Wouldn't it be
simpler to use one or the other?

> 
> Cc: linux-api@vger.kernel.org
> Signed-off-by: Ming Lei <ming.lei@canonical.com>
> ---
>  drivers/block/loop.c      | 63 ++++++++++++++++++++++++++++++++++++++++++++++-
>  drivers/block/loop.h      |  2 ++
>  include/uapi/linux/loop.h |  1 +
>  3 files changed, 65 insertions(+), 1 deletion(-)
> 
> diff --git a/drivers/block/loop.c b/drivers/block/loop.c
> index 1875aad..799cc23 100644
> --- a/drivers/block/loop.c
> +++ b/drivers/block/loop.c
> @@ -164,6 +164,47 @@ static loff_t get_loop_size(struct loop_device *lo, struct file *file)
>  	return get_size(lo->lo_offset, lo->lo_sizelimit, file);
>  }
>  
> +static void __loop_update_dio(struct loop_device *lo, bool dio)
> +{
> +	struct file *file = lo->lo_backing_file;
> +	struct inode *inode = file->f_mapping->host;
> +	bool use_dio;
> +	unsigned dio_align = inode->i_sb->s_bdev ?
> +		(bdev_io_min(inode->i_sb->s_bdev) - 1) : 0;
> +
> +	/*
> +	 * We support direct I/O only if lo_offset is aligned
> +	 * with the min I/O size of backing device.
> +	 *
> +	 * Request's offset and size will be checked in I/O path.
> +	 */
> +	if (dio) {
> +		if (!dio_align || (lo->lo_offset & dio_align))
> +			use_dio = false;
> +		else
> +			use_dio = true;
> +	} else {
> +		use_dio = false;
> +	}
> +
> +	/* flush dirty pages before changing direct IO */
> +	vfs_fsync(file, 0);
> +
> +	/*
> +	 * The flag of LO_FLAGS_DIRECT_IO is handled similarly with
> +	 * LO_FLAGS_READ_ONLY, both are set from kernel, and losetup
> +	 * will get updated by ioctl(LOOP_GET_STATUS)
> +	 */
> +	blk_mq_freeze_queue(lo->lo_queue);
> +	lo->use_dio = use_dio;
> +	lo->dio_align = dio_align;
> +	if (use_dio)
> +		lo->lo_flags |= LO_FLAGS_DIRECT_IO;
> +	else
> +		lo->lo_flags &= ~LO_FLAGS_DIRECT_IO;
> +	blk_mq_unfreeze_queue(lo->lo_queue);
> +}
> +
>  static int
>  figure_loop_size(struct loop_device *lo, loff_t offset, loff_t sizelimit)
>  {
> @@ -173,8 +214,12 @@ figure_loop_size(struct loop_device *lo, loff_t offset, loff_t sizelimit)
>  
>  	if (unlikely((loff_t)x != size))
>  		return -EFBIG;
> -	if (lo->lo_offset != offset)
> +	if (lo->lo_offset != offset) {
>  		lo->lo_offset = offset;
> +
> +		/* update dio if lo_offset is changed*/
> +		__loop_update_dio(lo, lo->use_dio);
> +	}
>  	if (lo->lo_sizelimit != sizelimit)
>  		lo->lo_sizelimit = sizelimit;
>  	set_capacity(lo->lo_disk, x);
> @@ -421,6 +466,11 @@ struct switch_request {
>  	struct completion wait;
>  };
>  
> +static inline void loop_update_dio(struct loop_device *lo)
> +{
> +	__loop_update_dio(lo, io_is_direct(lo->lo_backing_file));
> +}
> +
>  /*
>   * Do the actual switch; called from the BIO completion routine
>   */
> @@ -441,6 +491,7 @@ static void do_loop_switch(struct loop_device *lo, struct switch_request *p)
>  		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));
> +	loop_update_dio(lo);
>  }
>  
>  /*
> @@ -627,11 +678,19 @@ static ssize_t loop_attr_partscan_show(struct loop_device *lo, char *buf)
>  	return sprintf(buf, "%s\n", partscan ? "1" : "0");
>  }
>  
> +static ssize_t loop_attr_dio_show(struct loop_device *lo, char *buf)
> +{
> +	int dio = (lo->lo_flags & LO_FLAGS_DIRECT_IO);
> +
> +	return sprintf(buf, "%s\n", dio ? "1" : "0");
> +}
> +
>  LOOP_ATTR_RO(backing_file);
>  LOOP_ATTR_RO(offset);
>  LOOP_ATTR_RO(sizelimit);
>  LOOP_ATTR_RO(autoclear);
>  LOOP_ATTR_RO(partscan);
> +LOOP_ATTR_RO(dio);
>  
>  static struct attribute *loop_attrs[] = {
>  	&loop_attr_backing_file.attr,
> @@ -639,6 +698,7 @@ static struct attribute *loop_attrs[] = {
>  	&loop_attr_sizelimit.attr,
>  	&loop_attr_autoclear.attr,
>  	&loop_attr_partscan.attr,
> +	&loop_attr_dio.attr,
>  	NULL,
>  };
>  
> @@ -783,6 +843,7 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
>  	if (!(lo_flags & LO_FLAGS_READ_ONLY) && file->f_op->fsync)
>  		blk_queue_flush(lo->lo_queue, REQ_FLUSH);
>  
> +	loop_update_dio(lo);
>  	set_capacity(lo->lo_disk, size);
>  	bd_set_size(bdev, size << 9);
>  	loop_sysfs_init(lo);
> diff --git a/drivers/block/loop.h b/drivers/block/loop.h
> index b6c7d21..63f8e14 100644
> --- a/drivers/block/loop.h
> +++ b/drivers/block/loop.h
> @@ -58,6 +58,8 @@ struct loop_device {
>  	struct mutex		lo_ctl_mutex;
>  	struct kthread_worker	worker;
>  	struct task_struct	*worker_task;
> +	unsigned		dio_align;
> +	bool			use_dio;
>  
>  	struct request_queue	*lo_queue;
>  	struct blk_mq_tag_set	tag_set;
> diff --git a/include/uapi/linux/loop.h b/include/uapi/linux/loop.h
> index e0cecd2..949851c 100644
> --- a/include/uapi/linux/loop.h
> +++ b/include/uapi/linux/loop.h
> @@ -21,6 +21,7 @@ enum {
>  	LO_FLAGS_READ_ONLY	= 1,
>  	LO_FLAGS_AUTOCLEAR	= 4,
>  	LO_FLAGS_PARTSCAN	= 8,
> +	LO_FLAGS_DIRECT_IO	= 16,
>  };
>  
>  #include <asm/posix_types.h>	/* for __kernel_old_dev_t */
> 

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

* Re: [PATCH v8 4/6] block: loop: prepare for supporing direct IO
  2015-07-30 15:30   ` Dave Kleikamp
@ 2015-07-30 15:45     ` Ming Lei
  0 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-30 15:45 UTC (permalink / raw)
  To: Dave Kleikamp
  Cc: Jens Axboe, Linux Kernel Mailing List, Zach Brown,
	Christoph Hellwig, Maxim Patlasov, Andrew Morton, Alexander Viro,
	Tejun Heo, Dave Chinner, linux-api

On Thu, Jul 30, 2015 at 11:30 AM, Dave Kleikamp
<dave.kleikamp@oracle.com> wrote:
> On 07/30/2015 06:36 AM, Ming Lei wrote:
>> This patches provides one interface for enabling direct IO
>> from user space:
>>
>>       - userspace(such as losetup) can pass 'file' which is
>>       opened/fcntl as O_DIRECT
>>
>> Also __loop_update_dio() is introduced to check if direct I/O
>> can be used on current loop setting.
>>
>> The last big change is to introduce LO_FLAGS_DIRECT_IO flag
>> for userspace to know if direct IO is used to access backing
>> file.
>
> lo->use_dio and LO_FLAGS_DIRECT_IO seem redundant. Wouldn't it be
> simpler to use one or the other?

losetup is a stateless utility, which means it need to retrieve the dio status
via the flag of LO_FLAGS_DIRECT_IO, which is one API change.

>
>>
>> Cc: linux-api@vger.kernel.org
>> Signed-off-by: Ming Lei <ming.lei@canonical.com>
>> ---
>>  drivers/block/loop.c      | 63 ++++++++++++++++++++++++++++++++++++++++++++++-
>>  drivers/block/loop.h      |  2 ++
>>  include/uapi/linux/loop.h |  1 +
>>  3 files changed, 65 insertions(+), 1 deletion(-)
>>
>> diff --git a/drivers/block/loop.c b/drivers/block/loop.c
>> index 1875aad..799cc23 100644
>> --- a/drivers/block/loop.c
>> +++ b/drivers/block/loop.c
>> @@ -164,6 +164,47 @@ static loff_t get_loop_size(struct loop_device *lo, struct file *file)
>>       return get_size(lo->lo_offset, lo->lo_sizelimit, file);
>>  }
>>
>> +static void __loop_update_dio(struct loop_device *lo, bool dio)
>> +{
>> +     struct file *file = lo->lo_backing_file;
>> +     struct inode *inode = file->f_mapping->host;
>> +     bool use_dio;
>> +     unsigned dio_align = inode->i_sb->s_bdev ?
>> +             (bdev_io_min(inode->i_sb->s_bdev) - 1) : 0;
>> +
>> +     /*
>> +      * We support direct I/O only if lo_offset is aligned
>> +      * with the min I/O size of backing device.
>> +      *
>> +      * Request's offset and size will be checked in I/O path.
>> +      */
>> +     if (dio) {
>> +             if (!dio_align || (lo->lo_offset & dio_align))
>> +                     use_dio = false;
>> +             else
>> +                     use_dio = true;
>> +     } else {
>> +             use_dio = false;
>> +     }
>> +
>> +     /* flush dirty pages before changing direct IO */
>> +     vfs_fsync(file, 0);
>> +
>> +     /*
>> +      * The flag of LO_FLAGS_DIRECT_IO is handled similarly with
>> +      * LO_FLAGS_READ_ONLY, both are set from kernel, and losetup
>> +      * will get updated by ioctl(LOOP_GET_STATUS)
>> +      */
>> +     blk_mq_freeze_queue(lo->lo_queue);
>> +     lo->use_dio = use_dio;
>> +     lo->dio_align = dio_align;
>> +     if (use_dio)
>> +             lo->lo_flags |= LO_FLAGS_DIRECT_IO;
>> +     else
>> +             lo->lo_flags &= ~LO_FLAGS_DIRECT_IO;
>> +     blk_mq_unfreeze_queue(lo->lo_queue);
>> +}
>> +
>>  static int
>>  figure_loop_size(struct loop_device *lo, loff_t offset, loff_t sizelimit)
>>  {
>> @@ -173,8 +214,12 @@ figure_loop_size(struct loop_device *lo, loff_t offset, loff_t sizelimit)
>>
>>       if (unlikely((loff_t)x != size))
>>               return -EFBIG;
>> -     if (lo->lo_offset != offset)
>> +     if (lo->lo_offset != offset) {
>>               lo->lo_offset = offset;
>> +
>> +             /* update dio if lo_offset is changed*/
>> +             __loop_update_dio(lo, lo->use_dio);
>> +     }
>>       if (lo->lo_sizelimit != sizelimit)
>>               lo->lo_sizelimit = sizelimit;
>>       set_capacity(lo->lo_disk, x);
>> @@ -421,6 +466,11 @@ struct switch_request {
>>       struct completion wait;
>>  };
>>
>> +static inline void loop_update_dio(struct loop_device *lo)
>> +{
>> +     __loop_update_dio(lo, io_is_direct(lo->lo_backing_file));
>> +}
>> +
>>  /*
>>   * Do the actual switch; called from the BIO completion routine
>>   */
>> @@ -441,6 +491,7 @@ static void do_loop_switch(struct loop_device *lo, struct switch_request *p)
>>               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));
>> +     loop_update_dio(lo);
>>  }
>>
>>  /*
>> @@ -627,11 +678,19 @@ static ssize_t loop_attr_partscan_show(struct loop_device *lo, char *buf)
>>       return sprintf(buf, "%s\n", partscan ? "1" : "0");
>>  }
>>
>> +static ssize_t loop_attr_dio_show(struct loop_device *lo, char *buf)
>> +{
>> +     int dio = (lo->lo_flags & LO_FLAGS_DIRECT_IO);
>> +
>> +     return sprintf(buf, "%s\n", dio ? "1" : "0");
>> +}
>> +
>>  LOOP_ATTR_RO(backing_file);
>>  LOOP_ATTR_RO(offset);
>>  LOOP_ATTR_RO(sizelimit);
>>  LOOP_ATTR_RO(autoclear);
>>  LOOP_ATTR_RO(partscan);
>> +LOOP_ATTR_RO(dio);
>>
>>  static struct attribute *loop_attrs[] = {
>>       &loop_attr_backing_file.attr,
>> @@ -639,6 +698,7 @@ static struct attribute *loop_attrs[] = {
>>       &loop_attr_sizelimit.attr,
>>       &loop_attr_autoclear.attr,
>>       &loop_attr_partscan.attr,
>> +     &loop_attr_dio.attr,
>>       NULL,
>>  };
>>
>> @@ -783,6 +843,7 @@ static int loop_set_fd(struct loop_device *lo, fmode_t mode,
>>       if (!(lo_flags & LO_FLAGS_READ_ONLY) && file->f_op->fsync)
>>               blk_queue_flush(lo->lo_queue, REQ_FLUSH);
>>
>> +     loop_update_dio(lo);
>>       set_capacity(lo->lo_disk, size);
>>       bd_set_size(bdev, size << 9);
>>       loop_sysfs_init(lo);
>> diff --git a/drivers/block/loop.h b/drivers/block/loop.h
>> index b6c7d21..63f8e14 100644
>> --- a/drivers/block/loop.h
>> +++ b/drivers/block/loop.h
>> @@ -58,6 +58,8 @@ struct loop_device {
>>       struct mutex            lo_ctl_mutex;
>>       struct kthread_worker   worker;
>>       struct task_struct      *worker_task;
>> +     unsigned                dio_align;
>> +     bool                    use_dio;
>>
>>       struct request_queue    *lo_queue;
>>       struct blk_mq_tag_set   tag_set;
>> diff --git a/include/uapi/linux/loop.h b/include/uapi/linux/loop.h
>> index e0cecd2..949851c 100644
>> --- a/include/uapi/linux/loop.h
>> +++ b/include/uapi/linux/loop.h
>> @@ -21,6 +21,7 @@ enum {
>>       LO_FLAGS_READ_ONLY      = 1,
>>       LO_FLAGS_AUTOCLEAR      = 4,
>>       LO_FLAGS_PARTSCAN       = 8,
>> +     LO_FLAGS_DIRECT_IO      = 16,
>>  };
>>
>>  #include <asm/posix_types.h> /* for __kernel_old_dev_t */
>>

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

* Re: [PATCH v8 6/6] block: loop: support DIO & AIO
  2015-07-30 11:36 ` [PATCH v8 6/6] block: loop: support DIO & AIO Ming Lei
@ 2015-07-30 16:42   ` Christoph Hellwig
  2015-07-31  3:13     ` Ming Lei
  0 siblings, 1 reply; 13+ messages in thread
From: Christoph Hellwig @ 2015-07-30 16:42 UTC (permalink / raw)
  To: Ming Lei
  Cc: Jens Axboe, linux-kernel, Dave Kleikamp, Zach Brown,
	Christoph Hellwig, Maxim Patlasov, Andrew Morton, Alexander Viro,
	Tejun Heo, Dave Chinner

On Thu, Jul 30, 2015 at 07:36:24AM -0400, Ming Lei wrote:
> +	/*
> +	 * When working at direct I/O, under very unusual cases,
> +	 * such as unaligned direct I/O from application and
> +	 * access to loop block device with 'unaligned' offset & size,
> +	 * we have to fallback to non-dio mode.
> +	 *
> +	 * During the switch between dio and non-dio, page cache
> +	 * has to be flushed to the backing file.
> +	 */
> +	if (unlikely(lo->use_dio && lo->last_use_dio != cmd->use_aio))
> +		vfs_fsync(lo->lo_backing_file, 0);

Filesystems do the cache flushing for you.

> +static inline bool req_dio_aligned(struct loop_device *lo,
> +		const struct request *rq)
> +{
> +	return !((blk_rq_pos(rq) << 9) & lo->dio_align) &&
> +		!(blk_rq_bytes(rq) & lo->dio_align);
> +}
> +
>  static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
>  		const struct blk_mq_queue_data *bd)
>  {
> @@ -1554,6 +1658,13 @@ static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
>  	if (lo->lo_state != Lo_bound)
>  		return -EIO;
>  
> +	if (lo->use_dio && !lo->transfer &&
> +			req_dio_aligned(lo, bd->rq) &&
> +			!(cmd->rq->cmd_flags & (REQ_FLUSH | REQ_DISCARD)))
> +		cmd->use_aio = true;
> +	else
> +		cmd->use_aio = false;

But honestly run time switching between buffered I/O and direct I/O from
the same I/O stream is almost asking for triggering every possible
race in the dio vs buffered I/O synchronization.  And there have been
a lot of those..

I'd feel much more comfortable with a setup time check.

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

* Re: [PATCH v8 6/6] block: loop: support DIO & AIO
  2015-07-30 16:42   ` Christoph Hellwig
@ 2015-07-31  3:13     ` Ming Lei
  0 siblings, 0 replies; 13+ messages in thread
From: Ming Lei @ 2015-07-31  3:13 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Jens Axboe, Linux Kernel Mailing List, Dave Kleikamp, Zach Brown,
	Maxim Patlasov, Andrew Morton, Alexander Viro, Tejun Heo,
	Dave Chinner

On Thu, Jul 30, 2015 at 12:42 PM, Christoph Hellwig <hch@infradead.org> wrote:
> On Thu, Jul 30, 2015 at 07:36:24AM -0400, Ming Lei wrote:
>> +     /*
>> +      * When working at direct I/O, under very unusual cases,
>> +      * such as unaligned direct I/O from application and
>> +      * access to loop block device with 'unaligned' offset & size,
>> +      * we have to fallback to non-dio mode.
>> +      *
>> +      * During the switch between dio and non-dio, page cache
>> +      * has to be flushed to the backing file.
>> +      */
>> +     if (unlikely(lo->use_dio && lo->last_use_dio != cmd->use_aio))
>> +             vfs_fsync(lo->lo_backing_file, 0);
>
> Filesystems do the cache flushing for you.

If it depends on specific filesystem, it is better to do that here explicitly
and the page cache is still written back just one time(either here or
filesystem).

>
>> +static inline bool req_dio_aligned(struct loop_device *lo,
>> +             const struct request *rq)
>> +{
>> +     return !((blk_rq_pos(rq) << 9) & lo->dio_align) &&
>> +             !(blk_rq_bytes(rq) & lo->dio_align);
>> +}
>> +
>>  static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
>>               const struct blk_mq_queue_data *bd)
>>  {
>> @@ -1554,6 +1658,13 @@ static int loop_queue_rq(struct blk_mq_hw_ctx *hctx,
>>       if (lo->lo_state != Lo_bound)
>>               return -EIO;
>>
>> +     if (lo->use_dio && !lo->transfer &&
>> +                     req_dio_aligned(lo, bd->rq) &&
>> +                     !(cmd->rq->cmd_flags & (REQ_FLUSH | REQ_DISCARD)))
>> +             cmd->use_aio = true;
>> +     else
>> +             cmd->use_aio = false;
>
> But honestly run time switching between buffered I/O and direct I/O from
> the same I/O stream is almost asking for triggering every possible
> race in the dio vs buffered I/O synchronization.  And there have been
> a lot of those..

Yes, I agree, that is why I am a bit reluctant to do runtime switch
in v7.

I think of another way to aovid the race by serializing dio/buffered I/O
strictly:

- before switching to buffered I/O from dio, wait for completion of all
pending dio
- run fsync()
- switch to buffered I/O

Then concurrent dio and buffered I/O can't be possible, what do you
think about this approach?

>
> I'd feel much more comfortable with a setup time check.

Yes, that is what v7 did.  But during setup time, we can't know
the request's size and offset, and the simplest way is to just
support 512-byte sector size of backing device for avoding
runtime switching.

Christoph and Dave, or do you have other better idea?

Thanks,

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

end of thread, other threads:[~2015-07-31  3:13 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-07-30 11:36 [PATCH v8 0/6] block: loop: improve loop with AIO Ming Lei
2015-07-30 11:36 ` [PATCH v8 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
2015-07-30 11:36 ` [PATCH v8 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop Ming Lei
2015-07-30 11:36 ` [PATCH v8 3/6] block: loop: use kthread_work Ming Lei
2015-07-30 11:36 ` [PATCH v8 4/6] block: loop: prepare for supporing direct IO Ming Lei
2015-07-30 15:09   ` Christoph Hellwig
2015-07-30 15:21     ` Ming Lei
2015-07-30 15:30   ` Dave Kleikamp
2015-07-30 15:45     ` Ming Lei
2015-07-30 11:36 ` [PATCH v8 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO Ming Lei
2015-07-30 11:36 ` [PATCH v8 6/6] block: loop: support DIO & AIO Ming Lei
2015-07-30 16:42   ` Christoph Hellwig
2015-07-31  3:13     ` Ming Lei

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).