All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET v2] block: IO polling improvements
@ 2016-11-03 19:45 Jens Axboe
  2016-11-03 19:45 ` [PATCH 1/5] block: fast-path for small and simple direct I/O requests Jens Axboe
                   ` (4 more replies)
  0 siblings, 5 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-03 19:45 UTC (permalink / raw)
  To: axboe, linux-kernel, linux-block; +Cc: hch

Fixed a few bugs in this, and addressed some review comments. Patches
are against my 4.10 block branch, for-4.10/block.

Some performance results, running random reads against a fast device,
using either IRQ driven completions, classic polled, or the new hybrid
polled:

Method		IOPS		CPU%		IOPS/CPU%
=========================================================
IRQ		476K		48.45%		 9843
Poll		897K		99.9%		 8982
Hybrid		836K		68.1%		12273

Classic polling is still a bit faster, but is uses a lot more CPU. The
rightmost column is an efficiency metric, and with the new hybrid
scheme, we're both faster and more efficient than IRQ driven. We should
be able to close the speed gap as well, so you can have your cake and
eat it too.

I've adjusted the default, so that classic polling is the default poll
method. To enable hybrid polling for a device, you would do:

# echo 0 > /sys/block/<dev>/queue/io_poll_delay

Also see the last patch for the other values this can be set to.

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

* [PATCH 1/5] block: fast-path for small and simple direct I/O requests
  2016-11-03 19:45 [PATCHSET v2] block: IO polling improvements Jens Axboe
@ 2016-11-03 19:45 ` Jens Axboe
  2016-11-03 19:45 ` [PATCH 2/5] block: add scalable completion tracking of requests Jens Axboe
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-03 19:45 UTC (permalink / raw)
  To: axboe, linux-kernel, linux-block; +Cc: hch, Jens Axboe

From: Christoph Hellwig <hch@lst.de>

This patch adds a small and simple fast patch for small direct I/O
requests on block devices that don't use AIO.  Between the neat
bio_iov_iter_get_pages helper that avoids allocating a page array
for get_user_pages and the on-stack bio and biovec this avoid memory
allocations and atomic operations entirely in the direct I/O code
(lower levels might still do memory allocations and will usually
have at least some atomic operations, though).

Signed-off-by: Christoph Hellwig <hch@lst.de>
Signed-off-by: Jens Axboe <axboe@fb.com>
---
 fs/block_dev.c | 80 ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 80 insertions(+)

diff --git a/fs/block_dev.c b/fs/block_dev.c
index 05b553368bb4..d01413da5319 100644
--- a/fs/block_dev.c
+++ b/fs/block_dev.c
@@ -30,6 +30,7 @@
 #include <linux/cleancache.h>
 #include <linux/dax.h>
 #include <linux/badblocks.h>
+#include <linux/task_io_accounting_ops.h>
 #include <linux/falloc.h>
 #include <asm/uaccess.h>
 #include "internal.h"
@@ -175,12 +176,91 @@ static struct inode *bdev_file_inode(struct file *file)
 	return file->f_mapping->host;
 }
 
