linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 0/4] cgroup aware workqueues
@ 2016-03-18 22:14 Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 1/4] cgroup: Introduce a function to compare two tasks Bandan Das
                   ` (8 more replies)
  0 siblings, 9 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-18 22:14 UTC (permalink / raw)
  To: tj; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

At Linuxcon last year, based on our presentation "vhost: sharing is better" [1],
we had briefly discussed the idea of cgroup aware workqueues with Tejun. The
following patches are a result of the discussion. They are in no way complete in
that the changes are for unbounded workqueues only, but I just wanted to present my
unfinished work as RFC and get some feedback.

1/4 and 3/4 are simple cgroup changes and add a helper function.
2/4 is the main implementation.
4/4 changes vhost to use workqueues with support for cgroups.

Accounting:
When servicing a userspace task A attached to cgroup X, for cgroup
awareness, a worker thread could attach to all cgroups
of the task which it is servicing. This patch does it for unbound
workqueues which means all tasks that are bound to certain cgroups
could potentially be serviced by the same worker thread. However,
the same technique could be applicable to bounded workqueues as
well.

Example:
vhost creates a worker thread when invoked for a kvm guest. Since,
the guest is a normal process, the kernel thread servicing it should be
attached to the vm process' cgroups.

Design:

The fundamental addition is a cgroup aware worker pool and as stated above,
for the unbounded case only.

These changes don't populate the "numa awareness" fields/attrs and
unlike unbounded numa worker pools, cgroup worker pools are created
on demand. Every work request could potentially have a new cgroup
aware pool created for it based on the combination of cgroups it's attached
to. However, workqueues themselves are incognizant of the actual cgroups -
they rely on the cgroups provided helper functions either for 1. a match
of all the cgroups or 2. to attach a worker thread to all cgroups of
a userspace task. We do maintain a list of cgroup aware pools so that
when a new request comes in and a suitable worker pool needs to be
found, we search the list first before creating a new one. A worker
pool also stores a a list of all "task owners" - a list of processes
that we are serving currently.

Testing:
Create some qemu processes and attaching them to different
cgroups. Verifying that new worker pools are created for tasks
that are attached to different cgroups (and reuse for the ones
that belong to the same).

Some simple performace testing using netperf below. Although, these
numbers shouldn't be dependent on these patches. The cgroup attach
and match functions are not in hot paths for general usage which
is what this test does.

Netperf:
Two guests running netperf in parallel.
    	   	       	          Without patches                  With patches
   
TCP_STREAM (10^6 bits/second)         975.45			     978.88	
TCP_RR (Trans/second)		      20121			     18820.82
UDP_STREAM (10^6 bits/second)	      1287.82		  	     1184.5
UDP_RR (Trans/second)		      20766.72			     19667.08
Time a 4G iso download		      2m 33 seconds		     3m 02 seconds

Todo:
What about bounded workqueues ?
What happens when cgroups of a running process changes ?
sysfs variables
Sanity check the flush and destroy path.
More extensive testing
Can we optimize the search/match/attach functions ?
Better performance numbers ? (although the onese above don't look bad)

[1] http://events.linuxfoundation.org/sites/events/files/slides/kvm_forum_2015_vhost_sharing_is_better.pdf

Bandan Das (4):
  cgroup: Introduce a function to compare two tasks
  workqueue: introduce support for attaching to cgroups
  cgroup: use spin_lock_irq for cgroup match and attach fns
  vhost: use workqueues for the works

 drivers/vhost/vhost.c       | 103 ++++++++++++++++++---
 drivers/vhost/vhost.h       |   2 +
 include/linux/cgroup.h      |   1 +
 include/linux/workqueue.h   |   2 +
 kernel/cgroup.c             |  40 ++++++++-
 kernel/workqueue.c          | 212 +++++++++++++++++++++++++++++++++++++++++---
 kernel/workqueue_internal.h |   4 +
 7 files changed, 335 insertions(+), 29 deletions(-)

-- 
2.5.0

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

* [RFC PATCH 1/4] cgroup: Introduce a function to compare two tasks
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
@ 2016-03-18 22:14 ` Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 2/4] workqueue: introduce support for attaching to cgroups Bandan Das
                   ` (7 subsequent siblings)
  8 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-18 22:14 UTC (permalink / raw)
  To: tj; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

This function takes two tasks and iterates through all
hierarchies to check if they belong to the same cgroups.
It ignores the check for default hierarchies or for
hierarchies with no subsystems attached.

Signed-off-by: Bandan Das <bsd@redhat.com>
---
 include/linux/cgroup.h |  1 +
 kernel/cgroup.c        | 36 ++++++++++++++++++++++++++++++++++++
 2 files changed, 37 insertions(+)

diff --git a/include/linux/cgroup.h b/include/linux/cgroup.h
index 2162dca..6b66092 100644
--- a/include/linux/cgroup.h
+++ b/include/linux/cgroup.h
@@ -83,6 +83,7 @@ struct cgroup_subsys_state *css_tryget_online_from_dir(struct dentry *dentry,
 
 struct cgroup *cgroup_get_from_path(const char *path);
 
+bool cgroup_match_groups(struct task_struct *, struct task_struct *);
 int cgroup_attach_task_all(struct task_struct *from, struct task_struct *);
 int cgroup_transfer_tasks(struct cgroup *to, struct cgroup *from);
 
diff --git a/kernel/cgroup.c b/kernel/cgroup.c
index d27904c..3ffdbb4 100644
--- a/kernel/cgroup.c
+++ b/kernel/cgroup.c
@@ -2745,6 +2745,42 @@ out_unlock_threadgroup:
 }
 
 /**
+ * cgroup_match_groups - check if tsk1 and tsk2 belong to same cgroups
+ * in all hierarchies
+ */
+bool cgroup_match_groups(struct task_struct *tsk1, struct task_struct *tsk2)
+{
+	struct cgroup_root *root;
+	bool result = true;
+
+	mutex_lock(&cgroup_mutex);
+	for_each_root(root) {
+		struct cgroup *cg_tsk1;
+		struct cgroup *cg_tsk2;
+
+		if (root == &cgrp_dfl_root)
+			continue;
+
+		if (!root->subsys_mask)
+			continue;
+
+		spin_lock_bh(&css_set_lock);
+		cg_tsk1 = task_cgroup_from_root(tsk1, root);
+		cg_tsk2 = task_cgroup_from_root(tsk2, root);
+		spin_unlock_bh(&css_set_lock);
+
+		if (cg_tsk1 != cg_tsk2) {
+			result = false;
+			break;
+		}
+	}
+	mutex_unlock(&cgroup_mutex);
+
+	return result;
+}
+EXPORT_SYMBOL_GPL(cgroup_match_groups);
+
+/**
  * cgroup_attach_task_all - attach task 'tsk' to all cgroups of task 'from'
  * @from: attach to all cgroups of a given task
  * @tsk: the task to be attached
-- 
2.5.0

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

* [RFC PATCH 2/4] workqueue: introduce support for attaching to cgroups
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 1/4] cgroup: Introduce a function to compare two tasks Bandan Das
@ 2016-03-18 22:14 ` Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 3/4] cgroup: use spin_lock_irq for cgroup match and attach fns Bandan Das
                   ` (6 subsequent siblings)
  8 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-18 22:14 UTC (permalink / raw)
  To: tj; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

Introduce cgroup aware unbounded worker pools. Whenever a new worker thread is
created, create_worker attaches itself to the cgroups of the task that called
alloc_workqueue(). New worker pools are created if there's no match in the global
list of cgroup aware worker pools.

Signed-off-by: Bandan Das <bsd@redhat.com>
---
 include/linux/workqueue.h   |   2 +
 kernel/workqueue.c          | 212 +++++++++++++++++++++++++++++++++++++++++---
 kernel/workqueue_internal.h |   4 +
 3 files changed, 204 insertions(+), 14 deletions(-)

diff --git a/include/linux/workqueue.h b/include/linux/workqueue.h
index ca73c50..7afb72d 100644
--- a/include/linux/workqueue.h
+++ b/include/linux/workqueue.h
@@ -131,6 +131,7 @@ struct workqueue_attrs {
 	int			nice;		/* nice level */
 	cpumask_var_t		cpumask;	/* allowed CPUs */
 	bool			no_numa;	/* disable NUMA affinity */
+	bool                    cg_enabled;     /* cgroups aware */
 };
 
 static inline struct delayed_work *to_delayed_work(struct work_struct *work)
@@ -308,6 +309,7 @@ enum {
 	 * http://thread.gmane.org/gmane.linux.kernel/1480396
 	 */
 	WQ_POWER_EFFICIENT	= 1 << 7,
+	WQ_CGROUPS              = 1 << 8,
 
 	__WQ_DRAINING		= 1 << 16, /* internal: workqueue is draining */
 	__WQ_ORDERED		= 1 << 17, /* internal: workqueue is ordered */
diff --git a/kernel/workqueue.c b/kernel/workqueue.c
index 7ff5dc7..f052d85 100644
--- a/kernel/workqueue.c
+++ b/kernel/workqueue.c
@@ -48,6 +48,7 @@
 #include <linux/nodemask.h>
 #include <linux/moduleparam.h>
 #include <linux/uaccess.h>
+#include <linux/cgroup.h>
 
 #include "workqueue_internal.h"
 
@@ -139,8 +140,18 @@ enum {
  * MD: wq_mayday_lock protected.
  */
 
+/*
+ * list of tasks that "own" the cgroups that
+ * this pool is attached to
+ */
+struct cgroup_owners {
+	struct task_struct *owner;
+	struct list_head    link;
+};
+
 /* struct worker is defined in workqueue_internal.h */
 
+
 struct worker_pool {
 	spinlock_t		lock;		/* the pool lock */
 	int			cpu;		/* I: the associated cpu */
@@ -169,6 +180,8 @@ struct worker_pool {
 	struct worker		*manager;	/* L: purely informational */
 	struct mutex		attach_mutex;	/* attach/detach exclusion */
 	struct list_head	workers;	/* A: attached workers */
+	struct list_head        cg_owners;      /* tasks using this pool*/
+	struct list_head        unbound_node;   /* all cgroup aware pools */
 	struct completion	*detach_completion; /* all workers detached */
 
 	struct ida		worker_ida;	/* worker IDs for task name */
@@ -219,6 +232,7 @@ struct pool_workqueue {
 	 */
 	struct work_struct	unbound_release_work;
 	struct rcu_head		rcu;
+	struct task_struct      *owner; /*for cgroups */
 } __aligned(1 << WORK_STRUCT_FLAG_BITS);
 
 /*
@@ -299,6 +313,7 @@ static DEFINE_MUTEX(wq_pool_mutex);	/* protects pools and workqueues list */
 static DEFINE_SPINLOCK(wq_mayday_lock);	/* protects wq->maydays list */
 
 static LIST_HEAD(workqueues);		/* PR: list of all workqueues */
+static LIST_HEAD(unbound_cgpool);       /* list of cgroup aware worker pools */
 static bool workqueue_freezing;		/* PL: have wqs started freezing? */
 
 /* PL: allowable cpus for unbound wqs and work items */
@@ -425,6 +440,12 @@ static void workqueue_sysfs_unregister(struct workqueue_struct *wq);
 		if (({ assert_rcu_or_wq_mutex(wq); false; })) { }	\
 		else
 
+#define for_each_unbound_cgpool(pool)					 \
+	list_for_each_entry_rcu((pool), &(unbound_cgpool), unbound_node)
+
+#define for_each_task_cgpool(cgtask, pool)				 \
+	list_for_each_entry_rcu((cgtask), &(pool)->cg_owners, link)
+
 #ifdef CONFIG_DEBUG_OBJECTS_WORK
 
 static struct debug_obj_descr work_debug_descr;
@@ -700,6 +721,7 @@ static struct pool_workqueue *get_work_pwq(struct work_struct *work)
  *
  * Return: The worker_pool @work was last associated with.  %NULL if none.
  */
+
 static struct worker_pool *get_work_pool(struct work_struct *work)
 {
 	unsigned long data = atomic_long_read(&work->data);
@@ -757,6 +779,7 @@ static bool work_is_canceling(struct work_struct *work)
  * they're being called with pool->lock held.
  */
 
+
 static bool __need_more_worker(struct worker_pool *pool)
 {
 	return !atomic_read(&pool->nr_running);
@@ -1072,6 +1095,7 @@ static void get_pwq(struct pool_workqueue *pwq)
 static void put_pwq(struct pool_workqueue *pwq)
 {
 	lockdep_assert_held(&pwq->pool->lock);
+
 	if (likely(--pwq->refcnt))
 		return;
 	if (WARN_ON_ONCE(!(pwq->wq->flags & WQ_UNBOUND)))
@@ -1387,6 +1411,9 @@ retry:
 	/* pwq which will be used unless @work is executing elsewhere */
 	if (!(wq->flags & WQ_UNBOUND))
 		pwq = per_cpu_ptr(wq->cpu_pwqs, cpu);
+	else if (wq->flags & WQ_CGROUPS)
+		/* use the default pwq */
+		pwq = unbound_pwq_by_node(wq, NUMA_NO_NODE);
 	else
 		pwq = unbound_pwq_by_node(wq, cpu_to_node(cpu));
 
@@ -1674,6 +1701,8 @@ static struct worker *alloc_worker(int node)
 		/* on creation a worker is in !idle && prep state */
 		worker->flags = WORKER_PREP;
 	}
+	worker->attach_pending = false;
+	worker->attach_to = NULL;
 	return worker;
 }
 
@@ -1695,7 +1724,8 @@ static void worker_attach_to_pool(struct worker *worker,
 	 * set_cpus_allowed_ptr() will fail if the cpumask doesn't have any
 	 * online CPUs.  It'll be re-applied when any of the CPUs come up.
 	 */
-	set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
+	if (!pool->attrs->cg_enabled)
+		set_cpus_allowed_ptr(worker->task, pool->attrs->cpumask);
 
 	/*
 	 * The pool->attach_mutex ensures %POOL_DISASSOCIATED remains
@@ -1760,6 +1790,7 @@ static struct worker *create_worker(struct worker_pool *pool)
 	if (id < 0)
 		goto fail;
 
+	/* Note: if user specified cgroups, node is NUMA_NO_NODE */
 	worker = alloc_worker(pool->node);
 	if (!worker)
 		goto fail;
@@ -1779,7 +1810,11 @@ static struct worker *create_worker(struct worker_pool *pool)
 		goto fail;
 
 	set_user_nice(worker->task, pool->attrs->nice);
-	kthread_bind_mask(worker->task, pool->attrs->cpumask);
+	if (pool->attrs->cg_enabled) {
+		worker->attach_pending = true;
+		worker->attach_to = current;
+	} else
+		kthread_bind_mask(worker->task, pool->attrs->cpumask);
 
 	/* successful, attach the worker to the pool */
 	worker_attach_to_pool(worker, pool);
@@ -2172,6 +2207,7 @@ static int worker_thread(void *__worker)
 {
 	struct worker *worker = __worker;
 	struct worker_pool *pool = worker->pool;
+	int cgattach;
 
 	/* tell the scheduler that this is a workqueue worker */
 	worker->task->flags |= PF_WQ_WORKER;
@@ -2191,6 +2227,14 @@ woke_up:
 		return 0;
 	}
 
+	/* this is supposed to run only the first time to attach to cgroups */
+	if (worker->attach_pending) {
+		cgattach = cgroup_attach_task_all(worker->attach_to, current);
+		if (cgattach)
+			pr_warn("workqueue: worker cgroup attach failed but we will still run!");
+		worker->attach_pending = false;
+	}
+
 	worker_leave_idle(worker);
 recheck:
 	/* no more worker necessary? */
@@ -3181,6 +3225,7 @@ static int init_worker_pool(struct worker_pool *pool)
 	pool->watchdog_ts = jiffies;
 	INIT_LIST_HEAD(&pool->worklist);
 	INIT_LIST_HEAD(&pool->idle_list);
+	INIT_LIST_HEAD(&pool->cg_owners);
 	hash_init(pool->busy_hash);
 
 	init_timer_deferrable(&pool->idle_timer);
@@ -3251,13 +3296,22 @@ static void put_unbound_pool(struct worker_pool *pool)
 
 	/* sanity checks */
 	if (WARN_ON(!(pool->cpu < 0)) ||
-	    WARN_ON(!list_empty(&pool->worklist)))
+	    WARN_ON(!list_empty(&pool->worklist)) ||
+	    WARN_ON(!list_empty(&pool->cg_owners)))
 		return;
 
 	/* release id and unhash */
 	if (pool->id >= 0)
 		idr_remove(&worker_pool_idr, pool->id);
-	hash_del(&pool->hash_node);
+
+	/*
+	 * this pool is going down, so remove from the list of
+	 * cgroup aware pools
+	 */
+	if (pool->attrs->cg_enabled)
+		list_del(&pool->unbound_node);
+	else
+		hash_del(&pool->hash_node);
 
 	/*
 	 * Become the manager and destroy all workers.  Grabbing
@@ -3290,6 +3344,65 @@ static void put_unbound_pool(struct worker_pool *pool)
 	call_rcu_sched(&pool->rcu, rcu_free_pool);
 }
 
+static void remove_task_cgpool(struct worker_pool *pool,
+			       struct task_struct *tsk)
+{
+	struct cgroup_owners *iter;
+
+	if (pool->attrs->cg_enabled) {
+		for_each_task_cgpool(iter, pool) {
+			if (iter->owner == tsk) {
+				list_del(&iter->link);
+				break;
+			}
+		}
+	}
+}
+
+static bool attach_task_cgpool(struct worker_pool *pool,
+			       struct task_struct *tsk)
+{
+	bool result = true;
+	struct cgroup_owners *entry = kzalloc(sizeof(*entry), GFP_KERNEL);
+
+	if (!entry) {
+		result = false;
+		goto done;
+	}
+
+	entry->owner = tsk;
+	list_add_tail(&entry->link, &pool->cg_owners);
+
+done:
+	return result;
+}
+
+static struct worker_pool *find_cg_matching_pool(struct task_struct *tsk)
+{
+	struct worker_pool *pool = NULL, *iter;
+	bool found = false;
+
+	for_each_unbound_cgpool(iter) {
+		struct cgroup_owners *cgtask;
+
+		for_each_task_cgpool(cgtask, iter) {
+			if (cgtask->owner == tsk ||
+			    cgroup_match_groups(cgtask->owner, tsk)) {
+				found = true;
+				break;
+			}
+		}
+
+		if (found) {
+			pool = iter;
+			pool->refcnt++;
+			break;
+		}
+	}
+
+	return pool;
+}
+
 /**
  * get_unbound_pool - get a worker_pool with the specified attributes
  * @attrs: the attributes of the worker_pool to get
@@ -3310,9 +3423,19 @@ static struct worker_pool *get_unbound_pool(const struct workqueue_attrs *attrs)
 	struct worker_pool *pool;
 	int node;
 	int target_node = NUMA_NO_NODE;
+	bool cgroups_enabled = attrs->cg_enabled;
 
 	lockdep_assert_held(&wq_pool_mutex);
 
+	if (cgroups_enabled) {
+		/* "current" is the owner */
+		pool = find_cg_matching_pool(current);
+		if (!pool)
+			goto create;
+		else
+			return pool;
+	}
+
 	/* do we already have a matching pool? */
 	hash_for_each_possible(unbound_pool_hash, pool, hash_node, hash) {
 		if (wqattrs_equal(pool->attrs, attrs)) {
@@ -3332,6 +3455,8 @@ static struct worker_pool *get_unbound_pool(const struct workqueue_attrs *attrs)
 		}
 	}
 
+
+create:
 	/* nope, create a new one */
 	pool = kzalloc_node(sizeof(*pool), GFP_KERNEL, target_node);
 	if (!pool || init_worker_pool(pool) < 0)
@@ -3347,6 +3472,9 @@ static struct worker_pool *get_unbound_pool(const struct workqueue_attrs *attrs)
 	 */
 	pool->attrs->no_numa = false;
 
+	if (cgroups_enabled)
+		pool->attrs->cg_enabled = true;
+
 	if (worker_pool_assign_id(pool) < 0)
 		goto fail;
 
@@ -3355,7 +3483,10 @@ static struct worker_pool *get_unbound_pool(const struct workqueue_attrs *attrs)
 		goto fail;
 
 	/* install */
-	hash_add(unbound_pool_hash, &pool->hash_node, hash);
+	if (cgroups_enabled)
+		list_add_tail(&pool->unbound_node, &unbound_cgpool);
+	else
+		hash_add(unbound_pool_hash, &pool->hash_node, hash);
 
 	return pool;
 fail:
@@ -3390,6 +3521,8 @@ static void pwq_unbound_release_workfn(struct work_struct *work)
 	is_last = list_empty(&wq->pwqs);
 	mutex_unlock(&wq->mutex);
 
+	remove_task_cgpool(pool, pwq->owner);
+
 	mutex_lock(&wq_pool_mutex);
 	put_unbound_pool(pool);
 	mutex_unlock(&wq_pool_mutex);
@@ -3462,6 +3595,11 @@ static void init_pwq(struct pool_workqueue *pwq, struct workqueue_struct *wq,
 	pwq->wq = wq;
 	pwq->flush_color = -1;
 	pwq->refcnt = 1;
+	if (pool->attrs->cg_enabled) {
+		/* Add the current task to pool cg_owners */
+		WARN_ON(!attach_task_cgpool(pool, current));
+		pwq->owner = current;
+	}
 	INIT_LIST_HEAD(&pwq->delayed_works);
 	INIT_LIST_HEAD(&pwq->pwqs_node);
 	INIT_LIST_HEAD(&pwq->mayday_node);
@@ -3502,7 +3640,11 @@ static struct pool_workqueue *alloc_unbound_pwq(struct workqueue_struct *wq,
 	if (!pool)
 		return NULL;
 
-	pwq = kmem_cache_alloc_node(pwq_cache, GFP_KERNEL, pool->node);
+	if (wq->unbound_attrs->cg_enabled)
+		pwq = kmem_cache_alloc(pwq_cache, GFP_KERNEL);
+	else
+		pwq = kmem_cache_alloc_node(pwq_cache, GFP_KERNEL, pool->node);
+
 	if (!pwq) {
 		put_unbound_pool(pool);
 		return NULL;
@@ -3590,8 +3732,10 @@ static void apply_wqattrs_cleanup(struct apply_wqattrs_ctx *ctx)
 	if (ctx) {
 		int node;
 
-		for_each_node(node)
-			put_pwq_unlocked(ctx->pwq_tbl[node]);
+		if (ctx->attrs->cg_enabled) {
+			for_each_node(node)
+				put_pwq_unlocked(ctx->pwq_tbl[node]);
+		}
 		put_pwq_unlocked(ctx->dfl_pwq);
 
 		free_workqueue_attrs(ctx->attrs);
@@ -3607,11 +3751,14 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
 {
 	struct apply_wqattrs_ctx *ctx;
 	struct workqueue_attrs *new_attrs, *tmp_attrs;
-	int node;
+	int node, numa_nodes = nr_node_ids;
+	bool cgroups_enabled = wq->unbound_attrs->cg_enabled;
 
 	lockdep_assert_held(&wq_pool_mutex);
 
-	ctx = kzalloc(sizeof(*ctx) + nr_node_ids * sizeof(ctx->pwq_tbl[0]),
+	if (cgroups_enabled)
+		numa_nodes = 0;
+	ctx = kzalloc(sizeof(*ctx) + numa_nodes * sizeof(ctx->pwq_tbl[0]),
 		      GFP_KERNEL);
 
 	new_attrs = alloc_workqueue_attrs(GFP_KERNEL);
@@ -3623,6 +3770,7 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
 	 * Calculate the attrs of the default pwq.
 	 * If the user configured cpumask doesn't overlap with the
 	 * wq_unbound_cpumask, we fallback to the wq_unbound_cpumask.
+	 * This does not copy attrs->cg_enabled
 	 */
 	copy_workqueue_attrs(new_attrs, attrs);
 	cpumask_and(new_attrs->cpumask, new_attrs->cpumask, wq_unbound_cpumask);
@@ -3640,11 +3788,16 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
 	 * If something goes wrong during CPU up/down, we'll fall back to
 	 * the default pwq covering whole @attrs->cpumask.  Always create
 	 * it even if we don't use it immediately.
+	 * For cgroups aware wqs, there will be on only one pwq
 	 */
+	new_attrs->cg_enabled = cgroups_enabled;
 	ctx->dfl_pwq = alloc_unbound_pwq(wq, new_attrs);
 	if (!ctx->dfl_pwq)
 		goto out_free;
 
+	if (cgroups_enabled)
+		goto done;
+
 	for_each_node(node) {
 		if (wq_calc_node_cpumask(new_attrs, node, -1, tmp_attrs->cpumask)) {
 			ctx->pwq_tbl[node] = alloc_unbound_pwq(wq, tmp_attrs);
@@ -3656,8 +3809,10 @@ apply_wqattrs_prepare(struct workqueue_struct *wq,
 		}
 	}
 
+done:
 	/* save the user configured attrs and sanitize it. */
 	copy_workqueue_attrs(new_attrs, attrs);
+	/* at this point, note that cg_enabled is untouched */
 	cpumask_and(new_attrs->cpumask, new_attrs->cpumask, cpu_possible_mask);
 	ctx->attrs = new_attrs;
 
@@ -3676,16 +3831,23 @@ out_free:
 static void apply_wqattrs_commit(struct apply_wqattrs_ctx *ctx)
 {
 	int node;
+	bool cgroups_enabled = ctx->wq->unbound_attrs->cg_enabled;
 
 	/* all pwqs have been created successfully, let's install'em */
 	mutex_lock(&ctx->wq->mutex);
 
 	copy_workqueue_attrs(ctx->wq->unbound_attrs, ctx->attrs);
 
-	/* save the previous pwq and install the new one */
+	/*
+	 * save the previous pwq and install the new one
+	 * if WQ_CGROUPS is set, then we don't allocate space for pwq_tbl at all
+	 * so in that case, only dfl_pwq is valid
+	 */
+	if (!cgroups_enabled) {
 	for_each_node(node)
 		ctx->pwq_tbl[node] = numa_pwq_tbl_install(ctx->wq, node,
 							  ctx->pwq_tbl[node]);
+	}
 
 	/* @dfl_pwq might not have been used, ensure it's linked */
 	link_pwq(ctx->dfl_pwq);
@@ -3882,6 +4044,7 @@ static int alloc_and_link_pwqs(struct workqueue_struct *wq)
 static int wq_clamp_max_active(int max_active, unsigned int flags,
 			       const char *name)
 {
+	/* Determine max for cgroups ? */
 	int lim = flags & WQ_UNBOUND ? WQ_UNBOUND_MAX_ACTIVE : WQ_MAX_ACTIVE;
 
 	if (max_active < 1 || max_active > lim)
@@ -3901,14 +4064,30 @@ struct workqueue_struct *__alloc_workqueue_key(const char *fmt,
 	va_list args;
 	struct workqueue_struct *wq;
 	struct pool_workqueue *pwq;
+	bool cgroups_enabled = false;
+
+#ifdef CONFIG_CGROUPS
+	/* Only unbound workqueues but not ordered */
+	if ((flags & WQ_CGROUPS) && (flags & WQ_UNBOUND) &&
+	    !(flags & __WQ_ORDERED))
+		cgroups_enabled = true;
+#endif
 
 	/* see the comment above the definition of WQ_POWER_EFFICIENT */
-	if ((flags & WQ_POWER_EFFICIENT) && wq_power_efficient)
+	if ((flags & WQ_POWER_EFFICIENT) && wq_power_efficient) {
 		flags |= WQ_UNBOUND;
+		if (cgroups_enabled) {
+			pr_warn("workqueue: disabling cgroups because WQ_POWER_EFFICIENT specified");
+			cgroups_enabled = false;
+		}
+	}
 
 	/* allocate wq and format name */
-	if (flags & WQ_UNBOUND)
-		tbl_size = nr_node_ids * sizeof(wq->numa_pwq_tbl[0]);
+	if (flags & WQ_UNBOUND) {
+		if (!cgroups_enabled)
+			tbl_size = nr_node_ids * sizeof(wq->numa_pwq_tbl[0]);
+		/* else let cgroups take care of us */
+	}
 
 	wq = kzalloc(sizeof(*wq) + tbl_size, GFP_KERNEL);
 	if (!wq)
@@ -3918,6 +4097,8 @@ struct workqueue_struct *__alloc_workqueue_key(const char *fmt,
 		wq->unbound_attrs = alloc_workqueue_attrs(GFP_KERNEL);
 		if (!wq->unbound_attrs)
 			goto err_free_wq;
+		if (cgroups_enabled)
+			wq->unbound_attrs->cg_enabled = true;
 	}
 
 	va_start(args, lock_name);
@@ -4980,6 +5161,9 @@ static ssize_t wq_pool_ids_show(struct device *dev,
 	const char *delim = "";
 	int node, written = 0;
 
+	if (wq->unbound_attrs->cg_enabled)
+		return 0;
+
 	rcu_read_lock_sched();
 	for_each_node(node) {
 		written += scnprintf(buf + written, PAGE_SIZE - written,
diff --git a/kernel/workqueue_internal.h b/kernel/workqueue_internal.h
index 4521587..49228cab 100644
--- a/kernel/workqueue_internal.h
+++ b/kernel/workqueue_internal.h
@@ -52,6 +52,10 @@ struct worker {
 
 	/* used only by rescuers to point to the target workqueue */
 	struct workqueue_struct	*rescue_wq;	/* I: the workqueue to rescue */
+
+	/* for cgroups */
+	bool attach_pending;
+	struct task_struct *attach_to;
 };
 
 /**
-- 
2.5.0

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

* [RFC PATCH 3/4] cgroup: use spin_lock_irq for cgroup match and attach fns
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 1/4] cgroup: Introduce a function to compare two tasks Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 2/4] workqueue: introduce support for attaching to cgroups Bandan Das
@ 2016-03-18 22:14 ` Bandan Das
  2016-03-18 22:14 ` [RFC PATCH 4/4] vhost: use workqueues for the works Bandan Das
                   ` (5 subsequent siblings)
  8 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-18 22:14 UTC (permalink / raw)
  To: tj; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

