All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH] block: kyber: make kyber more friendly with merging
@ 2018-05-22 14:48 Jianchao Wang
  2018-05-22 16:17 ` Holger Hoffstätte
  2018-05-22 20:02 ` Omar Sandoval
  0 siblings, 2 replies; 15+ messages in thread
From: Jianchao Wang @ 2018-05-22 14:48 UTC (permalink / raw)
  To: axboe; +Cc: 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. 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>
---
 block/kyber-iosched.c | 240 ++++++++++++++++++++++++++++++++++++++++++++------
 1 file changed, 212 insertions(+), 28 deletions(-)

diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
index 0d6d25e3..04da05b 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;
 
@@ -84,6 +93,7 @@ struct kyber_queue_data {
 	 */
 	struct sbitmap_queue domain_tokens[KYBER_NUM_DOMAINS];
 
+	struct kyber_ctx_queue *ctx_queue;
 	/*
 	 * Async request percentage, converted to per-word depth for
 	 * sbitmap_get_shallow().
@@ -99,6 +109,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];
@@ -284,6 +296,19 @@ 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 void kyber_ctx_queue_init(struct kyber_queue_data *kqd)
+{
+	unsigned int i, j;
+
+	for (i = 0; i < nr_cpu_ids; i++) {
+		struct kyber_ctx_queue *kcq = &kqd->ctx_queue[i];
+
+		spin_lock_init(&kcq->lock);
+		for (j = 0; j < KYBER_NUM_DOMAINS; j++)
+			INIT_LIST_HEAD(&kcq->rq_list[j]);
+	}
+}
+
 static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 {
 	struct kyber_queue_data *kqd;
@@ -302,6 +327,13 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 	if (!kqd->cb)
 		goto err_kqd;
 
+	kqd->ctx_queue = kmalloc_array_node(nr_cpu_ids,
+			sizeof(struct kyber_ctx_queue), GFP_KERNEL, -1);
+	if (!kqd->ctx_queue)
+		goto err_cb;
+
+	kyber_ctx_queue_init(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
@@ -318,7 +350,7 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 		if (ret) {
 			while (--i >= 0)
 				sbitmap_queue_free(&kqd->domain_tokens[i]);
-			goto err_cb;
+			goto err_kcq;
 		}
 		sbitmap_queue_resize(&kqd->domain_tokens[i], kyber_depth[i]);
 	}
@@ -331,6 +363,8 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 
 	return kqd;
 
+err_kcq:
+	kfree(kqd->ctx_queue);
 err_cb:
 	blk_stat_free_callback(kqd->cb);
 err_kqd:
@@ -372,6 +406,7 @@ static void kyber_exit_sched(struct elevator_queue *e)
 
 	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
 		sbitmap_queue_free(&kqd->domain_tokens[i]);
+	kfree(kqd->ctx_queue);
 	blk_stat_free_callback(kqd->cb);
 	kfree(kqd);
 }
@@ -379,12 +414,33 @@ static void kyber_exit_sched(struct elevator_queue *e)
 static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
 {
 	struct kyber_hctx_data *khd;
+	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
 	int i;
+	int sd;
 
 	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
 	if (!khd)
 		return -ENOMEM;
 
+	khd->kcqs = kmalloc_array_node(nr_cpu_ids, sizeof(void *),
+					GFP_KERNEL, hctx->numa_node);
+	if (!khd->kcqs)
+		goto err_khd;
+
+	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++;
+	}
+	/*
+	 * clone the mapping between hctx and ctx to khd and kcq
+	 */
+	for (i = 0; i < hctx->nr_ctx; i++) {
+		khd->kcqs[i] = &kqd->ctx_queue[hctx->ctxs[i]->cpu];
+		khd->kcqs[i]->index = i;
+	}
 	spin_lock_init(&khd->lock);
 
 	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
@@ -402,10 +458,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);
 }
 
@@ -446,11 +516,95 @@ static void kyber_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
 	}
 }
 