+#define DIO_INLINE_BIO_VECS 4
+
+static void blkdev_bio_end_io_simple(struct bio *bio)
+{
+	struct task_struct *waiter = bio->bi_private;
+
+	WRITE_ONCE(bio->bi_private, NULL);
+	wake_up_process(waiter);
+}
+
+static ssize_t
+__blkdev_direct_IO_simple(struct kiocb *iocb, struct iov_iter *iter,
+		int nr_pages)
+{
+	struct file *file = iocb->ki_filp;
+	struct block_device *bdev = I_BDEV(bdev_file_inode(file));
+	unsigned blkbits = blksize_bits(bdev_logical_block_size(bdev));
+	struct bio_vec inline_vecs[DIO_INLINE_BIO_VECS], *bvec;
+	loff_t pos = iocb->ki_pos;
+	bool should_dirty = false;
+	struct bio bio;
+	ssize_t ret;
+	blk_qc_t qc;
+	int i;
+
+	if ((pos | iov_iter_alignment(iter)) & ((1 << blkbits) - 1))
+		return -EINVAL;
+
+	bio_init(&bio);
+	bio.bi_max_vecs = nr_pages;
+	bio.bi_io_vec = inline_vecs;
+	bio.bi_bdev = bdev;
+	bio.bi_iter.bi_sector = pos >> blkbits;
+	bio.bi_private = current;
+	bio.bi_end_io = blkdev_bio_end_io_simple;
+
+	ret = bio_iov_iter_get_pages(&bio, iter);
+	if (unlikely(ret))
+		return ret;
+	ret = bio.bi_iter.bi_size;
+
+	if (iov_iter_rw(iter) == READ) {
+		bio_set_op_attrs(&bio, REQ_OP_READ, 0);
+		if (iter_is_iovec(iter))
+			should_dirty = true;
+	} else {
+		bio_set_op_attrs(&bio, REQ_OP_WRITE, REQ_SYNC | REQ_IDLE);
+		task_io_account_write(ret);
+	}
+
+	qc = submit_bio(&bio);
+	for (;;) {
+		set_current_state(TASK_UNINTERRUPTIBLE);
+		if (!READ_ONCE(bio.bi_private))
+			break;
+		if (!(iocb->ki_flags & IOCB_HIPRI) ||
+		    !blk_poll(bdev_get_queue(bdev), qc))
+			io_schedule();
+	}
+	__set_current_state(TASK_RUNNING);
+
+	bio_for_each_segment_all(bvec, &bio, i) {
+		if (should_dirty && !PageCompound(bvec->bv_page))
+			set_page_dirty_lock(bvec->bv_page);
+		put_page(bvec->bv_page);
+	}
+
+	if (unlikely(bio.bi_error))
+		return bio.bi_error;
+	iocb->ki_pos += ret;
+	return ret;
+}
+
 static ssize_t
 blkdev_direct_IO(struct kiocb *iocb, struct iov_iter *iter)
 {
 	struct file *file = iocb->ki_filp;
 	struct inode *inode = bdev_file_inode(file);
+	int nr_pages;
 
+	nr_pages = iov_iter_npages(iter, BIO_MAX_PAGES);
+	if (!nr_pages)
+		return 0;
+	if (is_sync_kiocb(iocb) && nr_pages <= DIO_INLINE_BIO_VECS)
+		return __blkdev_direct_IO_simple(iocb, iter, nr_pages);
 	return __blockdev_direct_IO(iocb, inode, I_BDEV(inode), iter,
 				    blkdev_get_block, NULL, NULL,
 				    DIO_SKIP_DIO_COUNT);
-- 
2.7.4

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

* [PATCH 2/5] block: add scalable completion tracking of requests
  2016-11-03 19:45 [PATCHSET v2] block: IO polling improvements Jens Axboe
  2016-11-03 19:45 ` [PATCH 1/5] block: fast-path for small and simple direct I/O requests Jens Axboe
@ 2016-11-03 19:45 ` Jens Axboe
  2016-11-03 19:45 ` [PATCH 3/5] block: move poll code to blk-mq Jens Axboe
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-03 19:45 UTC (permalink / raw)
  To: axboe, linux-kernel, linux-block; +Cc: hch, Jens Axboe

For legacy block, we simply track them in the request queue. For
blk-mq, we track them on a per-sw queue basis, which we can then
sum up through the hardware queues and finally to a per device
state.

The stats are tracked in, roughly, 0.1s interval windows.

Add sysfs files to display the stats.

The feature is off by default, to avoid any extra overhead. In-kernel
users of it can turn it on by setting QUEUE_FLAG_STATS in the queue
flags. We currently don't turn it on if someone just reads any of
the stats files, that is something we could add as well.

Signed-off-by: Jens Axboe <axboe@fb.com>
---
 block/Makefile            |   2 +-
 block/blk-core.c          |  14 ++-
 block/blk-mq-sysfs.c      |  47 +++++++++
 block/blk-mq.c            |  19 ++++
 block/blk-mq.h            |   3 +
 block/blk-stat.c          | 245 ++++++++++++++++++++++++++++++++++++++++++++++
 block/blk-stat.h          |  43 ++++++++
 block/blk-sysfs.c         |  26 +++++
 include/linux/blk_types.h |  16 +++
 include/linux/blkdev.h    |   7 ++
 10 files changed, 419 insertions(+), 3 deletions(-)
 create mode 100644 block/blk-stat.c
 create mode 100644 block/blk-stat.h

diff --git a/block/Makefile b/block/Makefile
index 934dac73fb37..2528c596f7ec 100644
--- a/block/Makefile
+++ b/block/Makefile
@@ -5,7 +5,7 @@
 obj-$(CONFIG_BLOCK) := bio.o elevator.o blk-core.o blk-tag.o blk-sysfs.o \
 			blk-flush.o blk-settings.o blk-ioc.o blk-map.o \
 			blk-exec.o blk-merge.o blk-softirq.o blk-timeout.o \
-			blk-lib.o blk-mq.o blk-mq-tag.o \
+			blk-lib.o blk-mq.o blk-mq-tag.o blk-stat.o \
 			blk-mq-sysfs.o blk-mq-cpumap.o ioctl.o \
 			genhd.o scsi_ioctl.o partition-generic.o ioprio.o \
 			badblocks.o partitions/
diff --git a/block/blk-core.c b/block/blk-core.c
index 0bfaa54d3e9f..9dc7e9314d4c 100644
--- a/block/blk-core.c
+++ b/block/blk-core.c
@@ -2462,6 +2462,11 @@ void blk_start_request(struct request *req)
 {
 	blk_dequeue_request(req);
 
+	if (test_bit(QUEUE_FLAG_STATS, &req->q->queue_flags)) {
+		blk_stat_set_issue_time(&req->issue_stat);
+		req->rq_flags |= RQF_STATS;
+	}
+
 	/*
 	 * We are now handing the request to the hardware, initialize
 	 * resid_len to full count and add the timeout handler.
@@ -2681,8 +2686,13 @@ EXPORT_SYMBOL_GPL(blk_unprep_request);
  */
 void blk_finish_request(struct request *req, int error)
 {
+	struct request_queue *q = req->q;
+
+	if (req->rq_flags & RQF_STATS)
+		blk_stat_add(&q->rq_stats[rq_data_dir(req)], req);
+
 	if (req->rq_flags & RQF_QUEUED)
-		blk_queue_end_tag(req->q, req);
+		blk_queue_end_tag(q, req);
 
 	BUG_ON(blk_queued_rq(req));
 
@@ -2702,7 +2712,7 @@ void blk_finish_request(struct request *req, int error)
 		if (blk_bidi_rq(req))
 			__blk_put_request(req->next_rq->q, req->next_rq);
 
-		__blk_put_request(req->q, req);
+		__blk_put_request(q, req);
 	}
 }
 EXPORT_SYMBOL(blk_finish_request);
diff --git a/block/blk-mq-sysfs.c b/block/blk-mq-sysfs.c
index 01fb455d3377..eacd3af72099 100644
--- a/block/blk-mq-sysfs.c
+++ b/block/blk-mq-sysfs.c
@@ -259,6 +259,47 @@ static ssize_t blk_mq_hw_sysfs_cpus_show(struct blk_mq_hw_ctx *hctx, char *page)
 	return ret;
 }
 
+static void blk_mq_stat_clear(struct blk_mq_hw_ctx *hctx)
+{
+	struct blk_mq_ctx *ctx;
+	unsigned int i;
+
+	hctx_for_each_ctx(hctx, ctx, i) {
+		blk_stat_init(&ctx->stat[BLK_STAT_READ]);
+		blk_stat_init(&ctx->stat[BLK_STAT_WRITE]);
+	}
+}
+
+static ssize_t blk_mq_hw_sysfs_stat_store(struct blk_mq_hw_ctx *hctx,
+					  const char *page, size_t count)
+{
+	blk_mq_stat_clear(hctx);
+	return count;
+}
+
+static ssize_t print_stat(char *page, struct blk_rq_stat *stat, const char *pre)
+{
+	return sprintf(page, "%s samples=%llu, mean=%lld, min=%lld, max=%lld\n",
+			pre, (long long) stat->nr_samples,
+			(long long) stat->mean, (long long) stat->min,
+			(long long) stat->max);
+}
+
+static ssize_t blk_mq_hw_sysfs_stat_show(struct blk_mq_hw_ctx *hctx, char *page)
+{
+	struct blk_rq_stat stat[2];
+	ssize_t ret;
+
+	blk_stat_init(&stat[BLK_STAT_READ]);
+	blk_stat_init(&stat[BLK_STAT_WRITE]);
+
+	blk_hctx_stat_get(hctx, stat);
+
+	ret = print_stat(page, &stat[BLK_STAT_READ], "read :");
+	ret += print_stat(page + ret, &stat[BLK_STAT_WRITE], "write:");
+	return ret;
+}
+
 static struct blk_mq_ctx_sysfs_entry blk_mq_sysfs_dispatched = {
 	.attr = {.name = "dispatched", .mode = S_IRUGO },
 	.show = blk_mq_sysfs_dispatched_show,
@@ -317,6 +358,11 @@ static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_poll = {
 	.show = blk_mq_hw_sysfs_poll_show,
 	.store = blk_mq_hw_sysfs_poll_store,
 };
+static struct blk_mq_hw_ctx_sysfs_entry blk_mq_hw_sysfs_stat = {
+	.attr = {.name = "stats", .mode = S_IRUGO | S_IWUSR },
+	.show = blk_mq_hw_sysfs_stat_show,
+	.store = blk_mq_hw_sysfs_stat_store,
+};
 
 static struct attribute *default_hw_ctx_attrs[] = {
 	&blk_mq_hw_sysfs_queued.attr,
@@ -327,6 +373,7 @@ static struct attribute *default_hw_ctx_attrs[] = {
 	&blk_mq_hw_sysfs_cpus.attr,
 	&blk_mq_hw_sysfs_active.attr,
 	&blk_mq_hw_sysfs_poll.attr,
+	&blk_mq_hw_sysfs_stat.attr,
 	NULL,
 };
 
diff --git a/block/blk-mq.c b/block/blk-mq.c
index 2da1a0ee3318..476b18e1991c 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -30,6 +30,7 @@
 #include "blk.h"
 #include "blk-mq.h"
 #include "blk-mq-tag.h"
+#include "blk-stat.h"
 
 static DEFINE_MUTEX(all_q_mutex);
 static LIST_HEAD(all_q_list);
@@ -376,10 +377,21 @@ static void blk_mq_ipi_complete_request(struct request *rq)
 	put_cpu();
 }
 
+static void blk_mq_stat_add(struct request *rq)
+{
+	if (rq->rq_flags & RQF_STATS) {
+		struct blk_rq_stat *stat = &rq->mq_ctx->stat[rq_data_dir(rq)];
+
+		blk_stat_add(stat, rq);
+	}
+}
+
 static void __blk_mq_complete_request(struct request *rq)
 {
 	struct request_queue *q = rq->q;
 
+	blk_mq_stat_add(rq);
+
 	if (!q->softirq_done_fn)
 		blk_mq_end_request(rq, rq->errors);
 	else
@@ -423,6 +435,11 @@ void blk_mq_start_request(struct request *rq)
 	if (unlikely(blk_bidi_rq(rq)))
 		rq->next_rq->resid_len = blk_rq_bytes(rq->next_rq);
 
+	if (test_bit(QUEUE_FLAG_STATS, &q->queue_flags)) {
+		blk_stat_set_issue_time(&rq->issue_stat);
+		rq->rq_flags |= RQF_STATS;
+	}
+
 	blk_add_timer(rq);
 
 	/*
@@ -1708,6 +1725,8 @@ static void blk_mq_init_cpu_queues(struct request_queue *q,
 		spin_lock_init(&__ctx->lock);
 		INIT_LIST_HEAD(&__ctx->rq_list);
 		__ctx->queue = q;
+		blk_stat_init(&__ctx->stat[BLK_STAT_READ]);
+		blk_stat_init(&__ctx->stat[BLK_STAT_WRITE]);
 
 		/* If the cpu isn't online, the cpu is mapped to first hctx */
 		if (!cpu_online(i))
diff --git a/block/blk-mq.h b/block/blk-mq.h
index e5d25249028c..8cf16cb69f64 100644
--- a/block/blk-mq.h
+++ b/block/blk-mq.h
@@ -1,6 +1,8 @@
 #ifndef INT_BLK_MQ_H
 #define INT_BLK_MQ_H
 
+#include "blk-stat.h"
+
 struct blk_mq_tag_set;
 
 struct blk_mq_ctx {
@@ -18,6 +20,7 @@ struct blk_mq_ctx {
 
 	/* incremented at completion time */
 	unsigned long		____cacheline_aligned_in_smp rq_completed[2];
+	struct blk_rq_stat	stat[2];
 
 	struct request_queue	*queue;
 	struct kobject		kobj;
diff --git a/block/blk-stat.c b/block/blk-stat.c
new file mode 100644
index 000000000000..1ace5c662f3e
--- /dev/null
+++ b/block/blk-stat.c
@@ -0,0 +1,245 @@
+/*
+ * Block stat tracking code
+ *
+ * Copyright (C) 2016 Jens Axboe
+ */
+#include <linux/kernel.h>
+#include <linux/blk-mq.h>
+
+#include "blk-stat.h"
+#include "blk-mq.h"
+
+static void blk_stat_flush_batch(struct blk_rq_stat *stat)
+{
+	if (!stat->nr_batch)
+		return;
+	if (!stat->nr_samples)
+		stat->mean = div64_s64(stat->batch, stat->nr_batch);
+	else {
+		stat->mean = div64_s64((stat->mean * stat->nr_samples) +
+					stat->batch,
+					stat->nr_samples + stat->nr_batch);
+	}
+
+	stat->nr_samples += stat->nr_batch;
+	stat->nr_batch = stat->batch = 0;
+}
+
+void blk_stat_sum(struct blk_rq_stat *dst, struct blk_rq_stat *src)
+{
+	if (!src->nr_samples)
+		return;
+
+	blk_stat_flush_batch(src);
+
+	dst->min = min(dst->min, src->min);
+	dst->max = max(dst->max, src->max);
+
+	if (!dst->nr_samples)
+		dst->mean = src->mean;
+	else {
+		dst->mean = div64_s64((src->mean * src->nr_samples) +
+					(dst->mean * dst->nr_samples),
+					dst->nr_samples + src->nr_samples);
+	}
+	dst->nr_samples += src->nr_samples;
+}
+
+static void blk_mq_stat_get(struct request_queue *q, struct blk_rq_stat *dst)
+{
+	struct blk_mq_hw_ctx *hctx;
+	struct blk_mq_ctx *ctx;
+	uint64_t latest = 0;
+	int i, j, nr;
+
+	blk_stat_init(&dst[BLK_STAT_READ]);
+	blk_stat_init(&dst[BLK_STAT_WRITE]);
+
+	nr = 0;
+	do {
+		uint64_t newest = 0;
+
+		queue_for_each_hw_ctx(q, hctx, i) {
+			hctx_for_each_ctx(hctx, ctx, j) {
+				if (!ctx->stat[BLK_STAT_READ].nr_samples &&
+				    !ctx->stat[BLK_STAT_WRITE].nr_samples)
+					continue;
+				if (ctx->stat[BLK_STAT_READ].time > newest)
+					newest = ctx->stat[BLK_STAT_READ].time;
+				if (ctx->stat[BLK_STAT_WRITE].time > newest)
+					newest = ctx->stat[BLK_STAT_WRITE].time;
+			}
+		}
+
+		/*
+		 * No samples
+		 */
+		if (!newest)
+			break;
+
+		if (newest > latest)
+			latest = newest;
+
+		queue_for_each_hw_ctx(q, hctx, i) {
+			hctx_for_each_ctx(hctx, ctx, j) {
+				if (ctx->stat[BLK_STAT_READ].time == newest) {
+					blk_stat_sum(&dst[BLK_STAT_READ],
+						     &ctx->stat[BLK_STAT_READ]);
+					nr++;
+				}
+				if (ctx->stat[BLK_STAT_WRITE].time == newest) {
+					blk_stat_sum(&dst[BLK_STAT_WRITE],
+						     &ctx->stat[BLK_STAT_WRITE]);
+					nr++;
+				}
+			}
+		}
+		/*
+		 * If we race on finding an entry, just loop back again.
+		 * Should be very rare.
+		 */
+	} while (!nr);
+
+	dst[BLK_STAT_READ].time = dst[BLK_STAT_WRITE].time = latest;
+}
+
+void blk_queue_stat_get(struct request_queue *q, struct blk_rq_stat *dst)
+{
+	if (q->mq_ops)
+		blk_mq_stat_get(q, dst);
+	else {
+		memcpy(&dst[BLK_STAT_READ], &q->rq_stats[BLK_STAT_READ],
+				sizeof(struct blk_rq_stat));
+		memcpy(&dst[BLK_STAT_WRITE], &q->rq_stats[BLK_STAT_WRITE],
+				sizeof(struct blk_rq_stat));
+	}
+}
+
+void blk_hctx_stat_get(struct blk_mq_hw_ctx *hctx, struct blk_rq_stat *dst)
+{
+	struct blk_mq_ctx *ctx;
+	unsigned int i, nr;
+
+	nr = 0;
+	do {
+		uint64_t newest = 0;
+
+		hctx_for_each_ctx(hctx, ctx, i) {
+			if (!ctx->stat[BLK_STAT_READ].nr_samples &&
+			    !ctx->stat[BLK_STAT_WRITE].nr_samples)
+				continue;
+
+			if (ctx->stat[BLK_STAT_READ].time > newest)
+				newest = ctx->stat[BLK_STAT_READ].time;
+			if (ctx->stat[BLK_STAT_WRITE].time > newest)
+				newest = ctx->stat[BLK_STAT_WRITE].time;
+		}
+
+		if (!newest)
+			break;
+
+		hctx_for_each_ctx(hctx, ctx, i) {
+			if (ctx->stat[BLK_STAT_READ].time == newest) {
+				blk_stat_sum(&dst[BLK_STAT_READ],
+						&ctx->stat[BLK_STAT_READ]);
+				nr++;
+			}
+			if (ctx->stat[BLK_STAT_WRITE].time == newest) {
+				blk_stat_sum(&dst[BLK_STAT_WRITE],
+						&ctx->stat[BLK_STAT_WRITE]);
+				nr++;
+			}
+		}
+		/*
+		 * If we race on finding an entry, just loop back again.
+		 * Should be very rare, as the window is only updated
+		 * occasionally
+		 */
+	} while (!nr);
+}
+
+static void __blk_stat_init(struct blk_rq_stat *stat, s64 time_now)
+{
+	stat->min = -1ULL;
+	stat->max = stat->nr_samples = stat->mean = 0;
+	stat->batch = stat->nr_batch = 0;
+	stat->time = time_now & BLK_STAT_NSEC_MASK;
+}
+
+void blk_stat_init(struct blk_rq_stat *stat)
+{
+	__blk_stat_init(stat, ktime_to_ns(ktime_get()));
+}
+
+static bool __blk_stat_is_current(struct blk_rq_stat *stat, s64 now)
+{
+	return (now & BLK_STAT_NSEC_MASK) == (stat->time & BLK_STAT_NSEC_MASK);
+}
+
+bool blk_stat_is_current(struct blk_rq_stat *stat)
+{
+	return __blk_stat_is_current(stat, ktime_to_ns(ktime_get()));
+}
+
+void blk_stat_add(struct blk_rq_stat *stat, struct request *rq)
+{
+	s64 now, value;
+
+	now = __blk_stat_time(ktime_to_ns(ktime_get()));
+	if (now < blk_stat_time(&rq->issue_stat))
+		return;
+
+	if (!__blk_stat_is_current(stat, now))
+		__blk_stat_init(stat, now);
+
+	value = now - blk_stat_time(&rq->issue_stat);
+	if (value > stat->max)
+		stat->max = value;
+	if (value < stat->min)
+		stat->min = value;
+
+	if (stat->batch + value < stat->batch ||
+	    stat->nr_batch + 1 == BLK_RQ_STAT_BATCH)
+		blk_stat_flush_batch(stat);
+
+	stat->batch += value;
+	stat->nr_batch++;
+}
+
+void blk_stat_clear(struct request_queue *q)
+{
+	if (q->mq_ops) {
+		struct blk_mq_hw_ctx *hctx;
+		struct blk_mq_ctx *ctx;
+		int i, j;
+
+		queue_for_each_hw_ctx(q, hctx, i) {
+			hctx_for_each_ctx(hctx, ctx, j) {
+				blk_stat_init(&ctx->stat[BLK_STAT_READ]);
+				blk_stat_init(&ctx->stat[BLK_STAT_WRITE]);
+			}
+		}
+	} else {
+		blk_stat_init(&q->rq_stats[BLK_STAT_READ]);
+		blk_stat_init(&q->rq_stats[BLK_STAT_WRITE]);
+	}
+}
+
+void blk_stat_set_issue_time(struct blk_issue_stat *stat)
+{
+	stat->time = (stat->time & BLK_STAT_MASK) |
+			(ktime_to_ns(ktime_get()) & BLK_STAT_TIME_MASK);
+}
+
+/*
+ * Enable stat tracking, return whether it was enabled
+ */
+bool blk_stat_enable(struct request_queue *q)
+{
+	if (!test_bit(QUEUE_FLAG_STATS, &q->queue_flags)) {
+		set_bit(QUEUE_FLAG_STATS, &q->queue_flags);
+		return false;
+	}
+
+	return true;
+}
diff --git a/block/blk-stat.h b/block/blk-stat.h
new file mode 100644
index 000000000000..5a9c85a39e02
--- /dev/null
+++ b/block/blk-stat.h
@@ -0,0 +1,43 @@
+#ifndef BLK_STAT_H
+#define BLK_STAT_H
+
+/*
+ * ~0.13s window as a power-of-2 (2^27 nsecs)
+ */
+#define BLK_STAT_NSEC		134217728ULL
+#define BLK_STAT_NSEC_MASK	~(BLK_STAT_NSEC - 1)
+
+/*
+ * Upper 3 bits can be used elsewhere
+ */
+#define BLK_STAT_RES_BITS	3
+#define BLK_STAT_SHIFT		(64 - BLK_STAT_RES_BITS)
+#define BLK_STAT_TIME_MASK	((1ULL << BLK_STAT_SHIFT) - 1)
+#define BLK_STAT_MASK		~BLK_STAT_TIME_MASK
+
+enum {
+	BLK_STAT_READ	= 0,
+	BLK_STAT_WRITE,
+};
+
+void blk_stat_add(struct blk_rq_stat *, struct request *);
+void blk_hctx_stat_get(struct blk_mq_hw_ctx *, struct blk_rq_stat *);
+void blk_queue_stat_get(struct request_queue *, struct blk_rq_stat *);
+void blk_stat_clear(struct request_queue *);
+void blk_stat_init(struct blk_rq_stat *);
+void blk_stat_sum(struct blk_rq_stat *, struct blk_rq_stat *);
+bool blk_stat_is_current(struct blk_rq_stat *);
+void blk_stat_set_issue_time(struct blk_issue_stat *);
+bool blk_stat_enable(struct request_queue *);
+
+static inline u64 __blk_stat_time(u64 time)
+{
+	return time & BLK_STAT_TIME_MASK;
+}
+
+static inline u64 blk_stat_time(struct blk_issue_stat *stat)
+{
+	return __blk_stat_time(stat->time);
+}
+
+#endif
diff --git a/block/blk-sysfs.c b/block/blk-sysfs.c
index 488c2e28feb8..9cdb7247727a 100644
--- a/block/blk-sysfs.c
+++ b/block/blk-sysfs.c
@@ -401,6 +401,26 @@ static ssize_t queue_dax_show(struct request_queue *q, char *page)
 	return queue_var_show(blk_queue_dax(q), page);
 }
 
+static ssize_t print_stat(char *page, struct blk_rq_stat *stat, const char *pre)
+{
+	return sprintf(page, "%s samples=%llu, mean=%lld, min=%lld, max=%lld\n",
+			pre, (long long) stat->nr_samples,
+			(long long) stat->mean, (long long) stat->min,
+			(long long) stat->max);
+}
+
+static ssize_t queue_stats_show(struct request_queue *q, char *page)
+{
+	struct blk_rq_stat stat[2];
+	ssize_t ret;
+
+	blk_queue_stat_get(q, stat);
+
+	ret = print_stat(page, &stat[BLK_STAT_READ], "read :");
+	ret += print_stat(page + ret, &stat[BLK_STAT_WRITE], "write:");
+	return ret;
+}
+
 static struct queue_sysfs_entry queue_requests_entry = {
 	.attr = {.name = "nr_requests", .mode = S_IRUGO | S_IWUSR },
 	.show = queue_requests_show,
@@ -553,6 +573,11 @@ static struct queue_sysfs_entry queue_dax_entry = {
 	.show = queue_dax_show,
 };
 
+static struct queue_sysfs_entry queue_stats_entry = {
+	.attr = {.name = "stats", .mode = S_IRUGO },
+	.show = queue_stats_show,
+};
+
 static struct attribute *default_attrs[] = {
 	&queue_requests_entry.attr,
 	&queue_ra_entry.attr,
@@ -582,6 +607,7 @@ static struct attribute *default_attrs[] = {
 	&queue_poll_entry.attr,
 	&queue_wc_entry.attr,
 	&queue_dax_entry.attr,
+	&queue_stats_entry.attr,
 	NULL,
 };
 
diff --git a/include/linux/blk_types.h b/include/linux/blk_types.h
index 562ac46cb790..4d0044d09984 100644
--- a/include/linux/blk_types.h
+++ b/include/linux/blk_types.h
@@ -250,4 +250,20 @@ static inline unsigned int blk_qc_t_to_tag(blk_qc_t cookie)
 	return cookie & ((1u << BLK_QC_T_SHIFT) - 1);
 }
 
+struct blk_issue_stat {
+	u64 time;
+};
+
+#define BLK_RQ_STAT_BATCH	64
+
+struct blk_rq_stat {
+	s64 mean;
+	u64 min;
+	u64 max;
+	s32 nr_samples;
+	s32 nr_batch;
+	u64 batch;
+	s64 time;
+};
+
 #endif /* __LINUX_BLK_TYPES_H */
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index 8396da2bb698..b400c31b4a10 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -117,6 +117,8 @@ typedef __u32 __bitwise req_flags_t;
 #define RQF_PM			((__force req_flags_t)(1 << 15))
 /* on IO scheduler merge hash */
 #define RQF_HASHED		((__force req_flags_t)(1 << 16))
+/* IO stats tracking on */
+#define RQF_STATS		((__force req_flags_t)(1 << 17))
 
 /* flags that prevent us from merging requests: */
 #define RQF_NOMERGE_FLAGS \
@@ -197,6 +199,7 @@ struct request {
 	struct gendisk *rq_disk;
 	struct hd_struct *part;
 	unsigned long start_time;
+	struct blk_issue_stat issue_stat;
 #ifdef CONFIG_BLK_CGROUP
 	struct request_list *rl;		/* rl this rq is alloced from */
 	unsigned long long start_time_ns;
@@ -490,6 +493,9 @@ struct request_queue {
 
 	unsigned int		nr_sorted;
 	unsigned int		in_flight[2];
+
+	struct blk_rq_stat	rq_stats[2];
+
 	/*
 	 * Number of active block driver functions for which blk_drain_queue()
 	 * must wait. Must be incremented around functions that unlock the
@@ -583,6 +589,7 @@ struct request_queue {
 #define QUEUE_FLAG_FUA	       24	/* device supports FUA writes */
 #define QUEUE_FLAG_FLUSH_NQ    25	/* flush not queueuable */
 #define QUEUE_FLAG_DAX         26	/* device supports DAX */
+#define QUEUE_FLAG_STATS       27	/* track rq completion times */
 
 #define QUEUE_FLAG_DEFAULT	((1 << QUEUE_FLAG_IO_STAT) |		\
 				 (1 << QUEUE_FLAG_STACKABLE)	|	\
-- 
2.7.4

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

* [PATCH 3/5] block: move poll code to blk-mq
  2016-11-03 19:45 [PATCHSET v2] block: IO polling improvements Jens Axboe
  2016-11-03 19:45 ` [PATCH 1/5] block: fast-path for small and simple direct I/O requests Jens Axboe
  2016-11-03 19:45 ` [PATCH 2/5] block: add scalable completion tracking of requests Jens Axboe
@ 2016-11-03 19:45 ` Jens Axboe
  2016-11-04 14:44   ` Christoph Hellwig
  2016-11-03 19:45 ` [PATCH 4/5] blk-mq: implement hybrid poll mode for sync O_DIRECT Jens Axboe
  2016-11-03 19:45 ` [PATCH 5/5] blk-mq: make the polling code adaptive Jens Axboe
  4 siblings, 1 reply; 9+ messages in thread
From: Jens Axboe @ 2016-11-03 19:45 UTC (permalink / raw)
  To: axboe, linux-kernel, linux-block; +Cc: hch, Jens Axboe

The poll code is blk-mq specific, let's move it to blk-mq.c. This
is a prep patch for improving the polling code.

Signed-off-by: Jens Axboe <axboe@fb.com>
Reviewed-by: Christoph Hellwig <hch@lst.de>
---
 block/blk-core.c | 36 +++++-------------------------------
 block/blk-mq.c   | 33 +++++++++++++++++++++++++++++++++
 block/blk-mq.h   |  2 ++
 3 files changed, 40 insertions(+), 31 deletions(-)

diff --git a/block/blk-core.c b/block/blk-core.c
index 9dc7e9314d4c..e500d4b0e3fd 100644
--- a/block/blk-core.c
+++ b/block/blk-core.c
@@ -3299,47 +3299,21 @@ EXPORT_SYMBOL(blk_finish_plug);
 
 bool blk_poll(struct request_queue *q, blk_qc_t cookie)
 {
-	struct blk_plug *plug;
-	long state;
-	unsigned int queue_num;
 	struct blk_mq_hw_ctx *hctx;
+	struct blk_plug *plug;
+	struct request *rq;
 
 	if (!q->mq_ops || !q->mq_ops->poll || !blk_qc_t_valid(cookie) ||
 	    !test_bit(QUEUE_FLAG_POLL, &q->queue_flags))
 		return false;
 
-	queue_num = blk_qc_t_to_queue_num(cookie);
-	hctx = q->queue_hw_ctx[queue_num];
-	hctx->poll_considered++;
-
 	plug = current->plug;
 	if (plug)
 		blk_flush_plug_list(plug, false);
 
-	state = current->state;
-	while (!need_resched()) {
-		int ret;
-
-		hctx->poll_invoked++;
-
-		ret = q->mq_ops->poll(hctx, blk_qc_t_to_tag(cookie));
-		if (ret > 0) {
-			hctx->poll_success++;
-			set_current_state(TASK_RUNNING);
-			return true;
-		}
-
-		if (signal_pending_state(state, current))
-			set_current_state(TASK_RUNNING);
-
-		if (current->state == TASK_RUNNING)
-			return true;
-		if (ret < 0)
-			break;
-		cpu_relax();
-	}
-
-	return false;
+	hctx = q->queue_hw_ctx[blk_qc_t_to_queue_num(cookie)];
+	rq = blk_mq_tag_to_rq(hctx->tags, blk_qc_t_to_tag(cookie));
+	return blk_mq_poll(hctx, rq);
 }
 EXPORT_SYMBOL_GPL(blk_poll);
 
diff --git a/block/blk-mq.c b/block/blk-mq.c
index 476b18e1991c..4883c2d97ea9 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -2357,6 +2357,39 @@ void blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set, int nr_hw_queues)
 }
 EXPORT_SYMBOL_GPL(blk_mq_update_nr_hw_queues);
 
+bool blk_mq_poll(struct blk_mq_hw_ctx *hctx, struct request *rq)
+{
+	struct request_queue *q = hctx->queue;
+	long state;
+
+	hctx->poll_considered++;
+
+	state = current->state;
+	while (!need_resched()) {
+		int ret;
+
+		hctx->poll_invoked++;
+
+		ret = q->mq_ops->poll(hctx, rq->tag);
+		if (ret > 0) {
+			hctx->poll_success++;
+			set_current_state(TASK_RUNNING);
+			return true;
+		}
+
+		if (signal_pending_state(state, current))
+			set_current_state(TASK_RUNNING);
+
+		if (current->state == TASK_RUNNING)
+			return true;
+		if (ret < 0)
+			break;
+		cpu_relax();
+	}
+
+	return false;
+}
+
 void blk_mq_disable_hotplug(void)
 {
 	mutex_lock(&all_q_mutex);
diff --git a/block/blk-mq.h b/block/blk-mq.h
index 8cf16cb69f64..79ea86e0ed49 100644
--- a/block/blk-mq.h
+++ b/block/blk-mq.h
@@ -61,6 +61,8 @@ extern void blk_mq_rq_timed_out(struct request *req, bool reserved);
 
 void blk_mq_release(struct request_queue *q);
 
+extern bool blk_mq_poll(struct blk_mq_hw_ctx *, struct request *);
+
 static inline struct blk_mq_ctx *__blk_mq_get_ctx(struct request_queue *q,
 					   unsigned int cpu)
 {
-- 
2.7.4

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

* [PATCH 4/5] blk-mq: implement hybrid poll mode for sync O_DIRECT
  2016-11-03 19:45 [PATCHSET v2] block: IO polling improvements Jens Axboe
                   ` (2 preceding siblings ...)
  2016-11-03 19:45 ` [PATCH 3/5] block: move poll code to blk-mq Jens Axboe
@ 2016-11-03 19:45 ` Jens Axboe
  2016-11-03 19:45 ` [PATCH 5/5] blk-mq: make the polling code adaptive Jens Axboe
  4 siblings, 0 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-03 19:45 UTC (permalink / raw)
  To: axboe, linux-kernel, linux-block; +Cc: hch, Jens Axboe

This patch enables a hybrid polling mode. Instead of polling after IO
submission, we can induce an artificial delay, and then poll after that.
For example, if the IO is presumed to complete in 8 usecs from now, we
can sleep for 4 usecs, wake up, and then do our polling. This still puts
a sleep/wakeup cycle in the IO path, but instead of the wakeup happening
after the IO has completed, it'll happen before. With this hybrid
scheme, we can achieve big latency reductions while still using the same
(or less) amount of CPU.

Signed-off-by: Jens Axboe <axboe@fb.com>
---
 block/blk-mq.c         | 47 +++++++++++++++++++++++++++++++++++++++++++++++
 block/blk-sysfs.c      | 29 +++++++++++++++++++++++++++++
 block/blk.h            |  1 +
 include/linux/blkdev.h |  1 +
 4 files changed, 78 insertions(+)

diff --git a/block/blk-mq.c b/block/blk-mq.c
index 4883c2d97ea9..4f1748d016a9 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -302,6 +302,7 @@ static void __blk_mq_free_request(struct blk_mq_hw_ctx *hctx,
 	rq->rq_flags = 0;
 
 	clear_bit(REQ_ATOM_STARTED, &rq->atomic_flags);
+	clear_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags);
 	blk_mq_put_tag(hctx, ctx, tag);
 	blk_queue_exit(q);
 }
@@ -2357,11 +2358,57 @@ void blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set, int nr_hw_queues)
 }
 EXPORT_SYMBOL_GPL(blk_mq_update_nr_hw_queues);
 
+static bool blk_mq_poll_hybrid_sleep(struct request_queue *q,
+				     struct request *rq)
+{
+	struct hrtimer_sleeper hs;
+	ktime_t kt;
+
+	if (!q->poll_nsec || test_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags))
+		return false;
+
+	set_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags);
+
+	/*
+	 * This will be replaced with the stats tracking code, using
+	 * 'avg_completion_time / 2' as the pre-sleep target.
+	 */
+	kt = ktime_set(0, q->poll_nsec);
+
+	hrtimer_init_on_stack(&hs.timer, CLOCK_MONOTONIC, HRTIMER_MODE_REL);
+	hrtimer_set_expires(&hs.timer, kt);
+
+	hrtimer_init_sleeper(&hs, current);
+	do {
+		if (test_bit(REQ_ATOM_COMPLETE, &rq->atomic_flags))
+			break;
+		set_current_state(TASK_UNINTERRUPTIBLE);
+		hrtimer_start_expires(&hs.timer, HRTIMER_MODE_REL);
+		if (hs.task)
+			io_schedule();
+		hrtimer_cancel(&hs.timer);
+	} while (hs.task && !signal_pending(current));
+
+	__set_current_state(TASK_RUNNING);
+	destroy_hrtimer_on_stack(&hs.timer);
+	return true;
+}
+
 bool blk_mq_poll(struct blk_mq_hw_ctx *hctx, struct request *rq)
 {
 	struct request_queue *q = hctx->queue;
 	long state;
 
+	/*
+	 * If we sleep, have the caller restart the poll loop to reset
+	 * the state. Like for the other success return cases, the
+	 * caller is responsible for checking if the IO completed. If
+	 * the IO isn't complete, we'll get called again and will go
+	 * straight to the busy poll loop.
+	 */
+	if (blk_mq_poll_hybrid_sleep(q, rq))
+		return true;
+
 	hctx->poll_considered++;
 
 	state = current->state;
diff --git a/block/blk-sysfs.c b/block/blk-sysfs.c
index 9cdb7247727a..edc0e491ee3b 100644
--- a/block/blk-sysfs.c
+++ b/block/blk-sysfs.c
@@ -336,6 +336,28 @@ queue_rq_affinity_store(struct request_queue *q, const char *page, size_t count)
 	return ret;
 }
 
+static ssize_t queue_poll_delay_show(struct request_queue *q, char *page)
+{
+	return queue_var_show(q->poll_nsec / 1000, page);
+}
+
+static ssize_t queue_poll_delay_store(struct request_queue *q, const char *page,
+				size_t count)
+{
+	unsigned long poll_usec;
+	ssize_t ret;
+
+	if (!q->mq_ops || !q->mq_ops->poll)
+		return -EINVAL;
+
+	ret = queue_var_store(&poll_usec, page, count);
+	if (ret < 0)
+		return ret;
+
+	q->poll_nsec = poll_usec * 1000;
+	return ret;
+}
+
 static ssize_t queue_poll_show(struct request_queue *q, char *page)
 {
 	return queue_var_show(test_bit(QUEUE_FLAG_POLL, &q->queue_flags), page);
@@ -562,6 +584,12 @@ static struct queue_sysfs_entry queue_poll_entry = {
 	.store = queue_poll_store,
 };
 
+static struct queue_sysfs_entry queue_poll_delay_entry = {
+	.attr = {.name = "io_poll_delay", .mode = S_IRUGO | S_IWUSR },
+	.show = queue_poll_delay_show,
+	.store = queue_poll_delay_store,
+};
+
 static struct queue_sysfs_entry queue_wc_entry = {
 	.attr = {.name = "write_cache", .mode = S_IRUGO | S_IWUSR },
 	.show = queue_wc_show,
@@ -608,6 +636,7 @@ static struct attribute *default_attrs[] = {
 	&queue_wc_entry.attr,
 	&queue_dax_entry.attr,
 	&queue_stats_entry.attr,
+	&queue_poll_delay_entry.attr,
 	NULL,
 };
 
diff --git a/block/blk.h b/block/blk.h
index aa132dea598c..041185e5f129 100644
--- a/block/blk.h
+++ b/block/blk.h
@@ -111,6 +111,7 @@ void blk_account_io_done(struct request *req);
 enum rq_atomic_flags {
 	REQ_ATOM_COMPLETE = 0,
 	REQ_ATOM_STARTED,
+	REQ_ATOM_POLL_SLEPT,
 };
 
 /*
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index b400c31b4a10..fbd8bb5e064c 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -504,6 +504,7 @@ struct request_queue {
 	unsigned int		request_fn_active;
 
 	unsigned int		rq_timeout;
+	unsigned int		poll_nsec;
 	struct timer_list	timeout;
 	struct work_struct	timeout_work;
 	struct list_head	timeout_list;
-- 
2.7.4

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

* [PATCH 5/5] blk-mq: make the polling code adaptive
  2016-11-03 19:45 [PATCHSET v2] block: IO polling improvements Jens Axboe
                   ` (3 preceding siblings ...)
  2016-11-03 19:45 ` [PATCH 4/5] blk-mq: implement hybrid poll mode for sync O_DIRECT Jens Axboe
@ 2016-11-03 19:45 ` Jens Axboe
  4 siblings, 0 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-03 19:45 UTC (permalink / raw)
  To: axboe, linux-kernel, linux-block; +Cc: hch, Jens Axboe

The previous commit introduced the hybrid sleep/poll mode. Take
that one step further, and use the completion latencies to
automatically sleep for half the mean completion time. This is
a good approximation.

This changes the 'io_poll_delay' sysfs file a bit to expose the
various options. Depending on the value, the polling code will
behave differently:

-1	Never enter hybrid sleep mode
 0	Use half of the completion mean for the sleep delay
>0	Use this specific value as the sleep delay

Signed-off-by: Jens Axboe <axboe@fb.com>
---
 block/blk-mq.c         | 67 +++++++++++++++++++++++++++++++++++++++++++++++---
 block/blk-sysfs.c      | 26 ++++++++++++++------
 include/linux/blkdev.h |  2 +-
 3 files changed, 83 insertions(+), 12 deletions(-)

diff --git a/block/blk-mq.c b/block/blk-mq.c
index 4f1748d016a9..9aee5ba63894 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -2023,6 +2023,11 @@ struct request_queue *blk_mq_init_allocated_queue(struct blk_mq_tag_set *set,
 	 */
 	q->nr_requests = set->queue_depth;
 
+	/*
+	 * Default to classic polling
+	 */
+	q->poll_nsec = -1;
+
 	if (set->ops->complete)
 		blk_queue_softirq_done(q, set->ops->complete);
 
@@ -2358,13 +2363,69 @@ void blk_mq_update_nr_hw_queues(struct blk_mq_tag_set *set, int nr_hw_queues)
 }
 EXPORT_SYMBOL_GPL(blk_mq_update_nr_hw_queues);
 
+static unsigned long blk_mq_poll_nsecs(struct request_queue *q,
+				       struct blk_mq_hw_ctx *hctx,
+				       struct request *rq)
+{
+	struct blk_rq_stat stat[2];
+	unsigned long ret = 0;
+
+	/*
+	 * If stats collection isn't on, don't sleep but turn it on for
+	 * future users
+	 */
+	if (!blk_stat_enable(q))
+		return 0;
+
+	/*
+	 * We don't have to do this once per IO, should optimize this
+	 * to just use the current window of stats until it changes
+	 */
+	memset(&stat, 0, sizeof(stat));
+	blk_hctx_stat_get(hctx, stat);
+
+	/*
+	 * As an optimistic guess, use half of the mean service time
+	 * for this type of request. We can (and should) make this smarter.
+	 * For instance, if the completion latencies are tight, we can
+	 * get closer than just half the mean. This is especially
+	 * important on devices where the completion latencies are longer
+	 * than ~10 usec.
+	 */
+	if (req_op(rq) == REQ_OP_READ && stat[BLK_STAT_READ].nr_samples)
+		ret = (stat[BLK_STAT_READ].mean + 1) / 2;
+	else if (req_op(rq) == REQ_OP_WRITE && stat[BLK_STAT_WRITE].nr_samples)
+		ret = (stat[BLK_STAT_WRITE].mean + 1) / 2;
+
+	return ret;
+}
+
 static bool blk_mq_poll_hybrid_sleep(struct request_queue *q,
+				     struct blk_mq_hw_ctx *hctx,
 				     struct request *rq)
 {
 	struct hrtimer_sleeper hs;
+	unsigned int nsecs;
 	ktime_t kt;
 
-	if (!q->poll_nsec || test_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags))
+	if (test_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags))
+		return false;
+
+	/*
+	 * poll_nsec can be:
+	 *
+	 * -1:	don't ever hybrid sleep
+	 *  0:	use half of prev avg
+	 * >0:	use this specific value
+	 */
+	if (q->poll_nsec == -1)
+		return false;
+	else if (q->poll_nsec > 0)
+		nsecs = q->poll_nsec;
+	else
+		nsecs = blk_mq_poll_nsecs(q, hctx, rq);
+
+	if (!nsecs)
 		return false;
 
 	set_bit(REQ_ATOM_POLL_SLEPT, &rq->atomic_flags);