Since these functions will be called from the worker
thread context, using spin_lock_bh triggers WARN_ONs.

Signed-off-by: Bandan Das <bsd@redhat.com>
---
 kernel/cgroup.c | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/kernel/cgroup.c b/kernel/cgroup.c
index 3ffdbb4..24b34e8 100644
--- a/kernel/cgroup.c
+++ b/kernel/cgroup.c
@@ -2764,10 +2764,10 @@ bool cgroup_match_groups(struct task_struct *tsk1, struct task_struct *tsk2)
 		if (!root->subsys_mask)
 			continue;
 
-		spin_lock_bh(&css_set_lock);
+		spin_lock_irq(&css_set_lock);
 		cg_tsk1 = task_cgroup_from_root(tsk1, root);
 		cg_tsk2 = task_cgroup_from_root(tsk2, root);
-		spin_unlock_bh(&css_set_lock);
+		spin_unlock_irq(&css_set_lock);
 
 		if (cg_tsk1 != cg_tsk2) {
 			result = false;
@@ -2797,9 +2797,9 @@ int cgroup_attach_task_all(struct task_struct *from, struct task_struct *tsk)
 		if (root == &cgrp_dfl_root)
 			continue;
 
-		spin_lock_bh(&css_set_lock);
+		spin_lock_irq(&css_set_lock);
 		from_cgrp = task_cgroup_from_root(from, root);
-		spin_unlock_bh(&css_set_lock);
+		spin_unlock_irq(&css_set_lock);
 
 		retval = cgroup_attach_task(from_cgrp, tsk, false);
 		if (retval)
-- 
2.5.0

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

* [RFC PATCH 4/4] vhost: use workqueues for the works
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
                   ` (2 preceding siblings ...)
  2016-03-18 22:14 ` [RFC PATCH 3/4] cgroup: use spin_lock_irq for cgroup match and attach fns Bandan Das
@ 2016-03-18 22:14 ` Bandan Das
  2016-03-20 18:10 ` [RFC PATCH 0/4] cgroup aware workqueues Tejun Heo
                   ` (4 subsequent siblings)
  8 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-18 22:14 UTC (permalink / raw)
  To: tj; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

Use the common workqueue machanism for processing vhost work
by creating a unbound workqueue/vhost device. The backend workers
could still share work and are not visible to vhost.

Signed-off-by: Bandan Das <bsd@redhat.com>
---
 drivers/vhost/vhost.c | 103 +++++++++++++++++++++++++++++++++++++++++++-------
 drivers/vhost/vhost.h |   2 +
 2 files changed, 92 insertions(+), 13 deletions(-)

diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
index ad2146a..162e25e 100644
--- a/drivers/vhost/vhost.c
+++ b/drivers/vhost/vhost.c
@@ -30,6 +30,10 @@
 
 #include "vhost.h"
 
+static int cmwq_worker = 1;
+module_param(cmwq_worker, int, 0444);
+MODULE_PARM_DESC(cmwq_worker, "Use cmwq for worker threads - Experimental, 1 - Enable; 0 - Disable");
+
 static ushort max_mem_regions = 64;
 module_param(max_mem_regions, ushort, 0444);
 MODULE_PARM_DESC(max_mem_regions,
@@ -238,7 +242,10 @@ void vhost_work_queue(struct vhost_dev *dev, struct vhost_work *work)
 		list_add_tail(&work->node, &dev->work_list);
 		work->queue_seq++;
 		spin_unlock_irqrestore(&dev->work_lock, flags);
-		wake_up_process(dev->worker);
+		if (cmwq_worker)
+			queue_work(dev->qworker, &dev->qwork);
+		else
+			wake_up_process(dev->worker);
 	} else {
 		spin_unlock_irqrestore(&dev->work_lock, flags);
 	}
@@ -370,6 +377,52 @@ static void vhost_dev_free_iovecs(struct vhost_dev *dev)
 		vhost_vq_free_iovecs(dev->vqs[i]);
 }
 
+static void vhost_wq_worker(struct work_struct *qwork)
+{
+	struct vhost_dev *dev =
+		container_of(qwork, struct vhost_dev, qwork);
+	struct vhost_work *work = NULL;
+	unsigned uninitialized_var(seq);
+	struct mm_struct *prev_mm = NULL;
+	mm_segment_t oldfs = get_fs();
+
+	set_fs(USER_DS);
+
+	for (;;) {
+		spin_lock_irq(&dev->work_lock);
+		if (list_empty(&dev->work_list)) {
+			spin_unlock(&dev->work_lock);
+			break;
+		}
+
+		work = list_first_entry(&dev->work_list,
+					struct vhost_work, node);
+		list_del_init(&work->node);
+		seq = work->queue_seq;
+
+		if (prev_mm != dev->mm) {
+			if (prev_mm)
+				unuse_mm(prev_mm);
+			prev_mm = dev->mm;
+			use_mm(prev_mm);
+		}
+		spin_unlock(&dev->work_lock);
+
+		if (work) {
+			work->fn(work);
+			spin_lock_irq(&dev->work_lock);
+			work->done_seq = seq;
+			if (work->flushing)
+				wake_up_all(&work->done);
+			spin_unlock_irq(&dev->work_lock);
+		}
+	}
+
+	if (prev_mm)
+		unuse_mm(prev_mm);
+	set_fs(oldfs);
+}
+
 void vhost_dev_init(struct vhost_dev *dev,
 		    struct vhost_virtqueue **vqs, int nvqs)
 {
@@ -386,6 +439,10 @@ void vhost_dev_init(struct vhost_dev *dev,
 	spin_lock_init(&dev->work_lock);
 	INIT_LIST_HEAD(&dev->work_list);
 	dev->worker = NULL;
+	dev->qworker = NULL;
+
+	if (cmwq_worker)
+		INIT_WORK(&dev->qwork, vhost_wq_worker);
 
 	for (i = 0; i < dev->nvqs; ++i) {
 		vq = dev->vqs[i];
@@ -445,7 +502,8 @@ EXPORT_SYMBOL_GPL(vhost_dev_has_owner);
 /* Caller should have device mutex */
 long vhost_dev_set_owner(struct vhost_dev *dev)
 {
-	struct task_struct *worker;
+	struct task_struct *worker = NULL;
+	struct workqueue_struct *qworker;
 	int err;
 
 	/* Is there an owner already? */
@@ -456,18 +514,31 @@ long vhost_dev_set_owner(struct vhost_dev *dev)
 
 	/* No owner, become one */
 	dev->mm = get_task_mm(current);
-	worker = kthread_create(vhost_worker, dev, "vhost-%d", current->pid);
-	if (IS_ERR(worker)) {
-		err = PTR_ERR(worker);
-		goto err_worker;
-	}
+	if (cmwq_worker) {
+		qworker = alloc_workqueue("vhost-wq-%d",
+					  WQ_UNBOUND|WQ_CGROUPS,
+					  0, current->pid);
+		if (!qworker) {
+			err = -ENOMEM;
+			goto err_worker;
+		}
+		dev->qworker = qworker;
+	} else {
+		worker = kthread_create(vhost_worker, dev,
+					"vhost-%d", current->pid);
+		if (IS_ERR(worker)) {
+			err = PTR_ERR(worker);
+			goto err_worker;
+		}
 
-	dev->worker = worker;
-	wake_up_process(worker);	/* avoid contributing to loadavg */
+		dev->worker = worker;
+		/* avoid contributing to loadavg */
+		wake_up_process(worker);
 
-	err = vhost_attach_cgroups(dev);
-	if (err)
-		goto err_cgroup;
+		err = vhost_attach_cgroups(dev);
+		if (err)
+			goto err_cgroup;
+	}
 
 	err = vhost_dev_alloc_iovecs(dev);
 	if (err)
@@ -475,7 +546,8 @@ long vhost_dev_set_owner(struct vhost_dev *dev)
 
 	return 0;
 err_cgroup:
-	kthread_stop(worker);
+	if (worker)
+		kthread_stop(worker);
 	dev->worker = NULL;
 err_worker:
 	if (dev->mm)
@@ -556,6 +628,11 @@ void vhost_dev_cleanup(struct vhost_dev *dev, bool locked)
 		kthread_stop(dev->worker);
 		dev->worker = NULL;
 	}
+	if (dev->qworker) {
+		/* destroy does flush */
+		destroy_workqueue(dev->qworker);
+		dev->qworker = NULL;
+	}
 	if (dev->mm)
 		mmput(dev->mm);
 	dev->mm = NULL;
diff --git a/drivers/vhost/vhost.h b/drivers/vhost/vhost.h
index d3f7674..e2ce0c3 100644
--- a/drivers/vhost/vhost.h
+++ b/drivers/vhost/vhost.h
@@ -127,6 +127,8 @@ struct vhost_dev {
 	spinlock_t work_lock;
 	struct list_head work_list;
 	struct task_struct *worker;
+	struct workqueue_struct *qworker;
+	struct work_struct qwork;
 };
 
 void vhost_dev_init(struct vhost_dev *, struct vhost_virtqueue **vqs, int nvqs);
-- 
2.5.0

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
                   ` (3 preceding siblings ...)
  2016-03-18 22:14 ` [RFC PATCH 4/4] vhost: use workqueues for the works Bandan Das
@ 2016-03-20 18:10 ` Tejun Heo
  2016-03-21 17:35   ` Bandan Das
  2016-03-21  7:58 ` Michael Rapoport
                   ` (3 subsequent siblings)
  8 siblings, 1 reply; 23+ messages in thread
From: Tejun Heo @ 2016-03-20 18:10 UTC (permalink / raw)
  To: Bandan Das; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

Hello,

On Fri, Mar 18, 2016 at 06:14:47PM -0400, Bandan Das wrote:
> These changes don't populate the "numa awareness" fields/attrs and
> unlike unbounded numa worker pools, cgroup worker pools are created
> on demand. Every work request could potentially have a new cgroup

Hmmm... I don't get it.  Why would this be exclusive with numa
support?  Can't cgroup be just another attribute in addition to numa?

> aware pool created for it based on the combination of cgroups it's attached
> to. However, workqueues themselves are incognizant of the actual cgroups -
> they rely on the cgroups provided helper functions either for 1. a match
> of all the cgroups or 2. to attach a worker thread to all cgroups of
> a userspace task. We do maintain a list of cgroup aware pools so that
> when a new request comes in and a suitable worker pool needs to be
> found, we search the list first before creating a new one. A worker
> pool also stores a a list of all "task owners" - a list of processes
> that we are serving currently.

Why is this separate from the normal lookup mechanism?  Can't it be
hashed together?

> Todo:
> What about bounded workqueues ?

I don't think it'd matter.  This is only interesting for work items
which may consume a significant amount of resources, which shouldn't
be served by per-cpu workers anyway.

> What happens when cgroups of a running process changes ?

Existing work items will be served with the old association.  New work
items will be served with the new association.  This is consistent
with how other attributes are handled too.

> Better performance numbers ? (although the onese above don't look bad)

Where is performance regression coming from?  Why is there *any*
performance penalty?

Thanks.

-- 
tejun

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
                   ` (4 preceding siblings ...)
  2016-03-20 18:10 ` [RFC PATCH 0/4] cgroup aware workqueues Tejun Heo
@ 2016-03-21  7:58 ` Michael Rapoport
  2016-03-21  8:29 ` Christian Borntraeger
                   ` (2 subsequent siblings)
  8 siblings, 0 replies; 23+ messages in thread
From: Michael Rapoport @ 2016-03-21  7:58 UTC (permalink / raw)
  To: Bandan Das; +Cc: tj, linux-kernel, kvm, mst, jiangshanlai

Hi Bandan,

> From: Bandan Das <bsd@redhat.com>
> 
> At Linuxcon last year, based on our presentation "vhost: sharing is 
better" [1],
> we had briefly discussed the idea of cgroup aware workqueues with Tejun. 
The
> following patches are a result of the discussion. They are in no way 
complete in
> that the changes are for unbounded workqueues only, but I just wanted to 
present my
> unfinished work as RFC and get some feedback.
> 
> 1/4 and 3/4 are simple cgroup changes and add a helper function.
> 2/4 is the main implementation.
> 4/4 changes vhost to use workqueues with support for cgroups.
>
> Example:
> vhost creates a worker thread when invoked for a kvm guest. Since,
> the guest is a normal process, the kernel thread servicing it should be
> attached to the vm process' cgroups.

I did some performance evaluation of different threading models in vhost, 
and in most tests replacing vhost kthread's with workqueues degrades the 
performance. Moreover, having thread management inside the vhost provides 
opportunity for optimization, at least for some workloads...
That said, I believe that switching vhost to use workqueues is not that 
good idea after all.
 
> Netperf:
> Two guests running netperf in parallel.
>                                  Without patches                  With 
patches
> 
> TCP_STREAM (10^6 bits/second)         975.45              978.88 
> TCP_RR (Trans/second)            20121              18820.82
> UDP_STREAM (10^6 bits/second)         1287.82                1184.5
> UDP_RR (Trans/second)            20766.72              19667.08
> Time a 4G iso download            2m 33 seconds           3m 02 seconds

--
Sincerely yours,
Mike.

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
                   ` (5 preceding siblings ...)
  2016-03-21  7:58 ` Michael Rapoport
@ 2016-03-21  8:29 ` Christian Borntraeger
  2016-03-21 17:49   ` Bandan Das
       [not found] ` <201603210758.u2L7wiXA028101@d06av09.portsmouth.uk.ibm.com>
       [not found] ` <201603210758.u2L7wiY9003907@d06av07.portsmouth.uk.ibm.com>
  8 siblings, 1 reply; 23+ messages in thread
From: Christian Borntraeger @ 2016-03-21  8:29 UTC (permalink / raw)
  To: Bandan Das, tj; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

On 03/18/2016 11:14 PM, Bandan Das wrote:
[..]
> Netperf:
> Two guests running netperf in parallel.
>     	   	       	          Without patches                  With patches
>    
> TCP_STREAM (10^6 bits/second)         975.45			     978.88	
> TCP_RR (Trans/second)		      20121			     18820.82
> UDP_STREAM (10^6 bits/second)	      1287.82		  	     1184.5
> UDP_RR (Trans/second)		      20766.72			     19667.08
> Time a 4G iso download		      2m 33 seconds		     3m 02 seconds

So TCP stream stays the same everything else shows a regression? Not good.
Have you an idea why this happens?

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-20 18:10 ` [RFC PATCH 0/4] cgroup aware workqueues Tejun Heo
@ 2016-03-21 17:35   ` Bandan Das
  0 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-21 17:35 UTC (permalink / raw)
  To: Tejun Heo; +Cc: linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

Tejun Heo <tj@kernel.org> writes:

> Hello,
>
> On Fri, Mar 18, 2016 at 06:14:47PM -0400, Bandan Das wrote:
>> These changes don't populate the "numa awareness" fields/attrs and
>> unlike unbounded numa worker pools, cgroup worker pools are created
>> on demand. Every work request could potentially have a new cgroup
>
> Hmmm... I don't get it.  Why would this be exclusive with numa
> support?  Can't cgroup be just another attribute in addition to numa?

Yes, I think it can. I am not certain what would be a good representation
of the cgroup information; maybe, all cgroups could be represented by just a
simple bitmap just like numa attrs ? The other thing that was on my mind is
what happens when there's no intersection between the cgroups
of a task and the numa locality. For example, if node 1 with cpus 0,1,2,3 is local to
task A but it's cgroups want to attach to cpus 4-5, then who wins in this case ?
Or a simple logic would be to always attach to cgroups as the last step.


>> aware pool created for it based on the combination of cgroups it's attached
>> to. However, workqueues themselves are incognizant of the actual cgroups -
>> they rely on the cgroups provided helper functions either for 1. a match
>> of all the cgroups or 2. to attach a worker thread to all cgroups of
>> a userspace task. We do maintain a list of cgroup aware pools so that
>> when a new request comes in and a suitable worker pool needs to be
>> found, we search the list first before creating a new one. A worker
>> pool also stores a a list of all "task owners" - a list of processes
>> that we are serving currently.
>
> Why is this separate from the normal lookup mechanism?  Can't it be
> hashed together?
>
>> Todo:
>> What about bounded workqueues ?
>
> I don't think it'd matter.  This is only interesting for work items
> which may consume a significant amount of resources, which shouldn't
> be served by per-cpu workers anyway.

Ok.

>> What happens when cgroups of a running process changes ?
>
> Existing work items will be served with the old association.  New work
> items will be served with the new association.  This is consistent
> with how other attributes are handled too.

In the current implementation, the cgroup info is fetched just once
when alloc_workqueue is called. So, there's no way of knowing if the
cgroups changed. Maybe I should rethink this too.

>> Better performance numbers ? (although the onese above don't look bad)
>
> Where is performance regression coming from?  Why is there *any*
> performance penalty?

I am still investigating this but creating more worker threads could
be one.

/* do  we need to manage? */                                                                                                                                                                                
        if (unlikely(!may_start_working(pool)) && manage_workers(worker))
           goto recheck;                         

Since all work gets queued to the default pwq in this implementation,
we do end up creating workers in the middle of a run.

> Thanks.

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
       [not found] ` <201603210758.u2L7wiXA028101@d06av09.portsmouth.uk.ibm.com>
@ 2016-03-21 17:43   ` Bandan Das
  2016-03-22  7:12     ` vhost threading model (was: Re: [RFC PATCH 0/4] cgroup aware workqueues) Michael Rapoport
       [not found]     ` <201603220712.u2M7CCfq004548@d06av03.portsmouth.uk.ibm.com>
  0 siblings, 2 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-21 17:43 UTC (permalink / raw)
  To: Michael Rapoport; +Cc: tj, linux-kernel, kvm, mst, jiangshanlai

"Michael Rapoport" <RAPOPORT@il.ibm.com> writes:

> Hi Bandan,
>
>> From: Bandan Das <bsd@redhat.com>
>> 
>> At Linuxcon last year, based on our presentation "vhost: sharing is 
> better" [1],
>> we had briefly discussed the idea of cgroup aware workqueues with Tejun. 
> The
>> following patches are a result of the discussion. They are in no way 
> complete in
>> that the changes are for unbounded workqueues only, but I just wanted to 
> present my
>> unfinished work as RFC and get some feedback.
>> 
>> 1/4 and 3/4 are simple cgroup changes and add a helper function.
>> 2/4 is the main implementation.
>> 4/4 changes vhost to use workqueues with support for cgroups.
>>
>> Example:
>> vhost creates a worker thread when invoked for a kvm guest. Since,
>> the guest is a normal process, the kernel thread servicing it should be
>> attached to the vm process' cgroups.
>
> I did some performance evaluation of different threading models in vhost, 
> and in most tests replacing vhost kthread's with workqueues degrades the

Workqueues us kthread_create internally and if calling one over the
other impacts performace, I think we should investigate that. Which
patches did you use ? Note that an earlier version of workqueue patches
that I posted used per-cpu workqueues.

> performance. Moreover, having thread management inside the vhost provides

What exactly is the advantage doing our own thread management ? Do you have
any examples ? (Besides for doing our own scheduling like in the original Elvis
paper which I don't think is gonna happen). Also, note here that there is
a possibility to affect how our work gets executed by using optional switches to
alloc_workqueue() so all is not lost.

> opportunity for optimization, at least for some workloads...
> That said, I believe that switching vhost to use workqueues is not that 
> good idea after all.
>  
>> Netperf:
>> Two guests running netperf in parallel.
>>                                  Without patches                  With 
> patches
>> 
>> TCP_STREAM (10^6 bits/second)         975.45              978.88 
>> TCP_RR (Trans/second)            20121              18820.82
>> UDP_STREAM (10^6 bits/second)         1287.82                1184.5
>> UDP_RR (Trans/second)            20766.72              19667.08
>> Time a 4G iso download            2m 33 seconds           3m 02 seconds
>
> --
> Sincerely yours,
> Mike.
>
> --
> To unsubscribe from this list: send the line "unsubscribe kvm" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-21  8:29 ` Christian Borntraeger
@ 2016-03-21 17:49   ` Bandan Das
  0 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-21 17:49 UTC (permalink / raw)
  To: Christian Borntraeger; +Cc: tj, linux-kernel, kvm, mst, jiangshanlai, RAPOPORT

Christian Borntraeger <borntraeger@de.ibm.com> writes:

> On 03/18/2016 11:14 PM, Bandan Das wrote:
> [..]
>> Netperf:
>> Two guests running netperf in parallel.
>>     	   	       	          Without patches                  With patches
>>    
>> TCP_STREAM (10^6 bits/second)         975.45			     978.88	
>> TCP_RR (Trans/second)		      20121			     18820.82
>> UDP_STREAM (10^6 bits/second)	      1287.82		  	     1184.5
>> UDP_RR (Trans/second)		      20766.72			     19667.08
>> Time a 4G iso download		      2m 33 seconds		     3m 02 seconds
>
> So TCP stream stays the same everything else shows a regression? Not good.
> Have you an idea why this happens?

I am not sure yet but my guess is the way these patches implement cgroup
support. I will run some tests just with workqueues (and without these patches) and
see if the newer numbers are consistent with these.

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

* vhost threading model (was: Re: [RFC PATCH 0/4] cgroup aware workqueues)
  2016-03-21 17:43   ` Bandan Das
@ 2016-03-22  7:12     ` Michael Rapoport
       [not found]     ` <201603220712.u2M7CCfq004548@d06av03.portsmouth.uk.ibm.com>
  1 sibling, 0 replies; 23+ messages in thread
From: Michael Rapoport @ 2016-03-22  7:12 UTC (permalink / raw)
  To: Bandan Das; +Cc: jiangshanlai, kvm, linux-kernel, mst, tj

>  Bandan Das <bsd@redhat.com> wrote on 03/21/2016 07:43:41 PM:
> > "Michael Rapoport" <RAPOPORT@il.ibm.com> writes:
> >
> > Hi Bandan,
> >
> >> From: Bandan Das <bsd@redhat.com>
> >> 
> >> At Linuxcon last year, based on our presentation "vhost: sharing is 
> > better" [1],
> >> we had briefly discussed the idea of cgroup aware workqueues with 
Tejun. 
> > The
> >> following patches are a result of the discussion. They are in no way 
> > complete in
> >> that the changes are for unbounded workqueues only, but I just wanted 
to 
> > present my
> >> unfinished work as RFC and get some feedback.
> >> 
> >> 1/4 and 3/4 are simple cgroup changes and add a helper function.
> >> 2/4 is the main implementation.
> >> 4/4 changes vhost to use workqueues with support for cgroups.
> >>
> >> Example:
> >> vhost creates a worker thread when invoked for a kvm guest. Since,
> >> the guest is a normal process, the kernel thread servicing it should 
be
> >> attached to the vm process' cgroups.
> >
> > I did some performance evaluation of different threading models in 
vhost, 
> > and in most tests replacing vhost kthread's with workqueues degrades 
the
> 
> Workqueues us kthread_create internally and if calling one over the
> other impacts performace, I think we should investigate that.

Agree. Didn't have time to do it myself yet...

> Which patches did you use ? Note that an earlier version of workqueue 
patches
> that I posted used per-cpu workqueues.

I've used your earlier version of workqueue patches, then I modified it to 
use unbound workqueues, and then I even restored to some extent original 
vhost workqueue usage. In all the cases I saw performance degradation 
relatively to the baseline.
 
> > performance. Moreover, having thread management inside the vhost 
provides
> 
> What exactly is the advantage doing our own thread management ? Do you 
have
> any examples ? (Besides for doing our own scheduling like in the 
original Elvis
> paper which I don't think is gonna happen). Also, note here that there 
is
> a possibility to affect how our work gets executed by using optional 
switches to
> alloc_workqueue() so all is not lost.

Well, Elvis is a _theoretical_ example that showed that I/O scheduling in 
the vhost improves performance.
I'm not saying we should take Evlis and try to squeeze it into the vhost, 
I just want to say that we cannot switch vhost to use workqueues if it 
causes performance degradation.

My opinion is that we need to give it some more thought, much more 
performance evaluation, so that we can find the best model.
 
> > opportunity for optimization, at least for some workloads...
> > That said, I believe that switching vhost to use workqueues is not 
that 
> > good idea after all.
> > 

--
Sincerely yours,
Mike.

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

* Re: vhost threading model
       [not found]     ` <201603220712.u2M7CCfq004548@d06av03.portsmouth.uk.ibm.com>
@ 2016-03-22 19:00       ` Bandan Das
  2016-03-23 11:13         ` Michael Rapoport
  0 siblings, 1 reply; 23+ messages in thread
From: Bandan Das @ 2016-03-22 19:00 UTC (permalink / raw)
  To: Michael Rapoport; +Cc: jiangshanlai, kvm, linux-kernel, mst, tj

"Michael Rapoport" <RAPOPORT@il.ibm.com> writes:
...
>
> Well, Elvis is a _theoretical_ example that showed that I/O scheduling in 
> the vhost improves performance.
> I'm not saying we should take Evlis and try to squeeze it into the vhost, 
> I just want to say that we cannot switch vhost to use workqueues if it 
> causes performance degradation.
>
> My opinion is that we need to give it some more thought, much more 
> performance evaluation, so that we can find the best model.

Exactly, I think we are outright discarding using workqueues even
without investigating it in detail even though it would be a cleaner
implementation using a common framework and thereby more chances of
an acceptable solution for upstream.

Anyway, if we don't want to go the workqueues way for vhost, cgroups
support for workqueues is still something worth having on its own.

>> > opportunity for optimization, at least for some workloads...
>> > That said, I believe that switching vhost to use workqueues is not 
> that 
>> > good idea after all.
>> > 
>
> --
> Sincerely yours,
> Mike.
>
> --
> To unsubscribe from this list: send the line "unsubscribe kvm" in
> the body of a message to majordomo@vger.kernel.org
> More majordomo info at  http://vger.kernel.org/majordomo-info.html

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

* Re: vhost threading model
  2016-03-22 19:00       ` vhost threading model Bandan Das
@ 2016-03-23 11:13         ` Michael Rapoport
  0 siblings, 0 replies; 23+ messages in thread
From: Michael Rapoport @ 2016-03-23 11:13 UTC (permalink / raw)
  To: Bandan Das; +Cc: jiangshanlai, kvm, linux-kernel, mst, tj

> Bandan Das <bsd@redhat.com> wrote on 03/22/2016 09:00:50 PM:
> > "Michael Rapoport" <RAPOPORT@il.ibm.com> writes:
> >
> > Well, Elvis is a _theoretical_ example that showed that I/O scheduling 
in 
> > the vhost improves performance.
> > I'm not saying we should take Evlis and try to squeeze it into the 
vhost, 
> > I just want to say that we cannot switch vhost to use workqueues if it 

> > causes performance degradation.
> >
> > My opinion is that we need to give it some more thought, much more 
> > performance evaluation, so that we can find the best model.
> 
> Exactly, I think we are outright discarding using workqueues even
> without investigating it in detail even though it would be a cleaner
> implementation using a common framework and thereby more chances of
> an acceptable solution for upstream.

I'm not suggesting to discard using workqueues.
All I'm saying that among several options for vhost threading model we 
should find the one with best "performance/complexity" ratio :)
 
> Anyway, if we don't want to go the workqueues way for vhost, cgroups
> support for workqueues is still something worth having on its own.

No objection to that.
 
> >> > opportunity for optimization, at least for some workloads...
> >> > That said, I believe that switching vhost to use workqueues is not 
> > that 
> >> > good idea after all.
> >> > 
> >

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
       [not found] ` <201603210758.u2L7wiY9003907@d06av07.portsmouth.uk.ibm.com>
@ 2016-03-30 17:04   ` Tejun Heo
  2016-03-31  6:17     ` Michael Rapoport
  0 siblings, 1 reply; 23+ messages in thread
From: Tejun Heo @ 2016-03-30 17:04 UTC (permalink / raw)
  To: Michael Rapoport; +Cc: Bandan Das, linux-kernel, kvm, mst, jiangshanlai

Hello,

On Mon, Mar 21, 2016 at 09:58:39AM +0200, Michael Rapoport wrote:
> I did some performance evaluation of different threading models in vhost, 
> and in most tests replacing vhost kthread's with workqueues degrades the 
> performance. Moreover, having thread management inside the vhost provides 

There really shouldn't be any difference when using unbound
workqueues.  workqueue becomes a convenience thing which manages
worker pools and there shouldn't be any difference between workqueue
workers and kthreads in terms of behavior.

> opportunity for optimization, at least for some workloads...

What sort of optimizations are we talking about?

Thanks.

-- 
tejun

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-30 17:04   ` [RFC PATCH 0/4] cgroup aware workqueues Tejun Heo
@ 2016-03-31  6:17     ` Michael Rapoport
  2016-03-31 17:14       ` Tejun Heo
  0 siblings, 1 reply; 23+ messages in thread
From: Michael Rapoport @ 2016-03-31  6:17 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Bandan Das, linux-kernel, kvm, mst, jiangshanlai

Hello,

> Tejun Heo <htejun@gmail.com> wrote on 03/30/2016 08:04:19 PM:
> 
> Hello,
> 
> On Mon, Mar 21, 2016 at 09:58:39AM +0200, Michael Rapoport wrote:
> > I did some performance evaluation of different threading models in 
vhost, 
> > and in most tests replacing vhost kthread's with workqueues degrades 
the 
> > performance. Moreover, having thread management inside the vhost 
provides 
> 
> There really shouldn't be any difference when using unbound
> workqueues.  workqueue becomes a convenience thing which manages
> worker pools and there shouldn't be any difference between workqueue
> workers and kthreads in terms of behavior.

I agree that there really shouldn't be any performance difference, but the 
tests I've run show otherwise. I have no idea why and I hadn't time yet to 
investigate it.
 
> > opportunity for optimization, at least for some workloads...
> 
> What sort of optimizations are we talking about?

Well, if we take Evlis (1) as for the theoretical base, there could be 
benefit of doing I/O scheduling inside the vhost.

[1] https://www.usenix.org/system/files/conference/atc13/atc13-harel.pdf

> Thanks.
> 
> -- 
> tejun

--
Sincerely yours,
Mike.

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-31  6:17     ` Michael Rapoport
@ 2016-03-31 17:14       ` Tejun Heo
  2016-03-31 18:45         ` Bandan Das
                           ` (2 more replies)
  0 siblings, 3 replies; 23+ messages in thread
From: Tejun Heo @ 2016-03-31 17:14 UTC (permalink / raw)
  To: Michael Rapoport; +Cc: Bandan Das, linux-kernel, kvm, mst, jiangshanlai

Hello, Michael.

On Thu, Mar 31, 2016 at 08:17:13AM +0200, Michael Rapoport wrote:
> > There really shouldn't be any difference when using unbound
> > workqueues.  workqueue becomes a convenience thing which manages
> > worker pools and there shouldn't be any difference between workqueue
> > workers and kthreads in terms of behavior.
> 
> I agree that there really shouldn't be any performance difference, but the 
> tests I've run show otherwise. I have no idea why and I hadn't time yet to 
> investigate it.

I'd be happy to help digging into what's going on.  If kvm wants full
control over the worker thread, kvm can use workqueue as a pure
threadpool.  Schedule a work item to grab a worker thread with the
matching attributes and keep using it as it'd a kthread.  While that
wouldn't be able to take advantage of work item flushing and so on,
it'd still be a simpler way to manage worker threads and the extra
stuff like cgroup membership handling doesn't have to be duplicated.

> > > opportunity for optimization, at least for some workloads...
> > 
> > What sort of optimizations are we talking about?
> 
> Well, if we take Evlis (1) as for the theoretical base, there could be 
> benefit of doing I/O scheduling inside the vhost.

Yeah, if that actually is beneficial, take full control of the
kworker thread.

Thanks.

-- 
tejun

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-31 17:14       ` Tejun Heo
@ 2016-03-31 18:45         ` Bandan Das
  2016-04-03 10:43           ` Michael Rapoport
       [not found]           ` <201604031043.u33AhpSF023771@d06av06.portsmouth.uk.ibm.com>
  2016-04-03 10:43         ` Michael Rapoport
  2016-05-27  9:22         ` Michael Rapoport
  2 siblings, 2 replies; 23+ messages in thread
From: Bandan Das @ 2016-03-31 18:45 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Michael Rapoport, linux-kernel, kvm, mst, jiangshanlai

Tejun Heo <tj@kernel.org> writes:

> Hello, Michael.
>
> On Thu, Mar 31, 2016 at 08:17:13AM +0200, Michael Rapoport wrote:
>> > There really shouldn't be any difference when using unbound
>> > workqueues.  workqueue becomes a convenience thing which manages
>> > worker pools and there shouldn't be any difference between workqueue
>> > workers and kthreads in terms of behavior.
>> 
>> I agree that there really shouldn't be any performance difference, but the 
>> tests I've run show otherwise. I have no idea why and I hadn't time yet to 
>> investigate it.
>
> I'd be happy to help digging into what's going on.  If kvm wants full
> control over the worker thread, kvm can use workqueue as a pure
> threadpool.  Schedule a work item to grab a worker thread with the
> matching attributes and keep using it as it'd a kthread.  While that
> wouldn't be able to take advantage of work item flushing and so on,
> it'd still be a simpler way to manage worker threads and the extra
> stuff like cgroup membership handling doesn't have to be duplicated.
>
>> > > opportunity for optimization, at least for some workloads...
>> > 
>> > What sort of optimizations are we talking about?
>> 
>> Well, if we take Evlis (1) as for the theoretical base, there could be 
>> benefit of doing I/O scheduling inside the vhost.
>
> Yeah, if that actually is beneficial, take full control of the
> kworker thread.

Well, even if it actually is beneficial (which I am sure it is), it seems a
little impractical to block current improvements based on a future prospect
that (as far as I know), no one is working on ?

There have been discussions about this in the past and iirc, most people agree
about not going the byos* route. But I am still all for such a proposal and if
it's good/clean enough, I think we can definitely tear down what we have and
throw it away! The I/O scheduling part is intrusive enough that even the current
code base has to be changed quite a bit.

*byos = bring your own scheduling ;)

> Thanks.

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-31 17:14       ` Tejun Heo
  2016-03-31 18:45         ` Bandan Das
@ 2016-04-03 10:43         ` Michael Rapoport
  2016-05-27  9:22         ` Michael Rapoport
  2 siblings, 0 replies; 23+ messages in thread
From: Michael Rapoport @ 2016-04-03 10:43 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Bandan Das, linux-kernel, kvm, mst, jiangshanlai

Hi Tejun,

> Tejun Heo <htejun@gmail.com> wrote on 03/31/2016 08:14:35 PM:
> 
> Hello, Michael.
> 
> On Thu, Mar 31, 2016 at 08:17:13AM +0200, Michael Rapoport wrote:
> > > There really shouldn't be any difference when using unbound
> > > workqueues.  workqueue becomes a convenience thing which manages
> > > worker pools and there shouldn't be any difference between workqueue
> > > workers and kthreads in terms of behavior.
> > 
> > I agree that there really shouldn't be any performance difference, but 
the 
> > tests I've run show otherwise. I have no idea why and I hadn't time 
yet to 
> > investigate it.
> 
> I'd be happy to help digging into what's going on.  If kvm wants full
> control over the worker thread, kvm can use workqueue as a pure
> threadpool.  Schedule a work item to grab a worker thread with the
> matching attributes and keep using it as it'd a kthread.  While that
> wouldn't be able to take advantage of work item flushing and so on,
> it'd still be a simpler way to manage worker threads and the extra
> stuff like cgroup membership handling doesn't have to be duplicated.

My concern is that we trade-off performance for simpler management of 
worker threads.
With the three models I've tested (current vhost models, workqueues-based 
(1) and shared threads based (2)), workqueues-based ones gave the worst 
performance results :(
 
> > > > opportunity for optimization, at least for some workloads...
> > > 
> > > What sort of optimizations are we talking about?
> > 
> > Well, if we take Evlis (1) as for the theoretical base, there could be 

> > benefit of doing I/O scheduling inside the vhost.
> 
> Yeah, if that actually is beneficial, take full control of the
> kworker thread.
> 
> Thanks.

[1] http://thread.gmane.org/gmane.linux.network/286858
[2] http://thread.gmane.org/gmane.linux.kernel.cgroups/13808
 
> -- 
> tejun
> 

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-31 18:45         ` Bandan Das
@ 2016-04-03 10:43           ` Michael Rapoport
       [not found]           ` <201604031043.u33AhpSF023771@d06av06.portsmouth.uk.ibm.com>
  1 sibling, 0 replies; 23+ messages in thread
From: Michael Rapoport @ 2016-04-03 10:43 UTC (permalink / raw)
  To: Bandan Das; +Cc: jiangshanlai, kvm, linux-kernel, mst, Tejun Heo

Hi Bandan,

>  Bandan Das <bsd@redhat.com> wrote on 03/31/2016 09:45:43 PM:
> >
> >> > > opportunity for optimization, at least for some workloads...
> >> > 
> >> > What sort of optimizations are we talking about?
> >> 
> >> Well, if we take Evlis (1) as for the theoretical base, there could 
be 
> >> benefit of doing I/O scheduling inside the vhost.
> >
> > Yeah, if that actually is beneficial, take full control of the
> > kworker thread.
> 
> Well, even if it actually is beneficial (which I am sure it is), it 
seems a
> little impractical to block current improvements based on a future 
prospect
> that (as far as I know), no one is working on ?

I'm not suggesting to block current improvements based on a future 
prospect. But, unfortunately, there's regression rather than improvement 
with the results you've posted.

And, I thought you are working on comparing different approaches to vhost 
threading, like workqueues and shared vhost thread (1) ;-)
Anyway, I'm working on this in a background, and, frankly, I cannot say I 
have a clear vision of the best route.
 
> There have been discussions about this in the past and iirc, most people 
agree
> about not going the byos* route. But I am still all for such a proposal 
and if
> it's good/clean enough, I think we can definitely tear down what we have 
and
> throw it away! The I/O scheduling part is intrusive enough that even the 
current
> code base has to be changed quite a bit.

The "byos" route seems more promising with respect to possible performance 
gains, but it will definitely add complexity, and I cannot say if the 
added complexity will be worth performance improvements.

Meanwhile, I'd suggest we better understand what causes regression with 
your current patches and maybe then we'll be smarter to get to the right 
direction. :)
 
> *byos = bring your own scheduling ;)
> 
> > Thanks.

--
Sincerely yours,
Mike.

[1] https://lwn.net/Articles/650857/ 

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
       [not found]           ` <201604031043.u33AhpSF023771@d06av06.portsmouth.uk.ibm.com>
@ 2016-04-04 17:00             ` Bandan Das
  0 siblings, 0 replies; 23+ messages in thread
From: Bandan Das @ 2016-04-04 17:00 UTC (permalink / raw)
  To: Michael Rapoport; +Cc: jiangshanlai, kvm, linux-kernel, mst, Tejun Heo

...
>> There have been discussions about this in the past and iirc, most people 
> agree
>> about not going the byos* route. But I am still all for such a proposal 
> and if
>> it's good/clean enough, I think we can definitely tear down what we have 
> and
>> throw it away! The I/O scheduling part is intrusive enough that even the 
> current
>> code base has to be changed quite a bit.
>
> The "byos" route seems more promising with respect to possible performance 
> gains, but it will definitely add complexity, and I cannot say if the 
> added complexity will be worth performance improvements.
>
> Meanwhile, I'd suggest we better understand what causes regression with 
> your current patches and maybe then we'll be smarter to get to the right 
> direction. :)
>

Agreed, let's try to understand the cause of the "underperformance" with wqs.
I disabled WQ_CGROUPS that effectively disables my changes and I can still
consistently reproduce the lower numbers.

>> *byos = bring your own scheduling ;)
>> 
>> > Thanks.
>
> --
> Sincerely yours,
> Mike.
>
> [1] https://lwn.net/Articles/650857/ 

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-03-31 17:14       ` Tejun Heo
  2016-03-31 18:45         ` Bandan Das
  2016-04-03 10:43         ` Michael Rapoport
@ 2016-05-27  9:22         ` Michael Rapoport
  2016-05-27 14:17           ` Tejun Heo
  2 siblings, 1 reply; 23+ messages in thread
From: Michael Rapoport @ 2016-05-27  9:22 UTC (permalink / raw)
  To: Tejun Heo; +Cc: Bandan Das, linux-kernel, kvm, mst, jiangshanlai

> Tejun Heo <htejun@gmail.com> wrote on 03/31/2016 08:14:35 PM:
>
> Hello, Michael.
> 
> On Thu, Mar 31, 2016 at 08:17:13AM +0200, Michael Rapoport wrote:
> > > There really shouldn't be any difference when using unbound
> > > workqueues.  workqueue becomes a convenience thing which manages
> > > worker pools and there shouldn't be any difference between workqueue
> > > workers and kthreads in terms of behavior.
> > 
> > I agree that there really shouldn't be any performance difference, but 
the 
> > tests I've run show otherwise. I have no idea why and I hadn't time 
yet to 
> > investigate it.
> 
> I'd be happy to help digging into what's going on.  If kvm wants full
> control over the worker thread, kvm can use workqueue as a pure
> threadpool.  Schedule a work item to grab a worker thread with the
> matching attributes and keep using it as it'd a kthread.  While that
> wouldn't be able to take advantage of work item flushing and so on,
> it'd still be a simpler way to manage worker threads and the extra
> stuff like cgroup membership handling doesn't have to be duplicated.
> 
> > > > opportunity for optimization, at least for some workloads...
> > > 
> > > What sort of optimizations are we talking about?
> > 
> > Well, if we take Evlis (1) as for the theoretical base, there could be 

> > benefit of doing I/O scheduling inside the vhost.
> 
> Yeah, if that actually is beneficial, take full control of the
> kworker thread.

It me took a while, but at last I had time to run some benchmarks.
I've compared guest-to-guest netperf with 3 variants of vhost 
implementation:
(1) vanilla 4.4 (baseline)
(2) 4.4 + unbound workqueues based on Bandans patches [1]
(3) 4.4 + "grabbed" worker thread. This is my POC implementation that 
actually follows your proposal to take full control over the worker 
thread.

I've run two guests without any CPU pinning and without any actual 
interaction with cgroups
Here's the results (in MBits/sec):

size |   64  |   256   |  1024   |  4096   |  16384
-----+-------+---------+---------+---------+---------
(1)  | 496.8 | 1346.31 | 6058.49 | 13736.2 | 13541.4
(2)  | 493.3 | 1604.03 | 5723.68 | 10181.4 | 15572.4
(3)  | 489.7 | 1437.86 | 6251.12 | 12774.2 | 12867.9 


>From what I see, for different packet sizes there's different approach 
that outperforms the others.
Moreover, I'd expect that in case when vhost completely takes over the 
worker thread there would no be difference vs. current state.

Tejun, can you help explaining these results? 

[1] http://thread.gmane.org/gmane.linux.network/286858

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

* Re: [RFC PATCH 0/4] cgroup aware workqueues
  2016-05-27  9:22         ` Michael Rapoport
@ 2016-05-27 14:17           ` Tejun Heo
  0 siblings, 0 replies; 23+ messages in thread
From: Tejun Heo @ 2016-05-27 14:17 UTC (permalink / raw)
  To: Michael Rapoport; +Cc: Bandan Das, linux-kernel, kvm, mst, jiangshanlai

On Fri, May 27, 2016 at 12:22:19PM +0300, Michael Rapoport wrote:
> I've run two guests without any CPU pinning and without any actual 
> interaction with cgroups
> Here's the results (in MBits/sec):
> 
> size |   64  |   256   |  1024   |  4096   |  16384
> -----+-------+---------+---------+---------+---------
> (1)  | 496.8 | 1346.31 | 6058.49 | 13736.2 | 13541.4
> (2)  | 493.3 | 1604.03 | 5723.68 | 10181.4 | 15572.4
> (3)  | 489.7 | 1437.86 | 6251.12 | 12774.2 | 12867.9 
> 
> 
> From what I see, for different packet sizes there's different approach 
> that outperforms the others.
> Moreover, I'd expect that in case when vhost completely takes over the 
> worker thread there would no be difference vs. current state.
> 
> Tejun, can you help explaining these results? 

Heh, the only thing I can tell is the tests seem noisy and weren't run
enough times to draw any conclusion.  Even if the numbers were
consistent, I don't think anybody would be able to tell a lot from
just the results.  If the results can be made stable, perf would be a
good place to start.

Thanks.

-- 
tejun

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

end of thread, other threads:[~2016-05-27 14:17 UTC | newest]

Thread overview: 23+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2016-03-18 22:14 [RFC PATCH 0/4] cgroup aware workqueues Bandan Das
2016-03-18 22:14 ` [RFC PATCH 1/4] cgroup: Introduce a function to compare two tasks Bandan Das
2016-03-18 22:14 ` [RFC PATCH 2/4] workqueue: introduce support for attaching to cgroups Bandan Das
2016-03-18 22:14 ` [RFC PATCH 3/4] cgroup: use spin_lock_irq for cgroup match and attach fns Bandan Das
2016-03-18 22:14 ` [RFC PATCH 4/4] vhost: use workqueues for the works Bandan Das
2016-03-20 18:10 ` [RFC PATCH 0/4] cgroup aware workqueues Tejun Heo
2016-03-21 17:35   ` Bandan Das
2016-03-21  7:58 ` Michael Rapoport
2016-03-21  8:29 ` Christian Borntraeger
2016-03-21 17:49   ` Bandan Das
     [not found] ` <201603210758.u2L7wiXA028101@d06av09.portsmouth.uk.ibm.com>
2016-03-21 17:43   ` Bandan Das
2016-03-22  7:12     ` vhost threading model (was: Re: [RFC PATCH 0/4] cgroup aware workqueues) Michael Rapoport
     [not found]     ` <201603220712.u2M7CCfq004548@d06av03.portsmouth.uk.ibm.com>
2016-03-22 19:00       ` vhost threading model Bandan Das
2016-03-23 11:13         ` Michael Rapoport
     [not found] ` <201603210758.u2L7wiY9003907@d06av07.portsmouth.uk.ibm.com>
2016-03-30 17:04   ` [RFC PATCH 0/4] cgroup aware workqueues Tejun Heo
2016-03-31  6:17     ` Michael Rapoport
2016-03-31 17:14       ` Tejun Heo
2016-03-31 18:45         ` Bandan Das
2016-04-03 10:43           ` Michael Rapoport
     [not found]           ` <201604031043.u33AhpSF023771@d06av06.portsmouth.uk.ibm.com>
2016-04-04 17:00             ` Bandan Das
2016-04-03 10:43         ` Michael Rapoport
2016-05-27  9:22         ` Michael Rapoport
2016-05-27 14:17           ` Tejun Heo

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).