+static int bio_sched_domain(const struct bio *bio)
+{
+	unsigned int op = bio->bi_opf;
+
+	if ((op & REQ_OP_MASK) == REQ_OP_READ)
+		return KYBER_READ;
+	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
+		return KYBER_SYNC_WRITE;
+	else
+		return KYBER_OTHER;
+}
+
+/*
+ * Do limited merge trying here. Align with blk_mq_attempt_merge, reverse
+ * checking corresponding domain queue for 8 reqs.
+ */
+static bool kyber_bio_merge(struct blk_mq_hw_ctx *hctx, struct bio *bio)
+{
+	struct request_queue *q = hctx->queue;
+	struct kyber_hctx_data *khd = hctx->sched_data;
+	struct blk_mq_ctx *ctx = blk_mq_get_ctx(q);
+	struct kyber_ctx_queue *kcq = khd->kcqs[ctx->index_hw];
+	struct list_head *rq_list = &kcq->rq_list[bio_sched_domain(bio)];
+	struct request *rq;
+	int checked = 8;
+	bool merged = false;
+
+	spin_lock(&kcq->lock);
+	list_for_each_entry_reverse(rq, rq_list, queuelist) {
+
+		if (!checked--)
+			break;
+
+		if (!blk_rq_merge_ok(rq, bio))
+			continue;
+
+		switch (blk_try_merge(rq, bio)) {
+		case ELEVATOR_BACK_MERGE:
+			if (blk_mq_sched_allow_merge(q, rq, bio))
+				merged = bio_attempt_back_merge(q, rq, bio);
+			break;
+		case ELEVATOR_FRONT_MERGE:
+			if (blk_mq_sched_allow_merge(q, rq, bio))
+				merged = bio_attempt_front_merge(q, rq, bio);
+			break;
+		case ELEVATOR_DISCARD_MERGE:
+			merged = bio_attempt_discard_merge(q, rq, bio);
+			break;
+		default:
+			continue;
+		}
+		if (merged)
+			break;
+	}
+	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 = rq_sched_domain(rq);
+		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;
@@ -495,19 +649,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 +741,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 +762,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 +789,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 +799,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 +820,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 +837,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 +1013,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] 15+ messages in thread

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 14:48 [PATCH] block: kyber: make kyber more friendly with merging Jianchao Wang
@ 2018-05-22 16:17 ` Holger Hoffstätte
  2018-05-22 16:20   ` Jens Axboe
  2018-05-22 20:02 ` Omar Sandoval
  1 sibling, 1 reply; 15+ messages in thread
From: Holger Hoffstätte @ 2018-05-22 16:17 UTC (permalink / raw)
  To: Jianchao Wang, axboe; +Cc: linux-block, linux-kernel

On 05/22/18 16:48, 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. 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>

<snip>

This looks great but prevents kyber from being built as module,
which is AFAIK supposed to work (and works now):

..
  CC [M]  block/kyber-iosched.o
  Building modules, stage 2.
  MODPOST 313 modules
ERROR: "bio_attempt_back_merge" [block/kyber-iosched.ko] undefined!
ERROR: "bio_attempt_front_merge" [block/kyber-iosched.ko] undefined!
ERROR: "bio_attempt_discard_merge" [block/kyber-iosched.ko] undefined!
ERROR: "blk_try_merge" [block/kyber-iosched.ko] undefined!
ERROR: "blk_rq_merge_ok" [block/kyber-iosched.ko] undefined!
..

It does build fine when compiled in, obviously. :)

cheers,
Holger

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 16:17 ` Holger Hoffstätte
@ 2018-05-22 16:20   ` Jens Axboe
  2018-05-22 17:46     ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2018-05-22 16:20 UTC (permalink / raw)
  To: Holger Hoffstätte, Jianchao Wang; +Cc: linux-block, linux-kernel

On 5/22/18 10:17 AM, Holger Hoffstätte wrote:
> On 05/22/18 16:48, 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. 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>
> 
> <snip>
> 
> This looks great but prevents kyber from being built as module,
> which is AFAIK supposed to work (and works now):
> 
> ..
>   CC [M]  block/kyber-iosched.o
>   Building modules, stage 2.
>   MODPOST 313 modules
> ERROR: "bio_attempt_back_merge" [block/kyber-iosched.ko] undefined!
> ERROR: "bio_attempt_front_merge" [block/kyber-iosched.ko] undefined!
> ERROR: "bio_attempt_discard_merge" [block/kyber-iosched.ko] undefined!
> ERROR: "blk_try_merge" [block/kyber-iosched.ko] undefined!
> ERROR: "blk_rq_merge_ok" [block/kyber-iosched.ko] undefined!
> ..
> 
> It does build fine when compiled in, obviously. :)

It's basically duplicating the contents of blk_attempt_plug_merge().
I would suggest abstracting out the list loop and merge check
into a helper, that could then both be called from kyber and the
plug merge function.

-- 
Jens Axboe

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 16:20   ` Jens Axboe
@ 2018-05-22 17:46     ` Jens Axboe
  2018-05-22 18:32       ` Holger Hoffstätte
  0 siblings, 1 reply; 15+ messages in thread
From: Jens Axboe @ 2018-05-22 17:46 UTC (permalink / raw)
  To: Holger Hoffstätte, Jianchao Wang; +Cc: linux-block, linux-kernel

[-- Attachment #1: Type: text/plain, Size: 2686 bytes --]

On 5/22/18 10:20 AM, Jens Axboe wrote:
> On 5/22/18 10:17 AM, Holger Hoffstätte wrote:
>> On 05/22/18 16:48, 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. 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>
>>
>> <snip>
>>
>> This looks great but prevents kyber from being built as module,
>> which is AFAIK supposed to work (and works now):
>>
>> ..
>>   CC [M]  block/kyber-iosched.o
>>   Building modules, stage 2.
>>   MODPOST 313 modules
>> ERROR: "bio_attempt_back_merge" [block/kyber-iosched.ko] undefined!
>> ERROR: "bio_attempt_front_merge" [block/kyber-iosched.ko] undefined!
>> ERROR: "bio_attempt_discard_merge" [block/kyber-iosched.ko] undefined!
>> ERROR: "blk_try_merge" [block/kyber-iosched.ko] undefined!
>> ERROR: "blk_rq_merge_ok" [block/kyber-iosched.ko] undefined!
>> ..
>>
>> It does build fine when compiled in, obviously. :)
> 
> It's basically duplicating the contents of blk_attempt_plug_merge().
> I would suggest abstracting out the list loop and merge check
> into a helper, that could then both be called from kyber and the
> plug merge function.

See attached, prep patch and yours rebased on top of it.

-- 
Jens Axboe


