linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH V2 0/2] block: kyber: make kyber more friendly with merging
@ 2018-05-23  6:33 Jianchao Wang
  2018-05-23  6:33 ` [PATCH V2 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio merge helper Jianchao Wang
  2018-05-23  6:33 ` [PATCH V2 2/2] block: kyber: make kyber more friendly with merging Jianchao Wang
  0 siblings, 2 replies; 6+ messages in thread
From: Jianchao Wang @ 2018-05-23  6:33 UTC (permalink / raw)
  To: axboe; +Cc: osandov, holger, linux-block, linux-kernel

Hi Jens

This is the second version patchset to make the kyber io scheduler more firendly
with merging.

Most of time, kyber io scheduler will not leave any requests in ctx rq_list,
this is because even if tokens of one domain is used up, kyber will try to
dispatch requests from other domain and flush the rq_list there. So there is not
any change to do merge.

To improve this, setup kyber_ctx_queue (kcq) which is similar with blk_mq_ctx but has
rq_list for every domain, and build mapping between kcq and khd as ctx and hctx.
Then we could merge, insert and dispatch on rq_list of different domains separately.
At the same time, only flush the rq_list of kcq after get domain token
successfully, then the requests could be left in the rq_list of that domain and 
maybe merged with following io. In my local test on NVMe card, the sequential io
performance could be improved a lot, especially on high workload. More details
please refer to the second patch.

Changes V1 to V2:
 - Add patch from Jens which abstract out the blk-mq-sched rq list iteration bio
 merge helper interface and rebase the patch based on it.
 - merge bio_sched_domain and rq_sched_domain to avoid the duplicated code.
 - allocate kcqs per khd

Jens Axboe (1)
0001-blk-mq-abstract-out-blk-mq-sched-rq-list-iteration-b.patch

Jianchao Wang (1)
0002-block-kyber-make-kyber-more-friendly-with-merging.patch

 block/blk-mq-sched.c   |  34 ++++++---
 block/kyber-iosched.c  | 197 ++++++++++++++++++++++++++++++++++++++++---------
 include/linux/blk-mq.h |   3 +-
 3 files changed, 188 insertions(+), 46 deletions(-)


Thanks
Jianchao

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

