All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup
@ 2021-10-06 17:31 Jan Kara
  2021-10-06 17:31 ` [PATCH 1/8] block: Provide icq in request allocation data Jan Kara
                   ` (8 more replies)
  0 siblings, 9 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Hello!

Here is the third revision of my patches to fix how bfq weights apply on cgroup
throughput and on throughput of processes with different IO priorities. Since
v2 I've added some more patches so that now IO priorities also result in
service differentiation (previously they had no effect on service
differentiation on some workloads similarly to cgroup weights). The last patch
in the series still needs some work as in the current state it causes a
notable regression (~20-30%) with dbench benchmark for large numbers of
clients. I've verified that the last patch is indeed necessary for the service
differentiation with the workload described in its changelog. As we discussed
with Paolo, I have also found out that if I remove the "waker has enough
budget" condition from bfq_select_queue(), dbench performance is restored
and the service differentiation is still good. But we probably need some
justification or cleaner solution than just removing the condition so that
is still up to discussion. But first seven patches already noticeably improve
the situation for lots of workloads so IMO they stand on their own and
can be merged regardless of how we go about the last patch.

Changes since v2:
* Rebased on top of current Linus' tree
* Updated computation of scheduler tag proportions to work correctly even
  for processes within the same cgroup but with different IO priorities
* Added comment roughly explaining why we limit tag depth
* Added patch limiting waker / wakee detection in time so avoid at least the
  most obvious false positives
* Added patch to log waker / wakee detections in blktrace for better debugging
* Added patch properly account injected IO

Changes since v1:
* Fixed computation of appropriate proportion of scheduler tags for a cgroup
  to work with deeper cgroup hierarchies.

Original cover letter:

I was looking into why cgroup weights do not have any measurable impact on
writeback throughput from different cgroups. This actually a regression from
CFQ where things work more or less OK and weights have roughly the impact they
should. The problem can be reproduced e.g. by running the following easy fio
job in two cgroups with different weight:

[writer]
directory=/mnt/repro/
numjobs=1
rw=write
size=8g
time_based
runtime=30
ramp_time=10
blocksize=1m
direct=0
ioengine=sync

I can observe there's no significat difference in the amount of data written
from different cgroups despite their weights are in say 1:3 ratio.

After some debugging I've understood the dynamics of the system. There are two
issues:

1) The amount of scheduler tags needs to be significantly larger than the
amount of device tags. Otherwise there are not enough requests waiting in BFQ
to be dispatched to the device and thus there's nothing to schedule on.

2) Even with enough scheduler tags, writers from two cgroups eventually start
contending on scheduler tag allocation. These are served on first come first
served basis so writers from both cgroups feed requests into bfq with
approximately the same speed. Since bfq prefers IO from heavier cgroup, that is
submitted and completed faster and eventually we end up in a situation when
there's no IO from the heavier cgroup in bfq and all scheduler tags are
consumed by requests from the lighter cgroup. At that point bfq just dispatches
lots of the IO from the lighter cgroup since there's no contender for disk
throughput. As a result observed throughput for both cgroups are the same.

This series fixes this problem by accounting how many scheduler tags are
allocated for each cgroup and if a cgroup has more tags allocated than its
fair share (based on weights) in its service tree, we heavily limit scheduler
tag bitmap depth for it so that it is not be able to starve other cgroups from
scheduler tags.

What do people think about this?

								Honza

Previous versions:
Link: http://lore.kernel.org/r/20210712171146.12231-1-jack@suse.cz # v1
Link: http://lore.kernel.org/r/20210715132047.20874-1-jack@suse.cz # v2

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

* [PATCH 1/8] block: Provide icq in request allocation data
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-06 17:31 ` [PATCH 2/8] bfq: Track number of allocated requests in bfq_entity Jan Kara
                   ` (7 subsequent siblings)
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Currently we lookup ICQ only after the request is allocated. However BFQ
will want to decide how many scheduler tags it allows a given bfq queue
(effectively a process) to consume based on cgroup weight. So lookup ICQ
earlier and provide it in struct blk_mq_alloc_data so that BFQ can use
it.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/blk-mq-sched.c | 18 ++++++++++--------
 block/blk-mq-sched.h |  3 ++-
 block/blk-mq.c       |  7 ++++---
 block/blk-mq.h       |  1 +
 4 files changed, 17 insertions(+), 12 deletions(-)

diff --git a/block/blk-mq-sched.c b/block/blk-mq-sched.c
index 0f006cabfd91..bbb6a677fdde 100644
--- a/block/blk-mq-sched.c
+++ b/block/blk-mq-sched.c
@@ -18,9 +18,8 @@
 #include "blk-mq-tag.h"
 #include "blk-wbt.h"
 
-void blk_mq_sched_assign_ioc(struct request *rq)
+struct io_cq *blk_mq_sched_lookup_icq(struct request_queue *q)
 {
-	struct request_queue *q = rq->q;
 	struct io_context *ioc;
 	struct io_cq *icq;
 
@@ -29,17 +28,20 @@ void blk_mq_sched_assign_ioc(struct request *rq)
 	 */
 	ioc = current->io_context;
 	if (!ioc)
-		return;
+		return NULL;
 
 	spin_lock_irq(&q->queue_lock);
 	icq = ioc_lookup_icq(ioc, q);
 	spin_unlock_irq(&q->queue_lock);
+	if (icq)
+		return icq;
+	return ioc_create_icq(ioc, q, GFP_ATOMIC);
+}
 
-	if (!icq) {
-		icq = ioc_create_icq(ioc, q, GFP_ATOMIC);
-		if (!icq)
-			return;
-	}
+void blk_mq_sched_assign_ioc(struct request *rq, struct io_cq *icq)
+{
+	if (!icq)
+		return;
 	get_io_context(icq->ioc);
 	rq->elv.icq = icq;
 }
diff --git a/block/blk-mq-sched.h b/block/blk-mq-sched.h
index 5246ae040704..4529991e55e6 100644
--- a/block/blk-mq-sched.h
+++ b/block/blk-mq-sched.h
@@ -7,7 +7,8 @@
 
 #define MAX_SCHED_RQ (16 * BLKDEV_MAX_RQ)
 
-void blk_mq_sched_assign_ioc(struct request *rq);
+struct io_cq *blk_mq_sched_lookup_icq(struct request_queue *q);
+void blk_mq_sched_assign_ioc(struct request *rq, struct io_cq *icq);
 
 bool blk_mq_sched_try_merge(struct request_queue *q, struct bio *bio,
 		unsigned int nr_segs, struct request **merged_request);