[-- Attachment #2: 0002-block-kyber-make-kyber-more-friendly-with-merging.patch --]
[-- Type: text/x-patch, Size: 13143 bytes --]

>From edb57af07cf619f27a3fee85705870875f853f58 Mon Sep 17 00:00:00 2001
From: Jianchao Wang <jianchao.w.wang@oracle.com>
Date: Tue, 22 May 2018 11:45:01 -0600
Subject: [PATCH 2/2] block: kyber: make kyber more friendly with merging

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. 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>
---
 block/kyber-iosched.c | 212 +++++++++++++++++++++++++++++++++++++++++++-------
 1 file changed, 184 insertions(+), 28 deletions(-)

diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
index 5b33dc394cc7..f28147be729b 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;
 
@@ -84,6 +93,7 @@ struct kyber_queue_data {
 	 */
 	struct sbitmap_queue domain_tokens[KYBER_NUM_DOMAINS];
 
+	struct kyber_ctx_queue *ctx_queue;
 	/*
 	 * Async request percentage, converted to per-word depth for
 	 * sbitmap_get_shallow().
@@ -99,6 +109,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];
@@ -284,6 +296,19 @@ 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 void kyber_ctx_queue_init(struct kyber_queue_data *kqd)
+{
+	unsigned int i, j;
+
+	for (i = 0; i < nr_cpu_ids; i++) {
+		struct kyber_ctx_queue *kcq = &kqd->ctx_queue[i];
+
+		spin_lock_init(&kcq->lock);
+		for (j = 0; j < KYBER_NUM_DOMAINS; j++)
+			INIT_LIST_HEAD(&kcq->rq_list[j]);
+	}
+}
+
 static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 {
 	struct kyber_queue_data *kqd;
@@ -302,6 +327,13 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 	if (!kqd->cb)
 		goto err_kqd;
 
+	kqd->ctx_queue = kmalloc_array_node(nr_cpu_ids,
+			sizeof(struct kyber_ctx_queue), GFP_KERNEL, -1);
+	if (!kqd->ctx_queue)
+		goto err_cb;
+
+	kyber_ctx_queue_init(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
@@ -318,7 +350,7 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 		if (ret) {
 			while (--i >= 0)
 				sbitmap_queue_free(&kqd->domain_tokens[i]);
-			goto err_cb;
+			goto err_kcq;
 		}
 		sbitmap_queue_resize(&kqd->domain_tokens[i], kyber_depth[i]);
 	}
@@ -331,6 +363,8 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
 
 	return kqd;
 
+err_kcq:
+	kfree(kqd->ctx_queue);
 err_cb:
 	blk_stat_free_callback(kqd->cb);
 err_kqd:
@@ -372,6 +406,7 @@ static void kyber_exit_sched(struct elevator_queue *e)
 
 	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
 		sbitmap_queue_free(&kqd->domain_tokens[i]);
+	kfree(kqd->ctx_queue);
 	blk_stat_free_callback(kqd->cb);
 	kfree(kqd);
 }
@@ -381,11 +416,31 @@ static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
 	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
 	struct kyber_hctx_data *khd;
 	int i;
+	int sd;
 
 	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
 	if (!khd)
 		return -ENOMEM;
 
+	khd->kcqs = kmalloc_array_node(nr_cpu_ids, sizeof(void *),
+					GFP_KERNEL, hctx->numa_node);
+	if (!khd->kcqs)
+		goto err_khd;
+
+	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++;
+	}
+	/*
+	 * clone the mapping between hctx and ctx to khd and kcq
+	 */
+	for (i = 0; i < hctx->nr_ctx; i++) {
+		khd->kcqs[i] = &kqd->ctx_queue[hctx->ctxs[i]->cpu];
+		khd->kcqs[i]->index = i;
+	}
 	spin_lock_init(&khd->lock);
 
 	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
@@ -405,10 +460,24 @@ static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
 					kqd->async_depth);
 
 	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);
 }
 
@@ -449,11 +518,68 @@ static void kyber_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
 	}
 }
 
+static int bio_sched_domain(const struct bio *bio)
+{
+	unsigned int op = bio->bi_opf;
+
+	if ((op & REQ_OP_MASK) == REQ_OP_READ)
+		return KYBER_READ;
+	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
+		return KYBER_SYNC_WRITE;
+	else
+		return KYBER_OTHER;
+}
+
+/*
+ * Do limited merge trying here. Align with blk_mq_attempt_merge, reverse
+ * checking corresponding domain queue for 8 reqs.
+ */
+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[bio_sched_domain(bio)];
+	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 = rq_sched_domain(rq);
+		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;
@@ -498,19 +624,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,
@@ -573,26 +716,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) {
@@ -601,8 +737,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;
 }
@@ -611,7 +764,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;
 
@@ -622,7 +774,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;
 	}
@@ -643,7 +795,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;
 	}
@@ -660,10 +812,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)					\
@@ -834,7 +988,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


[-- Attachment #3: 0001-blk-mq-abstract-out-blk-mq-sched-rq-list-iteration-b.patch --]
[-- Type: text/x-patch, Size: 3037 bytes --]

>From 82d8018412c12407fbfb134c7be5410ba5a76084 Mon Sep 17 00:00:00 2001
From: Jens Axboe <axboe@kernel.dk>
Date: Tue, 22 May 2018 11:41:46 -0600
Subject: [PATCH 1/2] blk-mq: abstract out blk-mq-sched rq list iteration bio
 merge helper

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 25c14c58385c..b0f2c2a40a0c 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 ebc34a5686dc..fb355173f3c7 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] 15+ messages in thread

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 17:46     ` Jens Axboe
@ 2018-05-22 18:32       ` Holger Hoffstätte
  2018-05-23  1:59         ` jianchao.wang
  0 siblings, 1 reply; 15+ messages in thread
From: Holger Hoffstätte @ 2018-05-22 18:32 UTC (permalink / raw)
  To: Jens Axboe, Jianchao Wang; +Cc: linux-block, linux-kernel

On 05/22/18 19:46, Jens Axboe wrote:
> On 5/22/18 10:20 AM, Jens Axboe wrote:
>> On 5/22/18 10:17 AM, Holger Hoffstätte wrote:
>>> On 05/22/18 16:48, 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. 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>
>>>
>>> <snip>
>>>
>>> This looks great but prevents kyber from being built as module,
>>> which is AFAIK supposed to work (and works now):
>>>
>>> ..
>>>   CC [M]  block/kyber-iosched.o
>>>   Building modules, stage 2.
>>>   MODPOST 313 modules
>>> ERROR: "bio_attempt_back_merge" [block/kyber-iosched.ko] undefined!
>>> ERROR: "bio_attempt_front_merge" [block/kyber-iosched.ko] undefined!
>>> ERROR: "bio_attempt_discard_merge" [block/kyber-iosched.ko] undefined!
>>> ERROR: "blk_try_merge" [block/kyber-iosched.ko] undefined!
>>> ERROR: "blk_rq_merge_ok" [block/kyber-iosched.ko] undefined!
>>> ..
>>>
>>> It does build fine when compiled in, obviously. :)
>>
>> It's basically duplicating the contents of blk_attempt_plug_merge().
>> I would suggest abstracting out the list loop and merge check
>> into a helper, that could then both be called from kyber and the
>> plug merge function.
> 
> See attached, prep patch and yours rebased on top of it.

That was quick. :)

Applies smoothly on top of my 4.16++ tree, now builds correctly as
module and is reproducibly (slightly) faster even on my pedestrian
SATA SSDs, now on par or occasionally even faster than mq-deadline.
What's not to like? So:

Tested-by: Holger Hoffstätte <holger@applied-asynchrony.com>

cheers,
Holger

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 14:48 [PATCH] block: kyber: make kyber more friendly with merging Jianchao Wang
  2018-05-22 16:17 ` Holger Hoffstätte