* [PATCH V2 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio merge helper
  2018-05-23  6:33 [PATCH V2 0/2] block: kyber: make kyber more friendly with merging Jianchao Wang
@ 2018-05-23  6:33 ` Jianchao Wang
  2018-05-29 18:55   ` Omar Sandoval
  2018-05-23  6:33 ` [PATCH V2 2/2] block: kyber: make kyber more friendly with merging Jianchao Wang
  1 sibling, 1 reply; 6+ messages in thread
From: Jianchao Wang @ 2018-05-23  6:33 UTC (permalink / raw)
  To: axboe; +Cc: osandov, holger, linux-block, linux-kernel

From: Jens Axboe <axboe@kernel.dk>

No functional changes in this patch, just a prep patch for utilizing
this in an IO scheduler.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 block/blk-mq-sched.c   | 34 ++++++++++++++++++++++++----------
 include/linux/blk-mq.h |  3 ++-
 2 files changed, 26 insertions(+), 11 deletions(-)

diff --git a/block/blk-mq-sched.c b/block/blk-mq-sched.c
index 25c14c5..b0f2c2a 100644
--- a/block/blk-mq-sched.c
+++ b/block/blk-mq-sched.c
@@ -268,19 +268,16 @@ bool blk_mq_sched_try_merge(struct request_queue *q, struct bio *bio,
 EXPORT_SYMBOL_GPL(blk_mq_sched_try_merge);
 
 /*
- * Reverse check our software queue for entries that we could potentially
- * merge with. Currently includes a hand-wavy stop count of 8, to not spend
- * too much time checking for merges.
+ * Iterate list of requests and see if we can merge this bio with any
+ * of them.
  */
-static bool blk_mq_attempt_merge(struct request_queue *q,
-				 struct blk_mq_ctx *ctx, struct bio *bio)
+bool blk_mq_bio_list_merge(struct request_queue *q, struct list_head *list,
+			   struct bio *bio)
 {
 	struct request *rq;
 	int checked = 8;
 
-	lockdep_assert_held(&ctx->lock);
-
-	list_for_each_entry_reverse(rq, &ctx->rq_list, queuelist) {
+	list_for_each_entry_reverse(rq, list, queuelist) {
 		bool merged = false;
 
 		if (!checked--)
@@ -305,13 +302,30 @@ static bool blk_mq_attempt_merge(struct request_queue *q,
 			continue;
 		}
 
-		if (merged)
-			ctx->rq_merged++;
 		return merged;
 	}
 
 	return false;
 }
+EXPORT_SYMBOL_GPL(blk_mq_bio_list_merge);
+
+/*
+ * Reverse check our software queue for entries that we could potentially
+ * merge with. Currently includes a hand-wavy stop count of 8, to not spend
+ * too much time checking for merges.
+ */
+static bool blk_mq_attempt_merge(struct request_queue *q,
+				 struct blk_mq_ctx *ctx, struct bio *bio)
+{
+	lockdep_assert_held(&ctx->lock);
+
+	if (blk_mq_bio_list_merge(q, &ctx->rq_list, bio)) {
+		ctx->rq_merged++;
+		return true;
+	}
+
+	return false;
+}
 
 bool __blk_mq_sched_bio_merge(struct request_queue *q, struct bio *bio)
 {
diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
index ebc34a5..fb35517 100644
--- a/include/linux/blk-mq.h
+++ b/include/linux/blk-mq.h
@@ -259,7 +259,8 @@ void blk_mq_add_to_requeue_list(struct request *rq, bool at_head,
 void blk_mq_kick_requeue_list(struct request_queue *q);
 void blk_mq_delay_kick_requeue_list(struct request_queue *q, unsigned long msecs);
 void blk_mq_complete_request(struct request *rq);
-
+bool blk_mq_bio_list_merge(struct request_queue *q, struct list_head *list,
+			   struct bio *bio);
 bool blk_mq_queue_stopped(struct request_queue *q);
 void blk_mq_stop_hw_queue(struct blk_mq_hw_ctx *hctx);
 void blk_mq_start_hw_queue(struct blk_mq_hw_ctx *hctx);
-- 
2.7.4

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

* [PATCH V2 2/2] block: kyber: make kyber more friendly with merging
  2018-05-23  6:33 [PATCH V2 0/2] block: kyber: make kyber more friendly with merging Jianchao Wang
  2018-05-23  6:33 ` [PATCH V2 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio merge helper Jianchao Wang
@ 2018-05-23  6:33 ` Jianchao Wang
  2018-05-29 18:55   ` Omar Sandoval
  1 sibling, 1 reply; 6+ messages in thread
From: Jianchao Wang @ 2018-05-23  6:33 UTC (permalink / raw)
  To: axboe; +Cc: osandov, holger, linux-block, linux-kernel

Currently, kyber is very unfriendly with merging. kyber depends
on ctx rq_list to do merging, however, most of time, it will not
leave any requests in ctx rq_list. This is because even if tokens
of one domain is used up, kyber will try to dispatch requests
from other domain and flush the rq_list there.

To improve this, we setup kyber_ctx_queue (kcq) which is similar
with ctx, but it has rq_lists for different domain and build same
mapping between kcq and khd as the ctx & hctx. Then we could merge,
insert and dispatch for different domains separately. At the same
time, only flush the rq_list of kcq when get domain token successfully.
Then if one domain token is used up, the requests could be left in
the rq_list of that domain and maybe merged with following io.

Following is my test result on machine with 8 cores and NVMe card
INTEL SSDPEKKR128G7

fio size=256m ioengine=libaio iodepth=64 direct=1 numjobs=8
seq/random
+------+---------------------------------------------------------------+
|patch?| bw(MB/s) |   iops    | slat(usec) |    clat(usec)   |  merge  |
+----------------------------------------------------------------------+
| w/o  |  606/612 | 151k/153k |  6.89/7.03 | 3349.21/3305.40 |   0/0   |
+----------------------------------------------------------------------+
| w/   | 1083/616 | 277k/154k |  4.93/6.95 | 1830.62/3279.95 | 223k/3k |
+----------------------------------------------------------------------+
When set numjobs to 16, the bw and iops could reach 1662MB/s and 425k
on my platform.

Signed-off-by: Jianchao Wang <jianchao.w.wang@oracle.com>
Tested-by: Holger Hoffstätte <holger@applied-asynchrony.com>
---
 block/kyber-iosched.c | 197 +++++++++++++++++++++++++++++++++++++++++---------
 1 file changed, 162 insertions(+), 35 deletions(-)

diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
index 0d6d25e3..7ca1364 100644
--- a/block/kyber-iosched.c
+++ b/block/kyber-iosched.c
@@ -72,6 +72,15 @@ static const unsigned int kyber_batch_size[] = {
 	[KYBER_OTHER] = 8,
 };
 
+struct kyber_ctx_queue {
+	/*
+	 * Copied from blk_mq_ctx->index_hw
+	 */
+	unsigned int index;
+	spinlock_t lock;
+	struct list_head rq_list[KYBER_NUM_DOMAINS];
+} ____cacheline_aligned_in_smp;
+
 struct kyber_queue_data {
 	struct request_queue *q;
 
@@ -99,6 +108,8 @@ struct kyber_hctx_data {
 	struct list_head rqs[KYBER_NUM_DOMAINS];
 	unsigned int cur_domain;
 	unsigned int batching;
+	struct kyber_ctx_queue *kcqs;
+	struct sbitmap kcq_map[KYBER_NUM_DOMAINS];
 	wait_queue_entry_t domain_wait[KYBER_NUM_DOMAINS];
 	struct sbq_wait_state *domain_ws[KYBER_NUM_DOMAINS];
 	atomic_t wait_index[KYBER_NUM_DOMAINS];
@@ -107,10 +118,8 @@ struct kyber_hctx_data {
 static int kyber_domain_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
 			     void *key);
 
-static int rq_sched_domain(const struct request *rq)
+static int kyber_sched_domain(unsigned int op)
 {
-	unsigned int op = rq->cmd_flags;
-
 	if ((op & REQ_OP_MASK) == REQ_OP_READ)
 		return KYBER_READ;
 	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
@@ -284,6 +293,11 @@ static unsigned int kyber_sched_tags_shift(struct kyber_queue_data *kqd)
 	return kqd->q->queue_hw_ctx[0]->sched_tags->bitmap_tags.sb.shift;
 }
 
+static int kyber_bucket_fn(const struct request *rq)
+{
+	return kyber_sched_domain(rq->cmd_flags);
+}
+
 static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 {
 	struct kyber_queue_data *kqd;
@@ -297,11 +311,10 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 		goto err;
 	kqd->q = q;
 
-	kqd->cb = blk_stat_alloc_callback(kyber_stat_timer_fn, rq_sched_domain,
+	kqd->cb = blk_stat_alloc_callback(kyber_stat_timer_fn, kyber_bucket_fn,
 					  KYBER_NUM_DOMAINS, kqd);
 	if (!kqd->cb)
 		goto err_kqd;
-
 	/*
 	 * The maximum number of tokens for any scheduling domain is at least
 	 * the queue depth of a single hardware queue. If the hardware doesn't
@@ -376,15 +389,45 @@ static void kyber_exit_sched(struct elevator_queue *e)
 	kfree(kqd);
 }
 
+static void kyber_ctx_queue_init(struct kyber_ctx_queue *kcq)
+{
+	unsigned int i;
+
+	spin_lock_init(&kcq->lock);
+	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
+		INIT_LIST_HEAD(&kcq->rq_list[i]);
+}
+
 static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
 {
 	struct kyber_hctx_data *khd;
-	int i;
+	int i, sd;
 
 	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
 	if (!khd)
 		return -ENOMEM;
 
+	khd->kcqs = kmalloc_array_node(hctx->nr_ctx,
+			sizeof(struct kyber_ctx_queue),
+			GFP_KERNEL, hctx->numa_node);
+	if (!khd->kcqs)
+		goto err_khd;
+	/*
+	 * clone the mapping between hctx and ctx to khd and kcq
+	 */
+	for (i = 0; i < hctx->nr_ctx; i++) {
+		kyber_ctx_queue_init(&khd->kcqs[i]);
+		khd->kcqs[i].index = i;
+	}
+
+	sd = 0;
+	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
+		if (sbitmap_init_node(&khd->kcq_map[i], hctx->nr_ctx,
+				ilog2(8), GFP_KERNEL, hctx->numa_node))
+			goto err_kcq_map;
+		sd++;
+	}
+
 	spin_lock_init(&khd->lock);
 
 	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
@@ -402,10 +445,24 @@ static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
 	hctx->sched_data = khd;
 
 	return 0;
+
+err_kcq_map:
+	for (i = 0; i < sd; i++)
+		sbitmap_free(&khd->kcq_map[i]);
+	kfree(khd->kcqs);
+err_khd:
+	kfree(khd);
+	return -ENOMEM;
 }
 
 static void kyber_exit_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
 {
+	struct kyber_hctx_data *khd = hctx->sched_data;
+	int i;
+
+	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
+		sbitmap_free(&khd->kcq_map[i]);
+	kfree(khd->kcqs);
 	kfree(hctx->sched_data);
 }
 
@@ -427,7 +484,7 @@ static void rq_clear_domain_token(struct kyber_queue_data *kqd,
 
 	nr = rq_get_domain_token(rq);
 	if (nr != -1) {
-		sched_domain = rq_sched_domain(rq);
+		sched_domain = kyber_sched_domain(rq->cmd_flags);
 		sbitmap_queue_clear(&kqd->domain_tokens[sched_domain], nr,
 				    rq->mq_ctx->cpu);
 	}
@@ -446,11 +503,51 @@ static void kyber_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
 	}
 }
 
+static bool kyber_bio_merge(struct blk_mq_hw_ctx *hctx, struct bio *bio)
+{
+	struct kyber_hctx_data *khd = hctx->sched_data;
+	struct blk_mq_ctx *ctx = blk_mq_get_ctx(hctx->queue);
+	struct kyber_ctx_queue *kcq = &khd->kcqs[ctx->index_hw];
+	struct list_head *rq_list = &kcq->rq_list[kyber_sched_domain(bio->bi_opf)];
+	bool merged;
+
+	spin_lock(&kcq->lock);
+	merged = blk_mq_bio_list_merge(hctx->queue, rq_list, bio);
+	spin_unlock(&kcq->lock);
+	blk_mq_put_ctx(ctx);
+
+	return merged;
+}
+
 static void kyber_prepare_request(struct request *rq, struct bio *bio)
 {
 	rq_set_domain_token(rq, -1);
 }
 
+static void kyber_insert_requests(struct blk_mq_hw_ctx *hctx,
+		struct list_head *rq_list, bool at_head)
+{
+	struct kyber_hctx_data *khd = hctx->sched_data;
+	struct kyber_ctx_queue *kcq;
+	struct request *rq, *next;
+	struct list_head *head;
+	unsigned int sched_domain;
+
+	list_for_each_entry_safe(rq, next, rq_list, queuelist) {
+		sched_domain = kyber_sched_domain(rq->cmd_flags);
+		kcq = &khd->kcqs[rq->mq_ctx->index_hw];
+		head = &kcq->rq_list[sched_domain];
+		spin_lock(&kcq->lock);
+		if (at_head)
+			list_move(&rq->queuelist, head);
+		else
+			list_move_tail(&rq->queuelist, head);
+		sbitmap_set_bit(&khd->kcq_map[sched_domain], kcq->index);
+		blk_mq_sched_request_inserted(rq);
+		spin_unlock(&kcq->lock);
+	}
+}
+
 static void kyber_finish_request(struct request *rq)
 {
 	struct kyber_queue_data *kqd = rq->q->elevator->elevator_data;
@@ -469,7 +566,7 @@ static void kyber_completed_request(struct request *rq)
 	 * Check if this request met our latency goal. If not, quickly gather
 	 * some statistics and start throttling.
 	 */
-	sched_domain = rq_sched_domain(rq);
+	sched_domain = kyber_sched_domain(rq->cmd_flags);
 	switch (sched_domain) {
 	case KYBER_READ:
 		target = kqd->read_lat_nsec;
@@ -495,19 +592,36 @@ static void kyber_completed_request(struct request *rq)
 		blk_stat_activate_msecs(kqd->cb, 10);
 }
 
-static void kyber_flush_busy_ctxs(struct kyber_hctx_data *khd,
-				  struct blk_mq_hw_ctx *hctx)
+struct flush_kcq_data {
+	struct kyber_hctx_data *khd;
+	unsigned int sched_domain;
+	struct list_head *list;
+};
+
+static bool flush_busy_kcq(struct sbitmap *sb, unsigned int bitnr, void *data)
 {
-	LIST_HEAD(rq_list);
-	struct request *rq, *next;
+	struct flush_kcq_data *flush_data = data;
+	struct kyber_ctx_queue *kcq = &flush_data->khd->kcqs[bitnr];
 
-	blk_mq_flush_busy_ctxs(hctx, &rq_list);
-	list_for_each_entry_safe(rq, next, &rq_list, queuelist) {
-		unsigned int sched_domain;
+	spin_lock(&kcq->lock);
+	list_splice_tail_init(&kcq->rq_list[flush_data->sched_domain],
+			flush_data->list);
+	sbitmap_clear_bit(sb, bitnr);
+	spin_unlock(&kcq->lock);
 
-		sched_domain = rq_sched_domain(rq);
-		list_move_tail(&rq->queuelist, &khd->rqs[sched_domain]);
-	}
+	return true;
+}
+
+static void kyber_flush_busy_kcqs(struct kyber_hctx_data *khd,
+		unsigned int sched_domain, struct list_head *list)
+{
+	struct flush_kcq_data data = {
+		.khd = khd,
+		.sched_domain = sched_domain,
+		.list = list,
+	};
+
+	sbitmap_for_each_set(&khd->kcq_map[sched_domain], flush_busy_kcq, &data);
 }
 
 static int kyber_domain_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
@@ -570,26 +684,19 @@ static int kyber_get_domain_token(struct kyber_queue_data *kqd,
 static struct request *
 kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
 			  struct kyber_hctx_data *khd,
-			  struct blk_mq_hw_ctx *hctx,
-			  bool *flushed)
+			  struct blk_mq_hw_ctx *hctx)
 {
 	struct list_head *rqs;
 	struct request *rq;
 	int nr;
 
 	rqs = &khd->rqs[khd->cur_domain];
-	rq = list_first_entry_or_null(rqs, struct request, queuelist);
 
 	/*
-	 * If there wasn't already a pending request and we haven't flushed the
-	 * software queues yet, flush the software queues and check again.
+	 * If we do have cur_domain rqs on khd or kcq list, then try to require
+	 * the token
 	 */
-	if (!rq && !*flushed) {
-		kyber_flush_busy_ctxs(khd, hctx);
-		*flushed = true;
-		rq = list_first_entry_or_null(rqs, struct request, queuelist);
-	}
-
+	rq = list_first_entry_or_null(rqs, struct request, queuelist);
 	if (rq) {
 		nr = kyber_get_domain_token(kqd, khd, hctx);
 		if (nr >= 0) {
@@ -598,8 +705,25 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
 			list_del_init(&rq->queuelist);
 			return rq;
 		}
+	} else if (sbitmap_any_bit_set(&khd->kcq_map[khd->cur_domain])) {
+		nr = kyber_get_domain_token(kqd, khd, hctx);
+		if (nr >= 0) {
+			kyber_flush_busy_kcqs(khd, khd->cur_domain, rqs);
+			rq = list_first_entry_or_null(rqs, struct request, queuelist);
+			/*
+			 * khd->lock and kcq->lock will ensure that, if kcq_map[cur_domain]
+			 * is set, we must be able to get requests from the kcq
+			 */
+			khd->batching++;
+			rq_set_domain_token(rq, nr);
+			list_del_init(&rq->queuelist);
+			return rq;
+		}
+		/*
+		 * if not get domain token, the rqs could be left on kcqs to merged
+		 * with following ios.
+		 */
 	}
-
 	/* There were either no pending requests or no tokens. */
 	return NULL;
 }
@@ -608,7 +732,6 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
 {
 	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
 	struct kyber_hctx_data *khd = hctx->sched_data;
-	bool flushed = false;
 	struct request *rq;
 	int i;
 
@@ -619,7 +742,7 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
 	 * from the batch.
 	 */
 	if (khd->batching < kyber_batch_size[khd->cur_domain]) {
-		rq = kyber_dispatch_cur_domain(kqd, khd, hctx, &flushed);
+		rq = kyber_dispatch_cur_domain(kqd, khd, hctx);
 		if (rq)
 			goto out;
 	}
@@ -640,7 +763,7 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
 		else
 			khd->cur_domain++;
 
-		rq = kyber_dispatch_cur_domain(kqd, khd, hctx, &flushed);
+		rq = kyber_dispatch_cur_domain(kqd, khd, hctx);
 		if (rq)
 			goto out;
 	}
@@ -657,10 +780,12 @@ static bool kyber_has_work(struct blk_mq_hw_ctx *hctx)
 	int i;
 
 	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
-		if (!list_empty_careful(&khd->rqs[i]))
+		if (!list_empty_careful(&khd->rqs[i]) ||
+				sbitmap_any_bit_set(&khd->kcq_map[i]))
 			return true;
 	}
-	return sbitmap_any_bit_set(&hctx->ctx_map);
+
+	return false;
 }
 
 #define KYBER_LAT_SHOW_STORE(op)					\
@@ -831,7 +956,9 @@ static struct elevator_type kyber_sched = {
 		.init_hctx = kyber_init_hctx,
 		.exit_hctx = kyber_exit_hctx,
 		.limit_depth = kyber_limit_depth,
+		.bio_merge = kyber_bio_merge,
 		.prepare_request = kyber_prepare_request,
+		.insert_requests = kyber_insert_requests,
 		.finish_request = kyber_finish_request,
 		.requeue_request = kyber_finish_request,
 		.completed_request = kyber_completed_request,
-- 
2.7.4

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

* Re: [PATCH V2 2/2] block: kyber: make kyber more friendly with merging
  2018-05-23  6:33 ` [PATCH V2 2/2] block: kyber: make kyber more friendly with merging Jianchao Wang
@ 2018-05-29 18:55   ` Omar Sandoval
  2018-05-30  3:10     ` jianchao.wang
  0 siblings, 1 reply; 6+ messages in thread
From: Omar Sandoval @ 2018-05-29 18:55 UTC (permalink / raw)
  To: Jianchao Wang; +Cc: axboe, holger, linux-block, linux-kernel

On Wed, May 23, 2018 at 02:33:22PM +0800, Jianchao Wang wrote:
> Currently, kyber is very unfriendly with merging. kyber depends
> on ctx rq_list to do merging, however, most of time, it will not
> leave any requests in ctx rq_list. This is because even if tokens
> of one domain is used up, kyber will try to dispatch requests
> from other domain and flush the rq_list there.
> 
> To improve this, we setup kyber_ctx_queue (kcq) which is similar
> with ctx, but it has rq_lists for different domain and build same
> mapping between kcq and khd as the ctx & hctx. Then we could merge,
> insert and dispatch for different domains separately. At the same
> time, only flush the rq_list of kcq when get domain token successfully.
> Then if one domain token is used up, the requests could be left in
> the rq_list of that domain and maybe merged with following io.
> 
> Following is my test result on machine with 8 cores and NVMe card
> INTEL SSDPEKKR128G7
> 
> fio size=256m ioengine=libaio iodepth=64 direct=1 numjobs=8
> seq/random
> +------+---------------------------------------------------------------+
> |patch?| bw(MB/s) |   iops    | slat(usec) |    clat(usec)   |  merge  |
> +----------------------------------------------------------------------+
> | w/o  |  606/612 | 151k/153k |  6.89/7.03 | 3349.21/3305.40 |   0/0   |
> +----------------------------------------------------------------------+
> | w/   | 1083/616 | 277k/154k |  4.93/6.95 | 1830.62/3279.95 | 223k/3k |
> +----------------------------------------------------------------------+
> When set numjobs to 16, the bw and iops could reach 1662MB/s and 425k
> on my platform.
> 
> Signed-off-by: Jianchao Wang <jianchao.w.wang@oracle.com>
> Tested-by: Holger Hoffstätte <holger@applied-asynchrony.com>

Thanks, the per-hctx kqcs make much more sense. This is pretty cool! A
few nitpicks below.

> ---
>  block/kyber-iosched.c | 197 +++++++++++++++++++++++++++++++++++++++++---------
>  1 file changed, 162 insertions(+), 35 deletions(-)
> 
> diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
> index 0d6d25e3..7ca1364 100644
> --- a/block/kyber-iosched.c
> +++ b/block/kyber-iosched.c
> @@ -72,6 +72,15 @@ static const unsigned int kyber_batch_size[] = {
>  	[KYBER_OTHER] = 8,
>  };
>  
> +struct kyber_ctx_queue {

I'm not crazy about this name, but I can't really think of anything
better, so this will have to do.

> +	/*
> +	 * Copied from blk_mq_ctx->index_hw
> +	 */
> +	unsigned int index;

We can just calculate this as kcq - khd->kcqs in the one place we use
it.

> +	spinlock_t lock;
> +	struct list_head rq_list[KYBER_NUM_DOMAINS];
> +} ____cacheline_aligned_in_smp;
> +
>  struct kyber_queue_data {
>  	struct request_queue *q;
>  
> @@ -99,6 +108,8 @@ struct kyber_hctx_data {
>  	struct list_head rqs[KYBER_NUM_DOMAINS];
>  	unsigned int cur_domain;
>  	unsigned int batching;
> +	struct kyber_ctx_queue *kcqs;
> +	struct sbitmap kcq_map[KYBER_NUM_DOMAINS];
>  	wait_queue_entry_t domain_wait[KYBER_NUM_DOMAINS];
>  	struct sbq_wait_state *domain_ws[KYBER_NUM_DOMAINS];
>  	atomic_t wait_index[KYBER_NUM_DOMAINS];
> @@ -107,10 +118,8 @@ struct kyber_hctx_data {
>  static int kyber_domain_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
>  			     void *key);
>  
> -static int rq_sched_domain(const struct request *rq)
> +static int kyber_sched_domain(unsigned int op)

Please make this return an unsigned int since that's what we use for
sched_domain everywhere except the stats bucket function.

>  {
> -	unsigned int op = rq->cmd_flags;
> -
>  	if ((op & REQ_OP_MASK) == REQ_OP_READ)
>  		return KYBER_READ;
>  	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
> @@ -284,6 +293,11 @@ static unsigned int kyber_sched_tags_shift(struct kyber_queue_data *kqd)
>  	return kqd->q->queue_hw_ctx[0]->sched_tags->bitmap_tags.sb.shift;
>  }
>  
> +static int kyber_bucket_fn(const struct request *rq)
> +{
> +	return kyber_sched_domain(rq->cmd_flags);
> +}
> +
>  static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>  {
>  	struct kyber_queue_data *kqd;
> @@ -297,11 +311,10 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>  		goto err;
>  	kqd->q = q;
>  
> -	kqd->cb = blk_stat_alloc_callback(kyber_stat_timer_fn, rq_sched_domain,
> +	kqd->cb = blk_stat_alloc_callback(kyber_stat_timer_fn, kyber_bucket_fn,
>  					  KYBER_NUM_DOMAINS, kqd);
>  	if (!kqd->cb)
>  		goto err_kqd;
> -

Unrelated whitespace change.

>  	/*
>  	 * The maximum number of tokens for any scheduling domain is at least
>  	 * the queue depth of a single hardware queue. If the hardware doesn't
> @@ -376,15 +389,45 @@ static void kyber_exit_sched(struct elevator_queue *e)
>  	kfree(kqd);
>  }
>  
> +static void kyber_ctx_queue_init(struct kyber_ctx_queue *kcq)
> +{
> +	unsigned int i;
> +
> +	spin_lock_init(&kcq->lock);
> +	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
> +		INIT_LIST_HEAD(&kcq->rq_list[i]);
> +}
> +
>  static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>  {
>  	struct kyber_hctx_data *khd;
> -	int i;
> +	int i, sd;
>  
>  	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
>  	if (!khd)
>  		return -ENOMEM;
>  
> +	khd->kcqs = kmalloc_array_node(hctx->nr_ctx,
> +			sizeof(struct kyber_ctx_queue),
> +			GFP_KERNEL, hctx->numa_node);

Argument whitespace alignment is still wrong here. I guess you have to
pass --strict to checkpatch.pl to get it to warn about that. Usually I
wouldn't be nitpicky about this but I made sure to keep this file nice
and neat, and we need a v3 anyways :)

> +	if (!khd->kcqs)
> +		goto err_khd;
> +	/*
> +	 * clone the mapping between hctx and ctx to khd and kcq
> +	 */
> +	for (i = 0; i < hctx->nr_ctx; i++) {
> +		kyber_ctx_queue_init(&khd->kcqs[i]);
> +		khd->kcqs[i].index = i;
> +	}
> +
> +	sd = 0;
> +	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
> +		if (sbitmap_init_node(&khd->kcq_map[i], hctx->nr_ctx,
> +				ilog2(8), GFP_KERNEL, hctx->numa_node))

Whitespace again.

> +			goto err_kcq_map;

In this error case, I'd prefer if we did something similar to
kyber_queue_data_alloc() instead of having this obscurely-named sd
variable:

			while (--i >= 0)
				sbitmap_free(&khd->kcq_map[i]);
			goto err_kcqs;

Where err_kcqs is just the kfree(khd->kcqs).

> +		sd++;
> +	}
> +
>  	spin_lock_init(&khd->lock);
>  
>  	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
> @@ -402,10 +445,24 @@ static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>  	hctx->sched_data = khd;
>  
>  	return 0;
> +
> +err_kcq_map:
> +	for (i = 0; i < sd; i++)
> +		sbitmap_free(&khd->kcq_map[i]);
> +	kfree(khd->kcqs);
> +err_khd:
> +	kfree(khd);
> +	return -ENOMEM;
>  }
>  
>  static void kyber_exit_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>  {
> +	struct kyber_hctx_data *khd = hctx->sched_data;
> +	int i;
> +
> +	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
> +		sbitmap_free(&khd->kcq_map[i]);
> +	kfree(khd->kcqs);
>  	kfree(hctx->sched_data);
>  }
>  
> @@ -427,7 +484,7 @@ static void rq_clear_domain_token(struct kyber_queue_data *kqd,
>  
>  	nr = rq_get_domain_token(rq);
>  	if (nr != -1) {
> -		sched_domain = rq_sched_domain(rq);
> +		sched_domain = kyber_sched_domain(rq->cmd_flags);
>  		sbitmap_queue_clear(&kqd->domain_tokens[sched_domain], nr,
>  				    rq->mq_ctx->cpu);
>  	}
> @@ -446,11 +503,51 @@ static void kyber_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
>  	}
>  }
>  
> +static bool kyber_bio_merge(struct blk_mq_hw_ctx *hctx, struct bio *bio)
> +{
> +	struct kyber_hctx_data *khd = hctx->sched_data;
> +	struct blk_mq_ctx *ctx = blk_mq_get_ctx(hctx->queue);
> +	struct kyber_ctx_queue *kcq = &khd->kcqs[ctx->index_hw];
> +	struct list_head *rq_list = &kcq->rq_list[kyber_sched_domain(bio->bi_opf)];
> +	bool merged;
> +
> +	spin_lock(&kcq->lock);
> +	merged = blk_mq_bio_list_merge(hctx->queue, rq_list, bio);
> +	spin_unlock(&kcq->lock);
> +	blk_mq_put_ctx(ctx);
> +
> +	return merged;
> +}
> +
>  static void kyber_prepare_request(struct request *rq, struct bio *bio)
>  {
>  	rq_set_domain_token(rq, -1);
>  }
>  
> +static void kyber_insert_requests(struct blk_mq_hw_ctx *hctx,
> +		struct list_head *rq_list, bool at_head)

Whitespace.

> +{
> +	struct kyber_hctx_data *khd = hctx->sched_data;
> +	struct kyber_ctx_queue *kcq;
> +	struct request *rq, *next;
> +	struct list_head *head;
> +	unsigned int sched_domain;
> +
> +	list_for_each_entry_safe(rq, next, rq_list, queuelist) {

Please declare the variables which are local to the loop iteration
inside of the loop body. I.e.,

		unsigned int sched_domain;
		struct kyber_ctx_queue *kcq;
		struct list_head *head;

> +		sched_domain = kyber_sched_domain(rq->cmd_flags);
> +		kcq = &khd->kcqs[rq->mq_ctx->index_hw];
> +		head = &kcq->rq_list[sched_domain];
> +		spin_lock(&kcq->lock);
> +		if (at_head)
> +			list_move(&rq->queuelist, head);
> +		else
> +			list_move_tail(&rq->queuelist, head);
> +		sbitmap_set_bit(&khd->kcq_map[sched_domain], kcq->index);
> +		blk_mq_sched_request_inserted(rq);
> +		spin_unlock(&kcq->lock);
> +	}
> +}
> +
>  static void kyber_finish_request(struct request *rq)
>  {
>  	struct kyber_queue_data *kqd = rq->q->elevator->elevator_data;
> @@ -469,7 +566,7 @@ static void kyber_completed_request(struct request *rq)
>  	 * Check if this request met our latency goal. If not, quickly gather
>  	 * some statistics and start throttling.
>  	 */
> -	sched_domain = rq_sched_domain(rq);
> +	sched_domain = kyber_sched_domain(rq->cmd_flags);
>  	switch (sched_domain) {
>  	case KYBER_READ:
>  		target = kqd->read_lat_nsec;
> @@ -495,19 +592,36 @@ static void kyber_completed_request(struct request *rq)
>  		blk_stat_activate_msecs(kqd->cb, 10);
>  }
>  
> -static void kyber_flush_busy_ctxs(struct kyber_hctx_data *khd,
> -				  struct blk_mq_hw_ctx *hctx)
> +struct flush_kcq_data {
> +	struct kyber_hctx_data *khd;
> +	unsigned int sched_domain;
> +	struct list_head *list;
> +};
> +
> +static bool flush_busy_kcq(struct sbitmap *sb, unsigned int bitnr, void *data)
>  {
> -	LIST_HEAD(rq_list);
> -	struct request *rq, *next;
> +	struct flush_kcq_data *flush_data = data;
> +	struct kyber_ctx_queue *kcq = &flush_data->khd->kcqs[bitnr];
>  
> -	blk_mq_flush_busy_ctxs(hctx, &rq_list);
> -	list_for_each_entry_safe(rq, next, &rq_list, queuelist) {
> -		unsigned int sched_domain;
> +	spin_lock(&kcq->lock);
> +	list_splice_tail_init(&kcq->rq_list[flush_data->sched_domain],
> +			flush_data->list);
> +	sbitmap_clear_bit(sb, bitnr);
> +	spin_unlock(&kcq->lock);
>  
> -		sched_domain = rq_sched_domain(rq);
> -		list_move_tail(&rq->queuelist, &khd->rqs[sched_domain]);
> -	}
> +	return true;
> +}
> +
> +static void kyber_flush_busy_kcqs(struct kyber_hctx_data *khd,
> +		unsigned int sched_domain, struct list_head *list)
> +{
> +	struct flush_kcq_data data = {
> +		.khd = khd,
> +		.sched_domain = sched_domain,
> +		.list = list,
> +	};
> +
> +	sbitmap_for_each_set(&khd->kcq_map[sched_domain], flush_busy_kcq, &data);
>  }
>  
>  static int kyber_domain_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
> @@ -570,26 +684,19 @@ static int kyber_get_domain_token(struct kyber_queue_data *kqd,
>  static struct request *
>  kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
>  			  struct kyber_hctx_data *khd,
> -			  struct blk_mq_hw_ctx *hctx,
> -			  bool *flushed)
> +			  struct blk_mq_hw_ctx *hctx)
>  {
>  	struct list_head *rqs;
>  	struct request *rq;
>  	int nr;
>  
>  	rqs = &khd->rqs[khd->cur_domain];
> -	rq = list_first_entry_or_null(rqs, struct request, queuelist);
>  
>  	/*
> -	 * If there wasn't already a pending request and we haven't flushed the
> -	 * software queues yet, flush the software queues and check again.
> +	 * If we do have cur_domain rqs on khd or kcq list, then try to require
> +	 * the token
>  	 */
> -	if (!rq && !*flushed) {
> -		kyber_flush_busy_ctxs(khd, hctx);
> -		*flushed = true;
> -		rq = list_first_entry_or_null(rqs, struct request, queuelist);
> -	}
> -
> +	rq = list_first_entry_or_null(rqs, struct request, queuelist);
>  	if (rq) {
>  		nr = kyber_get_domain_token(kqd, khd, hctx);
>  		if (nr >= 0) {
> @@ -598,8 +705,25 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
>  			list_del_init(&rq->queuelist);
>  			return rq;
>  		}
> +	} else if (sbitmap_any_bit_set(&khd->kcq_map[khd->cur_domain])) {
> +		nr = kyber_get_domain_token(kqd, khd, hctx);
> +		if (nr >= 0) {
> +			kyber_flush_busy_kcqs(khd, khd->cur_domain, rqs);
> +			rq = list_first_entry_or_null(rqs, struct request, queuelist);

This should just be list_first_entry() if we're so sure that we will
always get a request.

> +			/*
> +			 * khd->lock and kcq->lock will ensure that, if kcq_map[cur_domain]
> +			 * is set, we must be able to get requests from the kcq
> +			 */
> +			khd->batching++;
> +			rq_set_domain_token(rq, nr);
> +			list_del_init(&rq->queuelist);
> +			return rq;
> +		}
> +		/*
> +		 * if not get domain token, the rqs could be left on kcqs to merged
> +		 * with following ios.
> +		 */
>  	}
> -
>  	/* There were either no pending requests or no tokens. */
>  	return NULL;
>  }

Clever logic in this function :) I think it needs a more complete
explanation rather than the little pieces sprinkled about, how about
this (applied on top of your patch):

diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
index f60bb4ce1fe6..f4f17527b7be 100644
--- a/block/kyber-iosched.c
+++ b/block/kyber-iosched.c
@@ -696,8 +696,12 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
 	rqs = &khd->rqs[khd->cur_domain];
 
 	/*
-	 * If we do have cur_domain rqs on khd or kcq list, then try to require
-	 * the token
+	 * If we already have a flushed request, then we just need to get a
+	 * token for it. Otherwise, if there are pending requests in the kcqs,
+	 * flush the kcqs, but only if we can get a token. If not, we should
+	 * leave the requests in the kcqs so that they can be merged. Note that
+	 * khd->lock serializes the flushes, so if we observed any bit set in
+	 * the kcq_map, we will always get a request.
 	 */
 	rq = list_first_entry_or_null(rqs, struct request, queuelist);
 	if (rq) {
@@ -712,20 +716,12 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
 		nr = kyber_get_domain_token(kqd, khd, hctx);
 		if (nr >= 0) {
 			kyber_flush_busy_kcqs(khd, khd->cur_domain, rqs);
-			rq = list_first_entry_or_null(rqs, struct request, queuelist);
-			/*
-			 * khd->lock and kcq->lock will ensure that, if kcq_map[cur_domain]
-			 * is set, we must be able to get requests from the kcq
-			 */
+			rq = list_first_entry(rqs, struct request, queuelist);
 			khd->batching++;
 			rq_set_domain_token(rq, nr);
 			list_del_init(&rq->queuelist);
 			return rq;
 		}
-		/*
-		 * if not get domain token, the rqs could be left on kcqs to merged
-		 * with following ios.
-		 */
 	}
 	/* There were either no pending requests or no tokens. */
 	return NULL;

> @@ -608,7 +732,6 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
>  {
>  	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
>  	struct kyber_hctx_data *khd = hctx->sched_data;
> -	bool flushed = false;
>  	struct request *rq;
>  	int i;
>  
> @@ -619,7 +742,7 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
>  	 * from the batch.
>  	 */
>  	if (khd->batching < kyber_batch_size[khd->cur_domain]) {
> -		rq = kyber_dispatch_cur_domain(kqd, khd, hctx, &flushed);
> +		rq = kyber_dispatch_cur_domain(kqd, khd, hctx);
>  		if (rq)
>  			goto out;
>  	}
> @@ -640,7 +763,7 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
>  		else
>  			khd->cur_domain++;
>  
> -		rq = kyber_dispatch_cur_domain(kqd, khd, hctx, &flushed);
> +		rq = kyber_dispatch_cur_domain(kqd, khd, hctx);
>  		if (rq)
>  			goto out;
>  	}
> @@ -657,10 +780,12 @@ static bool kyber_has_work(struct blk_mq_hw_ctx *hctx)
>  	int i;
>  
>  	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
> -		if (!list_empty_careful(&khd->rqs[i]))
> +		if (!list_empty_careful(&khd->rqs[i]) ||
> +				sbitmap_any_bit_set(&khd->kcq_map[i]))

Whitespace.

>  			return true;
>  	}
> -	return sbitmap_any_bit_set(&hctx->ctx_map);
> +
> +	return false;
>  }
>  
>  #define KYBER_LAT_SHOW_STORE(op)					\
> @@ -831,7 +956,9 @@ static struct elevator_type kyber_sched = {
>  		.init_hctx = kyber_init_hctx,
>  		.exit_hctx = kyber_exit_hctx,
>  		.limit_depth = kyber_limit_depth,
> +		.bio_merge = kyber_bio_merge,
>  		.prepare_request = kyber_prepare_request,
> +		.insert_requests = kyber_insert_requests,
>  		.finish_request = kyber_finish_request,
>  		.requeue_request = kyber_finish_request,
>  		.completed_request = kyber_completed_request,
> -- 
> 2.7.4
> 

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

* Re: [PATCH V2 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio merge helper
  2018-05-23  6:33 ` [PATCH V2 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio merge helper Jianchao Wang
@ 2018-05-29 18:55   ` Omar Sandoval
  0 siblings, 0 replies; 6+ messages in thread
From: Omar Sandoval @ 2018-05-29 18:55 UTC (permalink / raw)
  To: Jianchao Wang; +Cc: axboe, holger, linux-block, linux-kernel

On Wed, May 23, 2018 at 02:33:21PM +0800, Jianchao Wang wrote:
> From: Jens Axboe <axboe@kernel.dk>
> 
> No functional changes in this patch, just a prep patch for utilizing
> this in an IO scheduler.

Reviewed-by: Omar Sandoval <osandov@fb.com>

> Signed-off-by: Jens Axboe <axboe@kernel.dk>
> ---
>  block/blk-mq-sched.c   | 34 ++++++++++++++++++++++++----------
>  include/linux/blk-mq.h |  3 ++-
>  2 files changed, 26 insertions(+), 11 deletions(-)
> 
> diff --git a/block/blk-mq-sched.c b/block/blk-mq-sched.c
> index 25c14c5..b0f2c2a 100644
> --- a/block/blk-mq-sched.c
> +++ b/block/blk-mq-sched.c
> @@ -268,19 +268,16 @@ bool blk_mq_sched_try_merge(struct request_queue *q, struct bio *bio,
>  EXPORT_SYMBOL_GPL(blk_mq_sched_try_merge);
>  
>  /*
> - * Reverse check our software queue for entries that we could potentially
> - * merge with. Currently includes a hand-wavy stop count of 8, to not spend
> - * too much time checking for merges.
> + * Iterate list of requests and see if we can merge this bio with any
> + * of them.
>   */
> -static bool blk_mq_attempt_merge(struct request_queue *q,
> -				 struct blk_mq_ctx *ctx, struct bio *bio)
> +bool blk_mq_bio_list_merge(struct request_queue *q, struct list_head *list,
> +			   struct bio *bio)
>  {
>  	struct request *rq;
>  	int checked = 8;
>  
> -	lockdep_assert_held(&ctx->lock);
> -
> -	list_for_each_entry_reverse(rq, &ctx->rq_list, queuelist) {
> +	list_for_each_entry_reverse(rq, list, queuelist) {
>  		bool merged = false;
>  
>  		if (!checked--)
> @@ -305,13 +302,30 @@ static bool blk_mq_attempt_merge(struct request_queue *q,
>  			continue;
>  		}
>  
> -		if (merged)
> -			ctx->rq_merged++;
>  		return merged;
>  	}
>  
>  	return false;
>  }
> +EXPORT_SYMBOL_GPL(blk_mq_bio_list_merge);
> +
> +/*
> + * Reverse check our software queue for entries that we could potentially
> + * merge with. Currently includes a hand-wavy stop count of 8, to not spend
> + * too much time checking for merges.
> + */
> +static bool blk_mq_attempt_merge(struct request_queue *q,
> +				 struct blk_mq_ctx *ctx, struct bio *bio)
> +{
> +	lockdep_assert_held(&ctx->lock);
> +
> +	if (blk_mq_bio_list_merge(q, &ctx->rq_list, bio)) {
> +		ctx->rq_merged++;
> +		return true;
> +	}
> +
> +	return false;
> +}
>  
>  bool __blk_mq_sched_bio_merge(struct request_queue *q, struct bio *bio)
>  {
> diff --git a/include/linux/blk-mq.h b/include/linux/blk-mq.h
> index ebc34a5..fb35517 100644
> --- a/include/linux/blk-mq.h
> +++ b/include/linux/blk-mq.h
> @@ -259,7 +259,8 @@ void blk_mq_add_to_requeue_list(struct request *rq, bool at_head,
>  void blk_mq_kick_requeue_list(struct request_queue *q);
>  void blk_mq_delay_kick_requeue_list(struct request_queue *q, unsigned long msecs);
>  void blk_mq_complete_request(struct request *rq);
> -
> +bool blk_mq_bio_list_merge(struct request_queue *q, struct list_head *list,
> +			   struct bio *bio);
>  bool blk_mq_queue_stopped(struct request_queue *q);
>  void blk_mq_stop_hw_queue(struct blk_mq_hw_ctx *hctx);
>  void blk_mq_start_hw_queue(struct blk_mq_hw_ctx *hctx);
> -- 
> 2.7.4
> 

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

* Re: [PATCH V2 2/2] block: kyber: make kyber more friendly with merging
  2018-05-29 18:55   ` Omar Sandoval
@ 2018-05-30  3:10     ` jianchao.wang
  0 siblings, 0 replies; 6+ messages in thread
From: jianchao.wang @ 2018-05-30  3:10 UTC (permalink / raw)
  To: Omar Sandoval; +Cc: axboe, holger, linux-block, linux-kernel

Hi Omar

Thanks for your kindly and detailed comment.
That's really appreciated. :)

On 05/30/2018 02:55 AM, Omar Sandoval wrote:
> On Wed, May 23, 2018 at 02:33:22PM +0800, Jianchao Wang wrote:
>> Currently, kyber is very unfriendly with merging. kyber depends
>> on ctx rq_list to do merging, however, most of time, it will not
>> leave any requests in ctx rq_list. This is because even if tokens
>> of one domain is used up, kyber will try to dispatch requests
>> from other domain and flush the rq_list there.
>>
>> To improve this, we setup kyber_ctx_queue (kcq) which is similar
>> with ctx, but it has rq_lists for different domain and build same
>> mapping between kcq and khd as the ctx & hctx. Then we could merge,
>> insert and dispatch for different domains separately. At the same
>> time, only flush the rq_list of kcq when get domain token successfully.
>> Then if one domain token is used up, the requests could be left in
>> the rq_list of that domain and maybe merged with following io.
>>
>> Following is my test result on machine with 8 cores and NVMe card
>> INTEL SSDPEKKR128G7
>>
>> fio size=256m ioengine=libaio iodepth=64 direct=1 numjobs=8
>> seq/random
>> +------+---------------------------------------------------------------+
>> |patch?| bw(MB/s) |   iops    | slat(usec) |    clat(usec)   |  merge  |
>> +----------------------------------------------------------------------+
>> | w/o  |  606/612 | 151k/153k |  6.89/7.03 | 3349.21/3305.40 |   0/0   |
>> +----------------------------------------------------------------------+
>> | w/   | 1083/616 | 277k/154k |  4.93/6.95 | 1830.62/3279.95 | 223k/3k |
>> +----------------------------------------------------------------------+
>> When set numjobs to 16, the bw and iops could reach 1662MB/s and 425k
>> on my platform.
>>
>> Signed-off-by: Jianchao Wang <jianchao.w.wang@oracle.com>
>> Tested-by: Holger Hoffstätte <holger@applied-asynchrony.com>
> 
> Thanks, the per-hctx kqcs make much more sense. This is pretty cool! A
> few nitpicks below.
> 
>> ---
>>  block/kyber-iosched.c | 197 +++++++++++++++++++++++++++++++++++++++++---------
>>  1 file changed, 162 insertions(+), 35 deletions(-)
>>
>> diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
>> index 0d6d25e3..7ca1364 100644
>> --- a/block/kyber-iosched.c
>> +++ b/block/kyber-iosched.c
>> @@ -72,6 +72,15 @@ static const unsigned int kyber_batch_size[] = {
>>  	[KYBER_OTHER] = 8,
>>  };
>>  
>> +struct kyber_ctx_queue {
> 
> I'm not crazy about this name, but I can't really think of anything
> better, so this will have to do.

Yes, me too

> 
>> +	/*
>> +	 * Copied from blk_mq_ctx->index_hw
>> +	 */
>> +	unsigned int index;
> 
> We can just calculate this as kcq - khd->kcqs in the one place we use
> it.

except for this, we could also use req->mq_ctx->index_hw which has been
used to get kcq.

> 
>> +	spinlock_t lock;
>> +	struct list_head rq_list[KYBER_NUM_DOMAINS];
>> +} ____cacheline_aligned_in_smp;
>> +
>>  struct kyber_queue_data {
>>  	struct request_queue *q;
>>  
>> @@ -99,6 +108,8 @@ struct kyber_hctx_data {
>>  	struct list_head rqs[KYBER_NUM_DOMAINS];
>>  	unsigned int cur_domain;
>>  	unsigned int batching;
>> +	struct kyber_ctx_queue *kcqs;
>> +	struct sbitmap kcq_map[KYBER_NUM_DOMAINS];
>>  	wait_queue_entry_t domain_wait[KYBER_NUM_DOMAINS];
>>  	struct sbq_wait_state *domain_ws[KYBER_NUM_DOMAINS];
>>  	atomic_t wait_index[KYBER_NUM_DOMAINS];
>> @@ -107,10 +118,8 @@ struct kyber_hctx_data {
>>  static int kyber_domain_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
>>  			     void *key);
>>  
>> -static int rq_sched_domain(const struct request *rq)
>> +static int kyber_sched_domain(unsigned int op)
> 
> Please make this return an unsigned int since that's what we use for
> sched_domain everywhere except the stats bucket function.

Yes, done

> 
>>  {
>> -	unsigned int op = rq->cmd_flags;
>> -
>>  	if ((op & REQ_OP_MASK) == REQ_OP_READ)
>>  		return KYBER_READ;
>>  	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
>> @@ -284,6 +293,11 @@ static unsigned int kyber_sched_tags_shift(struct kyber_queue_data *kqd)
>>  	return kqd->q->queue_hw_ctx[0]->sched_tags->bitmap_tags.sb.shift;
>>  }
>>  
>> +static int kyber_bucket_fn(const struct request *rq)
>> +{
>> +	return kyber_sched_domain(rq->cmd_flags);
>> +}
>> +
>>  static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>>  {
>>  	struct kyber_queue_data *kqd;
>> @@ -297,11 +311,10 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>>  		goto err;
>>  	kqd->q = q;
>>  
>> -	kqd->cb = blk_stat_alloc_callback(kyber_stat_timer_fn, rq_sched_domain,
>> +	kqd->cb = blk_stat_alloc_callback(kyber_stat_timer_fn, kyber_bucket_fn,
>>  					  KYBER_NUM_DOMAINS, kqd);
>>  	if (!kqd->cb)
>>  		goto err_kqd;
>> -
> > Unrelated whitespace change.

Done

> 
>>  	/*
>>  	 * The maximum number of tokens for any scheduling domain is at least
>>  	 * the queue depth of a single hardware queue. If the hardware doesn't
>> @@ -376,15 +389,45 @@ static void kyber_exit_sched(struct elevator_queue *e)
>>  	kfree(kqd);
>>  }
>>  
>> +static void kyber_ctx_queue_init(struct kyber_ctx_queue *kcq)
>> +{
>> +	unsigned int i;
>> +
>> +	spin_lock_init(&kcq->lock);
>> +	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
>> +		INIT_LIST_HEAD(&kcq->rq_list[i]);
>> +}
>> +
>>  static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>>  {
>>  	struct kyber_hctx_data *khd;
>> -	int i;
>> +	int i, sd;
>>  
>>  	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
>>  	if (!khd)
>>  		return -ENOMEM;
>>  
>> +	khd->kcqs = kmalloc_array_node(hctx->nr_ctx,
>> +			sizeof(struct kyber_ctx_queue),
>> +			GFP_KERNEL, hctx->numa_node);
> 
> Argument whitespace alignment is still wrong here. I guess you have to
> pass --strict to checkpatch.pl to get it to warn about that. Usually I
> wouldn't be nitpicky about this but I made sure to keep this file nice
> and neat, and we need a v3 anyways :)

Yes, absolutely.

> 
>> +	if (!khd->kcqs)
>> +		goto err_khd;
>> +	/*
>> +	 * clone the mapping between hctx and ctx to khd and kcq
>> +	 */
>> +	for (i = 0; i < hctx->nr_ctx; i++) {
>> +		kyber_ctx_queue_init(&khd->kcqs[i]);
>> +		khd->kcqs[i].index = i;
>> +	}
>> +
>> +	sd = 0;
>> +	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
>> +		if (sbitmap_init_node(&khd->kcq_map[i], hctx->nr_ctx,
>> +				ilog2(8), GFP_KERNEL, hctx->numa_node))
> 
> Whitespace again.
> 
>> +			goto err_kcq_map;
> 
> In this error case, I'd prefer if we did something similar to
> kyber_queue_data_alloc() instead of having this obscurely-named sd
> variable:
> 
> 			while (--i >= 0)
> 				sbitmap_free(&khd->kcq_map[i]);
> 			goto err_kcqs;
> 
> Where err_kcqs is just the kfree(khd->kcqs).

Yes, done

> 
>> +		sd++;
>> +	}
>> +
>>  	spin_lock_init(&khd->lock);
>>  
>>  	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
>> @@ -402,10 +445,24 @@ static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>>  	hctx->sched_data = khd;
>>  
>>  	return 0;
>> +
>> +err_kcq_map:
>> +	for (i = 0; i < sd; i++)
>> +		sbitmap_free(&khd->kcq_map[i]);
>> +	kfree(khd->kcqs);
>> +err_khd:
>> +	kfree(khd);
>> +	return -ENOMEM;
>>  }
>>  
>>  static void kyber_exit_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>>  {
>> +	struct kyber_hctx_data *khd = hctx->sched_data;
>> +	int i;
>> +
>> +	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
>> +		sbitmap_free(&khd->kcq_map[i]);
>> +	kfree(khd->kcqs);
>>  	kfree(hctx->sched_data);
>>  }
>>  
>> @@ -427,7 +484,7 @@ static void rq_clear_domain_token(struct kyber_queue_data *kqd,
>>  
>>  	nr = rq_get_domain_token(rq);
>>  	if (nr != -1) {
>> -		sched_domain = rq_sched_domain(rq);
>> +		sched_domain = kyber_sched_domain(rq->cmd_flags);
>>  		sbitmap_queue_clear(&kqd->domain_tokens[sched_domain], nr,
>>  				    rq->mq_ctx->cpu);
>>  	}
>> @@ -446,11 +503,51 @@ static void kyber_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
>>  	}
>>  }
>>  
>> +static bool kyber_bio_merge(struct blk_mq_hw_ctx *hctx, struct bio *bio)
>> +{
>> +	struct kyber_hctx_data *khd = hctx->sched_data;
>> +	struct blk_mq_ctx *ctx = blk_mq_get_ctx(hctx->queue);
>> +	struct kyber_ctx_queue *kcq = &khd->kcqs[ctx->index_hw];
>> +	struct list_head *rq_list = &kcq->rq_list[kyber_sched_domain(bio->bi_opf)];
>> +	bool merged;
>> +
>> +	spin_lock(&kcq->lock);
>> +	merged = blk_mq_bio_list_merge(hctx->queue, rq_list, bio);
>> +	spin_unlock(&kcq->lock);
>> +	blk_mq_put_ctx(ctx);
>> +
>> +	return merged;
>> +}
>> +
>>  static void kyber_prepare_request(struct request *rq, struct bio *bio)
>>  {
>>  	rq_set_domain_token(rq, -1);
>>  }
>>  
>> +static void kyber_insert_requests(struct blk_mq_hw_ctx *hctx,
>> +		struct list_head *rq_list, bool at_head)
> 
> Whitespace.
> 
>> +{
>> +	struct kyber_hctx_data *khd = hctx->sched_data;
>> +	struct kyber_ctx_queue *kcq;
>> +	struct request *rq, *next;
>> +	struct list_head *head;
>> +	unsigned int sched_domain;
>> +
>> +	list_for_each_entry_safe(rq, next, rq_list, queuelist) {
> 
> Please declare the variables which are local to the loop iteration
> inside of the loop body. I.e.,
> 
> 		unsigned int sched_domain;
> 		struct kyber_ctx_queue *kcq;
> 		struct list_head *head;
> 

Done

>> +		sched_domain = kyber_sched_domain(rq->cmd_flags);
>> +		kcq = &khd->kcqs[rq->mq_ctx->index_hw];
>> +		head = &kcq->rq_list[sched_domain];
>> +		spin_lock(&kcq->lock);
>> +		if (at_head)
>> +			list_move(&rq->queuelist, head);
>> +		else
>> +			list_move_tail(&rq->queuelist, head);
>> +		sbitmap_set_bit(&khd->kcq_map[sched_domain], kcq->index);
>> +		blk_mq_sched_request_inserted(rq);
>> +		spin_unlock(&kcq->lock);
>> +	}
>> +}
>> +
>>  static void kyber_finish_request(struct request *rq)
>>  {
>>  	struct kyber_queue_data *kqd = rq->q->elevator->elevator_data;
>> @@ -469,7 +566,7 @@ static void kyber_completed_request(struct request *rq)
>>  	 * Check if this request met our latency goal. If not, quickly gather
>>  	 * some statistics and start throttling.
>>  	 */
>> -	sched_domain = rq_sched_domain(rq);
>> +	sched_domain = kyber_sched_domain(rq->cmd_flags);
>>  	switch (sched_domain) {
>>  	case KYBER_READ:
>>  		target = kqd->read_lat_nsec;
>> @@ -495,19 +592,36 @@ static void kyber_completed_request(struct request *rq)
>>  		blk_stat_activate_msecs(kqd->cb, 10);
>>  }
>>  
>> -static void kyber_flush_busy_ctxs(struct kyber_hctx_data *khd,
>> -				  struct blk_mq_hw_ctx *hctx)
>> +struct flush_kcq_data {
>> +	struct kyber_hctx_data *khd;
>> +	unsigned int sched_domain;
>> +	struct list_head *list;
>> +};
>> +
>> +static bool flush_busy_kcq(struct sbitmap *sb, unsigned int bitnr, void *data)
>>  {
>> -	LIST_HEAD(rq_list);
>> -	struct request *rq, *next;
>> +	struct flush_kcq_data *flush_data = data;
>> +	struct kyber_ctx_queue *kcq = &flush_data->khd->kcqs[bitnr];
>>  
>> -	blk_mq_flush_busy_ctxs(hctx, &rq_list);
>> -	list_for_each_entry_safe(rq, next, &rq_list, queuelist) {
>> -		unsigned int sched_domain;
>> +	spin_lock(&kcq->lock);
>> +	list_splice_tail_init(&kcq->rq_list[flush_data->sched_domain],
>> +			flush_data->list);
>> +	sbitmap_clear_bit(sb, bitnr);
>> +	spin_unlock(&kcq->lock);
>>  
>> -		sched_domain = rq_sched_domain(rq);
>> -		list_move_tail(&rq->queuelist, &khd->rqs[sched_domain]);
>> -	}
>> +	return true;
>> +}
>> +
>> +static void kyber_flush_busy_kcqs(struct kyber_hctx_data *khd,
>> +		unsigned int sched_domain, struct list_head *list)
>> +{
>> +	struct flush_kcq_data data = {
>> +		.khd = khd,
>> +		.sched_domain = sched_domain,
>> +		.list = list,
>> +	};
>> +
>> +	sbitmap_for_each_set(&khd->kcq_map[sched_domain], flush_busy_kcq, &data);
>>  }
>>  
>>  static int kyber_domain_wake(wait_queue_entry_t *wait, unsigned mode, int flags,
>> @@ -570,26 +684,19 @@ static int kyber_get_domain_token(struct kyber_queue_data *kqd,
>>  static struct request *
>>  kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
>>  			  struct kyber_hctx_data *khd,
>> -			  struct blk_mq_hw_ctx *hctx,
>> -			  bool *flushed)
>> +			  struct blk_mq_hw_ctx *hctx)
>>  {
>>  	struct list_head *rqs;
>>  	struct request *rq;
>>  	int nr;
>>  
>>  	rqs = &khd->rqs[khd->cur_domain];
>> -	rq = list_first_entry_or_null(rqs, struct request, queuelist);
>>  
>>  	/*
>> -	 * If there wasn't already a pending request and we haven't flushed the
>> -	 * software queues yet, flush the software queues and check again.
>> +	 * If we do have cur_domain rqs on khd or kcq list, then try to require
>> +	 * the token
>>  	 */
>> -	if (!rq && !*flushed) {
>> -		kyber_flush_busy_ctxs(khd, hctx);
>> -		*flushed = true;
>> -		rq = list_first_entry_or_null(rqs, struct request, queuelist);
>> -	}
>> -
>> +	rq = list_first_entry_or_null(rqs, struct request, queuelist);
>>  	if (rq) {
>>  		nr = kyber_get_domain_token(kqd, khd, hctx);
>>  		if (nr >= 0) {
>> @@ -598,8 +705,25 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
>>  			list_del_init(&rq->queuelist);
>>  			return rq;
>>  		}
>> +	} else if (sbitmap_any_bit_set(&khd->kcq_map[khd->cur_domain])) {
>> +		nr = kyber_get_domain_token(kqd, khd, hctx);
>> +		if (nr >= 0) {
>> +			kyber_flush_busy_kcqs(khd, khd->cur_domain, rqs);
>> +			rq = list_first_entry_or_null(rqs, struct request, queuelist);
> 
> This should just be list_first_entry() if we're so sure that we will
> always get a request.
> 
>> +			/*
>> +			 * khd->lock and kcq->lock will ensure that, if kcq_map[cur_domain]
>> +			 * is set, we must be able to get requests from the kcq
>> +			 */
>> +			khd->batching++;
>> +			rq_set_domain_token(rq, nr);
>> +			list_del_init(&rq->queuelist);
>> +			return rq;
>> +		}
>> +		/*
>> +		 * if not get domain token, the rqs could be left on kcqs to merged
>> +		 * with following ios.
>> +		 */
>>  	}
>> -
>>  	/* There were either no pending requests or no tokens. */
>>  	return NULL;
>>  }
> 
> Clever logic in this function :) I think it needs a more complete
> explanation rather than the little pieces sprinkled about, how about
> this (applied on top of your patch):

Yes, this is better, applied following changes. :)

> 
> diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
> index f60bb4ce1fe6..f4f17527b7be 100644
> --- a/block/kyber-iosched.c
> +++ b/block/kyber-iosched.c
> @@ -696,8 +696,12 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
>  	rqs = &khd->rqs[khd->cur_domain];
>  
>  	/*
> -	 * If we do have cur_domain rqs on khd or kcq list, then try to require
> -	 * the token
> +	 * If we already have a flushed request, then we just need to get a
> +	 * token for it. Otherwise, if there are pending requests in the kcqs,
> +	 * flush the kcqs, but only if we can get a token. If not, we should
> +	 * leave the requests in the kcqs so that they can be merged. Note that
> +	 * khd->lock serializes the flushes, so if we observed any bit set in
> +	 * the kcq_map, we will always get a request.
>  	 */
>  	rq = list_first_entry_or_null(rqs, struct request, queuelist);
>  	if (rq) {
> @@ -712,20 +716,12 @@ kyber_dispatch_cur_domain(struct kyber_queue_data *kqd,
>  		nr = kyber_get_domain_token(kqd, khd, hctx);
>  		if (nr >= 0) {
>  			kyber_flush_busy_kcqs(khd, khd->cur_domain, rqs);
> -			rq = list_first_entry_or_null(rqs, struct request, queuelist);
> -			/*
> -			 * khd->lock and kcq->lock will ensure that, if kcq_map[cur_domain]
> -			 * is set, we must be able to get requests from the kcq
> -			 */
> +			rq = list_first_entry(rqs, struct request, queuelist);
>  			khd->batching++;
>  			rq_set_domain_token(rq, nr);
>  			list_del_init(&rq->queuelist);
>  			return rq;
>  		}
> -		/*
> -		 * if not get domain token, the rqs could be left on kcqs to merged
> -		 * with following ios.
> -		 */
>  	}
>  	/* There were either no pending requests or no tokens. */
>  	return NULL;
> >> @@ -608,7 +732,6 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
>>  {
>>  	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
>>  	struct kyber_hctx_data *khd = hctx->sched_data;
>> -	bool flushed = false;
>>  	struct request *rq;
>>  	int i;
>>  
>> @@ -619,7 +742,7 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
>>  	 * from the batch.
>>  	 */
>>  	if (khd->batching < kyber_batch_size[khd->cur_domain]) {
>> -		rq = kyber_dispatch_cur_domain(kqd, khd, hctx, &flushed);
>> +		rq = kyber_dispatch_cur_domain(kqd, khd, hctx);
>>  		if (rq)
>>  			goto out;
>>  	}
>> @@ -640,7 +763,7 @@ static struct request *kyber_dispatch_request(struct blk_mq_hw_ctx *hctx)
>>  		else
>>  			khd->cur_domain++;
>>  
>> -		rq = kyber_dispatch_cur_domain(kqd, khd, hctx, &flushed);
>> +		rq = kyber_dispatch_cur_domain(kqd, khd, hctx);
>>  		if (rq)
>>  			goto out;
>>  	}
>> @@ -657,10 +780,12 @@ static bool kyber_has_work(struct blk_mq_hw_ctx *hctx)
>>  	int i;
>>  
>>  	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
>> -		if (!list_empty_careful(&khd->rqs[i]))
>> +		if (!list_empty_careful(&khd->rqs[i]) ||
>> +				sbitmap_any_bit_set(&khd->kcq_map[i]))
> 
> Whitespace.

Have corrected all of the similar issues and passed the checkpatch.pl with --strict option.


Thanks again for your directive.
Jianchao

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

end of thread, other threads:[~2018-05-30  3:10 UTC | newest]

Thread overview: 6+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-05-23  6:33 [PATCH V2 0/2] block: kyber: make kyber more friendly with merging Jianchao Wang
2018-05-23  6:33 ` [PATCH V2 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio merge helper Jianchao Wang
2018-05-29 18:55   ` Omar Sandoval
2018-05-23  6:33 ` [PATCH V2 2/2] block: kyber: make kyber more friendly with merging Jianchao Wang
2018-05-29 18:55   ` Omar Sandoval
2018-05-30  3:10     ` jianchao.wang

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).