diff --git a/block/blk-mq.c b/block/blk-mq.c
index 108a352051be..bf7dfd36d327 100644
--- a/block/blk-mq.c
+++ b/block/blk-mq.c
@@ -333,9 +333,7 @@ static struct request *blk_mq_rq_ctx_init(struct blk_mq_alloc_data *data,
 
 		rq->elv.icq = NULL;
 		if (e && e->type->ops.prepare_request) {
-			if (e->type->icq_cache)
-				blk_mq_sched_assign_ioc(rq);
-
+			blk_mq_sched_assign_ioc(rq, data->icq);
 			e->type->ops.prepare_request(rq);
 			rq->rq_flags |= RQF_ELVPRIV;
 		}
@@ -360,6 +358,9 @@ static struct request *__blk_mq_alloc_request(struct blk_mq_alloc_data *data)
 		data->flags |= BLK_MQ_REQ_NOWAIT;
 
 	if (e) {
+		if (!op_is_flush(data->cmd_flags) && e->type->icq_cache &&
+		    e->type->ops.prepare_request)
+			data->icq = blk_mq_sched_lookup_icq(q);
 		/*
 		 * Flush/passthrough requests are special and go directly to the
 		 * dispatch list. Don't include reserved tags in the
diff --git a/block/blk-mq.h b/block/blk-mq.h
index d08779f77a26..c502232384c6 100644
--- a/block/blk-mq.h
+++ b/block/blk-mq.h
@@ -151,6 +151,7 @@ static inline struct blk_mq_ctx *blk_mq_get_ctx(struct request_queue *q)
 struct blk_mq_alloc_data {
 	/* input parameter */
 	struct request_queue *q;
+	struct io_cq *icq;
 	blk_mq_req_flags_t flags;
 	unsigned int shallow_depth;
 	unsigned int cmd_flags;
-- 
2.26.2


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

* [PATCH 2/8] bfq: Track number of allocated requests in bfq_entity
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
  2021-10-06 17:31 ` [PATCH 1/8] block: Provide icq in request allocation data Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-06 17:31 ` [PATCH 3/8] bfq: Store full bitmap depth in bfq_data Jan Kara
                   ` (6 subsequent siblings)
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

When we want to limit number of requests used by each bfqq and also
cgroup, we need to track also number of requests used by each cgroup.
So track number of allocated requests for each bfq_entity.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.c | 28 ++++++++++++++++++++++------
 block/bfq-iosched.h |  5 +++--
 2 files changed, 25 insertions(+), 8 deletions(-)

diff --git a/block/bfq-iosched.c b/block/bfq-iosched.c
index 480e1a134859..4d9e04edb614 100644
--- a/block/bfq-iosched.c
+++ b/block/bfq-iosched.c
@@ -1113,7 +1113,8 @@ bfq_bfqq_resume_state(struct bfq_queue *bfqq, struct bfq_data *bfqd,
 
 static int bfqq_process_refs(struct bfq_queue *bfqq)
 {
-	return bfqq->ref - bfqq->allocated - bfqq->entity.on_st_or_in_serv -
+	return bfqq->ref - bfqq->entity.allocated -
+		bfqq->entity.on_st_or_in_serv -
 		(bfqq->weight_counter != NULL) - bfqq->stable_ref;
 }
 
@@ -5878,6 +5879,22 @@ static void bfq_rq_enqueued(struct bfq_data *bfqd, struct bfq_queue *bfqq,
 	}
 }
 
+static void bfqq_request_allocated(struct bfq_queue *bfqq)
+{
+	struct bfq_entity *entity = &bfqq->entity;
+
+	for_each_entity(entity)
+		entity->allocated++;
+}
+
+static void bfqq_request_freed(struct bfq_queue *bfqq)
+{
+	struct bfq_entity *entity = &bfqq->entity;
+
+	for_each_entity(entity)
+		entity->allocated--;
+}
+
 /* returns true if it causes the idle timer to be disabled */
 static bool __bfq_insert_request(struct bfq_data *bfqd, struct request *rq)
 {
@@ -5891,8 +5908,8 @@ static bool __bfq_insert_request(struct bfq_data *bfqd, struct request *rq)
 		 * Release the request's reference to the old bfqq
 		 * and make sure one is taken to the shared queue.
 		 */
-		new_bfqq->allocated++;
-		bfqq->allocated--;
+		bfqq_request_allocated(new_bfqq);
+		bfqq_request_freed(bfqq);
 		new_bfqq->ref++;
 		/*
 		 * If the bic associated with the process
@@ -6251,8 +6268,7 @@ static void bfq_completed_request(struct bfq_queue *bfqq, struct bfq_data *bfqd)
 
 static void bfq_finish_requeue_request_body(struct bfq_queue *bfqq)
 {
-	bfqq->allocated--;
-
+	bfqq_request_freed(bfqq);
 	bfq_put_queue(bfqq);
 }
 
@@ -6672,7 +6688,7 @@ static struct bfq_queue *bfq_init_rq(struct request *rq)
 		}
 	}
 
-	bfqq->allocated++;
+	bfqq_request_allocated(bfqq);
 	bfqq->ref++;
 	bfq_log_bfqq(bfqd, bfqq, "get_request %p: bfqq %p, %d",
 		     rq, bfqq, bfqq->ref);
diff --git a/block/bfq-iosched.h b/block/bfq-iosched.h
index a73488eec8a4..3787cfb0febb 100644
--- a/block/bfq-iosched.h
+++ b/block/bfq-iosched.h
@@ -170,6 +170,9 @@ struct bfq_entity {
 	/* budget, used also to calculate F_i: F_i = S_i + @budget / @weight */
 	int budget;
 
+	/* Number of requests allocated in the subtree of this entity */
+	int allocated;
+
 	/* device weight, if non-zero, it overrides the default weight of
 	 * bfq_group_data */
 	int dev_weight;
@@ -266,8 +269,6 @@ struct bfq_queue {
 	struct request *next_rq;
 	/* number of sync and async requests queued */
 	int queued[2];
-	/* number of requests currently allocated */
-	int allocated;
 	/* number of pending metadata requests */
 	int meta_pending;
 	/* fifo list of requests in sort_list */
-- 
2.26.2


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

* [PATCH 3/8] bfq: Store full bitmap depth in bfq_data
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
  2021-10-06 17:31 ` [PATCH 1/8] block: Provide icq in request allocation data Jan Kara
  2021-10-06 17:31 ` [PATCH 2/8] bfq: Track number of allocated requests in bfq_entity Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-06 17:31 ` [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup Jan Kara
                   ` (5 subsequent siblings)
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Store bitmap depth shift inside bfq_data so that we can use it in
bfq_limit_depth() for proportioning when limiting number of available
request tags for a cgroup.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.c | 10 ++++++----
 block/bfq-iosched.h |  1 +
 2 files changed, 7 insertions(+), 4 deletions(-)

diff --git a/block/bfq-iosched.c b/block/bfq-iosched.c
index 4d9e04edb614..c93a74b8e9c8 100644
--- a/block/bfq-iosched.c
+++ b/block/bfq-iosched.c
@@ -6855,7 +6855,9 @@ static unsigned int bfq_update_depths(struct bfq_data *bfqd,
 				      struct sbitmap_queue *bt)
 {
 	unsigned int i, j, min_shallow = UINT_MAX;
+	unsigned int depth = 1U << bt->sb.shift;
 
+	bfqd->full_depth_shift = bt->sb.shift;
 	/*
 	 * In-word depths if no bfq_queue is being weight-raised:
 	 * leaving 25% of tags only for sync reads.
@@ -6867,13 +6869,13 @@ static unsigned int bfq_update_depths(struct bfq_data *bfqd,
 	 * limit 'something'.
 	 */
 	/* no more than 50% of tags for async I/O */
-	bfqd->word_depths[0][0] = max((1U << bt->sb.shift) >> 1, 1U);
+	bfqd->word_depths[0][0] = max(depth >> 1, 1U);
 	/*
 	 * no more than 75% of tags for sync writes (25% extra tags
 	 * w.r.t. async I/O, to prevent async I/O from starving sync
 	 * writes)
 	 */
-	bfqd->word_depths[0][1] = max(((1U << bt->sb.shift) * 3) >> 2, 1U);
+	bfqd->word_depths[0][1] = max((depth * 3) >> 2, 1U);
 
 	/*
 	 * In-word depths in case some bfq_queue is being weight-
@@ -6883,9 +6885,9 @@ static unsigned int bfq_update_depths(struct bfq_data *bfqd,
 	 * shortage.
 	 */
 	/* no more than ~18% of tags for async I/O */
-	bfqd->word_depths[1][0] = max(((1U << bt->sb.shift) * 3) >> 4, 1U);
+	bfqd->word_depths[1][0] = max((depth * 3) >> 4, 1U);
 	/* no more than ~37% of tags for sync writes (~20% extra tags) */
-	bfqd->word_depths[1][1] = max(((1U << bt->sb.shift) * 6) >> 4, 1U);
+	bfqd->word_depths[1][1] = max((depth * 6) >> 4, 1U);
 
 	for (i = 0; i < 2; i++)
 		for (j = 0; j < 2; j++)
diff --git a/block/bfq-iosched.h b/block/bfq-iosched.h
index 3787cfb0febb..820cb8c2d1fe 100644
--- a/block/bfq-iosched.h
+++ b/block/bfq-iosched.h
@@ -769,6 +769,7 @@ struct bfq_data {
 	 * function)
 	 */
 	unsigned int word_depths[2][2];
+	unsigned int full_depth_shift;
 };
 
 enum bfqq_state_flags {
-- 
2.26.2


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

* [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
                   ` (2 preceding siblings ...)
  2021-10-06 17:31 ` [PATCH 3/8] bfq: Store full bitmap depth in bfq_data Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-11-02 18:16   ` Michal Koutný
  2021-10-06 17:31 ` [PATCH 5/8] bfq: Limit waker detection in time Jan Kara
                   ` (4 subsequent siblings)
  8 siblings, 1 reply; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

When cgroup IO scheduling is used with BFQ it does not really provide
service differentiation if the cgroup drives a big IO depth. That for
example happens with writeback which asynchronously submits lots of IO
but it can happen with AIO as well. The problem is that if we have two
cgroups that submit IO with different weights, the cgroup with higher
weight properly gets more IO time and is able to dispatch more IO.
However this causes lower weight cgroup to accumulate more requests
inside BFQ and eventually lower weight cgroup consumes most of IO
scheduler tags. At that point higher weight cgroup stops getting better
service as it is mostly blocked waiting for a scheduler tag while its
queues inside BFQ are empty and thus lower weight cgroup gets served.

Check how many requests submitting cgroup has allocated in
bfq_limit_depth() and if it consumes more requests than what would
correspond to its weight limit available depth to 1 so that the cgroup
cannot consume many more requests. With this limitation the higher
weight cgroup gets proper service even with writeback.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.c | 137 ++++++++++++++++++++++++++++++++++++++------
 1 file changed, 118 insertions(+), 19 deletions(-)

diff --git a/block/bfq-iosched.c b/block/bfq-iosched.c
index c93a74b8e9c8..3806409610ca 100644
--- a/block/bfq-iosched.c
+++ b/block/bfq-iosched.c
@@ -565,26 +565,134 @@ static struct request *bfq_choose_req(struct bfq_data *bfqd,
 	}
 }
 
+#define BFQ_LIMIT_INLINE_DEPTH 16
+
+#ifdef CONFIG_BFQ_GROUP_IOSCHED
+static bool bfqq_request_over_limit(struct bfq_queue *bfqq, int limit)
+{
+	struct bfq_data *bfqd = bfqq->bfqd;
+	struct bfq_entity *entity = &bfqq->entity;
+	struct bfq_entity *inline_entities[BFQ_LIMIT_INLINE_DEPTH];
+	struct bfq_entity **entities = inline_entities;
+	int depth, level;
+	int class_idx = bfqq->ioprio_class - 1;
+	struct bfq_sched_data *sched_data;
+	unsigned long wsum;
+	bool ret = false;
+
+	if (!entity->on_st_or_in_serv)
+		return false;
+
+	/* +1 for bfqq entity, root cgroup not included */
+	depth = bfqg_to_blkg(bfqq_group(bfqq))->blkcg->css.cgroup->level + 1;
+	if (depth > BFQ_LIMIT_INLINE_DEPTH) {
+		entities = kmalloc_array(depth, sizeof(*entities), GFP_NOIO);
+		if (!entities)
+			return false;
+	}
+
+	spin_lock_irq(&bfqd->lock);
+	sched_data = entity->sched_data;
+	/* Gather our ancestors as we need to traverse them in reverse order */
+	level = 0;
+	for_each_entity(entity) {
+		/*
+		 * If at some level entity is not even active, allow request
+ 		 * queueing so that BFQ knows there's work to do and activate
+		 * entities.
+		 */
+		if (!entity->on_st_or_in_serv)
+			goto out;
+		/* Uh, more parents than cgroup subsystem thinks? */
+		if (WARN_ON_ONCE(level >= depth))
+			break;
+		entities[level++] = entity;
+	}
+	WARN_ON_ONCE(level != depth);
+	for (level--; level >= 0; level--) {
+		entity = entities[level];
+		if (level > 0) {
+			wsum = bfq_entity_service_tree(entity)->wsum;
+		} else {
+			int i;
+			/*
+			 * For bfqq itself we take into account service trees
+			 * of all higher priority classes and multiply their
+			 * weights so that low prio queue from higher class
+			 * gets more requests than high prio queue from lower
+			 * class.
+			 */
+			wsum = 0;
+			for (i = 0; i <= class_idx; i++) {
+				wsum = wsum * IOPRIO_BE_NR +
+					sched_data->service_tree[i].wsum;
+			}
+		}
+		limit = DIV_ROUND_CLOSEST(limit * entity->weight, wsum);
+		if (entity->allocated >= limit) {
+			bfq_log_bfqq(bfqq->bfqd, bfqq,
+				"too many requests: allocated %d limit %d level %d",
+				entity->allocated, limit, level);
+			ret = true;
+			break;
+		}
+	}
+out:
+	spin_unlock_irq(&bfqd->lock);
+	if (entities != inline_entities)
+		kfree(entities);
+	return ret;
+}
+#else
+static bool bfqq_request_over_limit(struct bfq_queue *bfqq, int limit)
+{
+	return false;
+}
+#endif
+
 /*
  * Async I/O can easily starve sync I/O (both sync reads and sync
  * writes), by consuming all tags. Similarly, storms of sync writes,
  * such as those that sync(2) may trigger, can starve sync reads.
  * Limit depths of async I/O and sync writes so as to counter both
  * problems.
+ *
+ * Also if a bfq queue or its parent cgroup consume more tags than would be
+ * appropriate for their weight, we trim the available tag depth to 1. This
+ * avoids a situation where one cgroup can starve another cgroup from tags and
+ * thus block service differentiation among cgroups. Note that because the
+ * queue / cgroup already has many requests allocated and queued, this does not
+ * significantly affect service guarantees coming from the BFQ scheduling
+ * algorithm.
  */
 static void bfq_limit_depth(unsigned int op, struct blk_mq_alloc_data *data)
 {
 	struct bfq_data *bfqd = data->q->elevator->elevator_data;
+	struct bfq_io_cq *bic = data->icq ? icq_to_bic(data->icq) : NULL;
+	struct bfq_queue *bfqq = bic ? bic_to_bfqq(bic, op_is_sync(op)) : NULL;
+	int depth;
+	unsigned limit = data->q->nr_requests;
+
+	/* Sync reads have full depth available */
+	if (op_is_sync(op) && !op_is_write(op)) {
+		depth = 0;
+	} else {
+		depth = bfqd->word_depths[!!bfqd->wr_busy_queues][op_is_sync(op)];
+		limit = (limit * depth) >> bfqd->full_depth_shift;
+	}
 
-	if (op_is_sync(op) && !op_is_write(op))
-		return;
-
-	data->shallow_depth =
-		bfqd->word_depths[!!bfqd->wr_busy_queues][op_is_sync(op)];
+	/*
+	 * Does queue (or any parent entity) exceed number of requests that
+	 * should be available to it? Heavily limit depth so that it cannot
+	 * consume more available requests and thus starve other entities.
+	 */
+	if (bfqq && bfqq_request_over_limit(bfqq, limit))
+		depth = 1;
 
 	bfq_log(bfqd, "[%s] wr_busy %d sync %d depth %u",
-			__func__, bfqd->wr_busy_queues, op_is_sync(op),
-			data->shallow_depth);
+		__func__, bfqd->wr_busy_queues, op_is_sync(op), depth);
+	if (depth)
+		data->shallow_depth = depth;
 }
 
 static struct bfq_queue *
@@ -6851,10 +6959,8 @@ void bfq_put_async_queues(struct bfq_data *bfqd, struct bfq_group *bfqg)
  * See the comments on bfq_limit_depth for the purpose of
  * the depths set in the function. Return minimum shallow depth we'll use.
  */
-static unsigned int bfq_update_depths(struct bfq_data *bfqd,
-				      struct sbitmap_queue *bt)
+static void bfq_update_depths(struct bfq_data *bfqd, struct sbitmap_queue *bt)
 {
-	unsigned int i, j, min_shallow = UINT_MAX;
 	unsigned int depth = 1U << bt->sb.shift;
 
 	bfqd->full_depth_shift = bt->sb.shift;
@@ -6888,22 +6994,15 @@ static unsigned int bfq_update_depths(struct bfq_data *bfqd,
 	bfqd->word_depths[1][0] = max((depth * 3) >> 4, 1U);
 	/* no more than ~37% of tags for sync writes (~20% extra tags) */
 	bfqd->word_depths[1][1] = max((depth * 6) >> 4, 1U);
-
-	for (i = 0; i < 2; i++)
-		for (j = 0; j < 2; j++)
-			min_shallow = min(min_shallow, bfqd->word_depths[i][j]);
-
-	return min_shallow;
 }
 
 static void bfq_depth_updated(struct blk_mq_hw_ctx *hctx)
 {
 	struct bfq_data *bfqd = hctx->queue->elevator->elevator_data;
 	struct blk_mq_tags *tags = hctx->sched_tags;
-	unsigned int min_shallow;
 
-	min_shallow = bfq_update_depths(bfqd, tags->bitmap_tags);
-	sbitmap_queue_min_shallow_depth(tags->bitmap_tags, min_shallow);
+	bfq_update_depths(bfqd, tags->bitmap_tags);
+	sbitmap_queue_min_shallow_depth(tags->bitmap_tags, 1);
 }
 
 static int bfq_init_hctx(struct blk_mq_hw_ctx *hctx, unsigned int index)
-- 
2.26.2


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

* [PATCH 5/8] bfq: Limit waker detection in time
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
                   ` (3 preceding siblings ...)
  2021-10-06 17:31 ` [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-06 17:31 ` [PATCH 6/8] bfq: Provide helper to generate bfqq name Jan Kara
                   ` (3 subsequent siblings)
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Currently, when process A starts issuing requests shortly after process
B has completed some IO three times in a row, we decide that B is a
"waker" of A meaning that completing IO of B is needed for A to make
progress and generally stop separating A's and B's IO much. This logic
is useful to avoid unnecessary idling and thus throughput loss for cases
where workload needs to switch e.g. between the process and the
journaling thread doing IO. However the detection heuristic tends to
frequently give false positives when A and B are fighting IO bandwidth
and other processes aren't doing much IO as we are basically deemed to
eventually accumulate three occurences of a situation where one process
starts issuing requests after the other has completed some IO. To reduce
these false positives, cancel the waker detection also if we didn't
accumulate three detected wakeups within given timeout. The rationale is
that if wakeups are really rare, the pointless idling doesn't hurt
throughput that much anyway.

This significantly reduces false waker detection for workload like:

[global]
directory=/mnt/repro/
rw=write
size=8g
time_based
runtime=30
ramp_time=10
blocksize=1m
direct=0
ioengine=sync

[slowwriter]
numjobs=1
fsync=200

[fastwriter]
numjobs=1
fsync=200

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.c | 38 +++++++++++++++++++++++---------------
 block/bfq-iosched.h |  2 ++
 2 files changed, 25 insertions(+), 15 deletions(-)

diff --git a/block/bfq-iosched.c b/block/bfq-iosched.c
index 3806409610ca..6c5e9bafdb5d 100644
--- a/block/bfq-iosched.c
+++ b/block/bfq-iosched.c
@@ -2091,20 +2091,19 @@ static void bfq_update_io_intensity(struct bfq_queue *bfqq, u64 now_ns)
  * aspect, see the comments on the choice of the queue for injection
  * in bfq_select_queue().
  *
- * Turning back to the detection of a waker queue, a queue Q is deemed
- * as a waker queue for bfqq if, for three consecutive times, bfqq
- * happens to become non empty right after a request of Q has been
- * completed. In this respect, even if bfqq is empty, we do not check
- * for a waker if it still has some in-flight I/O. In fact, in this
- * case bfqq is actually still being served by the drive, and may
- * receive new I/O on the completion of some of the in-flight
- * requests. In particular, on the first time, Q is tentatively set as
- * a candidate waker queue, while on the third consecutive time that Q
- * is detected, the field waker_bfqq is set to Q, to confirm that Q is
- * a waker queue for bfqq. These detection steps are performed only if
- * bfqq has a long think time, so as to make it more likely that
- * bfqq's I/O is actually being blocked by a synchronization. This
- * last filter, plus the above three-times requirement, make false
+ * Turning back to the detection of a waker queue, a queue Q is deemed as a
+ * waker queue for bfqq if, for three consecutive times, bfqq happens to become
+ * non empty right after a request of Q has been completed within given
+ * timeout. In this respect, even if bfqq is empty, we do not check for a waker
+ * if it still has some in-flight I/O. In fact, in this case bfqq is actually
+ * still being served by the drive, and may receive new I/O on the completion
+ * of some of the in-flight requests. In particular, on the first time, Q is
+ * tentatively set as a candidate waker queue, while on the third consecutive
+ * time that Q is detected, the field waker_bfqq is set to Q, to confirm that Q
+ * is a waker queue for bfqq. These detection steps are performed only if bfqq
+ * has a long think time, so as to make it more likely that bfqq's I/O is
+ * actually being blocked by a synchronization. This last filter, plus the
+ * above three-times requirement and time limit for detection, make false
  * positives less likely.
  *
  * NOTE
@@ -2136,8 +2135,16 @@ static void bfq_check_waker(struct bfq_data *bfqd, struct bfq_queue *bfqq,
 	    bfqd->last_completed_rq_bfqq == bfqq->waker_bfqq)
 		return;
 
+	/*
+	 * We reset waker detection logic also if too much time has passed
+ 	 * since the first detection. If wakeups are rare, pointless idling
+	 * doesn't hurt throughput that much. The condition below makes sure
+	 * we do not uselessly idle blocking waker in more than 1/64 cases. 
+	 */
 	if (bfqd->last_completed_rq_bfqq !=
-	    bfqq->tentative_waker_bfqq) {
+	    bfqq->tentative_waker_bfqq ||
+	    now_ns > bfqq->waker_detection_started +
+					128 * (u64)bfqd->bfq_slice_idle) {
 		/*
 		 * First synchronization detected with a
 		 * candidate waker queue, or with a different
@@ -2146,6 +2153,7 @@ static void bfq_check_waker(struct bfq_data *bfqd, struct bfq_queue *bfqq,
 		bfqq->tentative_waker_bfqq =
 			bfqd->last_completed_rq_bfqq;
 		bfqq->num_waker_detections = 1;
+		bfqq->waker_detection_started = now_ns;
 	} else /* Same tentative waker queue detected again */
 		bfqq->num_waker_detections++;
 
diff --git a/block/bfq-iosched.h b/block/bfq-iosched.h
index 820cb8c2d1fe..bb8180c52a31 100644
--- a/block/bfq-iosched.h
+++ b/block/bfq-iosched.h
@@ -388,6 +388,8 @@ struct bfq_queue {
 	struct bfq_queue *tentative_waker_bfqq;
 	/* number of times the same tentative waker has been detected */
 	unsigned int num_waker_detections;
+	/* time when we started considering this waker */
+	u64 waker_detection_started;
 
 	/* node for woken_list, see below */
 	struct hlist_node woken_list_node;
-- 
2.26.2


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

* [PATCH 6/8] bfq: Provide helper to generate bfqq name
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
                   ` (4 preceding siblings ...)
  2021-10-06 17:31 ` [PATCH 5/8] bfq: Limit waker detection in time Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-06 17:31 ` [PATCH 7/8] bfq: Log waker detections Jan Kara
                   ` (2 subsequent siblings)
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Instead of having helper formating bfqq pid, provide a helper to
generate full bfqq name as used in the traces. It saves some code
duplication and will save more in the coming tracepoints.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.h | 27 +++++++++++++--------------
 1 file changed, 13 insertions(+), 14 deletions(-)

diff --git a/block/bfq-iosched.h b/block/bfq-iosched.h
index bb8180c52a31..07288b9da389 100644
--- a/block/bfq-iosched.h
+++ b/block/bfq-iosched.h
@@ -25,7 +25,7 @@
 #define BFQ_DEFAULT_GRP_IOPRIO	0
 #define BFQ_DEFAULT_GRP_CLASS	IOPRIO_CLASS_BE
 
-#define MAX_PID_STR_LENGTH 12
+#define MAX_BFQQ_NAME_LENGTH 16
 
 /*
  * Soft real-time applications are extremely more latency sensitive
@@ -1083,26 +1083,27 @@ void bfq_add_bfqq_busy(struct bfq_data *bfqd, struct bfq_queue *bfqq);
 /* --------------- end of interface of B-WF2Q+ ---------------- */
 
 /* Logging facilities. */
-static inline void bfq_pid_to_str(int pid, char *str, int len)
+static inline void bfq_bfqq_name(struct bfq_queue *bfqq, char *str, int len)
 {
-	if (pid != -1)
-		snprintf(str, len, "%d", pid);
+	char type = bfq_bfqq_sync(bfqq) ? 'S' : 'A';
+
+	if (bfqq->pid != -1)
+		snprintf(str, len, "bfq%d%c", bfqq->pid, type);
 	else
-		snprintf(str, len, "SHARED-");
+		snprintf(str, len, "bfqSHARED-%c", type);
 }
 
 #ifdef CONFIG_BFQ_GROUP_IOSCHED
 struct bfq_group *bfqq_group(struct bfq_queue *bfqq);
 
 #define bfq_log_bfqq(bfqd, bfqq, fmt, args...)	do {			\
-	char pid_str[MAX_PID_STR_LENGTH];	\
+	char pid_str[MAX_BFQQ_NAME_LENGTH];				\
 	if (likely(!blk_trace_note_message_enabled((bfqd)->queue)))	\
 		break;							\
-	bfq_pid_to_str((bfqq)->pid, pid_str, MAX_PID_STR_LENGTH);	\
+	bfq_bfqq_name((bfqq), pid_str, MAX_BFQQ_NAME_LENGTH);		\
 	blk_add_cgroup_trace_msg((bfqd)->queue,				\
 			bfqg_to_blkg(bfqq_group(bfqq))->blkcg,		\
-			"bfq%s%c " fmt, pid_str,			\
-			bfq_bfqq_sync((bfqq)) ? 'S' : 'A', ##args);	\
+			"%s " fmt, pid_str, ##args);			\
 } while (0)
 
 #define bfq_log_bfqg(bfqd, bfqg, fmt, args...)	do {			\
@@ -1113,13 +1114,11 @@ struct bfq_group *bfqq_group(struct bfq_queue *bfqq);
 #else /* CONFIG_BFQ_GROUP_IOSCHED */
 
 #define bfq_log_bfqq(bfqd, bfqq, fmt, args...) do {	\
-	char pid_str[MAX_PID_STR_LENGTH];	\
+	char pid_str[MAX_BFQQ_NAME_LENGTH];				\
 	if (likely(!blk_trace_note_message_enabled((bfqd)->queue)))	\
 		break;							\
-	bfq_pid_to_str((bfqq)->pid, pid_str, MAX_PID_STR_LENGTH);	\
-	blk_add_trace_msg((bfqd)->queue, "bfq%s%c " fmt, pid_str,	\
-			bfq_bfqq_sync((bfqq)) ? 'S' : 'A',		\
-				##args);	\
+	bfq_bfqq_name((bfqq), pid_str, MAX_BFQQ_NAME_LENGTH);		\
+	blk_add_trace_msg((bfqd)->queue, "%s " fmt, pid_str, ##args);	\
 } while (0)
 #define bfq_log_bfqg(bfqd, bfqg, fmt, args...)		do {} while (0)
 
-- 
2.26.2


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

* [PATCH 7/8] bfq: Log waker detections
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
                   ` (5 preceding siblings ...)
  2021-10-06 17:31 ` [PATCH 6/8] bfq: Provide helper to generate bfqq name Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-06 17:31 ` [PATCH 8/8] bfq: Do not let waker requests skip proper accounting Jan Kara
  2021-10-07 16:33 ` [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Paolo Valente
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Waker - wakee relationships are important in deciding whether one queue
can preempt the other one. Print information about detected waker-wakee
relationships so that scheduling decisions can be better understood from
block traces.

Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.c | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/block/bfq-iosched.c b/block/bfq-iosched.c
index 6c5e9bafdb5d..886befc35b57 100644
--- a/block/bfq-iosched.c
+++ b/block/bfq-iosched.c
@@ -2127,6 +2127,8 @@ static void bfq_update_io_intensity(struct bfq_queue *bfqq, u64 now_ns)
 static void bfq_check_waker(struct bfq_data *bfqd, struct bfq_queue *bfqq,
 			    u64 now_ns)
 {
+	char waker_name[MAX_BFQQ_NAME_LENGTH];
+
 	if (!bfqd->last_completed_rq_bfqq ||
 	    bfqd->last_completed_rq_bfqq == bfqq ||
 	    bfq_bfqq_has_short_ttime(bfqq) ||
@@ -2154,12 +2156,18 @@ static void bfq_check_waker(struct bfq_data *bfqd, struct bfq_queue *bfqq,
 			bfqd->last_completed_rq_bfqq;
 		bfqq->num_waker_detections = 1;
 		bfqq->waker_detection_started = now_ns;
+		bfq_bfqq_name(bfqq->tentative_waker_bfqq, waker_name,
+			      MAX_BFQQ_NAME_LENGTH);
+		bfq_log_bfqq(bfqd, bfqq, "set tenative waker %s", waker_name);
 	} else /* Same tentative waker queue detected again */
 		bfqq->num_waker_detections++;
 
 	if (bfqq->num_waker_detections == 3) {
 		bfqq->waker_bfqq = bfqd->last_completed_rq_bfqq;
 		bfqq->tentative_waker_bfqq = NULL;
+		bfq_bfqq_name(bfqq->waker_bfqq, waker_name,
+			      MAX_BFQQ_NAME_LENGTH);
+		bfq_log_bfqq(bfqd, bfqq, "set waker %s", waker_name);
 
 		/*
 		 * If the waker queue disappears, then
-- 
2.26.2


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

* [PATCH 8/8] bfq: Do not let waker requests skip proper accounting
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
                   ` (6 preceding siblings ...)
  2021-10-06 17:31 ` [PATCH 7/8] bfq: Log waker detections Jan Kara
@ 2021-10-06 17:31 ` Jan Kara
  2021-10-07 16:33 ` [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Paolo Valente
  8 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-10-06 17:31 UTC (permalink / raw)
  To: Paolo Valente; +Cc: linux-block, Jens Axboe, Michal Koutný, Jan Kara

Commit 7cc4ffc55564 ("block, bfq: put reqs of waker and woken in
dispatch list") added a condition to bfq_insert_request() which added
waker's requests directly to dispatch list. The rationale was that
completing waker's IO is needed to get more IO for the current queue.
Although this rationale is valid, there is a hole in it. The waker does
not necessarily serve the IO only for the current queue and maybe it's
current IO is not needed for current queue to make progress. Furthermore
injecting IO like this completely bypasses any service accounting within
bfq and thus we do not properly track how much service is waker's queue
getting or that the waker is actually doing any IO. Depending on the
conditions this can result in the waker getting too much or too few
service.

Consider for example the following job file:

[global]
directory=/mnt/repro/
rw=write
size=8g
time_based
runtime=30
ramp_time=10
blocksize=1m
direct=0
ioengine=sync

[slowwriter]
numjobs=1
prioclass=2
prio=7
fsync=200

[fastwriter]
numjobs=1
prioclass=2
prio=0
fsync=200

Despite processes have very different IO priorities, they get the same
about of service. The reason is that bfq identifies these processes as
having waker-wakee relationship and once that happens, IO from
fastwriter gets injected during slowwriter's time slice. As a result bfq
is not aware that fastwriter has any IO to do and constantly schedules
only slowwriter's queue. Thus fastwriter is forced to compete with
slowwriter's IO all the time instead of getting its share of time based
on IO priority.

Drop the special injection condition from bfq_insert_request(). As a
result, requests will be tracked and queued in a normal way and on next
dispatch bfq_select_queue() can decide whether the waker's inserted
requests should be injected during the current queue's timeslice or not.

Fixes: 7cc4ffc55564 ("block, bfq: put reqs of waker and woken in dispatch list")
Signed-off-by: Jan Kara <jack@suse.cz>
---
 block/bfq-iosched.c | 44 +-------------------------------------------
 1 file changed, 1 insertion(+), 43 deletions(-)

diff --git a/block/bfq-iosched.c b/block/bfq-iosched.c
index 886befc35b57..803a0c313f0f 100644
--- a/block/bfq-iosched.c
+++ b/block/bfq-iosched.c
@@ -6132,48 +6132,7 @@ static void bfq_insert_request(struct blk_mq_hw_ctx *hctx, struct request *rq,
 
 	spin_lock_irq(&bfqd->lock);
 	bfqq = bfq_init_rq(rq);
-
-	/*
-	 * Reqs with at_head or passthrough flags set are to be put
-	 * directly into dispatch list. Additional case for putting rq
-	 * directly into the dispatch queue: the only active
-	 * bfq_queues are bfqq and either its waker bfq_queue or one
-	 * of its woken bfq_queues. The rationale behind this
-	 * additional condition is as follows:
-	 * - consider a bfq_queue, say Q1, detected as a waker of
-	 *   another bfq_queue, say Q2
-	 * - by definition of a waker, Q1 blocks the I/O of Q2, i.e.,
-	 *   some I/O of Q1 needs to be completed for new I/O of Q2
-	 *   to arrive.  A notable example of waker is journald
-	 * - so, Q1 and Q2 are in any respect the queues of two
-	 *   cooperating processes (or of two cooperating sets of
-	 *   processes): the goal of Q1's I/O is doing what needs to
-	 *   be done so that new Q2's I/O can finally be
-	 *   issued. Therefore, if the service of Q1's I/O is delayed,
-	 *   then Q2's I/O is delayed too.  Conversely, if Q2's I/O is
-	 *   delayed, the goal of Q1's I/O is hindered.
-	 * - as a consequence, if some I/O of Q1/Q2 arrives while
-	 *   Q2/Q1 is the only queue in service, there is absolutely
-	 *   no point in delaying the service of such an I/O. The
-	 *   only possible result is a throughput loss
-	 * - so, when the above condition holds, the best option is to
-	 *   have the new I/O dispatched as soon as possible
-	 * - the most effective and efficient way to attain the above
-	 *   goal is to put the new I/O directly in the dispatch
-	 *   list
-	 * - as an additional restriction, Q1 and Q2 must be the only
-	 *   busy queues for this commit to put the I/O of Q2/Q1 in
-	 *   the dispatch list.  This is necessary, because, if also
-	 *   other queues are waiting for service, then putting new
-	 *   I/O directly in the dispatch list may evidently cause a
-	 *   violation of service guarantees for the other queues
-	 */
-	if (!bfqq ||
-	    (bfqq != bfqd->in_service_queue &&
-	     bfqd->in_service_queue != NULL &&
-	     bfq_tot_busy_queues(bfqd) == 1 + bfq_bfqq_busy(bfqq) &&
-	     (bfqq->waker_bfqq == bfqd->in_service_queue ||
-	      bfqd->in_service_queue->waker_bfqq == bfqq)) || at_head) {
+	if (!bfqq || at_head) {
 		if (at_head)
 			list_add(&rq->queuelist, &bfqd->dispatch);
 		else
@@ -6200,7 +6159,6 @@ static void bfq_insert_request(struct blk_mq_hw_ctx *hctx, struct request *rq,
 	 * merge).
 	 */
 	cmd_flags = rq->cmd_flags;
-
 	spin_unlock_irq(&bfqd->lock);
 
 	bfq_update_insert_stats(q, bfqq, idle_timer_disabled,
-- 
2.26.2


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

* Re: [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup
  2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
                   ` (7 preceding siblings ...)
  2021-10-06 17:31 ` [PATCH 8/8] bfq: Do not let waker requests skip proper accounting Jan Kara
@ 2021-10-07 16:33 ` Paolo Valente
  2021-10-25  7:58   ` Paolo Valente
  8 siblings, 1 reply; 17+ messages in thread
From: Paolo Valente @ 2021-10-07 16:33 UTC (permalink / raw)
  To: Jan Kara; +Cc: linux-block, Jens Axboe, Michal Koutný



> Il giorno 6 ott 2021, alle ore 19:31, Jan Kara <jack@suse.cz> ha scritto:
> 
> Hello!
> 
> Here is the third revision of my patches to fix how bfq weights apply on cgroup
> throughput and on throughput of processes with different IO priorities. Since
> v2 I've added some more patches so that now IO priorities also result in
> service differentiation (previously they had no effect on service
> differentiation on some workloads similarly to cgroup weights). The last patch
> in the series still needs some work as in the current state it causes a
> notable regression (~20-30%) with dbench benchmark for large numbers of
> clients. I've verified that the last patch is indeed necessary for the service
> differentiation with the workload described in its changelog. As we discussed
> with Paolo, I have also found out that if I remove the "waker has enough
> budget" condition from bfq_select_queue(), dbench performance is restored
> and the service differentiation is still good. But we probably need some
> justification or cleaner solution than just removing the condition so that
> is still up to discussion. But first seven patches already noticeably improve
> the situation for lots of workloads so IMO they stand on their own and
> can be merged regardless of how we go about the last patch.
> 

Hi Jan,
I have just one more (easy-to-resolve) doubt: you seem to have tested
these patches mostly on the throughput side.  Did you run a
startup-latency test as well?  I can run some for you, if you prefer
so. Just give me a few days.

When that is cleared, your first seven patches are ok for me.
Actually I think they are a very good and relevant contribution.
Patch number eight probably deserve some ore analysis, as you pointed
out.  As I already told you in our call, we can look at that budget
condition together.  And figure out the best tests to use, to check
whether I/O control does not get lost too much.

Thanks,
Paolo

> Changes since v2:
> * Rebased on top of current Linus' tree
> * Updated computation of scheduler tag proportions to work correctly even
>  for processes within the same cgroup but with different IO priorities
> * Added comment roughly explaining why we limit tag depth
> * Added patch limiting waker / wakee detection in time so avoid at least the
>  most obvious false positives
> * Added patch to log waker / wakee detections in blktrace for better debugging
> * Added patch properly account injected IO
> 
> Changes since v1:
> * Fixed computation of appropriate proportion of scheduler tags for a cgroup
>  to work with deeper cgroup hierarchies.
> 
> Original cover letter:
> 
> I was looking into why cgroup weights do not have any measurable impact on
> writeback throughput from different cgroups. This actually a regression from
> CFQ where things work more or less OK and weights have roughly the impact they
> should. The problem can be reproduced e.g. by running the following easy fio
> job in two cgroups with different weight:
> 
> [writer]
> directory=/mnt/repro/
> numjobs=1
> rw=write
> size=8g
> time_based
> runtime=30
> ramp_time=10
> blocksize=1m
> direct=0
> ioengine=sync
> 
> I can observe there's no significat difference in the amount of data written
> from different cgroups despite their weights are in say 1:3 ratio.
> 
> After some debugging I've understood the dynamics of the system. There are two
> issues:
> 
> 1) The amount of scheduler tags needs to be significantly larger than the
> amount of device tags. Otherwise there are not enough requests waiting in BFQ
> to be dispatched to the device and thus there's nothing to schedule on.
> 
> 2) Even with enough scheduler tags, writers from two cgroups eventually start
> contending on scheduler tag allocation. These are served on first come first
> served basis so writers from both cgroups feed requests into bfq with
> approximately the same speed. Since bfq prefers IO from heavier cgroup, that is
> submitted and completed faster and eventually we end up in a situation when
> there's no IO from the heavier cgroup in bfq and all scheduler tags are
> consumed by requests from the lighter cgroup. At that point bfq just dispatches
> lots of the IO from the lighter cgroup since there's no contender for disk
> throughput. As a result observed throughput for both cgroups are the same.
> 
> This series fixes this problem by accounting how many scheduler tags are
> allocated for each cgroup and if a cgroup has more tags allocated than its
> fair share (based on weights) in its service tree, we heavily limit scheduler
> tag bitmap depth for it so that it is not be able to starve other cgroups from
> scheduler tags.
> 
> What do people think about this?
> 
> 								Honza
> 
> Previous versions:
> Link: http://lore.kernel.org/r/20210712171146.12231-1-jack@suse.cz # v1
> Link: http://lore.kernel.org/r/20210715132047.20874-1-jack@suse.cz # v2


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

* Re: [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup
  2021-10-07 16:33 ` [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Paolo Valente
@ 2021-10-25  7:58   ` Paolo Valente
  2021-10-25 11:14     ` Jan Kara
  0 siblings, 1 reply; 17+ messages in thread
From: Paolo Valente @ 2021-10-25  7:58 UTC (permalink / raw)
  To: Jan Kara
  Cc: linux-block, Jens Axboe, Michal Koutný,
	237826, 224833, Giacomo Guiduzzi, 238290, PAOLO CROTTI



> Il giorno 7 ott 2021, alle ore 18:33, Paolo Valente <paolo.valente@linaro.org> ha scritto:
> 
> 
> 
>> Il giorno 6 ott 2021, alle ore 19:31, Jan Kara <jack@suse.cz> ha scritto:
>> 
>> Hello!
>> 
>> Here is the third revision of my patches to fix how bfq weights apply on cgroup
>> throughput and on throughput of processes with different IO priorities. Since
>> v2 I've added some more patches so that now IO priorities also result in
>> service differentiation (previously they had no effect on service
>> differentiation on some workloads similarly to cgroup weights). The last patch
>> in the series still needs some work as in the current state it causes a
>> notable regression (~20-30%) with dbench benchmark for large numbers of
>> clients. I've verified that the last patch is indeed necessary for the service
>> differentiation with the workload described in its changelog. As we discussed
>> with Paolo, I have also found out that if I remove the "waker has enough
>> budget" condition from bfq_select_queue(), dbench performance is restored
>> and the service differentiation is still good. But we probably need some
>> justification or cleaner solution than just removing the condition so that
>> is still up to discussion. But first seven patches already noticeably improve
>> the situation for lots of workloads so IMO they stand on their own and
>> can be merged regardless of how we go about the last patch.
>> 
> 
> Hi Jan,
> I have just one more (easy-to-resolve) doubt: you seem to have tested
> these patches mostly on the throughput side.  Did you run a
> startup-latency test as well?  I can run some for you, if you prefer
> so. Just give me a few days.
> 

We are finally testing your patches a little bit right now, for
regressions with our typical benchmarks ...

Paolo

> When that is cleared, your first seven patches are ok for me.
> Actually I think they are a very good and relevant contribution.
> Patch number eight probably deserve some ore analysis, as you pointed
> out.  As I already told you in our call, we can look at that budget
> condition together.  And figure out the best tests to use, to check
> whether I/O control does not get lost too much.
> 
> Thanks,
> Paolo
> 
>> Changes since v2:
>> * Rebased on top of current Linus' tree
>> * Updated computation of scheduler tag proportions to work correctly even
>> for processes within the same cgroup but with different IO priorities
>> * Added comment roughly explaining why we limit tag depth
>> * Added patch limiting waker / wakee detection in time so avoid at least the
>> most obvious false positives
>> * Added patch to log waker / wakee detections in blktrace for better debugging
>> * Added patch properly account injected IO
>> 
>> Changes since v1:
>> * Fixed computation of appropriate proportion of scheduler tags for a cgroup
>> to work with deeper cgroup hierarchies.
>> 
>> Original cover letter:
>> 
>> I was looking into why cgroup weights do not have any measurable impact on
>> writeback throughput from different cgroups. This actually a regression from
>> CFQ where things work more or less OK and weights have roughly the impact they
>> should. The problem can be reproduced e.g. by running the following easy fio
>> job in two cgroups with different weight:
>> 
>> [writer]
>> directory=/mnt/repro/
>> numjobs=1
>> rw=write
>> size=8g
>> time_based
>> runtime=30
>> ramp_time=10
>> blocksize=1m
>> direct=0
>> ioengine=sync
>> 
>> I can observe there's no significat difference in the amount of data written
>> from different cgroups despite their weights are in say 1:3 ratio.
>> 
>> After some debugging I've understood the dynamics of the system. There are two
>> issues:
>> 
>> 1) The amount of scheduler tags needs to be significantly larger than the
>> amount of device tags. Otherwise there are not enough requests waiting in BFQ
>> to be dispatched to the device and thus there's nothing to schedule on.
>> 
>> 2) Even with enough scheduler tags, writers from two cgroups eventually start
>> contending on scheduler tag allocation. These are served on first come first
>> served basis so writers from both cgroups feed requests into bfq with
>> approximately the same speed. Since bfq prefers IO from heavier cgroup, that is
>> submitted and completed faster and eventually we end up in a situation when
>> there's no IO from the heavier cgroup in bfq and all scheduler tags are
>> consumed by requests from the lighter cgroup. At that point bfq just dispatches
>> lots of the IO from the lighter cgroup since there's no contender for disk
>> throughput. As a result observed throughput for both cgroups are the same.
>> 
>> This series fixes this problem by accounting how many scheduler tags are
>> allocated for each cgroup and if a cgroup has more tags allocated than its
>> fair share (based on weights) in its service tree, we heavily limit scheduler
>> tag bitmap depth for it so that it is not be able to starve other cgroups from
>> scheduler tags.
>> 
>> What do people think about this?
>> 
>> 								Honza
>> 
>> Previous versions:
>> Link: http://lore.kernel.org/r/20210712171146.12231-1-jack@suse.cz # v1
>> Link: http://lore.kernel.org/r/20210715132047.20874-1-jack@suse.cz # v2


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

* Re: [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup
  2021-10-25  7:58   ` Paolo Valente
@ 2021-10-25 11:14     ` Jan Kara
  2021-11-10 10:24       ` Paolo Valente
  0 siblings, 1 reply; 17+ messages in thread
From: Jan Kara @ 2021-10-25 11:14 UTC (permalink / raw)
  To: Paolo Valente
  Cc: Jan Kara, linux-block, Jens Axboe, Michal Koutný,
	237826, 224833, Giacomo Guiduzzi, 238290, PAOLO CROTTI

On Mon 25-10-21 09:58:11, Paolo Valente wrote:
> > Il giorno 7 ott 2021, alle ore 18:33, Paolo Valente <paolo.valente@linaro.org> ha scritto:
> >> Il giorno 6 ott 2021, alle ore 19:31, Jan Kara <jack@suse.cz> ha scritto:
> >> 
> >> Hello!
> >> 
> >> Here is the third revision of my patches to fix how bfq weights apply on cgroup
> >> throughput and on throughput of processes with different IO priorities. Since
> >> v2 I've added some more patches so that now IO priorities also result in
> >> service differentiation (previously they had no effect on service
> >> differentiation on some workloads similarly to cgroup weights). The last patch
> >> in the series still needs some work as in the current state it causes a
> >> notable regression (~20-30%) with dbench benchmark for large numbers of
> >> clients. I've verified that the last patch is indeed necessary for the service
> >> differentiation with the workload described in its changelog. As we discussed
> >> with Paolo, I have also found out that if I remove the "waker has enough
> >> budget" condition from bfq_select_queue(), dbench performance is restored
> >> and the service differentiation is still good. But we probably need some
> >> justification or cleaner solution than just removing the condition so that
> >> is still up to discussion. But first seven patches already noticeably improve
> >> the situation for lots of workloads so IMO they stand on their own and
> >> can be merged regardless of how we go about the last patch.
> >> 
> > 
> > Hi Jan,
> > I have just one more (easy-to-resolve) doubt: you seem to have tested
> > these patches mostly on the throughput side.  Did you run a
> > startup-latency test as well?  I can run some for you, if you prefer
> > so. Just give me a few days.
> > 
> 
> We are finally testing your patches a little bit right now, for
> regressions with our typical benchmarks ...

Hum, strange I didn't get your previous email about benchmarks. You're
right I didn't run startup-latency AFAIR. Now that you've started them,
probably there's no big point in me queuing them as well. So thanks for
the benchmarking :)

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup
  2021-10-06 17:31 ` [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup Jan Kara
@ 2021-11-02 18:16   ` Michal Koutný
  2021-11-03 13:03     ` Jan Kara
  0 siblings, 1 reply; 17+ messages in thread
From: Michal Koutný @ 2021-11-02 18:16 UTC (permalink / raw)
  To: Jan Kara; +Cc: Paolo Valente, linux-block, Jens Axboe

Hello.

On Wed, Oct 06, 2021 at 07:31:43PM +0200, Jan Kara <jack@suse.cz> wrote:
> +	for (level--; level >= 0; level--) {
> +		entity = entities[level];
> +		if (level > 0) {
> +			wsum = bfq_entity_service_tree(entity)->wsum;
> +		} else {
> +			int i;
> +			/*
> +			 * For bfqq itself we take into account service trees
> +			 * of all higher priority classes and multiply their
> +			 * weights so that low prio queue from higher class
> +			 * gets more requests than high prio queue from lower
> +			 * class.
> +			 */
> +			wsum = 0;
> +			for (i = 0; i <= class_idx; i++) {
> +				wsum = wsum * IOPRIO_BE_NR +
> +					sched_data->service_tree[i].wsum;
> +			}
> +		}
> +		limit = DIV_ROUND_CLOSEST(limit * entity->weight, wsum);

This scheme caught my eye. You mutliply (tree) weights by a factor
depending on the class when counting the wsum but then you don't apply
the same scaling for the evaluated entity in the numerator.

IOW, I think there should be something like
	scale = (level > 0) ? 1 : int_pow(IOPRIO_BE_NR, BFQ_IOPRIO_CLASSES - bfq_class_idx(entity));
	limit = DIV_ROUND_CLOSEST(limit * entity->weight * scale, wsum);

For instance, if there are two cgroups (level=1) with weights 100 and
200, and each cgroup has a single IOPRIO_CLASS_BE entity (level=0) in
it, the `limit` distribution would honor the ratio of weights from
level=1 (100:200) but it would artificially lower the absolute amount of
allowed tags. If I am not mistaken, that would be reduced by factor
1/BFQ_IOPRIO_CLASSES.

Also if I consider it more broadly, is this supposed to match/extend
bfq_io_prio_to_weight() calculation?


Michal

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

* Re: [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup
  2021-11-02 18:16   ` Michal Koutný
@ 2021-11-03 13:03     ` Jan Kara
  2021-11-03 18:12       ` Michal Koutný
  0 siblings, 1 reply; 17+ messages in thread
From: Jan Kara @ 2021-11-03 13:03 UTC (permalink / raw)
  To: Michal Koutný; +Cc: Jan Kara, Paolo Valente, linux-block, Jens Axboe

Hello,

On Tue 02-11-21 19:16:58, Michal Koutný wrote:
> On Wed, Oct 06, 2021 at 07:31:43PM +0200, Jan Kara <jack@suse.cz> wrote:
> > +	for (level--; level >= 0; level--) {
> > +		entity = entities[level];
> > +		if (level > 0) {
> > +			wsum = bfq_entity_service_tree(entity)->wsum;
> > +		} else {
> > +			int i;
> > +			/*
> > +			 * For bfqq itself we take into account service trees
> > +			 * of all higher priority classes and multiply their
> > +			 * weights so that low prio queue from higher class
> > +			 * gets more requests than high prio queue from lower
> > +			 * class.
> > +			 */
> > +			wsum = 0;
> > +			for (i = 0; i <= class_idx; i++) {
> > +				wsum = wsum * IOPRIO_BE_NR +
> > +					sched_data->service_tree[i].wsum;
> > +			}
> > +		}
> > +		limit = DIV_ROUND_CLOSEST(limit * entity->weight, wsum);
> 
> This scheme caught my eye. You mutliply (tree) weights by a factor
> depending on the class when counting the wsum but then you don't apply
> the same scaling for the evaluated entity in the numerator.

Since we stop the loop at bfq_class_idx(entity) I don't think scaling of
the numerator makes sense - effectively when all the processes having IO to
submit (these are accounted in wsum) are in the same IO priority class, the
above code collapses to just:

  limit = limit * entity->weight / sched_data->service_tree[bfq_class_idx(entity)].wsum

I.e., we scale available tags proportionally to bfq_queue weight (which
scales linearly with IO priority).

When there are processes say both in RT and BE IO priority classes, then a
process in RT class still uses the above formula - i.e., as if all tags
available for a cgroup are split proportionally only among active tasks in
RT IO priority class. So in principle it can happen that there would be no
tag left for a process in lower IO priority class - and that is fine, we
don't care, because we don't want to submit IO from lower IO priority class
while there is still IO in higher IO priority class.

Now consider a situation for a process in BE IO priority class in this
setting. All processes in BE class can together occupy at most BE_wsum /
(RT_wsum * IOPRIO_BE_NR + BE_wsum) fraction of tags. This is admittedly
somewhat arbitrary fraction but it makes sure for each process in RT class
there are at least as many tags left as for the highest priority process in
BE class.

> IOW, I think there should be something like
> 	scale = (level > 0) ? 1 : int_pow(IOPRIO_BE_NR, BFQ_IOPRIO_CLASSES - bfq_class_idx(entity));
> 	limit = DIV_ROUND_CLOSEST(limit * entity->weight * scale, wsum);
> 
> For instance, if there are two cgroups (level=1) with weights 100 and
> 200, and each cgroup has a single IOPRIO_CLASS_BE entity (level=0) in
> it, the `limit` distribution would honor the ratio of weights from
> level=1 (100:200) but it would artificially lower the absolute amount of
> allowed tags. If I am not mistaken, that would be reduced by factor
> 1/BFQ_IOPRIO_CLASSES.

I don't see where my code would lead to available number of tags being
artifically lower as you write - in your example wsum for RT class would be
0 so effectively all terms of the formula for that class will cancel out.
As I wrote above, the highest active IO priority class effectively allows
processes in this class to consume all tags available for a cgroup. If
there are lower IO priority classes active as well, we allow them to
consume some tags but never allow them to consume all of them...

> Also if I consider it more broadly, is this supposed to match/extend
> bfq_io_prio_to_weight() calculation?

Yes, this is kind of an extension of bfq_io_prio_to_weight() that allows
some comparison of queues from different IO priority classes.

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup
  2021-11-03 13:03     ` Jan Kara
@ 2021-11-03 18:12       ` Michal Koutný
  2021-11-04 11:20         ` Jan Kara
  0 siblings, 1 reply; 17+ messages in thread
From: Michal Koutný @ 2021-11-03 18:12 UTC (permalink / raw)
  To: Jan Kara; +Cc: Paolo Valente, linux-block, Jens Axboe

On Wed, Nov 03, 2021 at 02:03:14PM +0100, Jan Kara <jack@suse.cz> wrote:
> Since we stop the loop at bfq_class_idx(entity)

Aha! I overlooked the for loop ends with the entity's class here and not
after the full range of classes.

> I.e., we scale available tags proportionally to bfq_queue weight (which
> scales linearly with IO priority).

Yes, you're working within the "order" of the entity's class and it's
always the last, i.e. least too, so the scale is 1.

> So in principle it can happen that there would be no tag left for a
> process in lower IO priority class - and that is fine, we don't care,
> because we don't want to submit IO from lower IO priority class while
> there is still IO in higher IO priority class.

Actually, can it ever happen that the higher class leaves some tags for
the lower? (IOW, is the CLS_wsum anytime exceeding sum of all active
entities of the CLS at the given point in time?) (1)

> Now consider a situation for a process in BE IO priority class in this
> setting. All processes in BE class can together occupy at most BE_wsum /
> (RT_wsum * IOPRIO_BE_NR + BE_wsum) fraction of tags. This is admittedly
> somewhat arbitrary fraction but it makes sure for each process in RT class
> there are at least as many tags left as for the highest priority process in
> BE class.

Can it happen that bfqq_request_over_limit() is called for a BE entity
before calling it for an RT entity (more precisely, not the
bfqq_request_over_limit() calls but actual allocation of tags)? (2)

> As I wrote above, the highest active IO priority class effectively allows
> processes in this class to consume all tags available for a cgroup. If
> there are lower IO priority classes active as well, we allow them to
> consume some tags but never allow them to consume all of them...

I assume this implies the answer to my previous question (2) is "yes"
and to the question (1) is: "numerically no, but lower class entity can
take some tags if it gets to draw them earlier".

> Yes, this is kind of an extension of bfq_io_prio_to_weight() that allows
> some comparison of queues from different IO priority classes.

I see there's no point using the same values for the weights in the
bfqq_request_over_limit() calculations as bfq_ioprio_to_weight()
calculates given the nature of strict ordering of classes above each
other. Your scoring makes sense to me now.

Reviewed-by: Michal Koutný <mkoutny@suse.com>

(this patch only)

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

* Re: [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup
  2021-11-03 18:12       ` Michal Koutný
@ 2021-11-04 11:20         ` Jan Kara
  0 siblings, 0 replies; 17+ messages in thread
From: Jan Kara @ 2021-11-04 11:20 UTC (permalink / raw)
  To: Michal Koutný; +Cc: Jan Kara, Paolo Valente, linux-block, Jens Axboe

On Wed 03-11-21 19:12:12, Michal Koutný wrote:
> On Wed, Nov 03, 2021 at 02:03:14PM +0100, Jan Kara <jack@suse.cz> wrote:
> > Since we stop the loop at bfq_class_idx(entity)
> 
> Aha! I overlooked the for loop ends with the entity's class here and not
> after the full range of classes.
> 
> > I.e., we scale available tags proportionally to bfq_queue weight (which
> > scales linearly with IO priority).
> 
> Yes, you're working within the "order" of the entity's class and it's
> always the last, i.e. least too, so the scale is 1.
> 
> > So in principle it can happen that there would be no tag left for a
> > process in lower IO priority class - and that is fine, we don't care,
> > because we don't want to submit IO from lower IO priority class while
> > there is still IO in higher IO priority class.
> 
> Actually, can it ever happen that the higher class leaves some tags for
> the lower? (IOW, is the CLS_wsum anytime exceeding sum of all active
> entities of the CLS at the given point in time?) (1)

Yes, that can happen. Here we compute just an upper bound on the number of
tags. Each entity can use less than this upper limit and thus there will be
tags left for entities in lower IO priority classes.

> > Now consider a situation for a process in BE IO priority class in this
> > setting. All processes in BE class can together occupy at most BE_wsum /
> > (RT_wsum * IOPRIO_BE_NR + BE_wsum) fraction of tags. This is admittedly
> > somewhat arbitrary fraction but it makes sure for each process in RT class
> > there are at least as many tags left as for the highest priority process in
> > BE class.
> 
> Can it happen that bfqq_request_over_limit() is called for a BE entity
> before calling it for an RT entity (more precisely, not the
> bfqq_request_over_limit() calls but actual allocation of tags)? (2)

Sure, that can happen. bfqq_request_over_limit() gets called (as well as
scheduler tag allocation happens) at the moment process calls submit_bio().
Time when each process calls submit_bio() is completely out of our control.
It can even happen that BE process submits lots of IO, we let it allocate
lots of tags (because there isn't any other process in the service trees)
and then RT process submits its first IO - only at this moment tag limit
for BE process is reduced so BE process will block when trying to allocate
any further tag until it frees enough tags by completing IO. This is
actually the reason why we always allow a process to allocate at least some
tags so that it can enter service tree, then it can gradually allocate more
and more tags (because its tag allocation is not limited unlike the tag
allocation for BE process) until it uses appropriate share of tags.

> > As I wrote above, the highest active IO priority class effectively allows
> > processes in this class to consume all tags available for a cgroup. If
> > there are lower IO priority classes active as well, we allow them to
> > consume some tags but never allow them to consume all of them...
> 
> I assume this implies the answer to my previous question (2) is "yes"
> and to the question (1) is: "numerically no, but lower class entity can
> take some tags if it gets to draw them earlier".

Exactly.

> > Yes, this is kind of an extension of bfq_io_prio_to_weight() that allows
> > some comparison of queues from different IO priority classes.
> 
> I see there's no point using the same values for the weights in the
> bfqq_request_over_limit() calculations as bfq_ioprio_to_weight()
> calculates given the nature of strict ordering of classes above each
> other. Your scoring makes sense to me now.
> 
> Reviewed-by: Michal Koutný <mkoutny@suse.com>

Thanks!

								Honza
-- 
Jan Kara <jack@suse.com>
SUSE Labs, CR

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

* Re: [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup
  2021-10-25 11:14     ` Jan Kara
@ 2021-11-10 10:24       ` Paolo Valente
  0 siblings, 0 replies; 17+ messages in thread
From: Paolo Valente @ 2021-11-10 10:24 UTC (permalink / raw)
  To: Jan Kara
  Cc: linux-block, Jens Axboe, Michal Koutný,
	237826, 224833, Giacomo Guiduzzi, 238290, PAOLO CROTTI



> Il giorno 25 ott 2021, alle ore 13:14, Jan Kara <jack@suse.cz> ha scritto:
> 
> On Mon 25-10-21 09:58:11, Paolo Valente wrote:
>>> Il giorno 7 ott 2021, alle ore 18:33, Paolo Valente <paolo.valente@linaro.org> ha scritto:
>>>> Il giorno 6 ott 2021, alle ore 19:31, Jan Kara <jack@suse.cz> ha scritto:
>>>> 
>>>> Hello!
>>>> 
>>>> Here is the third revision of my patches to fix how bfq weights apply on cgroup
>>>> throughput and on throughput of processes with different IO priorities. Since
>>>> v2 I've added some more patches so that now IO priorities also result in
>>>> service differentiation (previously they had no effect on service
>>>> differentiation on some workloads similarly to cgroup weights). The last patch
>>>> in the series still needs some work as in the current state it causes a
>>>> notable regression (~20-30%) with dbench benchmark for large numbers of
>>>> clients. I've verified that the last patch is indeed necessary for the service
>>>> differentiation with the workload described in its changelog. As we discussed
>>>> with Paolo, I have also found out that if I remove the "waker has enough
>>>> budget" condition from bfq_select_queue(), dbench performance is restored
>>>> and the service differentiation is still good. But we probably need some
>>>> justification or cleaner solution than just removing the condition so that
>>>> is still up to discussion. But first seven patches already noticeably improve
>>>> the situation for lots of workloads so IMO they stand on their own and
>>>> can be merged regardless of how we go about the last patch.
>>>> 
>>> 
>>> Hi Jan,
>>> I have just one more (easy-to-resolve) doubt: you seem to have tested
>>> these patches mostly on the throughput side.  Did you run a
>>> startup-latency test as well?  I can run some for you, if you prefer
>>> so. Just give me a few days.
>>> 
>> 
>> We are finally testing your patches a little bit right now, for
>> regressions with our typical benchmarks ...
> 
> Hum, strange I didn't get your previous email about benchmarks. You're
> right I didn't run startup-latency AFAIR. Now that you've started them,
> probably there's no big point in me queuing them as well. So thanks for
> the benchmarking :)
> 

Hi Jan,
we have executed a lot of benchmarking, on various hardware.  No
regression found!

So, thank you very much for this important contribution, and:

Acked-by: Paolo Valente <paolo.valente@linaro.org>

> 								Honza
> -- 
> Jan Kara <jack@suse.com>
> SUSE Labs, CR


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

end of thread, other threads:[~2021-11-10 10:24 UTC | newest]

Thread overview: 17+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-10-06 17:31 [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Jan Kara
2021-10-06 17:31 ` [PATCH 1/8] block: Provide icq in request allocation data Jan Kara
2021-10-06 17:31 ` [PATCH 2/8] bfq: Track number of allocated requests in bfq_entity Jan Kara
2021-10-06 17:31 ` [PATCH 3/8] bfq: Store full bitmap depth in bfq_data Jan Kara
2021-10-06 17:31 ` [PATCH 4/8] bfq: Limit number of requests consumed by each cgroup Jan Kara
2021-11-02 18:16   ` Michal Koutný
2021-11-03 13:03     ` Jan Kara
2021-11-03 18:12       ` Michal Koutný
2021-11-04 11:20         ` Jan Kara
2021-10-06 17:31 ` [PATCH 5/8] bfq: Limit waker detection in time Jan Kara
2021-10-06 17:31 ` [PATCH 6/8] bfq: Provide helper to generate bfqq name Jan Kara
2021-10-06 17:31 ` [PATCH 7/8] bfq: Log waker detections Jan Kara
2021-10-06 17:31 ` [PATCH 8/8] bfq: Do not let waker requests skip proper accounting Jan Kara
2021-10-07 16:33 ` [PATCH 0/8 v3] bfq: Limit number of allocated scheduler tags per cgroup Paolo Valente
2021-10-25  7:58   ` Paolo Valente
2021-10-25 11:14     ` Jan Kara
2021-11-10 10:24       ` Paolo Valente

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.