@ 2018-05-22 20:02 ` Omar Sandoval
  2018-05-23  1:47   ` jianchao.wang
  1 sibling, 1 reply; 15+ messages in thread
From: Omar Sandoval @ 2018-05-22 20:02 UTC (permalink / raw)
  To: Jianchao Wang; +Cc: axboe, linux-block, linux-kernel

On Tue, May 22, 2018 at 10:48:29PM +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.

That's a great catch, I totally missed this.

This approach does end up duplicating a lot of code with the blk-mq core
even after Jens' change, so I'm curious if you tried other approaches.
One idea I had is to try the bio merge against the kqd->rqs lists. Since
that's per-queue, the locking overhead might be too high. Alternatively,
you could keep the software queues as-is but add our own version of
flush_busy_ctxs() that only removes requests of the domain that we want.
If one domain gets backed up, that might get messy with long iterations,
though.

Regarding this approach, a couple of comments below.

> 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. 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>
> ---
>  block/kyber-iosched.c | 240 ++++++++++++++++++++++++++++++++++++++++++++------
>  1 file changed, 212 insertions(+), 28 deletions(-)
> 
> diff --git a/block/kyber-iosched.c b/block/kyber-iosched.c
> index 0d6d25e3..04da05b 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;
>  
> @@ -84,6 +93,7 @@ struct kyber_queue_data {
>  	 */
>  	struct sbitmap_queue domain_tokens[KYBER_NUM_DOMAINS];
>  
> +	struct kyber_ctx_queue *ctx_queue;
>  	/*
>  	 * Async request percentage, converted to per-word depth for
>  	 * sbitmap_get_shallow().
> @@ -99,6 +109,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];
> @@ -284,6 +296,19 @@ 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 void kyber_ctx_queue_init(struct kyber_queue_data *kqd)
> +{
> +	unsigned int i, j;
> +
> +	for (i = 0; i < nr_cpu_ids; i++) {
> +		struct kyber_ctx_queue *kcq = &kqd->ctx_queue[i];
> +
> +		spin_lock_init(&kcq->lock);
> +		for (j = 0; j < KYBER_NUM_DOMAINS; j++)
> +			INIT_LIST_HEAD(&kcq->rq_list[j]);
> +	}
> +}
> +
>  static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>  {
>  	struct kyber_queue_data *kqd;
> @@ -302,6 +327,13 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>  	if (!kqd->cb)
>  		goto err_kqd;
>  
> +	kqd->ctx_queue = kmalloc_array_node(nr_cpu_ids,
> +			sizeof(struct kyber_ctx_queue), GFP_KERNEL, -1);

The whitespace here and in several other places is weird, please run
this through checkpatch.

> +	if (!kqd->ctx_queue)
> +		goto err_cb;
> +
> +	kyber_ctx_queue_init(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
> @@ -318,7 +350,7 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>  		if (ret) {
>  			while (--i >= 0)
>  				sbitmap_queue_free(&kqd->domain_tokens[i]);
> -			goto err_cb;
> +			goto err_kcq;
>  		}
>  		sbitmap_queue_resize(&kqd->domain_tokens[i], kyber_depth[i]);
>  	}
> @@ -331,6 +363,8 @@ static struct kyber_queue_data *kyber_queue_data_alloc(struct request_queue *q)
>  
>  	return kqd;
>  
> +err_kcq:
> +	kfree(kqd->ctx_queue);
>  err_cb:
>  	blk_stat_free_callback(kqd->cb);
>  err_kqd:
> @@ -372,6 +406,7 @@ static void kyber_exit_sched(struct elevator_queue *e)
>  
>  	for (i = 0; i < KYBER_NUM_DOMAINS; i++)
>  		sbitmap_queue_free(&kqd->domain_tokens[i]);
> +	kfree(kqd->ctx_queue);
>  	blk_stat_free_callback(kqd->cb);
>  	kfree(kqd);
>  }
> @@ -379,12 +414,33 @@ static void kyber_exit_sched(struct elevator_queue *e)
>  static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>  {
>  	struct kyber_hctx_data *khd;
> +	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
>  	int i;
> +	int sd;
>  
>  	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
>  	if (!khd)
>  		return -ENOMEM;
>  
> +	khd->kcqs = kmalloc_array_node(nr_cpu_ids, sizeof(void *),
> +					GFP_KERNEL, hctx->numa_node);
> +	if (!khd->kcqs)
> +		goto err_khd;

Why the double indirection of a percpu allocation per hardware queue
here? With, say, 56 cpus and that many hardware queues, that's 3136
pointers, which seems like overkill. Can't you just use the percpu array
in the kqd directly, or make it per-hardware queue instead?

> +	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++;
> +	}
> +	/*
> +	 * clone the mapping between hctx and ctx to khd and kcq
> +	 */
> +	for (i = 0; i < hctx->nr_ctx; i++) {
> +		khd->kcqs[i] = &kqd->ctx_queue[hctx->ctxs[i]->cpu];
> +		khd->kcqs[i]->index = i;
> +	}
>  	spin_lock_init(&khd->lock);
>  
>  	for (i = 0; i < KYBER_NUM_DOMAINS; i++) {
> @@ -402,10 +458,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);
>  }
>  
> @@ -446,11 +516,95 @@ static void kyber_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
>  	}
>  }
>  
> +static int bio_sched_domain(const struct bio *bio)
> +{
> +	unsigned int op = bio->bi_opf;
> +
> +	if ((op & REQ_OP_MASK) == REQ_OP_READ)
> +		return KYBER_READ;
> +	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
> +		return KYBER_SYNC_WRITE;
> +	else
> +		return KYBER_OTHER;
> +}

