linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v7 0/6] block: loop: improve loop with AIO
@ 2015-07-16 15:37 Ming Lei
  2015-07-16 15:37 ` [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
                   ` (6 more replies)
  0 siblings, 7 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig

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.

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'
        - 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
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      | 263 ++++++++++++++++++++++++++++++++++++----------
 drivers/block/loop.h      |  13 +--
 fs/direct-io.c            |   9 +-
 include/uapi/linux/loop.h |   2 +
 4 files changed, 221 insertions(+), 66 deletions(-)


Thanks,
Ming



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

* [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
@ 2015-07-16 15:37 ` Ming Lei
  2015-07-16 16:48   ` Dave Kleikamp
  2015-07-16 15:37 ` [PATCH v7 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop Ming Lei
                   ` (5 subsequent siblings)
  6 siblings, 1 reply; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig,
	Ming Lei, Dave Kleikamp

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.

Cc: 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] 20+ messages in thread

* [PATCH v7 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
  2015-07-16 15:37 ` [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
@ 2015-07-16 15:37 ` Ming Lei
  2015-07-16 15:37 ` [PATCH v7 3/6] block: loop: use kthread_work Ming Lei
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig, 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 f7a4c9d..e4381f8 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] 20+ messages in thread

* [PATCH v7 3/6] block: loop: use kthread_work
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
  2015-07-16 15:37 ` [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
  2015-07-16 15:37 ` [PATCH v7 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop Ming Lei
@ 2015-07-16 15:37 ` Ming Lei
  2015-07-16 15:37 ` [PATCH v7 4/6] block: loop: prepare for supporing direct IO Ming Lei
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig, 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 e4381f8..ce94b92 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] 20+ messages in thread

* [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (2 preceding siblings ...)
  2015-07-16 15:37 ` [PATCH v7 3/6] block: loop: use kthread_work Ming Lei
@ 2015-07-16 15:37 ` Ming Lei
  2015-07-27  8:40   ` Christoph Hellwig
  2015-07-16 15:37 ` [PATCH v7 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO Ming Lei
                   ` (2 subsequent siblings)
  6 siblings, 1 reply; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig,
	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      | 65 ++++++++++++++++++++++++++++++++++++++++++++++-
 drivers/block/loop.h      |  1 +
 include/uapi/linux/loop.h |  1 +
 3 files changed, 66 insertions(+), 1 deletion(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index ce94b92..35aa3dd 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -164,6 +164,49 @@ 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;
+
+	/*
+	 * loop block's logical block size is 512, now
+	 * we support direct I/O only if the backing
+	 * block devices' minimize I/O size is 512 and
+	 * the offset is aligned with 512.
+	 */
+	if (dio) {
+		if (inode->i_sb->s_bdev &&
+			bdev_io_min(inode->i_sb->s_bdev) == 512 &&
+			!(lo->lo_offset & 511))
+			use_dio = true;
+		else
+			use_dio = false;
+	} else {
+		use_dio = false;
+	}
+
+	if (lo->use_dio == use_dio)
+		return;
+
+	/* 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;
+	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 +216,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 +468,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 +493,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 +680,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 +700,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 +845,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..d1de221 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -58,6 +58,7 @@ struct loop_device {
 	struct mutex		lo_ctl_mutex;
 	struct kthread_worker	worker;
 	struct task_struct	*worker_task;
+	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] 20+ messages in thread

* [PATCH v7 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (3 preceding siblings ...)
  2015-07-16 15:37 ` [PATCH v7 4/6] block: loop: prepare for supporing direct IO Ming Lei
@ 2015-07-16 15:37 ` Ming Lei
  2015-07-16 15:37 ` [PATCH v7 6/6] block: loop: support DIO & AIO Ming Lei
  2015-07-23  9:00 ` [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
  6 siblings, 0 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig,
	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 35aa3dd..3f26a0a 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -1214,6 +1214,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)
 {
@@ -1257,6 +1271,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] 20+ messages in thread

* [PATCH v7 6/6] block: loop: support DIO & AIO
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (4 preceding siblings ...)
  2015-07-16 15:37 ` [PATCH v7 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO Ming Lei
@ 2015-07-16 15:37 ` Ming Lei
  2015-07-23  9:00 ` [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
  6 siblings, 0 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-16 15:37 UTC (permalink / raw)
  To: Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig, 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 | 98 ++++++++++++++++++++++++++++++++++++++++++++++++++--
 drivers/block/loop.h |  2 ++
 2 files changed, 97 insertions(+), 3 deletions(-)

diff --git a/drivers/block/loop.c b/drivers/block/loop.c
index 3f26a0a..8099049 100644
--- a/drivers/block/loop.c
+++ b/drivers/block/loop.c
@@ -436,6 +436,90 @@ 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);
+
+	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;
@@ -451,13 +535,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;
@@ -1556,6 +1640,12 @@ 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 &&
+			!(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;
@@ -1575,7 +1665,9 @@ static void loop_handle_cmd(struct loop_cmd *cmd)
  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 d1de221..fb2237c 100644
--- a/drivers/block/loop.h
+++ b/drivers/block/loop.h
@@ -69,6 +69,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] 20+ messages in thread

* Re: [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read
  2015-07-16 15:37 ` [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
@ 2015-07-16 16:48   ` Dave Kleikamp
  0 siblings, 0 replies; 20+ messages in thread
From: Dave Kleikamp @ 2015-07-16 16:48 UTC (permalink / raw)
  To: Ming Lei, Jens Axboe, linux-kernel
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig

On 07/16/2015 10:37 AM, Ming Lei wrote:
> 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.
> 
> Cc: Dave Kleikamp <dave.kleikamp@oracle.com>
> Reviewed-by: Christoph Hellwig <hch@lst.de>
> Signed-off-by: Ming Lei <ming.lei@canonical.com>

Reviewed-by: Dave Kleikamp <dave.kleikamp@oracle.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;
> 

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

* Re: [PATCH v7 0/6] block: loop: improve loop with AIO
  2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
                   ` (5 preceding siblings ...)
  2015-07-16 15:37 ` [PATCH v7 6/6] block: loop: support DIO & AIO Ming Lei
@ 2015-07-23  9:00 ` Ming Lei
  6 siblings, 0 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-23  9:00 UTC (permalink / raw)
  To: Jens Axboe, Linux Kernel Mailing List
  Cc: Justin M. Forbes, Jeff Moyer, Tejun Heo, Christoph Hellwig

On Thu, Jul 16, 2015 at 11:37 AM, Ming Lei <ming.lei@canonical.com> wrote:
> 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.
>
> 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'
>         - 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
> 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      | 263 ++++++++++++++++++++++++++++++++++++----------
>  drivers/block/loop.h      |  13 +--
>  fs/direct-io.c            |   9 +-
>  include/uapi/linux/loop.h |   2 +
>  4 files changed, 221 insertions(+), 66 deletions(-)

Hi Guys,

Gentle ping.

thanks,

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-16 15:37 ` [PATCH v7 4/6] block: loop: prepare for supporing direct IO Ming Lei
@ 2015-07-27  8:40   ` Christoph Hellwig
  2015-07-27  9:41     ` Ming Lei
  0 siblings, 1 reply; 20+ messages in thread
From: Christoph Hellwig @ 2015-07-27  8:40 UTC (permalink / raw)
  To: Ming Lei
  Cc: Jens Axboe, linux-kernel, Justin M. Forbes, Jeff Moyer,
	Tejun Heo, Christoph Hellwig, linux-api

> +	/*
> +	 * loop block's logical block size is 512, now
> +	 * we support direct I/O only if the backing
> +	 * block devices' minimize I/O size is 512 and
> +	 * the offset is aligned with 512.
> +	 */
> +	if (dio) {
> +		if (inode->i_sb->s_bdev &&
> +			bdev_io_min(inode->i_sb->s_bdev) == 512 &&
> +			!(lo->lo_offset & 511))

Why the hardcoded value?  I suspect this should be more like:

	if (dio && inode->i_sb->s_bdev &&
	    (lo->lo_offset & (bdev_io_min(inode->i_sb->s_bdev) - 1)) != 0)
		dio = false;

> +	blk_mq_freeze_queue(lo->lo_queue);
> +	lo->use_dio = use_dio;
> +	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);

Locking?

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-27  8:40   ` Christoph Hellwig
@ 2015-07-27  9:41     ` Ming Lei
  2015-07-27  9:45       ` Christoph Hellwig
  0 siblings, 1 reply; 20+ messages in thread
From: Ming Lei @ 2015-07-27  9:41 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Jens Axboe, Linux Kernel Mailing List, Justin M. Forbes,
	Jeff Moyer, Tejun Heo, linux-api

On Mon, Jul 27, 2015 at 4:40 AM, Christoph Hellwig <hch@infradead.org> wrote:
>> +     /*
>> +      * loop block's logical block size is 512, now
>> +      * we support direct I/O only if the backing
>> +      * block devices' minimize I/O size is 512 and
>> +      * the offset is aligned with 512.
>> +      */
>> +     if (dio) {
>> +             if (inode->i_sb->s_bdev &&
>> +                     bdev_io_min(inode->i_sb->s_bdev) == 512 &&
>> +                     !(lo->lo_offset & 511))
>
> Why the hardcoded value?  I suspect this should be more like:
>
>         if (dio && inode->i_sb->s_bdev &&
>             (lo->lo_offset & (bdev_io_min(inode->i_sb->s_bdev) - 1)) != 0)
>                 dio = false;

The above can't work if the backing device has a bigger sector size
(such as 4K), that is why loop's direct-io requires 512 min_io_size of
backing device.

>
>> +     blk_mq_freeze_queue(lo->lo_queue);
>> +     lo->use_dio = use_dio;
>> +     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);
>
> Locking?

__loop_update_dio() is only called inside ioctl path, so
mutex of lo->lo_ctl_mutex has been held already.

Thanks,

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-27  9:41     ` Ming Lei
@ 2015-07-27  9:45       ` Christoph Hellwig
  2015-07-27  9:53         ` Ming Lei
  0 siblings, 1 reply; 20+ messages in thread
From: Christoph Hellwig @ 2015-07-27  9:45 UTC (permalink / raw)
  To: Ming Lei
  Cc: Christoph Hellwig, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Mon, Jul 27, 2015 at 05:41:57AM -0400, Ming Lei wrote:
> > Why the hardcoded value?  I suspect this should be more like:
> >
> >         if (dio && inode->i_sb->s_bdev &&
> >             (lo->lo_offset & (bdev_io_min(inode->i_sb->s_bdev) - 1)) != 0)
> >                 dio = false;
> 
> The above can't work if the backing device has a bigger sector size
> (such as 4K), that is why loop's direct-io requires 512 min_io_size of
> backing device.

Why doesn't it work?  If the backing device sector size is 4k
and lo_offset is 0 or a multiple of 4k it should allow direct I/O,
and my code sniplet will allow that.


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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-27  9:45       ` Christoph Hellwig
@ 2015-07-27  9:53         ` Ming Lei
  2015-07-27 17:33           ` Christoph Hellwig
  2015-07-27 22:06           ` Dave Chinner
  0 siblings, 2 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-27  9:53 UTC (permalink / raw)
  To: Christoph Hellwig, Dave Chinner
  Cc: Jens Axboe, Linux Kernel Mailing List, Justin M. Forbes,
	Jeff Moyer, Tejun Heo, linux-api

On Mon, Jul 27, 2015 at 5:45 AM, Christoph Hellwig <hch@infradead.org> wrote:
> On Mon, Jul 27, 2015 at 05:41:57AM -0400, Ming Lei wrote:
>> > Why the hardcoded value?  I suspect this should be more like:
>> >
>> >         if (dio && inode->i_sb->s_bdev &&
>> >             (lo->lo_offset & (bdev_io_min(inode->i_sb->s_bdev) - 1)) != 0)
>> >                 dio = false;
>>
>> The above can't work if the backing device has a bigger sector size
>> (such as 4K), that is why loop's direct-io requires 512 min_io_size of
>> backing device.
>
> Why doesn't it work?  If the backing device sector size is 4k
> and lo_offset is 0 or a multiple of 4k it should allow direct I/O,
> and my code sniplet will allow that.

Because size has to be 4k aligned too.

And it can't work in the example posted by Dave Chinner:

> I have a 4k sector backing device and a 512 byte sector filesystem
> image. I can't do 512 byte direct IO to the filesystem image, so I
> can't run tools that handle fs images in files using direct Io on
> that file. Create a loop device with the filesystem image, and now I
> can do 512 byte direct IO to the filesystem image, because all that
> direct IO to the filesystem image is now buffered by the loop
> device.
>
> If the loop device does direct io in this situation, the backing
> filesystem rejects direct IO from the loop device because it is not
> sector (4k) sized/aligned. User now swears, shouts and curses you
> from afar.


Thanks,

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-27  9:53         ` Ming Lei
@ 2015-07-27 17:33           ` Christoph Hellwig
  2015-07-29  7:33             ` Ming Lei
  2015-07-27 22:06           ` Dave Chinner
  1 sibling, 1 reply; 20+ messages in thread
From: Christoph Hellwig @ 2015-07-27 17:33 UTC (permalink / raw)
  To: Ming Lei
  Cc: Christoph Hellwig, Dave Chinner, Jens Axboe,
	Linux Kernel Mailing List, Justin M. Forbes, Jeff Moyer,
	Tejun Heo, linux-api

On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
> Because size has to be 4k aligned too.

Yes.  But again I don't see any reason to limit us to a hardcoded 512
byte block size here, especially considering the patches to finally
allow enabling other block sizes from userspace.


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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-27  9:53         ` Ming Lei
  2015-07-27 17:33           ` Christoph Hellwig
@ 2015-07-27 22:06           ` Dave Chinner
  1 sibling, 0 replies; 20+ messages in thread
From: Dave Chinner @ 2015-07-27 22:06 UTC (permalink / raw)
  To: Ming Lei
  Cc: Christoph Hellwig, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
> On Mon, Jul 27, 2015 at 5:45 AM, Christoph Hellwig <hch@infradead.org> wrote:
> > On Mon, Jul 27, 2015 at 05:41:57AM -0400, Ming Lei wrote:
> >> > Why the hardcoded value?  I suspect this should be more like:
> >> >
> >> >         if (dio && inode->i_sb->s_bdev &&
> >> >             (lo->lo_offset & (bdev_io_min(inode->i_sb->s_bdev) - 1)) != 0)
> >> >                 dio = false;
> >>
> >> The above can't work if the backing device has a bigger sector size
> >> (such as 4K), that is why loop's direct-io requires 512 min_io_size of
> >> backing device.
> >
> > Why doesn't it work?  If the backing device sector size is 4k
> > and lo_offset is 0 or a multiple of 4k it should allow direct I/O,
> > and my code sniplet will allow that.
> 
> Because size has to be 4k aligned too.

So check that, too. Any >= 4k block size filesystem should be doing
mostly 4k aligned and sized IO...

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-27 17:33           ` Christoph Hellwig
@ 2015-07-29  7:33             ` Ming Lei
  2015-07-29  8:41               ` Dave Chinner
  0 siblings, 1 reply; 20+ messages in thread
From: Ming Lei @ 2015-07-29  7:33 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: Dave Chinner, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Mon, Jul 27, 2015 at 1:33 PM, Christoph Hellwig <hch@infradead.org> wrote:
> On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
>> Because size has to be 4k aligned too.
>
> Yes.  But again I don't see any reason to limit us to a hardcoded 512
> byte block size here, especially considering the patches to finally

>From loop block's view, the request size can be any count of 512-byte
sectors, then the transfer size to backing device can't  guarantee to be
4k aligned always.

> allow enabling other block sizes from userspace.

I have some questions about the patchset, and looks the author doesn't
reply it yet.

On Mon, Jul 27, 2015 at 6:06 PM, Dave Chinner <david@fromorbit.com> wrote:
>> Because size has to be 4k aligned too.
>
> So check that, too. Any >= 4k block size filesystem should be doing
> mostly 4k aligned and sized IO...

I guess you mean we only use direct IO for the 4k aligned and sized IO?
If so, that won't be efficient because the page cache has to be flushed
during the switch.

Thanks,
Ming



> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at  http://www.tux.org/lkml/

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-29  7:33             ` Ming Lei
@ 2015-07-29  8:41               ` Dave Chinner
  2015-07-29 11:21                 ` Ming Lei
  0 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2015-07-29  8:41 UTC (permalink / raw)
  To: Ming Lei
  Cc: Christoph Hellwig, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Wed, Jul 29, 2015 at 03:33:52AM -0400, Ming Lei wrote:
> On Mon, Jul 27, 2015 at 1:33 PM, Christoph Hellwig <hch@infradead.org> wrote:
> > On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
> >> Because size has to be 4k aligned too.
> >
> > Yes.  But again I don't see any reason to limit us to a hardcoded 512
> > byte block size here, especially considering the patches to finally
> 
> From loop block's view, the request size can be any count of 512-byte
> sectors, then the transfer size to backing device can't  guarantee to be
> 4k aligned always.

In theory, yes. In practise, doesn't happen very often.

> > allow enabling other block sizes from userspace.
> 
> I have some questions about the patchset, and looks the author doesn't
> reply it yet.
> 
> On Mon, Jul 27, 2015 at 6:06 PM, Dave Chinner <david@fromorbit.com> wrote:
> >> Because size has to be 4k aligned too.
> >
> > So check that, too. Any >= 4k block size filesystem should be doing
> > mostly 4k aligned and sized IO...
> 
> I guess you mean we only use direct IO for the 4k aligned and sized IO?
> If so, that won't be efficient because the page cache has to be flushed
> during the switch.

It will be extremely rare for a 4k block size filesystem to do
anything other than 4k aligned and sized IO. Think about it for a
minute: what does the page cache do to unaligned IO patterns (i.e.
buffered IO)?  It does IO in page sizes, and so if the application
if doing badly aligned or sized IO with buffered IO, then the
underlying device will only ever size page sized and aligned IO.

Hence sector aligned IO will only come from applications doing
direct IO.  If the application is doing direct IO and it's not
properly aligned, then it already is going to get sucky performance
because most filesystem serialise sub-block size direct IO because
concurrent sub-block IOs to the same block usually leads to data
corruption.

So, really, sector aligned/sized direct IO is a sucky performance
path before we even get to the loop device, so we don't really need
to care how fast the loop device handles this case. The loop device
just needs to ensure that it doesn't corrupt data when badly aligned
IOs come in... ;)

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-29  8:41               ` Dave Chinner
@ 2015-07-29 11:21                 ` Ming Lei
  2015-07-29 22:08                   ` Dave Chinner
  0 siblings, 1 reply; 20+ messages in thread
From: Ming Lei @ 2015-07-29 11:21 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Christoph Hellwig, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Wed, Jul 29, 2015 at 4:41 AM, Dave Chinner <david@fromorbit.com> wrote:
> On Wed, Jul 29, 2015 at 03:33:52AM -0400, Ming Lei wrote:
>> On Mon, Jul 27, 2015 at 1:33 PM, Christoph Hellwig <hch@infradead.org> wrote:
>> > On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
>> >> Because size has to be 4k aligned too.
>> >
>> > Yes.  But again I don't see any reason to limit us to a hardcoded 512
>> > byte block size here, especially considering the patches to finally
>>
>> From loop block's view, the request size can be any count of 512-byte
>> sectors, then the transfer size to backing device can't  guarantee to be
>> 4k aligned always.
>
> In theory, yes. In practise, doesn't happen very often.
>
>> > allow enabling other block sizes from userspace.
>>
>> I have some questions about the patchset, and looks the author doesn't
>> reply it yet.
>>
>> On Mon, Jul 27, 2015 at 6:06 PM, Dave Chinner <david@fromorbit.com> wrote:
>> >> Because size has to be 4k aligned too.
>> >
>> > So check that, too. Any >= 4k block size filesystem should be doing
>> > mostly 4k aligned and sized IO...
>>
>> I guess you mean we only use direct IO for the 4k aligned and sized IO?
>> If so, that won't be efficient because the page cache has to be flushed
>> during the switch.
>
> It will be extremely rare for a 4k block size filesystem to do
> anything other than 4k aligned and sized IO. Think about it for a
> minute: what does the page cache do to unaligned IO patterns (i.e.
> buffered IO)?  It does IO in page sizes, and so if the application
> if doing badly aligned or sized IO with buffered IO, then the
> underlying device will only ever size page sized and aligned IO.
>
> Hence sector aligned IO will only come from applications doing
> direct IO.  If the application is doing direct IO and it's not
> properly aligned, then it already is going to get sucky performance
> because most filesystem serialise sub-block size direct IO because
> concurrent sub-block IOs to the same block usually leads to data
> corruption.

The blocksize of filesysten over loop can be 512, 1024, 2048, and
suppose sector size of backing device is 4096, then filesystem
can see aligned direct IO when IO size/offset from application is aligned
with fs block size, but loop still can't do direct IO for all this
kind of requests
against backing file.

Another case is that application may access loop block directly, such
as 'dd if=/dev/loopN', but it may not be common, and maybe it needn't
to consider.

Thanks,

>
> So, really, sector aligned/sized direct IO is a sucky performance
> path before we even get to the loop device, so we don't really need
> to care how fast the loop device handles this case. The loop device
> just needs to ensure that it doesn't corrupt data when badly aligned
> IOs come in... ;)
>
> Cheers,
>
> Dave.
> --
> Dave Chinner
> david@fromorbit.com

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-29 11:21                 ` Ming Lei
@ 2015-07-29 22:08                   ` Dave Chinner
  2015-07-30  8:01                     ` Ming Lei
  0 siblings, 1 reply; 20+ messages in thread
From: Dave Chinner @ 2015-07-29 22:08 UTC (permalink / raw)
  To: Ming Lei
  Cc: Christoph Hellwig, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Wed, Jul 29, 2015 at 07:21:47AM -0400, Ming Lei wrote:
> On Wed, Jul 29, 2015 at 4:41 AM, Dave Chinner <david@fromorbit.com> wrote:
> > On Wed, Jul 29, 2015 at 03:33:52AM -0400, Ming Lei wrote:
> >> On Mon, Jul 27, 2015 at 1:33 PM, Christoph Hellwig <hch@infradead.org> wrote:
> >> > On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
> >> >> Because size has to be 4k aligned too.
> >> >
> >> > Yes.  But again I don't see any reason to limit us to a hardcoded 512
> >> > byte block size here, especially considering the patches to finally
> >>
> >> From loop block's view, the request size can be any count of 512-byte
> >> sectors, then the transfer size to backing device can't  guarantee to be
> >> 4k aligned always.
> >
> > In theory, yes. In practise, doesn't happen very often.
> >
> >> > allow enabling other block sizes from userspace.
> >>
> >> I have some questions about the patchset, and looks the author doesn't
> >> reply it yet.
> >>
> >> On Mon, Jul 27, 2015 at 6:06 PM, Dave Chinner <david@fromorbit.com> wrote:
> >> >> Because size has to be 4k aligned too.
> >> >
> >> > So check that, too. Any >= 4k block size filesystem should be doing
> >> > mostly 4k aligned and sized IO...
> >>
> >> I guess you mean we only use direct IO for the 4k aligned and sized IO?
> >> If so, that won't be efficient because the page cache has to be flushed
> >> during the switch.
> >
> > It will be extremely rare for a 4k block size filesystem to do
> > anything other than 4k aligned and sized IO. Think about it for a
> > minute: what does the page cache do to unaligned IO patterns (i.e.
> > buffered IO)?  It does IO in page sizes, and so if the application
> > if doing badly aligned or sized IO with buffered IO, then the
> > underlying device will only ever size page sized and aligned IO.
> >
> > Hence sector aligned IO will only come from applications doing
> > direct IO.  If the application is doing direct IO and it's not
> > properly aligned, then it already is going to get sucky performance
> > because most filesystem serialise sub-block size direct IO because
> > concurrent sub-block IOs to the same block usually leads to data
> > corruption.
> 
> The blocksize of filesysten over loop can be 512, 1024, 2048, and
> suppose sector size of backing device is 4096, then filesystem
> can see aligned direct IO when IO size/offset from application is aligned
> with fs block size, but loop still can't do direct IO for all this
> kind of requests
> against backing file.

Sure, but again you're talking about a fairly rare configuration.
The vast majority of filesystems use 4k block sizes, just like the
vast majority of applications use buffered IO. Don't jump through
hoops to optimise a case that probably doesn't need optimising. Make
it work correctly first, then optimise performance later when
someone has a need for it to be really fast.

> Another case is that application may access loop block directly, such
> as 'dd if=/dev/loopN', but it may not be common, and maybe it needn't
> to consider.

'dd if=/dev/loopN bs=4k....'

Cheers,

Dave.
-- 
Dave Chinner
david@fromorbit.com

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

* Re: [PATCH v7 4/6] block: loop: prepare for supporing direct IO
  2015-07-29 22:08                   ` Dave Chinner
@ 2015-07-30  8:01                     ` Ming Lei
  0 siblings, 0 replies; 20+ messages in thread
From: Ming Lei @ 2015-07-30  8:01 UTC (permalink / raw)
  To: Dave Chinner
  Cc: Christoph Hellwig, Jens Axboe, Linux Kernel Mailing List,
	Justin M. Forbes, Jeff Moyer, Tejun Heo, linux-api

On Wed, Jul 29, 2015 at 6:08 PM, Dave Chinner <david@fromorbit.com> wrote:
> On Wed, Jul 29, 2015 at 07:21:47AM -0400, Ming Lei wrote:
>> On Wed, Jul 29, 2015 at 4:41 AM, Dave Chinner <david@fromorbit.com> wrote:
>> > On Wed, Jul 29, 2015 at 03:33:52AM -0400, Ming Lei wrote:
>> >> On Mon, Jul 27, 2015 at 1:33 PM, Christoph Hellwig <hch@infradead.org> wrote:
>> >> > On Mon, Jul 27, 2015 at 05:53:33AM -0400, Ming Lei wrote:
>> >> >> Because size has to be 4k aligned too.
>> >> >
>> >> > Yes.  But again I don't see any reason to limit us to a hardcoded 512
>> >> > byte block size here, especially considering the patches to finally
>> >>
>> >> From loop block's view, the request size can be any count of 512-byte
>> >> sectors, then the transfer size to backing device can't  guarantee to be
>> >> 4k aligned always.
>> >
>> > In theory, yes. In practise, doesn't happen very often.
>> >
>> >> > allow enabling other block sizes from userspace.
>> >>
>> >> I have some questions about the patchset, and looks the author doesn't
>> >> reply it yet.
>> >>
>> >> On Mon, Jul 27, 2015 at 6:06 PM, Dave Chinner <david@fromorbit.com> wrote:
>> >> >> Because size has to be 4k aligned too.
>> >> >
>> >> > So check that, too. Any >= 4k block size filesystem should be doing
>> >> > mostly 4k aligned and sized IO...
>> >>
>> >> I guess you mean we only use direct IO for the 4k aligned and sized IO?
>> >> If so, that won't be efficient because the page cache has to be flushed
>> >> during the switch.
>> >
>> > It will be extremely rare for a 4k block size filesystem to do
>> > anything other than 4k aligned and sized IO. Think about it for a
>> > minute: what does the page cache do to unaligned IO patterns (i.e.
>> > buffered IO)?  It does IO in page sizes, and so if the application
>> > if doing badly aligned or sized IO with buffered IO, then the
>> > underlying device will only ever size page sized and aligned IO.
>> >
>> > Hence sector aligned IO will only come from applications doing
>> > direct IO.  If the application is doing direct IO and it's not
>> > properly aligned, then it already is going to get sucky performance
>> > because most filesystem serialise sub-block size direct IO because
>> > concurrent sub-block IOs to the same block usually leads to data
>> > corruption.
>>
>> The blocksize of filesysten over loop can be 512, 1024, 2048, and
>> suppose sector size of backing device is 4096, then filesystem
>> can see aligned direct IO when IO size/offset from application is aligned
>> with fs block size, but loop still can't do direct IO for all this
>> kind of requests
>> against backing file.
>
> Sure, but again you're talking about a fairly rare configuration.
> The vast majority of filesystems use 4k block sizes, just like the
> vast majority of applications use buffered IO. Don't jump through
> hoops to optimise a case that probably doesn't need optimising. Make
> it work correctly first, then optimise performance later when
> someone has a need for it to be really fast.

OK, I will support 1024, 2048 and 4096 sector size in v8.

>
>> Another case is that application may access loop block directly, such
>> as 'dd if=/dev/loopN', but it may not be common, and maybe it needn't
>> to consider.
>
> 'dd if=/dev/loopN bs=4k....'
>
> Cheers,
>
> Dave.
> --
> Dave Chinner
> david@fromorbit.com
> --
> To unsubscribe from this list: send the line "unsubscribe linux-kernel" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html
> Please read the FAQ at  http://www.tux.org/lkml/

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

end of thread, other threads:[~2015-07-30  8:01 UTC | newest]

Thread overview: 20+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-07-16 15:37 [PATCH v7 0/6] block: loop: improve loop with AIO Ming Lei
2015-07-16 15:37 ` [PATCH v7 1/6] fs: direct-io: don't dirtying pages for ITER_BVEC/ITER_KVEC direct read Ming Lei
2015-07-16 16:48   ` Dave Kleikamp
2015-07-16 15:37 ` [PATCH v7 2/6] block: loop: set QUEUE_FLAG_NOMERGES for request queue of loop Ming Lei
2015-07-16 15:37 ` [PATCH v7 3/6] block: loop: use kthread_work Ming Lei
2015-07-16 15:37 ` [PATCH v7 4/6] block: loop: prepare for supporing direct IO Ming Lei
2015-07-27  8:40   ` Christoph Hellwig
2015-07-27  9:41     ` Ming Lei
2015-07-27  9:45       ` Christoph Hellwig
2015-07-27  9:53         ` Ming Lei
2015-07-27 17:33           ` Christoph Hellwig
2015-07-29  7:33             ` Ming Lei
2015-07-29  8:41               ` Dave Chinner
2015-07-29 11:21                 ` Ming Lei
2015-07-29 22:08                   ` Dave Chinner
2015-07-30  8:01                     ` Ming Lei
2015-07-27 22:06           ` Dave Chinner
2015-07-16 15:37 ` [PATCH v7 5/6] block: loop: introduce ioctl command of LOOP_SET_DIRECT_IO Ming Lei
2015-07-16 15:37 ` [PATCH v7 6/6] block: loop: support DIO & AIO Ming Lei
2015-07-23  9:00 ` [PATCH v7 0/6] block: loop: improve loop with AIO 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).