@@ -2373,7 +2434,7 @@ static bool blk_mq_poll_hybrid_sleep(struct request_queue *q,
 	 * This will be replaced with the stats tracking code, using
 	 * 'avg_completion_time / 2' as the pre-sleep target.
 	 */
-	kt = ktime_set(0, q->poll_nsec);
+	kt = ktime_set(0, nsecs);
 
 	hrtimer_init_on_stack(&hs.timer, CLOCK_MONOTONIC, HRTIMER_MODE_REL);
 	hrtimer_set_expires(&hs.timer, kt);
@@ -2406,7 +2467,7 @@ bool blk_mq_poll(struct blk_mq_hw_ctx *hctx, struct request *rq)
 	 * the IO isn't complete, we'll get called again and will go
 	 * straight to the busy poll loop.
 	 */
-	if (blk_mq_poll_hybrid_sleep(q, rq))
+	if (blk_mq_poll_hybrid_sleep(q, hctx, rq))
 		return true;
 
 	hctx->poll_considered++;
diff --git a/block/blk-sysfs.c b/block/blk-sysfs.c
index edc0e491ee3b..66742ef0a323 100644
--- a/block/blk-sysfs.c
+++ b/block/blk-sysfs.c
@@ -338,24 +338,34 @@ queue_rq_affinity_store(struct request_queue *q, const char *page, size_t count)
 
 static ssize_t queue_poll_delay_show(struct request_queue *q, char *page)
 {
-	return queue_var_show(q->poll_nsec / 1000, page);
+	int val;
+
+	if (q->poll_nsec == -1)
+		val = -1;
+	else
+		val = q->poll_nsec / 1000;
+
+	return sprintf(page, "%d\n", val);
 }
 
 static ssize_t queue_poll_delay_store(struct request_queue *q, const char *page,
 				size_t count)
 {
-	unsigned long poll_usec;
-	ssize_t ret;
+	int err, val;
 
 	if (!q->mq_ops || !q->mq_ops->poll)
 		return -EINVAL;
 
-	ret = queue_var_store(&poll_usec, page, count);
-	if (ret < 0)
-		return ret;
+	err = kstrtoint(page, 10, &val);
+	if (err < 0)
+		return err;
 
-	q->poll_nsec = poll_usec * 1000;
-	return ret;
+	if (val == -1)
+		q->poll_nsec = -1;
+	else
+		q->poll_nsec = val * 1000;
+
+	return count;
 }
 
 static ssize_t queue_poll_show(struct request_queue *q, char *page)
diff --git a/include/linux/blkdev.h b/include/linux/blkdev.h
index fbd8bb5e064c..17d3f5c31b66 100644
--- a/include/linux/blkdev.h
+++ b/include/linux/blkdev.h
@@ -504,7 +504,7 @@ struct request_queue {
 	unsigned int		request_fn_active;
 
 	unsigned int		rq_timeout;
-	unsigned int		poll_nsec;
+	int			poll_nsec;
 	struct timer_list	timeout;
 	struct work_struct	timeout_work;
 	struct list_head	timeout_list;
-- 
2.7.4

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

* Re: [PATCH 3/5] block: move poll code to blk-mq
  2016-11-03 19:45 ` [PATCH 3/5] block: move poll code to blk-mq Jens Axboe
@ 2016-11-04 14:44   ` Christoph Hellwig
  2016-11-04 15:30       ` Jens Axboe
  0 siblings, 1 reply; 9+ messages in thread
From: Christoph Hellwig @ 2016-11-04 14:44 UTC (permalink / raw)
  To: Jens Axboe; +Cc: axboe, linux-kernel, linux-block, hch

On Thu, Nov 03, 2016 at 01:45:05PM -0600, Jens Axboe wrote:
> The poll code is blk-mq specific, let's move it to blk-mq.c. This
> is a prep patch for improving the polling code.
> 
> Signed-off-by: Jens Axboe <axboe@fb.com>
> Reviewed-by: Christoph Hellwig <hch@lst.de>

So I gave my ok earlier, but I spent some more time looking at this
this morning and now I wonder why we even bother to keep some
code in blk-core.c.  How about just renaming the whole damn thing
to blk_mq_poll and move it to blk-mq.c instead of that split?

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

* Re: [PATCH 3/5] block: move poll code to blk-mq
  2016-11-04 14:44   ` Christoph Hellwig
@ 2016-11-04 15:30       ` Jens Axboe
  0 siblings, 0 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-04 15:30 UTC (permalink / raw)
  To: Christoph Hellwig; +Cc: axboe, linux-kernel, linux-block

On 11/04/2016 08:44 AM, Christoph Hellwig wrote:
> On Thu, Nov 03, 2016 at 01:45:05PM -0600, Jens Axboe wrote:
>> The poll code is blk-mq specific, let's move it to blk-mq.c. This
>> is a prep patch for improving the polling code.
>>
>> Signed-off-by: Jens Axboe <axboe@fb.com>
>> Reviewed-by: Christoph Hellwig <hch@lst.de>
>
> So I gave my ok earlier, but I spent some more time looking at this
> this morning and now I wonder why we even bother to keep some
> code in blk-core.c.  How about just renaming the whole damn thing
> to blk_mq_poll and move it to blk-mq.c instead of that split?

I pondered that too, and I guess we might as well since the likelihood
of supporting polling on non-mq is pretty close to 0%.

-- 
Jens Axboe

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

* Re: [PATCH 3/5] block: move poll code to blk-mq
@ 2016-11-04 15:30       ` Jens Axboe
  0 siblings, 0 replies; 9+ messages in thread
From: Jens Axboe @ 2016-11-04 15:30 UTC (permalink / raw)
  To: Christoph Hellwig; +Cc: axboe, linux-kernel, linux-block

On 11/04/2016 08:44 AM, Christoph Hellwig wrote:
> On Thu, Nov 03, 2016 at 01:45:05PM -0600, Jens Axboe wrote:
>> The poll code is blk-mq specific, let's move it to blk-mq.c. This
>> is a prep patch for improving the polling code.
>>
>> Signed-off-by: Jens Axboe <axboe@fb.com>
>> Reviewed-by: Christoph Hellwig <hch@lst.de>
>
> So I gave my ok earlier, but I spent some more time looking at this
> this morning and now I wonder why we even bother to keep some
> code in blk-core.c.  How about just renaming the whole damn thing
> to blk_mq_poll and move it to blk-mq.c instead of that split?

I pondered that too, and I guess we might as well since the likelihood
of supporting polling on non-mq is pretty close to 0%.

-- 
Jens Axboe

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

end of thread, other threads:[~2016-11-04 15:30 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2016-11-03 19:45 [PATCHSET v2] block: IO polling improvements Jens Axboe
2016-11-03 19:45 ` [PATCH 1/5] block: fast-path for small and simple direct I/O requests Jens Axboe
2016-11-03 19:45 ` [PATCH 2/5] block: add scalable completion tracking of requests Jens Axboe
2016-11-03 19:45 ` [PATCH 3/5] block: move poll code to blk-mq Jens Axboe
2016-11-04 14:44   ` Christoph Hellwig
2016-11-04 15:30     ` Jens Axboe
2016-11-04 15:30       ` Jens Axboe
2016-11-03 19:45 ` [PATCH 4/5] blk-mq: implement hybrid poll mode for sync O_DIRECT Jens Axboe
2016-11-03 19:45 ` [PATCH 5/5] blk-mq: make the polling code adaptive Jens Axboe

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