Please add a common helper for rq_sched_domain() and bio_sched_domain()
instead of duplicating the logic.

> +/*
> + * Do limited merge trying here. Align with blk_mq_attempt_merge, reverse
> + * checking corresponding domain queue for 8 reqs.
> + */
> +static bool kyber_bio_merge(struct blk_mq_hw_ctx *hctx, struct bio *bio)
> +{
> +	struct request_queue *q = hctx->queue;
> +	struct kyber_hctx_data *khd = hctx->sched_data;
> +	struct blk_mq_ctx *ctx = blk_mq_get_ctx(q);
> +	struct kyber_ctx_queue *kcq = khd->kcqs[ctx->index_hw];
> +	struct list_head *rq_list = &kcq->rq_list[bio_sched_domain(bio)];
> +	struct request *rq;
> +	int checked = 8;
> +	bool merged = false;
> +
> +	spin_lock(&kcq->lock);
> +	list_for_each_entry_reverse(rq, rq_list, queuelist) {
> +
> +		if (!checked--)
> +			break;
> +
> +		if (!blk_rq_merge_ok(rq, bio))
> +			continue;
> +
> +		switch (blk_try_merge(rq, bio)) {
> +		case ELEVATOR_BACK_MERGE:
> +			if (blk_mq_sched_allow_merge(q, rq, bio))
> +				merged = bio_attempt_back_merge(q, rq, bio);
> +			break;
> +		case ELEVATOR_FRONT_MERGE:
> +			if (blk_mq_sched_allow_merge(q, rq, bio))
> +				merged = bio_attempt_front_merge(q, rq, bio);
> +			break;
> +		case ELEVATOR_DISCARD_MERGE:
> +			merged = bio_attempt_discard_merge(q, rq, bio);
> +			break;
> +		default:
> +			continue;
> +		}
> +		if (merged)
> +			break;
> +	}
> +	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 = rq_sched_domain(rq);
> +		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;
> @@ -495,19 +649,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 +741,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 +762,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 +789,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 +799,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 +820,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 +837,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]))

More weird whitespace.

>  			return true;
>  	}
> -	return sbitmap_any_bit_set(&hctx->ctx_map);
> +
> +	return false;
>  }
>  
>  #define KYBER_LAT_SHOW_STORE(op)					\
> @@ -831,7 +1013,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	[flat|nested] 15+ messages in thread

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 20:02 ` Omar Sandoval
@ 2018-05-23  1:47   ` jianchao.wang
  2018-05-30  8:22     ` Ming Lei
  0 siblings, 1 reply; 15+ messages in thread
From: jianchao.wang @ 2018-05-23  1:47 UTC (permalink / raw)
  To: Omar Sandoval; +Cc: axboe, linux-block, linux-kernel

Hi Omar

Thanks for your kindly response.

On 05/23/2018 04:02 AM, Omar Sandoval wrote:
> On Tue, May 22, 2018 at 10:48:29PM +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.
> 
> That's a great catch, I totally missed this.
> 
> This approach does end up duplicating a lot of code with the blk-mq core
> even after Jens' change, so I'm curious if you tried other approaches.
> One idea I had is to try the bio merge against the kqd->rqs lists. Since
> that's per-queue, the locking overhead might be too high. Alternatively,

Yes, I used to make a patch as you say, try the bio merge against kqd->rqs directly.
The patch looks even simpler. However, because the khd->lock is needed every time 
when try bio merge, there maybe high contending overhead on hkd->lock when cpu-hctx
mapping is not 1:1. 

> you could keep the software queues as-is but add our own version of
> flush_busy_ctxs() that only removes requests of the domain that we want.
> If one domain gets backed up, that might get messy with long iterations,
> though.

Yes, I also considered this approach :)
But the long iterations on every ctx->rq_list looks really inefficient.

> 
> Regarding this approach, a couple of comments below.
...
>>  }
>> @@ -379,12 +414,33 @@ static void kyber_exit_sched(struct elevator_queue *e)
>>  static int kyber_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int hctx_idx)
>>  {
>>  	struct kyber_hctx_data *khd;
>> +	struct kyber_queue_data *kqd = hctx->queue->elevator->elevator_data;
>>  	int i;
>> +	int sd;
>>  
>>  	khd = kmalloc_node(sizeof(*khd), GFP_KERNEL, hctx->numa_node);
>>  	if (!khd)
>>  		return -ENOMEM;
>>  
>> +	khd->kcqs = kmalloc_array_node(nr_cpu_ids, sizeof(void *),
>> +					GFP_KERNEL, hctx->numa_node);
>> +	if (!khd->kcqs)
>> +		goto err_khd;
> 
> Why the double indirection of a percpu allocation per hardware queue
> here? With, say, 56 cpus and that many hardware queues, that's 3136
> pointers, which seems like overkill. Can't you just use the percpu array
> in the kqd directly, or make it per-hardware queue instead?

oops, I forgot to change the nr_cpu_ids to hctx->nr_ctx.
The mapping between cpu and hctx has been setup when kyber_init_hctx is invoked,
so just need to allocate hctx->nr_ctx * struct kyber_ctx_queue per khd.

...
>> +static int bio_sched_domain(const struct bio *bio)
>> +{
>> +	unsigned int op = bio->bi_opf;
>> +
>> +	if ((op & REQ_OP_MASK) == REQ_OP_READ)
>> +		return KYBER_READ;
>> +	else if ((op & REQ_OP_MASK) == REQ_OP_WRITE && op_is_sync(op))
>> +		return KYBER_SYNC_WRITE;
>> +	else
>> +		return KYBER_OTHER;
>> +}
> 
> Please add a common helper for rq_sched_domain() and bio_sched_domain()
> instead of duplicating the logic.
> 

Yes, I will do it in next version.

Thanks
Jianchao

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-22 18:32       ` Holger Hoffstätte
@ 2018-05-23  1:59         ` jianchao.wang
  0 siblings, 0 replies; 15+ messages in thread
From: jianchao.wang @ 2018-05-23  1:59 UTC (permalink / raw)
  To: Holger Hoffstätte, Jens Axboe; +Cc: linux-block, linux-kernel

Hi Jens and Holger

Thank for your kindly response.
That's really appreciated.

I will post next version based on Jens' patch.

Thanks
Jianchao

On 05/23/2018 02:32 AM, Holger Hoffstätte wrote:
>>>> This looks great but prevents kyber from being built as module,
>>>> which is AFAIK supposed to work (and works now):
>>>>
>>>> ..
>>>>   CC [M]  block/kyber-iosched.o
>>>>   Building modules, stage 2.
>>>>   MODPOST 313 modules
>>>> ERROR: "bio_attempt_back_merge" [block/kyber-iosched.ko] undefined!
>>>> ERROR: "bio_attempt_front_merge" [block/kyber-iosched.ko] undefined!
>>>> ERROR: "bio_attempt_discard_merge" [block/kyber-iosched.ko] undefined!
>>>> ERROR: "blk_try_merge" [block/kyber-iosched.ko] undefined!
>>>> ERROR: "blk_rq_merge_ok" [block/kyber-iosched.ko] undefined!
>>>> ..
>>>>
>>>> It does build fine when compiled in, obviously. :)
>>> It's basically duplicating the contents of blk_attempt_plug_merge().
>>> I would suggest abstracting out the list loop and merge check
>>> into a helper, that could then both be called from kyber and the
>>> plug merge function.
>> See attached, prep patch and yours rebased on top of it.
> That was quick. :)
> 
> Applies smoothly on top of my 4.16++ tree, now builds correctly as
> module and is reproducibly (slightly) faster even on my pedestrian
> SATA SSDs, now on par or occasionally even faster than mq-deadline.
> What's not to like? So:
> 
> Tested-by: Holger Hoffstätte <holger@applied-asynchrony.com>

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-23  1:47   ` jianchao.wang
@ 2018-05-30  8:22     ` Ming Lei
  2018-05-30  8:36       ` jianchao.wang
  0 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2018-05-30  8:22 UTC (permalink / raw)
  To: jianchao.wang
  Cc: Omar Sandoval, Jens Axboe, linux-block, Linux Kernel Mailing List

On Wed, May 23, 2018 at 9:47 AM, jianchao.wang
<jianchao.w.wang@oracle.com> wrote:
> Hi Omar
>
> Thanks for your kindly response.
>
> On 05/23/2018 04:02 AM, Omar Sandoval wrote:
>> On Tue, May 22, 2018 at 10:48:29PM +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.
>>
>> That's a great catch, I totally missed this.
>>
>> This approach does end up duplicating a lot of code with the blk-mq core
>> even after Jens' change, so I'm curious if you tried other approaches.
>> One idea I had is to try the bio merge against the kqd->rqs lists. Since
>> that's per-queue, the locking overhead might be too high. Alternatively,
>
> Yes, I used to make a patch as you say, try the bio merge against kqd->rqs directly.
> The patch looks even simpler. However, because the khd->lock is needed every time
> when try bio merge, there maybe high contending overhead on hkd->lock when cpu-hctx
> mapping is not 1:1.
>
>> you could keep the software queues as-is but add our own version of
>> flush_busy_ctxs() that only removes requests of the domain that we want.
>> If one domain gets backed up, that might get messy with long iterations,
>> though.
>
> Yes, I also considered this approach :)
> But the long iterations on every ctx->rq_list looks really inefficient.

Right, this list can be quite long if dispatch token is used up.

You might try to introduce per-domain list into ctx directly, then 'none'
may benefit from this change too since bio merge should be done
on the per-domain list actually.


Thanks,
Ming Lei

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-30  8:22     ` Ming Lei
@ 2018-05-30  8:36       ` jianchao.wang
  2018-05-30  9:13         ` Ming Lei
  0 siblings, 1 reply; 15+ messages in thread
From: jianchao.wang @ 2018-05-30  8:36 UTC (permalink / raw)
  To: Ming Lei
  Cc: Omar Sandoval, Jens Axboe, linux-block, Linux Kernel Mailing List

Hi ming

Thanks for your kindly response.

On 05/30/2018 04:22 PM, Ming Lei wrote:
>>> you could keep the software queues as-is but add our own version of
>>> flush_busy_ctxs() that only removes requests of the domain that we want.
>>> If one domain gets backed up, that might get messy with long iterations,
>>> though.
>> Yes, I also considered this approach :)
>> But the long iterations on every ctx->rq_list looks really inefficient.
> Right, this list can be quite long if dispatch token is used up.
> 
> You might try to introduce per-domain list into ctx directly, then 'none'
> may benefit from this change too since bio merge should be done
> on the per-domain list actually.

Yes, it maybe good for merging of 'none', because the rq_list is split into 3
lists, and not need to iterate the whole rq_list any more.
But what's about the dispatch when there is no io scheduler.
We will dispatch request from ctx one by one at the moment.
If we have per-domain list in ctx, we have to introduce some policies to determine
which domain to dispatch, and these policies should be in io scheduler actually.

Thanks
Jianchao

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-30  8:36       ` jianchao.wang
@ 2018-05-30  9:13         ` Ming Lei
  2018-05-30  9:20           ` jianchao.wang
  0 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2018-05-30  9:13 UTC (permalink / raw)
  To: jianchao.wang
  Cc: Omar Sandoval, Jens Axboe, linux-block, Linux Kernel Mailing List

On Wed, May 30, 2018 at 4:36 PM, jianchao.wang
<jianchao.w.wang@oracle.com> wrote:
> Hi ming
>
> Thanks for your kindly response.
>
> On 05/30/2018 04:22 PM, Ming Lei wrote:
>>>> you could keep the software queues as-is but add our own version of
>>>> flush_busy_ctxs() that only removes requests of the domain that we want.
>>>> If one domain gets backed up, that might get messy with long iterations,
>>>> though.
>>> Yes, I also considered this approach :)
>>> But the long iterations on every ctx->rq_list looks really inefficient.
>> Right, this list can be quite long if dispatch token is used up.
>>
>> You might try to introduce per-domain list into ctx directly, then 'none'
>> may benefit from this change too since bio merge should be done
>> on the per-domain list actually.
>
> Yes, it maybe good for merging of 'none', because the rq_list is split into 3
> lists, and not need to iterate the whole rq_list any more.
> But what's about the dispatch when there is no io scheduler.

blk_mq_flush_busy_ctxs() and blk_mq_dequeue_from_ctx() should work
fine in case of 'none' if per-domain list is added to ctx. Then we can make
none to be a bit fair on READ/WRITE.

> We will dispatch request from ctx one by one at the moment.
> If we have per-domain list in ctx, we have to introduce some policies to determine
> which domain to dispatch, and these policies should be in io scheduler actually.

The policy is done by IO scheduler, and you can just pick up request
from ctx/domain list easily by introducing one blk-mq core API.

Thanks,
Ming Lei

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-30  9:13         ` Ming Lei
@ 2018-05-30  9:20           ` jianchao.wang
  2018-05-30  9:44             ` Ming Lei
  0 siblings, 1 reply; 15+ messages in thread
From: jianchao.wang @ 2018-05-30  9:20 UTC (permalink / raw)
  To: Ming Lei
  Cc: Omar Sandoval, Jens Axboe, linux-block, Linux Kernel Mailing List

Hi ming

On 05/30/2018 05:13 PM, Ming Lei wrote:
>> Yes, it maybe good for merging of 'none', because the rq_list is split into 3
>> lists, and not need to iterate the whole rq_list any more.
>> But what's about the dispatch when there is no io scheduler.
> blk_mq_flush_busy_ctxs() and blk_mq_dequeue_from_ctx() should work
> fine in case of 'none' if per-domain list is added to ctx. Then we can make
> none to be a bit fair on READ/WRITE.
> 

But how to determine when to dispatch READ, WRITE or other more, when there is no io scheduler ?

Thanks
Jianchao

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-30  9:20           ` jianchao.wang
@ 2018-05-30  9:44             ` Ming Lei
  2018-05-30 14:55               ` jianchao.wang
  0 siblings, 1 reply; 15+ messages in thread
From: Ming Lei @ 2018-05-30  9:44 UTC (permalink / raw)
  To: jianchao.wang
  Cc: Omar Sandoval, Jens Axboe, linux-block, Linux Kernel Mailing List

On Wed, May 30, 2018 at 5:20 PM, jianchao.wang
<jianchao.w.wang@oracle.com> wrote:
> Hi ming
>
> On 05/30/2018 05:13 PM, Ming Lei wrote:
>>> Yes, it maybe good for merging of 'none', because the rq_list is split into 3
>>> lists, and not need to iterate the whole rq_list any more.
>>> But what's about the dispatch when there is no io scheduler.
>> blk_mq_flush_busy_ctxs() and blk_mq_dequeue_from_ctx() should work
>> fine in case of 'none' if per-domain list is added to ctx. Then we can make
>> none to be a bit fair on READ/WRITE.
>>
>
> But how to determine when to dispatch READ, WRITE or other more, when there is no io scheduler ?
>

For blk-mq, no io scheduler means 'none' actually, and it works like a
scheduler too, but just shares driver tags, IMO.

Wrt. the current code of 'none', blk-mq just picks up one request from
ctx->rq_list
directly in FIFO style. If READ/WRITE lists are introduced, only
blk_mq_dequeue_from_ctx() is effected, there are several choices
left for us:

1) keep the FIFO style of current behaviour by using req->start_time_ns

2) READ/WRIRE fair style by picking up request from the lists in round-robin
order

3) or others

It just will make more choices for us, :-)

Thanks,
Ming Lei

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-30  9:44             ` Ming Lei
@ 2018-05-30 14:55               ` jianchao.wang
  2018-05-30 14:58                 ` Jens Axboe
  0 siblings, 1 reply; 15+ messages in thread
From: jianchao.wang @ 2018-05-30 14:55 UTC (permalink / raw)
  To: Ming Lei
  Cc: Omar Sandoval, Jens Axboe, linux-block, Linux Kernel Mailing List

Hi Ming

Thanks for your kindly and detailed response. :)

On 05/30/2018 05:44 PM, Ming Lei wrote:
> On Wed, May 30, 2018 at 5:20 PM, jianchao.wang
> <jianchao.w.wang@oracle.com> wrote:
>> Hi ming
>>
>> On 05/30/2018 05:13 PM, Ming Lei wrote:
>>>> Yes, it maybe good for merging of 'none', because the rq_list is split into 3
>>>> lists, and not need to iterate the whole rq_list any more.
>>>> But what's about the dispatch when there is no io scheduler.
>>> blk_mq_flush_busy_ctxs() and blk_mq_dequeue_from_ctx() should work
>>> fine in case of 'none' if per-domain list is added to ctx. Then we can make
>>> none to be a bit fair on READ/WRITE.
>>>
>>
>> But how to determine when to dispatch READ, WRITE or other more, when there is no io scheduler ?
>>
> 
> For blk-mq, no io scheduler means 'none' actually, and it works like a
> scheduler too, but just shares driver tags, IMO.
> > Wrt. the current code of 'none', blk-mq just picks up one request from
> ctx->rq_list
> directly in FIFO style. If READ/WRITE lists are introduced, only
> blk_mq_dequeue_from_ctx() is effected, there are several choices
> left for us:
> 
> 1) keep the FIFO style of current behaviour by using req->start_time_ns
> 
> 2) READ/WRIRE fair style by picking up request from the lists in round-robin
> order
> 
> 3) or others
> 
> It just will make more choices for us, :-)

OK, I got the point.

But is it necessary to introduce kind of dispatch policy which is more complicated 
than current simple FIFO style in ctx rq_list dispatching ? 
If we have this kind of requirement, why not introduce an io scheduler ?
ITOW, shouldn't we keep the blk-mq core code as simple as possible, and put most of the policy
into io scheduler ?

Thanks
Jianchao

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

* Re: [PATCH] block: kyber: make kyber more friendly with merging
  2018-05-30 14:55               ` jianchao.wang
@ 2018-05-30 14:58                 ` Jens Axboe
  0 siblings, 0 replies; 15+ messages in thread
From: Jens Axboe @ 2018-05-30 14:58 UTC (permalink / raw)
  To: jianchao.wang, Ming Lei
  Cc: Omar Sandoval, linux-block, Linux Kernel Mailing List

On 5/30/18 8:55 AM, jianchao.wang wrote:
> Hi Ming
> 
> Thanks for your kindly and detailed response. :)
> 
> On 05/30/2018 05:44 PM, Ming Lei wrote:
>> On Wed, May 30, 2018 at 5:20 PM, jianchao.wang
>> <jianchao.w.wang@oracle.com> wrote:
>>> Hi ming
>>>
>>> On 05/30/2018 05:13 PM, Ming Lei wrote:
>>>>> Yes, it maybe good for merging of 'none', because the rq_list is split into 3
>>>>> lists, and not need to iterate the whole rq_list any more.
>>>>> But what's about the dispatch when there is no io scheduler.
>>>> blk_mq_flush_busy_ctxs() and blk_mq_dequeue_from_ctx() should work
>>>> fine in case of 'none' if per-domain list is added to ctx. Then we can make
>>>> none to be a bit fair on READ/WRITE.
>>>>
>>>
>>> But how to determine when to dispatch READ, WRITE or other more, when there is no io scheduler ?
>>>
>>
>> For blk-mq, no io scheduler means 'none' actually, and it works like a
>> scheduler too, but just shares driver tags, IMO.
>>> Wrt. the current code of 'none', blk-mq just picks up one request from
>> ctx->rq_list
>> directly in FIFO style. If READ/WRITE lists are introduced, only
>> blk_mq_dequeue_from_ctx() is effected, there are several choices
>> left for us:
>>
>> 1) keep the FIFO style of current behaviour by using req->start_time_ns
>>
>> 2) READ/WRIRE fair style by picking up request from the lists in round-robin
>> order
>>
>> 3) or others
>>
>> It just will make more choices for us, :-)
> 
> OK, I got the point.
> 
> But is it necessary to introduce kind of dispatch policy which is more complicated 
> than current simple FIFO style in ctx rq_list dispatching ? 
> If we have this kind of requirement, why not introduce an io scheduler ?
> ITOW, shouldn't we keep the blk-mq core code as simple as possible, and put most of the policy
> into io scheduler ?

That is indeed the point, we're not going to introducing further logic
or merging to 'none'. With that comes various other heuristics, like
being discussed here, and that takes it even further away from the
light weight and non-intrusive nature of it.

-- 
Jens Axboe

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

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

Thread overview: 15+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-05-22 14:48 [PATCH] block: kyber: make kyber more friendly with merging Jianchao Wang
2018-05-22 16:17 ` Holger Hoffstätte
2018-05-22 16:20   ` Jens Axboe
2018-05-22 17:46     ` Jens Axboe
2018-05-22 18:32       ` Holger Hoffstätte
2018-05-23  1:59         ` jianchao.wang
2018-05-22 20:02 ` Omar Sandoval
2018-05-23  1:47   ` jianchao.wang
2018-05-30  8:22     ` Ming Lei
2018-05-30  8:36       ` jianchao.wang
2018-05-30  9:13         ` Ming Lei
2018-05-30  9:20           ` jianchao.wang
2018-05-30  9:44             ` Ming Lei
2018-05-30 14:55               ` jianchao.wang
2018-05-30 14:58                 ` 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.