All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
@ 2022-01-06  0:46 Daniel Jordan
  2022-01-06  0:46 ` [RFC 01/16] padata: Remove __init from multithreading functions Daniel Jordan
                   ` (16 more replies)
  0 siblings, 17 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan


Here's phase two of padata multithreaded jobs, which multithreads VFIO page
pinning and lays the groundwork for other padata users.  It's RFC because there
are still pieces missing and testing to do, and because of the last two
patches, which I'm hoping scheduler and cgroup folks can weigh in on.  Any and
all feedback is welcome.

---

Assigning a VFIO device to a guest requires pinning each and every page of the
guest's memory, which gets expensive for large guests even if the memory has
already been faulted in and cleared with something like qemu prealloc.

Some recent optimizations[0][1] have brought the cost down, but it's still a
significant bottleneck for guest initialization time.  Parallelize with padata
to take proper advantage of memory bandwidth, yielding up to 12x speedups for
VFIO page pinning and 10x speedups for overall qemu guest initialization.
Detailed performance results are in patch 8.

Phase one[4] of multithreaded jobs made deferred struct page init use all the
CPUs on x86.  That's a special case because it happens during boot when the
machine is waiting on page init to finish and there are generally no resource
controls to violate.

Page pinning, on the other hand, can be done by a user task (the "main thread"
in a job), so helper threads should honor the main thread's resource controls
that are relevant for pinning (CPU, memory) and give priority to other tasks on
the system.  This RFC has some but not all of the pieces to do that.

After this phase, it shouldn't take many lines to parallelize other
memory-proportional paths like struct page init for memory hotplug, munmap(),
hugetlb_fallocate(), and __ib_umem_release().

The first half of this series (more or less) has been running in our kernels
for about three years.


Changelog
---------

This addresses some comments on two earlier projects, ktask[2] and
cgroup-aware workqueues[3].

 - Fix undoing partially a completed chunk in the thread function, and use
   larger minimum chunk size (Alex Williamson)

 - Helper threads should honor the main thread's settings and resource controls,
   and shouldn't disturb other tasks (Michal Hocko, Pavel Machek)

 - Design comments, lockdep awareness (Peter Zijlstra, Jason Gunthorpe)

 - Implement remote charging in the CPU controller (Tejun Heo)


Series Rundown
--------------

     1  padata: Remove __init from multithreading functions
     2  padata: Return first error from a job
     3  padata: Add undo support
     4  padata: Detect deadlocks between main and helper threads

Get ready to parallelize.  In particular, pinning can fail, so make jobs
undo-able.

     5  vfio/type1: Pass mm to vfio_pin_pages_remote()
     6  vfio/type1: Refactor dma map removal
     7  vfio/type1: Parallelize vfio_pin_map_dma()
     8  vfio/type1: Cache locked_vm to ease mmap_lock contention

Do the parallelization itself.

     9  padata: Use kthreads in do_multithreaded
    10  padata: Helpers should respect main thread's CPU affinity
    11  padata: Cap helpers started to online CPUs
    12  sched, padata: Bound max threads with max_cfs_bandwidth_cpus()

Put caps on the number of helpers started according to the main thread's CPU
affinity, the system' online CPU count, and the main thread's CFS bandwidth
settings.  

    13  padata: Run helper threads at MAX_NICE
    14  padata: Nice helper threads one by one to prevent starvation

Prevent helpers from taking CPU away unfairly from other tasks for the sake of
an optimized kernel code path.

    15  sched/fair: Account kthread runtime debt for CFS bandwidth
    16  sched/fair: Consider kthread debt in cputime

A prototype for remote charging in CFS bandwidth and cpu.stat, described more
in the next section.  It's debatable whether these last two are required for
this series.  Patch 12 caps the number of helper threads started according to
the max effective CPUs allowed by the quota and period of the main thread's
task group.  In practice, I think this hits the sweet spot between complexity
and respecting CFS bandwidth limits so that patch 15 might just be dropped.
For instance, when running qemu with a vfio device, the restriction from patch
12 was enough to avoid the helpers breaching CFS bandwidth limits.  That leaves
patch 16, which on its own seems overkill for all the hunks it would require
from patch 15, so it could be dropped too.

Patch 12 isn't airtight, though, since other tasks running in the task group
alongside the main thread and helpers could still result in overage.  So,
patches 15-16 give an idea of what absolutely correct accounting in the CPU
controller might look like in case there are real situations that want it.


Remote Charging in the CPU Controller
-------------------------------------

CPU-intensive kthreads aren't generally accounted in the CPU controller, so
they escape settings such as weight and bandwidth when they do work on behalf
of a task group.

This problem arises with multithreaded jobs, but is also an issue in other
places.  CPU activity from async memory reclaim (kswapd, cswapd?[5]) should be
accounted to the cgroup that the memory belongs to, and similarly CPU activity
from net rx should be accounted to the task groups that correspond to the
packets being received.  There are also vague complaints from Android[6].

Each use case has its own requirements[7].  In padata and reclaim, the task
group to account to is known ahead of time, but net rx has to spend cycles
processing a packet before its destination task group is known, so any solution
should be able to work without knowing the task group in advance.  Furthermore,
the CPU controller shouldn't throttle reclaim or net rx in real time since both
are doing high priority work.  These make approaches that run kthreads directly
in a task group, like cgroup-aware workqueues[8] or a kernel path for
CLONE_INTO_CGROUP, infeasible.  Running kthreads directly in cgroups also has a
downside for padata because helpers' MAX_NICE priority is "shadowed" by the
priority of the group entities they're running under.

The proposed solution of remote charging can accrue debt to a task group to be
paid off or forgiven later, addressing all these issues.  A kthread calls the
interface

    void cpu_cgroup_remote_begin(struct task_struct *p,
                                 struct cgroup_subsys_state *css);

to begin remote charging to @css, causing @p's current sum_exec_runtime to be
updated and saved.  The @css arg isn't required and can be removed later to
facilitate the unknown cgroup case mentioned above.  Then the kthread calls
another interface

    void cpu_cgroup_remote_charge(struct task_struct *p,
                                  struct cgroup_subsys_state *css);

to account the sum_exec_runtime that @p has used since the first call.
Internally, a new field cfs_bandwidth::debt is added to keep track of unpaid
debt that's only used when the debt exceeds the quota in the current period.

Weight-based control isn't implemented for now since padata helpers run at
MAX_NICE and so always yield to anything higher priority, meaning they would
rarely compete with other task groups.

[ We have another use case to use remote charging for implementing
  CFS bandwidth control across multiple machines.  This is an entirely
  different topic that deserves its own thread. ]


TODO
----

 - Honor these other resource controls:
    - Memory controller limits for helpers via active_memcg.  I *think* this
      will turn out to be necessary despite helpers using the main thread's mm,
      but I need to look into it more.
    - cpuset.mems
    - NUMA memory policy

 - Make helpers aware of signals sent to the main thread

 - Test test test


Series based on 5.14.  I had to downgrade from 5.15 because of an intel iommu
bug that's since been fixed.

thanks,
Daniel


[0] https://lore.kernel.org/linux-mm/20210128182632.24562-1-joao.m.martins@oracle.com
[1] https://lore.kernel.org/lkml/20210219161305.36522-1-daniel.m.jordan@oracle.com/
[2] https://x-lore.kernel.org/all/20181105165558.11698-1-daniel.m.jordan@oracle.com/
[3] https://lore.kernel.org/linux-mm/20190605133650.28545-1-daniel.m.jordan@oracle.com/
[4] https://x-lore.kernel.org/all/20200527173608.2885243-1-daniel.m.jordan@oracle.com/
[5] https://x-lore.kernel.org/all/20200219181219.54356-1-hannes@cmpxchg.org/
[6] https://x-lore.kernel.org/all/20210407013856.GC21941@codeaurora.org/
[7] https://x-lore.kernel.org/all/20200219214112.4kt573kyzbvmbvn3@ca-dmjordan1.us.oracle.com/
[8] https://x-lore.kernel.org/all/20190605133650.28545-1-daniel.m.jordan@oracle.com/

Daniel Jordan (16):
  padata: Remove __init from multithreading functions
  padata: Return first error from a job
  padata: Add undo support
  padata: Detect deadlocks between main and helper threads
  vfio/type1: Pass mm to vfio_pin_pages_remote()
  vfio/type1: Refactor dma map removal
  vfio/type1: Parallelize vfio_pin_map_dma()
  vfio/type1: Cache locked_vm to ease mmap_lock contention
  padata: Use kthreads in do_multithreaded
  padata: Helpers should respect main thread's CPU affinity
  padata: Cap helpers started to online CPUs
  sched, padata: Bound max threads with max_cfs_bandwidth_cpus()
  padata: Run helper threads at MAX_NICE
  padata: Nice helper threads one by one to prevent starvation
  sched/fair: Account kthread runtime debt for CFS bandwidth
  sched/fair: Consider kthread debt in cputime

 drivers/vfio/Kconfig            |   1 +
 drivers/vfio/vfio_iommu_type1.c | 170 ++++++++++++++---
 include/linux/padata.h          |  31 +++-
 include/linux/sched.h           |   2 +
 include/linux/sched/cgroup.h    |  37 ++++
 kernel/padata.c                 | 311 +++++++++++++++++++++++++-------
 kernel/sched/core.c             |  58 ++++++
 kernel/sched/fair.c             |  99 +++++++++-
 kernel/sched/sched.h            |   5 +
 mm/page_alloc.c                 |   4 +-
 10 files changed, 620 insertions(+), 98 deletions(-)
 create mode 100644 include/linux/sched/cgroup.h


base-commit: 7d2a07b769330c34b4deabeed939325c77a7ec2f
-- 
2.34.1


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

* [RFC 01/16] padata: Remove __init from multithreading functions
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 02/16] padata: Return first error from a job Daniel Jordan
                   ` (15 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

A non-__init caller will need them soon.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 include/linux/padata.h |  2 +-
 kernel/padata.c        | 11 +++++------
 2 files changed, 6 insertions(+), 7 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index a433f13fc4bf..0dc031d54742 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -188,7 +188,7 @@ extern void padata_free_shell(struct padata_shell *ps);
 extern int padata_do_parallel(struct padata_shell *ps,
 			      struct padata_priv *padata, int *cb_cpu);
 extern void padata_do_serial(struct padata_priv *padata);
-extern void __init padata_do_multithreaded(struct padata_mt_job *job);
+extern void padata_do_multithreaded(struct padata_mt_job *job);
 extern int padata_set_cpumask(struct padata_instance *pinst, int cpumask_type,
 			      cpumask_var_t cpumask);
 #endif
diff --git a/kernel/padata.c b/kernel/padata.c
index d4d3ba6e1728..5d13920d2a12 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -58,7 +58,7 @@ struct padata_mt_job_state {
 };
 
 static void padata_free_pd(struct parallel_data *pd);
-static void __init padata_mt_helper(struct work_struct *work);
+static void padata_mt_helper(struct work_struct *work);
 
 static int padata_index_to_cpu(struct parallel_data *pd, int cpu_index)
 {
@@ -106,8 +106,7 @@ static void padata_work_init(struct padata_work *pw, work_func_t work_fn,
 	pw->pw_data = data;
 }
 
-static int __init padata_work_alloc_mt(int nworks, void *data,
-				       struct list_head *head)
+static int padata_work_alloc_mt(int nworks, void *data, struct list_head *head)
 {
 	int i;
 
@@ -132,7 +131,7 @@ static void padata_work_free(struct padata_work *pw)
 	list_add(&pw->pw_list, &padata_free_works);
 }
 
-static void __init padata_works_free(struct list_head *works)
+static void padata_works_free(struct list_head *works)
 {
 	struct padata_work *cur, *next;
 
@@ -438,7 +437,7 @@ static int padata_setup_cpumasks(struct padata_instance *pinst)
 	return err;
 }
 
-static void __init padata_mt_helper(struct work_struct *w)
+static void padata_mt_helper(struct work_struct *w)
 {
 	struct padata_work *pw = container_of(w, struct padata_work, pw_work);
 	struct padata_mt_job_state *ps = pw->pw_data;
@@ -478,7 +477,7 @@ static void __init padata_mt_helper(struct work_struct *w)
  *
  * See the definition of struct padata_mt_job for more details.
  */
-void __init padata_do_multithreaded(struct padata_mt_job *job)
+void padata_do_multithreaded(struct padata_mt_job *job)
 {
 	/* In case threads finish at different times. */
 	static const unsigned long load_balance_factor = 4;
-- 
2.34.1


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

* [RFC 02/16] padata: Return first error from a job
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
  2022-01-06  0:46 ` [RFC 01/16] padata: Remove __init from multithreading functions Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 03/16] padata: Add undo support Daniel Jordan
                   ` (14 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

The only current user of multithreaded jobs, deferred struct page init,
can't fail, but soon the world won't be perfect anymore.  Return the
first error encountered during a job.

Threads can fail for different reasons, which may need special handling
in the future, but returning the first will do for the upcoming new user
because the kernel unwinds the same way no matter the error.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 include/linux/padata.h |  5 +++--
 kernel/padata.c        | 22 ++++++++++++++++------
 mm/page_alloc.c        |  4 +++-
 3 files changed, 22 insertions(+), 9 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index 0dc031d54742..1c8670a24ccf 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -126,6 +126,7 @@ struct padata_shell {
  * struct padata_mt_job - represents one multithreaded job
  *
  * @thread_fn: Called for each chunk of work that a padata thread does.
+ *             Returns 0 or client-specific nonzero error code.
  * @fn_arg: The thread function argument.
  * @start: The start of the job (units are job-specific).
  * @size: size of this node's work (units are job-specific).
@@ -138,7 +139,7 @@ struct padata_shell {
  *               depending on task size and minimum chunk size.
  */
 struct padata_mt_job {
-	void (*thread_fn)(unsigned long start, unsigned long end, void *arg);
+	int (*thread_fn)(unsigned long start, unsigned long end, void *arg);
 	void			*fn_arg;
 	unsigned long		start;
 	unsigned long		size;
@@ -188,7 +189,7 @@ extern void padata_free_shell(struct padata_shell *ps);
 extern int padata_do_parallel(struct padata_shell *ps,
 			      struct padata_priv *padata, int *cb_cpu);
 extern void padata_do_serial(struct padata_priv *padata);
-extern void padata_do_multithreaded(struct padata_mt_job *job);
+extern int padata_do_multithreaded(struct padata_mt_job *job);
 extern int padata_set_cpumask(struct padata_instance *pinst, int cpumask_type,
 			      cpumask_var_t cpumask);
 #endif
diff --git a/kernel/padata.c b/kernel/padata.c
index 5d13920d2a12..1596ca22b316 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -54,6 +54,7 @@ struct padata_mt_job_state {
 	struct padata_mt_job	*job;
 	int			nworks;
 	int			nworks_fini;
+	int			error; /* first error from thread_fn */
 	unsigned long		chunk_size;
 };
 
@@ -446,8 +447,9 @@ static void padata_mt_helper(struct work_struct *w)
 
 	spin_lock(&ps->lock);
 
-	while (job->size > 0) {
+	while (job->size > 0 && ps->error == 0) {
 		unsigned long start, size, end;
+		int ret;
 
 		start = job->start;
 		/* So end is chunk size aligned if enough work remains. */
@@ -459,8 +461,12 @@ static void padata_mt_helper(struct work_struct *w)
 		job->size -= size;
 
 		spin_unlock(&ps->lock);
-		job->thread_fn(start, end, job->fn_arg);
+		ret = job->thread_fn(start, end, job->fn_arg);
 		spin_lock(&ps->lock);
+
+		/* Save first error code only. */
+		if (ps->error == 0)
+			ps->error = ret;
 	}
 
 	++ps->nworks_fini;
@@ -476,8 +482,10 @@ static void padata_mt_helper(struct work_struct *w)
  * @job: Description of the job.
  *
  * See the definition of struct padata_mt_job for more details.
+ *
+ * Return: 0 or a client-specific nonzero error code.
  */
-void padata_do_multithreaded(struct padata_mt_job *job)
+int padata_do_multithreaded(struct padata_mt_job *job)
 {
 	/* In case threads finish at different times. */
 	static const unsigned long load_balance_factor = 4;
@@ -487,7 +495,7 @@ void padata_do_multithreaded(struct padata_mt_job *job)
 	int nworks;
 
 	if (job->size == 0)
-		return;
+		return 0;
 
 	/* Ensure at least one thread when size < min_chunk. */
 	nworks = max(job->size / job->min_chunk, 1ul);
@@ -495,8 +503,8 @@ void padata_do_multithreaded(struct padata_mt_job *job)
 
 	if (nworks == 1) {
 		/* Single thread, no coordination needed, cut to the chase. */
-		job->thread_fn(job->start, job->start + job->size, job->fn_arg);
-		return;
+		return job->thread_fn(job->start, job->start + job->size,
+				      job->fn_arg);
 	}
 
 	spin_lock_init(&ps.lock);
@@ -504,6 +512,7 @@ void padata_do_multithreaded(struct padata_mt_job *job)
 	ps.job	       = job;
 	ps.nworks      = padata_work_alloc_mt(nworks, &ps, &works);
 	ps.nworks_fini = 0;
+	ps.error       = 0;
 
 	/*
 	 * Chunk size is the amount of work a helper does per call to the
@@ -527,6 +536,7 @@ void padata_do_multithreaded(struct padata_mt_job *job)
 
 	destroy_work_on_stack(&my_work.pw_work);
 	padata_works_free(&works);
+	return ps.error;
 }
 
 static void __padata_list_init(struct padata_list *pd_list)
diff --git a/mm/page_alloc.c b/mm/page_alloc.c
index eeb3a9cb36bb..039786d840cf 100644
--- a/mm/page_alloc.c
+++ b/mm/page_alloc.c
@@ -2018,7 +2018,7 @@ deferred_init_maxorder(u64 *i, struct zone *zone, unsigned long *start_pfn,
 	return nr_pages;
 }
 
-static void __init
+static int __init
 deferred_init_memmap_chunk(unsigned long start_pfn, unsigned long end_pfn,
 			   void *arg)
 {
@@ -2036,6 +2036,8 @@ deferred_init_memmap_chunk(unsigned long start_pfn, unsigned long end_pfn,
 		deferred_init_maxorder(&i, zone, &spfn, &epfn);
 		cond_resched();
 	}
+
+	return 0;
 }
 
 /* An arch may override for more concurrency. */
-- 
2.34.1


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

* [RFC 03/16] padata: Add undo support
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
  2022-01-06  0:46 ` [RFC 01/16] padata: Remove __init from multithreading functions Daniel Jordan
  2022-01-06  0:46 ` [RFC 02/16] padata: Return first error from a job Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 04/16] padata: Detect deadlocks between main and helper threads Daniel Jordan
                   ` (13 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Jobs can fail midway through their work.  To recover, the finished
chunks of work need to be undone in a job-specific way.

Let padata_do_multithreaded callers specify an "undo" callback
responsible for undoing one chunk of a job.  To avoid multiple levels of
error handling, do not allow the callback to fail.  Undoing is
singlethreaded to keep it simple and because it's a slow path.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 include/linux/padata.h |   6 +++
 kernel/padata.c        | 113 +++++++++++++++++++++++++++++++++++------
 2 files changed, 103 insertions(+), 16 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index 1c8670a24ccf..2a9fa459463d 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -135,6 +135,10 @@ struct padata_shell {
  * @min_chunk: The minimum chunk size in job-specific units.  This allows
  *             the client to communicate the minimum amount of work that's
  *             appropriate for one worker thread to do at once.
+ * @undo_fn: A function that undoes one chunk of the task per call.  If
+ *           error(s) occur during the job, this is called on all successfully
+ *           completed chunks.  The chunk(s) in which failure occurs should be
+ *           handled in the thread function.
  * @max_threads: Max threads to use for the job, actual number may be less
  *               depending on task size and minimum chunk size.
  */
@@ -145,6 +149,8 @@ struct padata_mt_job {
 	unsigned long		size;
 	unsigned long		align;
 	unsigned long		min_chunk;
+
+	void (*undo_fn)(unsigned long start, unsigned long end, void *arg);
 	int			max_threads;
 };
 
diff --git a/kernel/padata.c b/kernel/padata.c
index 1596ca22b316..d0876f861464 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -29,6 +29,7 @@
 #include <linux/cpumask.h>
 #include <linux/err.h>
 #include <linux/cpu.h>
+#include <linux/list_sort.h>
 #include <linux/padata.h>
 #include <linux/mutex.h>
 #include <linux/sched.h>
@@ -42,6 +43,10 @@ struct padata_work {
 	struct work_struct	pw_work;
 	struct list_head	pw_list;  /* padata_free_works linkage */
 	void			*pw_data;
+	/* holds job units from padata_mt_job::start to pw_error_start */
+	unsigned long		pw_error_offset;
+	unsigned long		pw_error_start;
+	unsigned long		pw_error_end;
 };
 
 static DEFINE_SPINLOCK(padata_works_lock);
@@ -56,6 +61,9 @@ struct padata_mt_job_state {
 	int			nworks_fini;
 	int			error; /* first error from thread_fn */
 	unsigned long		chunk_size;
+	unsigned long		position;
+	unsigned long		remaining_size;
+	struct list_head	failed_works;
 };
 
 static void padata_free_pd(struct parallel_data *pd);
@@ -447,26 +455,38 @@ static void padata_mt_helper(struct work_struct *w)
 
 	spin_lock(&ps->lock);
 
-	while (job->size > 0 && ps->error == 0) {
-		unsigned long start, size, end;
+	while (ps->remaining_size > 0 && ps->error == 0) {
+		unsigned long position, position_offset, size, end;
 		int ret;
 
-		start = job->start;
+		position_offset = job->size - ps->remaining_size;
+		position = ps->position;
 		/* So end is chunk size aligned if enough work remains. */
-		size = roundup(start + 1, ps->chunk_size) - start;
-		size = min(size, job->size);
-		end = start + size;
+		size = roundup(position + 1, ps->chunk_size) - position;
+		size = min(size, ps->remaining_size);
+		end = position + size;
 
-		job->start = end;
-		job->size -= size;
+		ps->position = end;
+		ps->remaining_size -= size;
 
 		spin_unlock(&ps->lock);
-		ret = job->thread_fn(start, end, job->fn_arg);
+
+		ret = job->thread_fn(position, end, job->fn_arg);
+
 		spin_lock(&ps->lock);
 
-		/* Save first error code only. */
-		if (ps->error == 0)
-			ps->error = ret;
+		if (ret) {
+			/* Save first error code only. */
+			if (ps->error == 0)
+				ps->error = ret;
+			/* Save information about where the job failed. */
+			if (job->undo_fn) {
+				list_move(&pw->pw_list, &ps->failed_works);
+				pw->pw_error_start = position;
+				pw->pw_error_offset = position_offset;
+				pw->pw_error_end = end;
+			}
+		}
 	}
 
 	++ps->nworks_fini;
@@ -477,6 +497,60 @@ static void padata_mt_helper(struct work_struct *w)
 		complete(&ps->completion);
 }
 
+static int padata_error_cmp(void *unused, const struct list_head *a,
+			    const struct list_head *b)
+{
+	struct padata_work *work_a = list_entry(a, struct padata_work, pw_list);
+	struct padata_work *work_b = list_entry(b, struct padata_work, pw_list);
+
+	if (work_a->pw_error_offset < work_b->pw_error_offset)
+		return -1;
+	else if (work_a->pw_error_offset > work_b->pw_error_offset)
+		return 1;
+	return 0;
+}
+
+static void padata_undo(struct padata_mt_job_state *ps,
+			struct list_head *works_list,
+			struct padata_work *stack_work)
+{
+	struct list_head *failed_works = &ps->failed_works;
+	struct padata_mt_job *job = ps->job;
+	unsigned long undo_pos = job->start;
+
+	/* Sort so the failed ranges can be checked as we go. */
+	list_sort(NULL, failed_works, padata_error_cmp);
+
+	/* Undo completed work on this node, skipping failed ranges. */
+	while (undo_pos != ps->position) {
+		struct padata_work *failed_work;
+		unsigned long undo_end;
+
+		failed_work = list_first_entry_or_null(failed_works,
+						       struct padata_work,
+						       pw_list);
+		if (failed_work)
+			undo_end = failed_work->pw_error_start;
+		else
+			undo_end = ps->position;
+
+		if (undo_pos != undo_end)
+			job->undo_fn(undo_pos, undo_end, job->fn_arg);
+
+		if (failed_work) {
+			undo_pos = failed_work->pw_error_end;
+			/* main thread's stack_work stays off works_list */
+			if (failed_work == stack_work)
+				list_del(&failed_work->pw_list);
+			else
+				list_move(&failed_work->pw_list, works_list);
+		} else {
+			undo_pos = undo_end;
+		}
+	}
+	WARN_ON_ONCE(!list_empty(failed_works));
+}
+
 /**
  * padata_do_multithreaded - run a multithreaded job
  * @job: Description of the job.
@@ -509,10 +583,13 @@ int padata_do_multithreaded(struct padata_mt_job *job)
 
 	spin_lock_init(&ps.lock);
 	init_completion(&ps.completion);
-	ps.job	       = job;
-	ps.nworks      = padata_work_alloc_mt(nworks, &ps, &works);
-	ps.nworks_fini = 0;
-	ps.error       = 0;
+	INIT_LIST_HEAD(&ps.failed_works);
+	ps.job		  = job;
+	ps.nworks	  = padata_work_alloc_mt(nworks, &ps, &works);
+	ps.nworks_fini	  = 0;
+	ps.error	  = 0;
+	ps.position	  = job->start;
+	ps.remaining_size = job->size;
 
 	/*
 	 * Chunk size is the amount of work a helper does per call to the
@@ -529,11 +606,15 @@ int padata_do_multithreaded(struct padata_mt_job *job)
 
 	/* Use the current thread, which saves starting a workqueue worker. */
 	padata_work_init(&my_work, padata_mt_helper, &ps, PADATA_WORK_ONSTACK);
+	INIT_LIST_HEAD(&my_work.pw_list);
 	padata_mt_helper(&my_work.pw_work);
 
 	/* Wait for all the helpers to finish. */
 	wait_for_completion(&ps.completion);
 
+	if (ps.error && job->undo_fn)
+		padata_undo(&ps, &works, &my_work);
+
 	destroy_work_on_stack(&my_work.pw_work);
 	padata_works_free(&works);
 	return ps.error;
-- 
2.34.1


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

* [RFC 04/16] padata: Detect deadlocks between main and helper threads
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (2 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 03/16] padata: Add undo support Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 05/16] vfio/type1: Pass mm to vfio_pin_pages_remote() Daniel Jordan
                   ` (12 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

A caller of padata_do_multithreaded() can unwittingly introduce
deadlocks if it already holds lock(s) that thread_fn() takes.  Lockdep
can't detect such a dependency because it doesn't know that
padata_do_multithreaded() waits on the helper threads.

Use a lockdep_map to encode the dependency, following the pattern in
workqueue, CPU hotplug, and other parts of the kernel.  See
commit 4e6045f13478 ("workqueue: debug flushing deadlocks with lockdep")
for an example of a similar situation.

Each padata_do_multithreaded() callsite gets its own lock_class_key to
avoid false positives involving locks from different calls that don't
depend on each other.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Suggested-by: Peter Zijlstra <peterz@infradead.org>
---
 include/linux/padata.h | 22 +++++++++++++++++++++-
 kernel/padata.c        | 15 +++++++++++++--
 2 files changed, 34 insertions(+), 3 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index 2a9fa459463d..907d624a8ca4 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -17,6 +17,7 @@
 #include <linux/spinlock.h>
 #include <linux/list.h>
 #include <linux/kobject.h>
+#include <linux/lockdep.h>
 
 #define PADATA_CPU_SERIAL   0x01
 #define PADATA_CPU_PARALLEL 0x02
@@ -188,6 +189,23 @@ extern void __init padata_init(void);
 static inline void __init padata_init(void) {}
 #endif
 
+#ifdef CONFIG_LOCKDEP
+
+#define padata_do_multithreaded(job)					      \
+({									      \
+	static struct lock_class_key __key;				      \
+	const char *__map_name = "padata master waiting";		      \
+									      \
+	padata_do_multithreaded_job((job), &__key, __map_name);		      \
+})
+
+#else
+
+#define padata_do_multithreaded(job)					      \
+	padata_do_multithreaded_job((job), NULL, NULL)
+
+#endif
+
 extern struct padata_instance *padata_alloc(const char *name);
 extern void padata_free(struct padata_instance *pinst);
 extern struct padata_shell *padata_alloc_shell(struct padata_instance *pinst);
@@ -195,7 +213,9 @@ extern void padata_free_shell(struct padata_shell *ps);
 extern int padata_do_parallel(struct padata_shell *ps,
 			      struct padata_priv *padata, int *cb_cpu);
 extern void padata_do_serial(struct padata_priv *padata);
-extern int padata_do_multithreaded(struct padata_mt_job *job);
+extern int padata_do_multithreaded_job(struct padata_mt_job *job,
+				       struct lock_class_key *key,
+				       const char *map_name);
 extern int padata_set_cpumask(struct padata_instance *pinst, int cpumask_type,
 			      cpumask_var_t cpumask);
 #endif
diff --git a/kernel/padata.c b/kernel/padata.c
index d0876f861464..b458deb17121 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -64,6 +64,9 @@ struct padata_mt_job_state {
 	unsigned long		position;
 	unsigned long		remaining_size;
 	struct list_head	failed_works;
+#ifdef CONFIG_LOCKDEP
+	struct lockdep_map	lockdep_map;
+#endif
 };
 
 static void padata_free_pd(struct parallel_data *pd);
@@ -470,9 +473,11 @@ static void padata_mt_helper(struct work_struct *w)
 		ps->remaining_size -= size;
 
 		spin_unlock(&ps->lock);
+		lock_map_acquire(&ps->lockdep_map);
 
 		ret = job->thread_fn(position, end, job->fn_arg);
 
+		lock_map_release(&ps->lockdep_map);
 		spin_lock(&ps->lock);
 
 		if (ret) {
@@ -552,14 +557,16 @@ static void padata_undo(struct padata_mt_job_state *ps,
 }
 
 /**
- * padata_do_multithreaded - run a multithreaded job
+ * padata_do_multithreaded_job - run a multithreaded job
  * @job: Description of the job.
  *
  * See the definition of struct padata_mt_job for more details.
  *
  * Return: 0 or a client-specific nonzero error code.
  */
-int padata_do_multithreaded(struct padata_mt_job *job)
+int padata_do_multithreaded_job(struct padata_mt_job *job,
+				struct lock_class_key *key,
+				const char *map_name)
 {
 	/* In case threads finish at different times. */
 	static const unsigned long load_balance_factor = 4;
@@ -583,6 +590,7 @@ int padata_do_multithreaded(struct padata_mt_job *job)
 
 	spin_lock_init(&ps.lock);
 	init_completion(&ps.completion);
+	lockdep_init_map(&ps.lockdep_map, map_name, key, 0);
 	INIT_LIST_HEAD(&ps.failed_works);
 	ps.job		  = job;
 	ps.nworks	  = padata_work_alloc_mt(nworks, &ps, &works);
@@ -601,6 +609,9 @@ int padata_do_multithreaded(struct padata_mt_job *job)
 	ps.chunk_size = max(ps.chunk_size, job->min_chunk);
 	ps.chunk_size = roundup(ps.chunk_size, job->align);
 
+	lock_map_acquire(&ps.lockdep_map);
+	lock_map_release(&ps.lockdep_map);
+
 	list_for_each_entry(pw, &works, pw_list)
 		queue_work(system_unbound_wq, &pw->pw_work);
 
-- 
2.34.1


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

* [RFC 05/16] vfio/type1: Pass mm to vfio_pin_pages_remote()
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (3 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 04/16] padata: Detect deadlocks between main and helper threads Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 06/16] vfio/type1: Refactor dma map removal Daniel Jordan
                   ` (11 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Refactoring for later, when padata helpers need to use the main thread's
mm.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 drivers/vfio/vfio_iommu_type1.c | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)

diff --git a/drivers/vfio/vfio_iommu_type1.c b/drivers/vfio/vfio_iommu_type1.c
index 0b4f7c174c7a..26bb2d9b698b 100644
--- a/drivers/vfio/vfio_iommu_type1.c
+++ b/drivers/vfio/vfio_iommu_type1.c
@@ -649,10 +649,10 @@ static int vfio_wait_all_valid(struct vfio_iommu *iommu)
  */
 static long vfio_pin_pages_remote(struct vfio_dma *dma, unsigned long vaddr,
 				  long npage, unsigned long *pfn_base,
-				  unsigned long limit, struct vfio_batch *batch)
+				  unsigned long limit, struct vfio_batch *batch,
+				  struct mm_struct *mm)
 {
 	unsigned long pfn;
-	struct mm_struct *mm = current->mm;
 	long ret, pinned = 0, lock_acct = 0;
 	bool rsvd;
 	dma_addr_t iova = vaddr - dma->vaddr + dma->iova;
@@ -1500,7 +1500,7 @@ static int vfio_pin_map_dma(struct vfio_iommu *iommu, struct vfio_dma *dma,
 		/* Pin a contiguous chunk of memory */
 		npage = vfio_pin_pages_remote(dma, vaddr + dma->size,
 					      size >> PAGE_SHIFT, &pfn, limit,
-					      &batch);
+					      &batch, current->mm);
 		if (npage <= 0) {
 			WARN_ON(!npage);
 			ret = (int)npage;
@@ -1763,7 +1763,8 @@ static int vfio_iommu_replay(struct vfio_iommu *iommu,
 				npage = vfio_pin_pages_remote(dma, vaddr,
 							      n >> PAGE_SHIFT,
 							      &pfn, limit,
-							      &batch);
+							      &batch,
+							      current->mm);
 				if (npage <= 0) {
 					WARN_ON(!npage);
 					ret = (int)npage;
-- 
2.34.1


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

* [RFC 06/16] vfio/type1: Refactor dma map removal
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (4 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 05/16] vfio/type1: Pass mm to vfio_pin_pages_remote() Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 07/16] vfio/type1: Parallelize vfio_pin_map_dma() Daniel Jordan
                   ` (10 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Do these small refactors to prepare for multithreaded page pinning:

 * pass @iova and @end args to vfio_unmap_unpin()
 * set iommu_mapped outside of vfio_unmap_unpin()
 * split part of vfio_remove_dma() off into vfio_remove_dma_finish()

They all facilitate padata's undo callback, which needs to undo only the
parts of the job that each helper completed successfully.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 drivers/vfio/vfio_iommu_type1.c | 23 +++++++++++++++--------
 1 file changed, 15 insertions(+), 8 deletions(-)

diff --git a/drivers/vfio/vfio_iommu_type1.c b/drivers/vfio/vfio_iommu_type1.c
index 26bb2d9b698b..8440e7e2c36d 100644
--- a/drivers/vfio/vfio_iommu_type1.c
+++ b/drivers/vfio/vfio_iommu_type1.c
@@ -1078,16 +1078,16 @@ static size_t unmap_unpin_slow(struct vfio_domain *domain,
 }
 
 static long vfio_unmap_unpin(struct vfio_iommu *iommu, struct vfio_dma *dma,
+			     dma_addr_t iova, dma_addr_t end,
 			     bool do_accounting)
 {
-	dma_addr_t iova = dma->iova, end = dma->iova + dma->size;
 	struct vfio_domain *domain, *d;
 	LIST_HEAD(unmapped_region_list);
 	struct iommu_iotlb_gather iotlb_gather;
 	int unmapped_region_cnt = 0;
 	long unlocked = 0;
 
-	if (!dma->size)
+	if (iova == end)
 		return 0;
 
 	if (!IS_IOMMU_CAP_DOMAIN_IN_CONTAINER(iommu))
@@ -1104,7 +1104,7 @@ static long vfio_unmap_unpin(struct vfio_iommu *iommu, struct vfio_dma *dma,
 				      struct vfio_domain, next);
 
 	list_for_each_entry_continue(d, &iommu->domain_list, next) {
-		iommu_unmap(d->domain, dma->iova, dma->size);
+		iommu_unmap(d->domain, iova, end - iova);
 		cond_resched();
 	}
 
@@ -1147,8 +1147,6 @@ static long vfio_unmap_unpin(struct vfio_iommu *iommu, struct vfio_dma *dma,
 		}
 	}
 
-	dma->iommu_mapped = false;
-
 	if (unmapped_region_cnt) {
 		unlocked += vfio_sync_unpin(dma, domain, &unmapped_region_list,
 					    &iotlb_gather);
@@ -1161,10 +1159,11 @@ static long vfio_unmap_unpin(struct vfio_iommu *iommu, struct vfio_dma *dma,
 	return unlocked;
 }
 
-static void vfio_remove_dma(struct vfio_iommu *iommu, struct vfio_dma *dma)
+static void vfio_remove_dma_finish(struct vfio_iommu *iommu,
+				   struct vfio_dma *dma)
 {
 	WARN_ON(!RB_EMPTY_ROOT(&dma->pfn_list));
-	vfio_unmap_unpin(iommu, dma, true);
+	dma->iommu_mapped = false;
 	vfio_unlink_dma(iommu, dma);
 	put_task_struct(dma->task);
 	vfio_dma_bitmap_free(dma);
@@ -1176,6 +1175,12 @@ static void vfio_remove_dma(struct vfio_iommu *iommu, struct vfio_dma *dma)
 	iommu->dma_avail++;
 }
 
+static void vfio_remove_dma(struct vfio_iommu *iommu, struct vfio_dma *dma)
+{
+	vfio_unmap_unpin(iommu, dma, dma->iova, dma->iova + dma->size, true);
+	vfio_remove_dma_finish(iommu, dma);
+}
+
 static void vfio_update_pgsize_bitmap(struct vfio_iommu *iommu)
 {
 	struct vfio_domain *domain;
@@ -2466,7 +2471,9 @@ static void vfio_iommu_unmap_unpin_reaccount(struct vfio_iommu *iommu)
 		long locked = 0, unlocked = 0;
 
 		dma = rb_entry(n, struct vfio_dma, node);
-		unlocked += vfio_unmap_unpin(iommu, dma, false);
+		unlocked += vfio_unmap_unpin(iommu, dma, dma->iova,
+					     dma->iova + dma->size, false);
+		dma->iommu_mapped = false;
 		p = rb_first(&dma->pfn_list);
 		for (; p; p = rb_next(p)) {
 			struct vfio_pfn *vpfn = rb_entry(p, struct vfio_pfn,
-- 
2.34.1


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

* [RFC 07/16] vfio/type1: Parallelize vfio_pin_map_dma()
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (5 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 06/16] vfio/type1: Refactor dma map removal Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention Daniel Jordan
                   ` (9 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

The VFIO_IOMMU_MAP_DMA ioctl uses a single CPU to pin all pages in the
given range to facilitate DMA to/from the passed-through device.  The
pages may not have been faulted in and cleared, in which case the wall
time for this can be truly horrendous, but even if this was already done
(e.g. qemu prealloc), pinning pages for the largest guests still takes
significant time, even with recent optimizations to hugetlb gup[1] and
ioctl(VFIO_IOMMU_MAP_DMA) itself[2].

Parallelize with padata for faster guest initialization times.  Numbers
come later on.

[1] https://lore.kernel.org/linux-mm/20210128182632.24562-1-joao.m.martins@oracle.com
[2] https://lore.kernel.org/lkml/20210219161305.36522-1-daniel.m.jordan@oracle.com/

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Suggested-by: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
---
 drivers/vfio/Kconfig            |  1 +
 drivers/vfio/vfio_iommu_type1.c | 95 +++++++++++++++++++++++++++------
 2 files changed, 80 insertions(+), 16 deletions(-)

diff --git a/drivers/vfio/Kconfig b/drivers/vfio/Kconfig
index 67d0bf4efa16..39c7efb7b1b1 100644
--- a/drivers/vfio/Kconfig
+++ b/drivers/vfio/Kconfig
@@ -2,6 +2,7 @@
 config VFIO_IOMMU_TYPE1
 	tristate
 	depends on VFIO
+	select PADATA
 	default n
 
 config VFIO_IOMMU_SPAPR_TCE
diff --git a/drivers/vfio/vfio_iommu_type1.c b/drivers/vfio/vfio_iommu_type1.c
index 8440e7e2c36d..faee849f1cce 100644
--- a/drivers/vfio/vfio_iommu_type1.c
+++ b/drivers/vfio/vfio_iommu_type1.c
@@ -40,6 +40,7 @@
 #include <linux/notifier.h>
 #include <linux/dma-iommu.h>
 #include <linux/irqdomain.h>
+#include <linux/padata.h>
 
 #define DRIVER_VERSION  "0.2"
 #define DRIVER_AUTHOR   "Alex Williamson <alex.williamson@redhat.com>"
@@ -1488,24 +1489,44 @@ static int vfio_iommu_map(struct vfio_iommu *iommu, dma_addr_t iova,
 	return ret;
 }
 
-static int vfio_pin_map_dma(struct vfio_iommu *iommu, struct vfio_dma *dma,
-			    size_t map_size)
+struct vfio_pin_args {
+	struct vfio_iommu *iommu;
+	struct vfio_dma *dma;
+	unsigned long limit;
+	struct mm_struct *mm;
+};
+
+static void vfio_pin_map_dma_undo(unsigned long start_vaddr,
+				  unsigned long end_vaddr, void *arg)
+{
+	struct vfio_pin_args *args = arg;
+	struct vfio_dma *dma = args->dma;
+	dma_addr_t iova = dma->iova + (start_vaddr - dma->vaddr);
+	dma_addr_t end  = dma->iova + (end_vaddr   - dma->vaddr);
+
+	vfio_unmap_unpin(args->iommu, args->dma, iova, end, true);
+}
+
+static int vfio_pin_map_dma_chunk(unsigned long start_vaddr,
+				  unsigned long end_vaddr, void *arg)
 {
-	dma_addr_t iova = dma->iova;
-	unsigned long vaddr = dma->vaddr;
+	struct vfio_pin_args *args = arg;
+	struct vfio_dma *dma = args->dma;
+	dma_addr_t iova = dma->iova + (start_vaddr - dma->vaddr);
+	unsigned long unmapped_size = end_vaddr - start_vaddr;
+	unsigned long pfn, mapped_size = 0;
 	struct vfio_batch batch;
-	size_t size = map_size;
 	long npage;
-	unsigned long pfn, limit = rlimit(RLIMIT_MEMLOCK) >> PAGE_SHIFT;
 	int ret = 0;
 
 	vfio_batch_init(&batch);
 
-	while (size) {
+	while (unmapped_size) {
 		/* Pin a contiguous chunk of memory */
-		npage = vfio_pin_pages_remote(dma, vaddr + dma->size,
-					      size >> PAGE_SHIFT, &pfn, limit,
-					      &batch, current->mm);
+		npage = vfio_pin_pages_remote(dma, start_vaddr + mapped_size,
+					      unmapped_size >> PAGE_SHIFT,
+					      &pfn, args->limit, &batch,
+					      args->mm);
 		if (npage <= 0) {
 			WARN_ON(!npage);
 			ret = (int)npage;
@@ -1513,24 +1534,66 @@ static int vfio_pin_map_dma(struct vfio_iommu *iommu, struct vfio_dma *dma,
 		}
 
 		/* Map it! */
-		ret = vfio_iommu_map(iommu, iova + dma->size, pfn, npage,
-				     dma->prot);
+		ret = vfio_iommu_map(args->iommu, iova + mapped_size, pfn,
+				     npage, dma->prot);
 		if (ret) {
-			vfio_unpin_pages_remote(dma, iova + dma->size, pfn,
+			vfio_unpin_pages_remote(dma, iova + mapped_size, pfn,
 						npage, true);
 			vfio_batch_unpin(&batch, dma);
 			break;
 		}
 
-		size -= npage << PAGE_SHIFT;
-		dma->size += npage << PAGE_SHIFT;
+		unmapped_size -= npage << PAGE_SHIFT;
+		mapped_size   += npage << PAGE_SHIFT;
 	}
 
 	vfio_batch_fini(&batch);
+
+	/*
+	 * Undo the successfully completed part of this chunk now.  padata will
+	 * undo previously completed chunks internally at the end of the job.
+	 */
+	if (ret) {
+		vfio_pin_map_dma_undo(start_vaddr, start_vaddr + mapped_size,
+				      args);
+		return ret;
+	}
+
+	return 0;
+}
+
+/* Small-memory guests benefited from this relatively small value in testing. */
+#define VFIO_MIN_CHUNK		(1ul << 27)
+
+/* The sweet spot between performance and efficiency on the test machines. */
+#define VFIO_MAX_THREADS	16
+
+static int vfio_pin_map_dma(struct vfio_iommu *iommu, struct vfio_dma *dma,
+			    size_t map_size)
+{
+	unsigned long limit = rlimit(RLIMIT_MEMLOCK) >> PAGE_SHIFT;
+	int ret = 0;
+	struct vfio_pin_args args = { iommu, dma, limit, current->mm };
+	/* Stay on PMD boundary in case THP is being used. */
+	struct padata_mt_job job = {
+		.thread_fn   = vfio_pin_map_dma_chunk,
+		.fn_arg      = &args,
+		.start       = dma->vaddr,
+		.size        = map_size,
+		.align       = PMD_SIZE,
+		.min_chunk   = VFIO_MIN_CHUNK,
+		.undo_fn     = vfio_pin_map_dma_undo,
+		.max_threads = VFIO_MAX_THREADS,
+	};
+
+	ret = padata_do_multithreaded(&job);
+
 	dma->iommu_mapped = true;
 
 	if (ret)
-		vfio_remove_dma(iommu, dma);
+		vfio_remove_dma_finish(iommu, dma);
+	else
+		dma->size += map_size;
 
 	return ret;
 }
-- 
2.34.1


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

* [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (6 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 07/16] vfio/type1: Parallelize vfio_pin_map_dma() Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:53   ` Jason Gunthorpe
  2022-01-06  0:46 ` [RFC 09/16] padata: Use kthreads in do_multithreaded Daniel Jordan
                   ` (8 subsequent siblings)
  16 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

padata threads hold mmap_lock as reader for the majority of their
runtime in order to call pin_user_pages_remote(), but they also
periodically take mmap_lock as writer for short periods to adjust
mm->locked_vm, hurting parallelism.

Alleviate the write-side contention with a per-thread cache of locked_vm
which allows taking mmap_lock as writer far less frequently.

Failure to refill the cache due to insufficient locked_vm will not cause
the entire pinning operation to error out.  This avoids spurious failure
in case some pinned pages aren't accounted to locked_vm.

Cache size is limited to provide some protection in the unlikely event
of a concurrent locked_vm accounting operation in the same address space
needlessly failing in case the cache takes more locked_vm than it needs.

Performance Testing
===================

The tests measure the time from qemu invocation to roughly the end of
qemu guest initialization, and cover all combinations of these
parameters:

 - guest memory type (hugetlb, THP)
 - guest memory size (16, 128, 360-or-980 G)
 - number of qemu prealloc threads (0, 16)

The goal is to find reasonable values for

 - number of padata threads (0, 8, 16, 24, 32)
 - locked_vm cache size in pages (0, 32768, 65536, 131072)

The winning compromises seem to be 16 threads and 65536 pages.  They
both balance between performance on the one hand and threading
efficiency or needless locked_vm accounting failures on the other.

Hardware info:

 - Intel Xeon Platinum 8167M (Skylake)
     2 nodes * 26 cores * 2 threads = 104 CPUs
     2.00GHz, performance scaling governor, turbo enabled
     384G/node = 768G memory

 - AMD EPYC 7J13 (Milan)
     2 nodes * 64 cores * 2 threads = 256 CPUs
     2.50GHz, performance scaling governor, turbo enabled
     ~1T/node = ~2T memory

The base kernel is 5.14.  I had to downgrade from 5.15 because of an
intel iommu bug that's since been fixed.  The qemu version is 6.2.0-rc4.

Key:

 qthr: number of qemu prealloc threads
 mem: guest memory size
 pin: wall time of the largest VFIO page pin (qemu does several)
 qemu init: wall time of qemu invocation to roughly the end of qemu init
 thr: number of padata threads

Summary Data
============

All tests in the summary section use 16 padata threads and 65536 pages
of locked_vm cache.

With these settings, there's some contention on pmd lock.  When
increasing the padata min chunk size from 128M to 1G to align threads on
PMD page table boundaries, the contention drops significantly but the
times get worse (don't you hate it when that happens?).  I'm planning to
look into this more.

qemu prealloc significantly reduces the pinning time, as expected, but
counterintuitively makes qemu on the base kernel take longer to
initialize a THP-backed guest than when qemu prealloc isn't used.
That's something to investigate, but not this series.

Intel
~~~~~

              base                    test
              ......................  ..........................................
                          qemu                                 qemu   qemu
       mem    pin         init             pin    pin          init   init
qthr   (G)    (s) (std)    (s) (std)   speedup    (s) (std) speedup    (s) (std)

hugetlb

   0    16    2.9 (0.0)    3.8 (0.0)     11.2x    0.3 (0.0)    5.2x    0.7 (0.0)
   0   128   26.6 (0.1)   28.0 (0.1)     12.0x    2.2 (0.0)    8.7x    3.2 (0.0)
   0   360   75.1 (0.5)   77.5 (0.5)     11.9x    6.3 (0.0)    9.2x    8.4 (0.0)
  16    16    0.1 (0.0)    0.7 (0.0)      2.5x    0.0 (0.0)    1.1x    0.7 (0.0)
  16   128    0.6 (0.0)    3.6 (0.0)      7.9x    0.1 (0.0)    1.2x    3.0 (0.0)
  16   360    1.8 (0.0)    9.4 (0.0)      8.5x    0.2 (0.0)    1.2x    7.8 (0.0)

THP

   0    16    3.3 (0.0)    4.2 (0.0)      7.3x    0.4 (0.0)    4.2x    1.0 (0.0)
   0   128   29.5 (0.2)   30.5 (0.2)     11.8x    2.5 (0.0)    9.6x    3.2 (0.0)
   0   360   83.8 (0.6)   85.1 (0.6)     11.9x    7.0 (0.0)   10.7x    8.0 (0.0)
  16    16    0.6 (0.0)    6.1 (0.0)      4.0x    0.1 (0.0)    1.1x    5.6 (0.1)
  16   128    5.1 (0.0)   44.5 (0.0)      9.6x    0.5 (0.0)    1.1x   40.3 (0.4)
  16   360   14.4 (0.0)  125.4 (0.3)      9.7x    1.5 (0.0)    1.1x  111.5 (0.8)

AMD
~~~

             base                     test
             .......................  ..........................................
                          qemu                                 qemu   qemu
       mem    pin         init             pin    pin          init   init
qthr   (G)    (s) (std)    (s) (std)   speedup    (s) (std) speedup    (s) (std)

hugetlb

   0    16    1.1 (0.0)    1.5 (0.0)      4.3x    0.2 (0.0)    2.6x    0.6 (0.0)
   0   128    9.6 (0.1)   10.2 (0.1)      4.3x    2.2 (0.0)    3.6x    2.8 (0.0)
   0   980   74.1 (0.7)   75.7 (0.7)      4.3x   17.1 (0.0)    3.9x   19.2 (0.0)
  16    16    0.0 (0.0)    0.6 (0.0)      3.2x    0.0 (0.0)    1.0x    0.6 (0.0)
  16   128    0.3 (0.0)    2.7 (0.0)      8.5x    0.0 (0.0)    1.1x    2.4 (0.0)
  16   980    2.0 (0.0)   18.2 (0.1)      8.1x    0.3 (0.0)    1.1x   16.4 (0.0)

THP

   0    16    1.2 (0.0)    1.7 (0.0)      4.0x    0.3 (0.0)    2.3x    0.7 (0.0)
   0   128   10.9 (0.1)   11.4 (0.1)      4.1x    2.7 (0.2)    3.7x    3.1 (0.2)
   0   980   85.3 (0.6)   86.1 (0.6)      4.7x   18.3 (0.0)    4.5x   19.0 (0.0)
  16    16    0.5 (0.3)    6.2 (0.4)      5.1x    0.1 (0.0)    1.1x    5.7 (0.1)
  16   128    3.4 (0.8)   45.5 (1.0)      8.5x    0.4 (0.1)    1.1x   42.1 (0.2)
  16   980   19.6 (0.9)  337.9 (0.7)      6.5x    3.0 (0.2)    1.1x  320.4 (0.7)

All Data
========

The first row in each table is the base kernel (0 threads).  The
remaining rows are all the test kernel and are sorted by fastest time.

Intel
~~~~~

hugetlb

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0   16        --    0          --    2.9 (0.0)         --    3.8 (0.0)
              65536   16       11.2x    0.3 (0.0)       5.2x    0.7 (0.0)
              65536   24       11.2x    0.3 (0.0)       5.2x    0.7 (0.0)
             131072   16       11.2x    0.3 (0.0)       5.1x    0.7 (0.0)
             131072   24       10.9x    0.3 (0.0)       5.1x    0.7 (0.0)
              32768   16       10.2x    0.3 (0.0)       5.1x    0.7 (0.0)
             131072   32       10.4x    0.3 (0.0)       5.1x    0.7 (0.0)
              65536   32       10.4x    0.3 (0.0)       5.1x    0.7 (0.0)
              32768   32       10.5x    0.3 (0.0)       5.1x    0.7 (0.0)
              32768   24       10.0x    0.3 (0.0)       5.1x    0.7 (0.0)
             131072    8        7.4x    0.4 (0.0)       4.2x    0.9 (0.0)
              65536    8        7.1x    0.4 (0.0)       4.1x    0.9 (0.0)
              32768    8        6.8x    0.4 (0.0)       4.1x    0.9 (0.0)
                  0    8        2.7x    1.1 (0.3)       2.3x    1.6 (0.3)
                  0   16        1.9x    1.6 (0.0)       1.7x    2.2 (0.0)
                  0   32        1.9x    1.6 (0.0)       1.7x    2.2 (0.0)
                  0   24        1.8x    1.6 (0.0)       1.7x    2.2 (0.0)
             131072    1        1.0x    2.9 (0.0)       1.0x    3.8 (0.0)
                  0    1        1.0x    2.9 (0.0)       1.0x    3.8 (0.0)
              65536    1        1.0x    2.9 (0.0)       1.0x    3.8 (0.0)
              32768    1        1.0x    3.0 (0.0)       1.0x    3.8 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  128        --    0          --   26.6 (0.1)         --   28.0 (0.1)
             131072   24       13.1x    2.0 (0.0)       9.2x    3.0 (0.0)
             131072   32       12.9x    2.1 (0.0)       9.2x    3.1 (0.0)
             131072   16       12.7x    2.1 (0.0)       9.1x    3.1 (0.0)
              65536   24       12.3x    2.2 (0.0)       8.9x    3.1 (0.0)
              65536   32       12.3x    2.2 (0.0)       8.9x    3.2 (0.0)
              65536   16       12.0x    2.2 (0.0)       8.7x    3.2 (0.0)
              32768   24       11.1x    2.4 (0.0)       8.3x    3.4 (0.0)
              32768   32       11.0x    2.4 (0.0)       8.2x    3.4 (0.0)
              32768   16       11.0x    2.4 (0.0)       8.2x    3.4 (0.0)
             131072    8        7.5x    3.6 (0.0)       6.1x    4.6 (0.0)
              65536    8        7.1x    3.7 (0.1)       5.9x    4.8 (0.0)
              32768    8        6.8x    3.9 (0.1)       5.7x    4.9 (0.1)
                  0    8        3.0x    8.9 (0.6)       2.8x   10.0 (0.7)
                  0   16        1.9x   13.8 (0.3)       1.9x   14.9 (0.3)
                  0   32        1.9x   14.1 (0.2)       1.8x   15.2 (0.3)
                  0   24        1.8x   14.4 (0.1)       1.8x   15.6 (0.1)
             131072    1        1.0x   26.4 (0.2)       1.0x   27.8 (0.2)
              65536    1        1.0x   26.5 (0.0)       1.0x   27.9 (0.0)
                  0    1        1.0x   26.6 (0.3)       1.0x   27.9 (0.3)
              32768    1        1.0x   26.6 (0.2)       1.0x   28.0 (0.2)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  360        --    0          --   75.1 (0.5)         --   77.5 (0.5)
             131072   24       13.0x    5.8 (0.0)       9.9x    7.8 (0.0)
             131072   32       12.9x    5.8 (0.0)       9.8x    7.9 (0.0)
             131072   16       12.6x    6.0 (0.0)       9.6x    8.1 (0.0)
              65536   24       12.4x    6.0 (0.0)       9.6x    8.1 (0.0)
              65536   32       12.1x    6.2 (0.0)       9.4x    8.3 (0.0)
              65536   16       11.9x    6.3 (0.0)       9.2x    8.4 (0.0)
              32768   24       11.3x    6.6 (0.0)       8.9x    8.7 (0.0)
              32768   16       10.9x    6.9 (0.0)       8.7x    9.0 (0.0)
              32768   32       10.7x    7.0 (0.1)       8.6x    9.1 (0.1)
             131072    8        7.4x   10.1 (0.0)       6.3x   12.3 (0.0)
              65536    8        7.2x   10.5 (0.1)       6.2x   12.6 (0.1)
              32768    8        6.8x   11.1 (0.1)       5.9x   13.2 (0.1)
                  0    8        3.2x   23.6 (0.3)       3.0x   25.7 (0.3)
                  0   32        1.9x   39.2 (0.2)       1.9x   41.5 (0.2)
                  0   16        1.9x   39.8 (0.4)       1.8x   42.0 (0.4)
                  0   24        1.8x   40.9 (0.4)       1.8x   43.1 (0.4)
              32768    1        1.0x   74.9 (0.5)       1.0x   77.3 (0.5)
             131072    1        1.0x   75.3 (0.6)       1.0x   77.7 (0.6)
                  0    1        1.0x   75.6 (0.2)       1.0x   78.1 (0.2)
              65536    1        1.0x   75.9 (0.1)       1.0x   78.4 (0.1)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16   16        --    0          --    0.1 (0.0)         --    0.7 (0.0)
              65536   24        8.3x    0.0 (0.0)       1.1x    0.7 (0.0)
              65536   32        6.3x    0.0 (0.0)       1.1x    0.7 (0.0)
             131072   32        4.2x    0.0 (0.0)       1.1x    0.7 (0.0)
              65536    8        4.2x    0.0 (0.0)       1.1x    0.7 (0.0)
             131072   24        4.2x    0.0 (0.0)       1.1x    0.7 (0.0)
              32768   16        3.9x    0.0 (0.0)       1.1x    0.7 (0.0)
              32768   32        3.5x    0.0 (0.0)       1.1x    0.7 (0.0)
              32768   24        4.0x    0.0 (0.0)       1.1x    0.7 (0.0)
              32768    8        2.6x    0.0 (0.0)       1.1x    0.7 (0.0)
                  0   16        3.1x    0.0 (0.0)       1.1x    0.7 (0.0)
             131072   16        2.7x    0.0 (0.0)       1.1x    0.7 (0.0)
              65536   16        2.5x    0.0 (0.0)       1.1x    0.7 (0.0)
                  0   24        2.5x    0.0 (0.0)       1.1x    0.7 (0.0)
                  0    8        2.8x    0.0 (0.0)       1.1x    0.7 (0.0)
             131072    8        2.2x    0.0 (0.0)       1.1x    0.7 (0.0)
                  0   32        2.3x    0.0 (0.0)       1.1x    0.7 (0.0)
              32768    1        0.9x    0.1 (0.0)       1.0x    0.8 (0.0)
             131072    1        0.9x    0.1 (0.0)       1.0x    0.8 (0.0)
              65536    1        0.9x    0.1 (0.0)       1.0x    0.8 (0.0)
                  0    1        0.9x    0.1 (0.0)       1.0x    0.8 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  128        --    0          --    0.6 (0.0)         --    3.6 (0.0)
             131072   24       13.4x    0.0 (0.0)       1.2x    3.0 (0.0)
              65536   32       11.8x    0.1 (0.0)       1.2x    3.0 (0.0)
             131072   32       11.8x    0.1 (0.0)       1.2x    3.0 (0.0)
              32768   32       10.4x    0.1 (0.0)       1.2x    3.0 (0.0)
              32768   24        9.3x    0.1 (0.0)       1.2x    3.0 (0.0)
             131072   16        8.7x    0.1 (0.0)       1.2x    3.0 (0.0)
              65536   16        7.9x    0.1 (0.0)       1.2x    3.0 (0.0)
              32768   16        7.7x    0.1 (0.0)       1.2x    3.0 (0.0)
              65536   24        7.6x    0.1 (0.0)       1.2x    3.0 (0.0)
             131072    8        5.7x    0.1 (0.0)       1.2x    3.0 (0.0)
              65536    8        4.9x    0.1 (0.0)       1.2x    3.1 (0.0)
              32768    8        4.6x    0.1 (0.0)       1.2x    3.1 (0.0)
                  0    8        3.9x    0.2 (0.0)       1.1x    3.1 (0.0)
                  0   16        3.1x    0.2 (0.1)       1.1x    3.1 (0.1)
                  0   24        2.9x    0.2 (0.0)       1.1x    3.2 (0.0)
                  0   32        2.6x    0.2 (0.0)       1.1x    3.2 (0.0)
             131072    1        0.9x    0.7 (0.0)       1.0x    3.6 (0.0)
              65536    1        0.9x    0.7 (0.0)       1.0x    3.6 (0.0)
              32768    1        0.9x    0.7 (0.0)       1.0x    3.6 (0.0)
                  0    1        0.9x    0.7 (0.0)       1.0x    3.6 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  360        --    0          --    1.8 (0.0)         --    9.4 (0.0)
             131072   32       15.1x    0.1 (0.0)       1.2x    7.7 (0.0)
              65536   32       13.5x    0.1 (0.0)       1.2x    7.7 (0.0)
              65536   24       11.6x    0.2 (0.0)       1.2x    7.8 (0.0)
             131072   16       11.5x    0.2 (0.0)       1.2x    7.8 (0.0)
              32768   32       11.3x    0.2 (0.0)       1.2x    7.8 (0.0)
              32768   24       10.5x    0.2 (0.0)       1.2x    7.8 (0.0)
             131072   24       10.4x    0.2 (0.0)       1.2x    7.8 (0.0)
              32768   16        8.8x    0.2 (0.0)       1.2x    7.8 (0.0)
              65536   16        8.5x    0.2 (0.0)       1.2x    7.8 (0.0)
             131072    8        6.1x    0.3 (0.0)       1.2x    7.9 (0.1)
              65536    8        5.5x    0.3 (0.0)       1.2x    7.9 (0.0)
              32768    8        5.3x    0.3 (0.0)       1.2x    7.9 (0.0)
                  0    8        4.8x    0.4 (0.1)       1.2x    8.0 (0.1)
                  0   16        3.3x    0.5 (0.1)       1.2x    8.1 (0.1)
                  0   24        3.1x    0.6 (0.0)       1.1x    8.2 (0.0)
                  0   32        2.7x    0.7 (0.0)       1.1x    8.3 (0.0)
             131072    1        0.9x    1.9 (0.0)       1.0x    9.5 (0.0)
              32768    1        0.9x    1.9 (0.0)       1.0x    9.5 (0.0)
              65536    1        0.9x    1.9 (0.0)       1.0x    9.6 (0.0)
                  0    1        0.9x    1.9 (0.0)       1.0x    9.5 (0.0)

THP

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0   16        --    0          --    3.3 (0.0)         --    4.2 (0.0)
              32768   32        7.5x    0.4 (0.0)       4.3x    1.0 (0.0)
             131072   32        7.6x    0.4 (0.0)       4.3x    1.0 (0.0)
              65536   16        7.3x    0.4 (0.0)       4.2x    1.0 (0.0)
              65536   32        7.5x    0.4 (0.0)       4.3x    1.0 (0.0)
             131072   16        7.2x    0.5 (0.0)       4.2x    1.0 (0.0)
              65536   24        7.0x    0.5 (0.0)       4.1x    1.0 (0.0)
             131072   24        6.9x    0.5 (0.0)       4.1x    1.0 (0.0)
              32768   16        6.3x    0.5 (0.0)       3.9x    1.1 (0.0)
              32768   24        5.7x    0.6 (0.0)       3.8x    1.1 (0.0)
              32768    8        5.0x    0.7 (0.0)       3.5x    1.2 (0.0)
              65536    8        5.4x    0.6 (0.0)       3.4x    1.2 (0.1)
             131072    8        5.7x    0.6 (0.0)       3.5x    1.2 (0.1)
                  0   32        2.0x    1.6 (0.1)       1.8x    2.3 (0.1)
                  0   24        1.9x    1.7 (0.0)       1.7x    2.5 (0.1)
                  0   16        1.8x    1.8 (0.3)       1.6x    2.6 (0.3)
                  0    8        1.9x    1.7 (0.3)       1.7x    2.5 (0.3)
                  0    1        1.0x    3.3 (0.0)       1.0x    4.2 (0.0)
             131072    1        1.0x    3.3 (0.0)       1.0x    4.2 (0.0)
              65536    1        1.0x    3.3 (0.0)       1.0x    4.2 (0.0)
              32768    1        1.0x    3.3 (0.0)       1.0x    4.2 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  128        --    0          --   29.5 (0.2)         --   30.5 (0.2)
             131072   24       12.9x    2.3 (0.0)      10.3x    2.9 (0.0)
             131072   32       12.8x    2.3 (0.0)      10.2x    3.0 (0.0)
             131072   16       12.5x    2.4 (0.0)      10.0x    3.0 (0.0)
              65536   24       12.1x    2.4 (0.0)       9.8x    3.1 (0.0)
              65536   32       12.0x    2.4 (0.0)       9.8x    3.1 (0.0)
              65536   16       11.8x    2.5 (0.0)       9.6x    3.2 (0.0)
              32768   24       11.1x    2.7 (0.0)       9.1x    3.3 (0.0)
              32768   32       10.7x    2.7 (0.0)       8.9x    3.4 (0.0)
              32768   16       10.6x    2.8 (0.0)       8.8x    3.5 (0.0)
             131072    8        7.3x    4.0 (0.0)       6.4x    4.8 (0.0)
              65536    8        7.1x    4.2 (0.0)       6.2x    4.9 (0.0)
              32768    8        6.6x    4.4 (0.0)       5.8x    5.2 (0.0)
                  0    8        3.6x    8.1 (0.7)       3.4x    9.0 (0.7)
                  0   32        2.2x   13.6 (1.9)       2.1x   14.5 (1.9)
                  0   16        2.1x   14.0 (3.2)       2.1x   14.8 (3.2)
                  0   24        2.1x   14.1 (3.1)       2.0x   15.0 (3.1)
                  0    1        1.0x   29.6 (0.2)       1.0x   30.6 (0.2)
              32768    1        1.0x   29.6 (0.2)       1.0x   30.7 (0.2)
             131072    1        1.0x   29.7 (0.0)       1.0x   30.7 (0.0)
              65536    1        1.0x   29.8 (0.1)       1.0x   30.8 (0.1)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  360        --    0          --   83.8 (0.6)         --   85.1 (0.6)
             131072   24       13.6x    6.2 (0.0)      12.0x    7.1 (0.0)
             131072   32       13.4x    6.2 (0.0)      11.9x    7.2 (0.0)
              65536   24       12.8x    6.6 (0.1)      11.3x    7.5 (0.1)
             131072   16       12.7x    6.6 (0.0)      11.3x    7.5 (0.0)
              65536   32       12.4x    6.8 (0.0)      11.0x    7.7 (0.0)
              65536   16       11.9x    7.0 (0.0)      10.7x    8.0 (0.0)
              32768   24       11.4x    7.4 (0.0)      10.3x    8.3 (0.0)
              32768   32       11.0x    7.6 (0.0)      10.0x    8.5 (0.0)
              32768   16       10.7x    7.8 (0.0)       9.7x    8.8 (0.0)
             131072    8        7.4x   11.4 (0.0)       6.8x   12.4 (0.0)
              65536    8        7.2x   11.7 (0.0)       6.7x   12.7 (0.0)
              32768    8        6.7x   12.6 (0.1)       6.2x   13.6 (0.1)
                  0    8        3.1x   27.2 (6.1)       3.0x   28.3 (6.1)
                  0   32        2.1x   39.9 (6.4)       2.1x   41.0 (6.4)
                  0   24        2.1x   40.6 (6.6)       2.0x   41.7 (6.6)
                  0   16        2.0x   42.6 (0.1)       1.9x   43.8 (0.1)
             131072    1        1.0x   83.9 (0.5)       1.0x   85.2 (0.5)
              65536    1        1.0x   84.2 (0.3)       1.0x   85.5 (0.3)
              32768    1        1.0x   84.6 (0.1)       1.0x   85.9 (0.1)
                  0    1        1.0x   84.9 (0.1)       1.0x   86.2 (0.1)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16   16        --    0          --    0.6 (0.0)         --    6.1 (0.0)
              65536   32        3.9x    0.1 (0.0)       1.1x    5.5 (0.0)
              32768   32        3.9x    0.1 (0.0)       1.1x    5.6 (0.0)
             131072   24        3.9x    0.1 (0.0)       1.1x    5.6 (0.0)
             131072   32        3.9x    0.1 (0.0)       1.1x    5.5 (0.0)
              65536   24        3.9x    0.1 (0.0)       1.1x    5.6 (0.0)
              32768   24        3.9x    0.1 (0.0)       1.1x    5.6 (0.1)
              65536   16        4.0x    0.1 (0.0)       1.1x    5.6 (0.1)
              32768   16        3.9x    0.1 (0.0)       1.1x    5.6 (0.0)
             131072   16        3.9x    0.1 (0.0)       1.1x    5.6 (0.1)
              65536    8        4.0x    0.1 (0.0)       1.1x    5.6 (0.0)
             131072    8        4.0x    0.1 (0.0)       1.1x    5.7 (0.1)
              32768    8        4.0x    0.1 (0.0)       1.1x    5.6 (0.0)
                  0   32        1.6x    0.4 (0.0)       1.0x    5.9 (0.1)
                  0   24        1.6x    0.4 (0.0)       1.0x    5.9 (0.0)
                  0   16        1.5x    0.4 (0.0)       1.0x    6.0 (0.0)
                  0    8        1.5x    0.4 (0.0)       1.0x    5.9 (0.1)
              65536    1        1.0x    0.6 (0.0)       1.0x    6.1 (0.1)
              32768    1        1.0x    0.6 (0.0)       1.0x    6.1 (0.0)
                  0    1        1.0x    0.6 (0.0)       1.0x    6.2 (0.0)
             131072    1        1.0x    0.6 (0.0)       1.0x    6.2 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  128        --    0          --    5.1 (0.0)         --   44.5 (0.0)
             131072   32       16.5x    0.3 (0.0)       1.1x   40.4 (0.3)
              65536   32       15.7x    0.3 (0.0)       1.1x   40.4 (0.6)
             131072   24       13.9x    0.4 (0.0)       1.1x   39.8 (0.3)
              32768   32       14.1x    0.4 (0.0)       1.1x   40.0 (0.5)
              65536   24       12.9x    0.4 (0.0)       1.1x   39.8 (0.5)
              32768   24       12.2x    0.4 (0.0)       1.1x   40.1 (0.1)
              65536   16        9.6x    0.5 (0.0)       1.1x   40.3 (0.4)
             131072   16        9.7x    0.5 (0.0)       1.1x   40.4 (0.5)
              32768   16        9.2x    0.5 (0.0)       1.1x   40.8 (0.5)
              65536    8        5.5x    0.9 (0.0)       1.1x   40.5 (0.5)
             131072    8        5.5x    0.9 (0.0)       1.1x   40.7 (0.6)
              32768    8        5.2x    1.0 (0.0)       1.1x   40.7 (0.3)
                  0   32        1.6x    3.1 (0.0)       1.0x   43.5 (0.8)
                  0   24        1.6x    3.2 (0.0)       1.0x   42.9 (0.5)
                  0   16        1.5x    3.3 (0.0)       1.0x   43.5 (0.4)
                  0    8        1.5x    3.5 (0.0)       1.0x   43.4 (0.5)
              65536    1        1.0x    5.0 (0.0)       1.0x   44.6 (0.1)
              32768    1        1.0x    5.0 (0.0)       1.0x   44.9 (0.2)
             131072    1        1.0x    5.0 (0.0)       1.0x   44.8 (0.2)
                  0    1        1.0x    5.0 (0.0)       1.0x   44.8 (0.3)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  360        --    0          --   14.4 (0.0)         --  125.4 (0.3)
             131072   32       16.5x    0.9 (0.0)       1.1x  112.0 (0.7)
              65536   32       14.9x    1.0 (0.0)       1.1x  113.3 (1.3)
              32768   32       14.0x    1.0 (0.0)       1.1x  112.6 (1.0)
             131072   24       13.5x    1.1 (0.0)       1.1x  111.3 (0.9)
              65536   24       13.3x    1.1 (0.0)       1.1x  112.3 (0.8)
              32768   24       12.4x    1.2 (0.0)       1.1x  111.1 (0.8)
              65536   16        9.7x    1.5 (0.0)       1.1x  111.5 (0.8)
             131072   16        9.7x    1.5 (0.0)       1.1x  112.1 (1.2)
              32768   16        9.3x    1.5 (0.0)       1.1x  113.2 (0.4)
             131072    8        5.5x    2.6 (0.0)       1.1x  114.8 (1.3)
              32768    8        5.5x    2.6 (0.0)       1.1x  114.1 (1.0)
              65536    8        5.4x    2.6 (0.0)       1.1x  115.0 (3.3)
                  0   32        1.6x    8.8 (0.0)       1.0x  120.7 (0.7)
                  0   24        1.6x    8.9 (0.0)       1.1x  119.4 (0.1)
                  0   16        1.5x    9.5 (0.0)       1.0x  120.1 (0.7)
                  0    8        1.4x   10.1 (0.2)       1.0x  123.6 (1.9)
              32768    1        1.0x   14.3 (0.0)       1.0x  126.2 (0.9)
              65536    1        1.0x   14.3 (0.0)       1.0x  125.4 (0.6)
             131072    1        1.0x   14.3 (0.0)       1.0x  126.5 (1.0)
                  0    1        1.0x   14.3 (0.0)       1.0x  124.7 (1.2)

AMD
~~~

hugetlb

           lockedvm                                    qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0   16        --    0          --    1.1 (0.0)         --    1.5 (0.0)
             131072    8        4.3x    0.2 (0.0)       2.5x    0.6 (0.0)
              65536   16        4.3x    0.2 (0.0)       2.6x    0.6 (0.0)
              65536    8        4.0x    0.3 (0.0)       2.5x    0.6 (0.0)
              65536   24        3.8x    0.3 (0.0)       2.4x    0.6 (0.0)
              32768   32        3.6x    0.3 (0.0)       2.3x    0.6 (0.0)
             131072   32        3.6x    0.3 (0.0)       2.3x    0.6 (0.0)
              65536   32        3.5x    0.3 (0.0)       2.3x    0.6 (0.0)
              32768    8        3.4x    0.3 (0.0)       2.3x    0.7 (0.0)
             131072   24        3.0x    0.3 (0.0)       2.1x    0.7 (0.0)
             131072   16        2.8x    0.4 (0.0)       2.0x    0.8 (0.1)
              32768   16        2.6x    0.4 (0.0)       1.9x    0.8 (0.0)
              32768   24        2.6x    0.4 (0.0)       1.9x    0.8 (0.0)
                  0   32        1.3x    0.8 (0.0)       1.2x    1.2 (0.0)
                  0   24        1.3x    0.8 (0.0)       1.2x    1.3 (0.0)
                  0   16        1.2x    0.9 (0.0)       1.2x    1.3 (0.0)
                  0    8        1.1x    0.9 (0.0)       1.1x    1.4 (0.0)
              32768    1        1.0x    1.1 (0.0)       1.0x    1.5 (0.0)
             131072    1        1.0x    1.1 (0.0)       1.0x    1.5 (0.0)
                  0    1        1.0x    1.1 (0.0)       1.0x    1.5 (0.0)
              65536    1        1.0x    1.1 (0.0)       1.0x    1.5 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  128        --    0          --    9.6 (0.1)         --   10.2 (0.1)
             131072   32        4.5x    2.1 (0.0)       3.9x    2.6 (0.0)
             131072    8        4.4x    2.2 (0.0)       3.7x    2.8 (0.1)
              65536   16        4.3x    2.2 (0.0)       3.6x    2.8 (0.0)
             131072   16        4.2x    2.3 (0.1)       3.6x    2.9 (0.0)
              65536    8        4.1x    2.3 (0.0)       3.6x    2.8 (0.0)
             131072   24        4.1x    2.4 (0.1)       3.5x    3.0 (0.1)
              65536   24        3.8x    2.5 (0.0)       3.4x    3.0 (0.0)
              65536   32        3.8x    2.5 (0.0)       3.3x    3.1 (0.0)
              32768   32        3.6x    2.6 (0.0)       3.3x    3.1 (0.0)
              32768    8        3.3x    2.9 (0.1)       2.9x    3.5 (0.1)
              32768   16        3.2x    3.0 (0.3)       2.9x    3.5 (0.3)
              32768   24        2.5x    3.8 (0.0)       2.3x    4.4 (0.0)
                  0   16        1.2x    7.8 (0.1)       1.2x    8.4 (0.1)
                  0    8        1.2x    8.3 (0.1)       1.1x    8.9 (0.1)
              32768    1        1.0x    9.6 (0.1)       1.0x   10.3 (0.1)
              65536    1        1.0x    9.6 (0.0)       1.0x   10.3 (0.1)
             131072    1        1.0x    9.7 (0.0)       1.0x   10.3 (0.0)
                  0    1        1.0x    9.7 (0.0)       1.0x   10.4 (0.0)
                  0   24        0.9x   10.2 (0.6)       0.9x   10.8 (0.6)
                  0   32        0.9x   10.5 (0.5)       0.9x   11.2 (0.5)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  980        --    0          --   74.1 (0.7)         --   75.7 (0.7)
             131072   16        4.7x   15.9 (0.1)       4.3x   17.4 (0.1)
             131072   24        4.6x   16.0 (0.0)       4.2x   18.1 (0.0)
             131072   32        4.6x   16.3 (0.0)       4.1x   18.4 (0.0)
             131072    8        4.4x   16.9 (0.1)       4.1x   18.5 (0.1)
              65536   16        4.3x   17.1 (0.0)       3.9x   19.2 (0.0)
              65536   24        4.3x   17.4 (0.0)       3.9x   19.5 (0.0)
              65536   32        4.2x   17.7 (0.0)       3.8x   19.9 (0.1)
              65536    8        4.1x   18.2 (0.0)       3.7x   20.4 (0.0)
              32768   24        3.7x   19.8 (0.1)       3.4x   22.0 (0.1)
              32768   16        3.7x   20.2 (0.2)       3.5x   21.8 (0.2)
              32768   32        3.6x   20.4 (0.1)       3.4x   22.5 (0.1)
              32768    8        3.4x   21.6 (0.5)       3.3x   23.1 (0.5)
                  0   16        1.2x   60.4 (0.6)       1.2x   62.0 (0.6)
                  0    8        1.1x   65.3 (1.0)       1.1x   67.6 (1.0)
                  0   24        1.0x   73.1 (2.7)       1.0x   75.4 (2.6)
             131072    1        1.0x   75.0 (0.7)       1.0x   77.3 (0.7)
              65536    1        1.0x   75.4 (0.7)       1.0x   77.7 (0.7)
                  0    1        1.0x   75.6 (0.7)       1.0x   77.8 (0.7)
              32768    1        1.0x   75.8 (0.0)       1.0x   78.0 (0.0)
                  0   32        0.8x   92.9 (1.2)       0.8x   95.3 (1.1)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16   16        --    0          --    0.0 (0.0)         --    0.6 (0.0)
             131072   24        5.6x    0.0 (0.0)       1.0x    0.6 (0.0)
              32768   16        4.6x    0.0 (0.0)       1.0x    0.6 (0.0)
              32768   32        4.8x    0.0 (0.0)       0.9x    0.6 (0.0)
             131072   16        4.6x    0.0 (0.0)       1.0x    0.6 (0.0)
             131072   32        4.3x    0.0 (0.0)       1.0x    0.6 (0.0)
             131072    8        4.5x    0.0 (0.0)       1.0x    0.6 (0.0)
              32768    8        4.4x    0.0 (0.0)       1.0x    0.6 (0.0)
              65536   24        3.7x    0.0 (0.0)       1.0x    0.6 (0.0)
              65536   16        3.2x    0.0 (0.0)       1.0x    0.6 (0.0)
              65536    8        2.8x    0.0 (0.0)       1.0x    0.6 (0.0)
              32768   24        3.0x    0.0 (0.0)       1.0x    0.6 (0.0)
              65536   32        2.6x    0.0 (0.0)       1.0x    0.6 (0.0)
                  0   32        2.1x    0.0 (0.0)       1.0x    0.6 (0.0)
                  0   16        2.3x    0.0 (0.0)       0.9x    0.6 (0.0)
                  0    8        2.2x    0.0 (0.0)       0.9x    0.6 (0.0)
                  0   24        1.2x    0.0 (0.0)       1.0x    0.6 (0.0)
             131072    1        1.0x    0.0 (0.0)       0.9x    0.6 (0.0)
              65536    1        1.0x    0.0 (0.0)       0.9x    0.7 (0.0)
              32768    1        0.8x    0.0 (0.0)       0.9x    0.6 (0.0)
                  0    1        0.9x    0.0 (0.0)       1.0x    0.6 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  128        --    0          --    0.3 (0.0)         --    2.7 (0.0)
             131072   24       10.4x    0.0 (0.0)       1.1x    2.4 (0.0)
              65536   16        8.5x    0.0 (0.0)       1.1x    2.4 (0.0)
              32768   24        7.7x    0.0 (0.0)       1.1x    2.4 (0.0)
              32768   32        7.6x    0.0 (0.0)       1.1x    2.4 (0.0)
              65536   24        6.1x    0.0 (0.0)       1.1x    2.4 (0.0)
             131072   16        5.8x    0.0 (0.0)       1.1x    2.4 (0.0)
             131072   32        5.6x    0.0 (0.0)       1.1x    2.4 (0.0)
              32768    8        5.2x    0.1 (0.0)       1.1x    2.4 (0.0)
              65536   32        4.8x    0.1 (0.0)       1.1x    2.5 (0.0)
              32768   16        4.9x    0.1 (0.0)       1.1x    2.4 (0.0)
             131072    8        4.4x    0.1 (0.0)       1.1x    2.4 (0.0)
              65536    8        4.2x    0.1 (0.0)       1.1x    2.4 (0.0)
                  0    8        2.9x    0.1 (0.0)       1.1x    2.4 (0.0)
                  0   16        2.9x    0.1 (0.0)       1.1x    2.5 (0.0)
                  0   24        2.8x    0.1 (0.0)       1.1x    2.4 (0.0)
                  0   32        1.2x    0.2 (0.0)       1.0x    2.6 (0.0)
              32768    1        1.0x    0.3 (0.0)       1.0x    2.7 (0.0)
             131072    1        1.0x    0.3 (0.0)       1.0x    2.7 (0.0)
              65536    1        1.0x    0.3 (0.0)       1.0x    2.7 (0.0)
                  0    1        0.9x    0.3 (0.0)       1.0x    2.7 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  980        --    0          --    2.0 (0.0)         --   18.2 (0.1)
             131072   32       11.2x    0.2 (0.0)       1.2x   15.7 (0.0)
             131072   16        9.4x    0.2 (0.0)       1.2x   15.7 (0.0)
              65536   24        9.2x    0.2 (0.0)       1.1x   16.3 (0.0)
              65536   16        8.1x    0.3 (0.0)       1.1x   16.4 (0.0)
              32768   16        7.1x    0.3 (0.0)       1.1x   15.8 (0.0)
             131072   24        7.1x    0.3 (0.0)       1.1x   15.8 (0.0)
              65536   32        6.2x    0.3 (0.0)       1.1x   16.4 (0.0)
              65536    8        5.7x    0.4 (0.0)       1.1x   16.5 (0.1)
              32768   32        5.6x    0.4 (0.0)       1.1x   16.5 (0.0)
              32768   24        5.6x    0.4 (0.0)       1.1x   15.9 (0.0)
             131072    8        5.0x    0.4 (0.0)       1.1x   16.0 (0.0)
              32768    8        3.0x    0.7 (0.0)       1.1x   16.3 (0.1)
                  0    8        2.8x    0.7 (0.0)       1.1x   16.2 (0.0)
                  0   16        2.7x    0.8 (0.1)       1.1x   16.9 (0.1)
                  0   24        1.6x    1.2 (0.4)       1.0x   17.4 (0.4)
              32768    1        1.0x    2.1 (0.0)       1.0x   18.1 (0.0)
                  0   32        1.0x    2.1 (0.0)       1.0x   17.7 (0.0)
              65536    1        1.0x    2.1 (0.0)       1.0x   18.2 (0.1)
             131072    1        1.0x    2.1 (0.0)       1.0x   18.3 (0.0)
                  0    1        0.9x    2.2 (0.0)       1.0x   17.7 (0.0)

THP

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0   16        --    0          --    1.2 (0.0)         --    1.7 (0.0)
             131072    8        4.3x    0.3 (0.0)       2.4x    0.7 (0.0)
             131072   32        3.1x    0.4 (0.0)       2.1x    0.8 (0.0)
              65536   16        4.0x    0.3 (0.0)       2.3x    0.7 (0.0)
              65536    8        3.9x    0.3 (0.0)       2.3x    0.7 (0.0)
              65536   24        3.3x    0.4 (0.0)       2.1x    0.8 (0.0)
              65536   32        3.3x    0.4 (0.0)       2.2x    0.8 (0.0)
              32768   16        2.6x    0.5 (0.0)       1.9x    0.9 (0.0)
             131072   24        3.3x    0.4 (0.0)       2.1x    0.8 (0.0)
              32768   32        3.3x    0.4 (0.0)       2.1x    0.8 (0.0)
             131072   16        3.1x    0.4 (0.0)       2.0x    0.8 (0.0)
              32768   24        2.5x    0.5 (0.0)       1.9x    0.9 (0.0)
              32768    8        3.2x    0.4 (0.0)       1.9x    0.9 (0.0)
                  0   24        1.3x    1.0 (0.0)       1.2x    1.4 (0.0)
                  0   32        1.2x    1.0 (0.0)       1.1x    1.5 (0.1)
                  0    8        1.2x    1.0 (0.0)       1.1x    1.5 (0.0)
                  0   16        1.2x    1.0 (0.0)       1.1x    1.5 (0.0)
             131072    1        1.0x    1.2 (0.0)       1.0x    1.7 (0.0)
              65536    1        1.0x    1.2 (0.0)       1.0x    1.7 (0.0)
                  0    1        1.0x    1.2 (0.0)       1.0x    1.7 (0.0)
              32768    1        1.0x    1.2 (0.0)       1.0x    1.7 (0.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  128        --    0          --   10.9 (0.1)         --   11.4 (0.1)
             131072   16        5.0x    2.2 (0.0)       4.3x    2.7 (0.0)
             131072   32        4.8x    2.3 (0.0)       4.2x    2.7 (0.0)
             131072   24        4.6x    2.4 (0.0)       4.1x    2.8 (0.1)
             131072    8        4.7x    2.3 (0.0)       4.1x    2.8 (0.0)
              65536   24        4.4x    2.5 (0.0)       3.9x    2.9 (0.0)
              65536   32        4.1x    2.7 (0.1)       3.7x    3.1 (0.1)
              65536   16        4.1x    2.7 (0.2)       3.7x    3.1 (0.2)
              65536    8        4.0x    2.7 (0.1)       3.6x    3.2 (0.1)
              32768   24        3.8x    2.9 (0.0)       3.4x    3.4 (0.0)
              32768   32        3.6x    3.0 (0.1)       3.3x    3.5 (0.1)
              32768    8        3.5x    3.1 (0.0)       3.2x    3.6 (0.1)
              32768   16        3.3x    3.3 (0.2)       3.1x    3.7 (0.2)
                  0   16        1.3x    8.3 (0.4)       1.3x    8.8 (0.4)
                  0    8        1.2x    8.8 (0.4)       1.2x    9.3 (0.4)
                  0   24        1.1x   10.1 (1.2)       1.1x   10.7 (1.3)
                  0   32        1.1x   10.3 (1.3)       1.1x   10.8 (1.3)
             131072    1        1.0x   10.9 (0.0)       1.0x   11.5 (0.0)
              32768    1        1.0x   11.0 (0.1)       1.0x   11.6 (0.1)
              65536    1        1.0x   11.1 (0.0)       1.0x   11.6 (0.0)
                  0    1        1.0x   11.1 (0.2)       1.0x   11.6 (0.2)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

   0  980        --    0          --   85.3 (0.6)         --   86.1 (0.6)
             131072   16        5.2x   16.4 (0.0)       5.0x   17.1 (0.0)
             131072   24        5.1x   16.7 (0.1)       4.9x   17.4 (0.1)
             131072   32        5.0x   17.1 (0.0)       4.8x   17.8 (0.0)
             131072    8        4.7x   18.3 (0.1)       4.5x   19.0 (0.1)
              65536   16        4.7x   18.3 (0.0)       4.5x   19.0 (0.0)
              65536   24        4.6x   18.5 (0.0)       4.5x   19.2 (0.0)
              65536   32        4.5x   18.8 (0.0)       4.4x   19.6 (0.0)
              65536    8        4.3x   19.6 (0.0)       4.2x   20.4 (0.0)
              32768   16        3.9x   21.6 (0.0)       3.9x   22.4 (0.0)
              32768   24        3.9x   22.1 (0.3)       3.8x   22.8 (0.3)
              32768   32        3.8x   22.4 (0.1)       3.7x   23.1 (0.1)
              32768    8        3.8x   22.7 (0.0)       3.7x   23.5 (0.0)
                  0   16        1.3x   64.6 (2.7)       1.3x   65.4 (2.7)
                  0    8        1.2x   70.0 (2.7)       1.2x   70.8 (2.7)
                  0   32        1.0x   82.4 (5.7)       1.0x   83.2 (5.7)
                  0   24        1.0x   83.4 (6.9)       1.0x   84.1 (6.9)
             131072    1        1.0x   84.2 (0.3)       1.0x   85.0 (0.3)
                  0    1        1.0x   84.8 (1.3)       1.0x   85.6 (1.3)
              65536    1        1.0x   84.9 (0.4)       1.0x   85.7 (0.4)
              32768    1        1.0x   85.6 (1.3)       1.0x   86.4 (1.3)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16   16        --    0          --    0.5 (0.3)         --    6.2 (0.4)
             131072   32        4.9x    0.1 (0.0)       1.1x    5.6 (0.0)
              65536   16        5.1x    0.1 (0.0)       1.1x    5.7 (0.1)
              65536   32        5.0x    0.1 (0.0)       1.1x    5.6 (0.1)
              32768   16        5.0x    0.1 (0.0)       1.1x    5.7 (0.0)
              32768    8        5.8x    0.1 (0.0)       1.1x    5.6 (0.0)
              65536   24        5.7x    0.1 (0.0)       1.1x    5.7 (0.0)
              32768   32        3.9x    0.1 (0.0)       1.0x    5.9 (0.1)
             131072   16        3.7x    0.1 (0.1)       1.0x    6.0 (0.3)
              65536    8        4.0x    0.1 (0.1)       1.1x    5.9 (0.1)
             131072   24        3.6x    0.1 (0.1)       1.0x    5.9 (0.5)
             131072    8        2.5x    0.2 (0.1)       1.0x    6.0 (0.6)
              32768   24        1.7x    0.3 (0.1)       1.0x    6.5 (0.2)
             131072    1        1.8x    0.3 (0.0)       1.1x    5.9 (0.0)
                  0   32        1.6x    0.3 (0.0)       1.0x    6.2 (0.2)
                  0    8        1.0x    0.5 (0.0)       1.0x    6.2 (0.1)
                  0   24        0.9x    0.5 (0.3)       1.0x    6.3 (0.5)
                  0    1        0.9x    0.5 (0.4)       1.0x    6.2 (0.5)
              32768    1        0.8x    0.6 (0.3)       1.0x    6.5 (0.4)
                  0   16        0.7x    0.7 (0.7)       0.9x    6.6 (0.8)
              65536    1        0.6x    0.9 (0.5)       0.9x    6.7 (0.7)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  128        --    0          --    3.4 (0.8)         --   45.5 (1.0)
             131072   32       11.7x    0.3 (0.1)       1.1x   42.1 (0.2)
              65536   16        8.5x    0.4 (0.1)       1.1x   42.1 (0.2)
              32768   32        8.6x    0.4 (0.2)       1.1x   43.0 (0.2)
              65536   32        8.9x    0.4 (0.1)       1.0x   43.6 (0.3)
              32768   24        7.9x    0.4 (0.1)       1.1x   42.3 (0.3)
              32768   16        6.5x    0.5 (0.2)       1.1x   42.5 (0.5)
              65536   24        6.7x    0.5 (0.2)       1.1x   42.6 (0.5)
             131072   24        5.8x    0.6 (0.5)       1.1x   42.5 (0.6)
             131072   16        5.0x    0.7 (0.6)       1.1x   42.4 (0.8)
             131072    8        3.8x    0.9 (0.4)       1.1x   42.7 (0.5)
              65536    8        3.2x    1.1 (0.5)       1.1x   42.9 (0.6)
              32768    8        3.1x    1.1 (0.4)       1.1x   43.3 (1.0)
                  0   32        1.1x    3.0 (0.2)       1.0x   45.1 (0.2)
                  0   24        1.2x    2.9 (0.1)       1.0x   44.6 (0.2)
                  0    8        1.0x    3.5 (1.1)       1.0x   45.5 (1.2)
              32768    1        1.0x    3.6 (0.9)       1.0x   45.5 (0.7)
             131072    1        1.0x    3.5 (1.1)       1.0x   45.6 (1.4)
                  0    1        0.9x    3.6 (0.5)       1.0x   45.6 (0.4)
                  0   16        0.9x    3.6 (0.2)       1.0x   45.7 (0.2)
              65536    1        0.9x    3.6 (1.0)       1.0x   45.8 (1.0)

           lockedvm                                     qemu   qemu
      mem     cache              pin    pin             init   init
qthr  (G)   (pages)  thr     speedup    (s) (std)    speedup    (s) (std)

  16  980        --    0          --   19.6 (0.9)         --  337.9 (0.7)
             131072   32        9.7x    2.0 (0.4)       1.0x  323.0 (0.7)
             131072   24        8.8x    2.2 (0.4)       1.0x  324.6 (0.8)
              65536   32        8.4x    2.3 (0.2)       1.0x  323.1 (0.5)
              32768   24        7.9x    2.5 (0.1)       1.1x  319.4 (1.0)
              65536   24        8.1x    2.4 (0.1)       1.0x  322.3 (0.8)
              32768   32        7.4x    2.6 (0.2)       1.1x  321.2 (0.8)
             131072   16        6.9x    2.8 (0.3)       1.0x  331.0 (8.8)
              65536   16        6.5x    3.0 (0.2)       1.1x  320.4 (0.7)
              32768   16        5.9x    3.3 (0.5)       1.0x  328.3 (1.5)
              65536    8        5.3x    3.7 (0.4)       1.1x  320.8 (1.0)
              32768    8        4.8x    4.1 (0.2)       1.0x  328.9 (0.8)
             131072    8        4.7x    4.1 (0.2)       1.1x  319.4 (0.9)
                  0    8        1.2x   16.9 (0.7)       1.0x  333.9 (3.1)
                  0   32        1.1x   18.0 (0.7)       1.0x  336.1 (0.8)
                  0   24        1.1x   18.0 (1.6)       1.0x  336.7 (1.7)
              65536    1        1.0x   19.0 (0.5)       1.0x  341.0 (0.3)
             131072    1        1.0x   19.7 (1.0)       1.0x  335.7 (1.0)
                  0   16        1.0x   19.8 (1.8)       1.0x  338.8 (1.8)
              32768    1        0.9x   20.7 (1.5)       1.0x  337.6 (1.9)
                  0    1        0.9x   21.3 (1.4)       1.0x  339.5 (1.8)

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 drivers/vfio/vfio_iommu_type1.c | 51 +++++++++++++++++++++++++++++----
 1 file changed, 45 insertions(+), 6 deletions(-)

diff --git a/drivers/vfio/vfio_iommu_type1.c b/drivers/vfio/vfio_iommu_type1.c
index faee849f1cce..c2edc5a4c727 100644
--- a/drivers/vfio/vfio_iommu_type1.c
+++ b/drivers/vfio/vfio_iommu_type1.c
@@ -651,7 +651,7 @@ static int vfio_wait_all_valid(struct vfio_iommu *iommu)
 static long vfio_pin_pages_remote(struct vfio_dma *dma, unsigned long vaddr,
 				  long npage, unsigned long *pfn_base,
 				  unsigned long limit, struct vfio_batch *batch,
-				  struct mm_struct *mm)
+				  struct mm_struct *mm, long *lock_cache)
 {
 	unsigned long pfn;
 	long ret, pinned = 0, lock_acct = 0;
@@ -709,15 +709,25 @@ static long vfio_pin_pages_remote(struct vfio_dma *dma, unsigned long vaddr,
 			 * the user.
 			 */
 			if (!rsvd && !vfio_find_vpfn(dma, iova)) {
-				if (!dma->lock_cap &&
+				if (!dma->lock_cap && *lock_cache == 0 &&
 				    mm->locked_vm + lock_acct + 1 > limit) {
 					pr_warn("%s: RLIMIT_MEMLOCK (%ld) exceeded\n",
 						__func__, limit << PAGE_SHIFT);
 					ret = -ENOMEM;
 					goto unpin_out;
 				}
-				lock_acct++;
-			}
+				/*
+				 * Draw from the cache if possible to avoid
+				 * taking the write-side mmap_lock in
+				 * vfio_lock_acct(), which will alleviate
+				 * contention with the read-side mmap_lock in
+				 * vaddr_get_pfn().
+				 */
+				if (*lock_cache > 0)
+					(*lock_cache)--;
+				else
+					lock_acct++;
+				}
 
 			pinned++;
 			npage--;
@@ -1507,6 +1517,13 @@ static void vfio_pin_map_dma_undo(unsigned long start_vaddr,
 	vfio_unmap_unpin(args->iommu, args->dma, iova, end, true);
 }
 
+/*
+ * Relieve mmap_lock contention when multithreading page pinning by caching
+ * locked_vm locally.  Bound the locked_vm that a thread will cache but not use
+ * with this constant, which compromises between performance and overaccounting.
+ */
+#define LOCKED_VM_CACHE_PAGES	65536
+
 static int vfio_pin_map_dma_chunk(unsigned long start_vaddr,
 				  unsigned long end_vaddr, void *arg)
 {
@@ -1515,6 +1532,7 @@ static int vfio_pin_map_dma_chunk(unsigned long start_vaddr,
 	dma_addr_t iova = dma->iova + (start_vaddr - dma->vaddr);
 	unsigned long unmapped_size = end_vaddr - start_vaddr;
 	unsigned long pfn, mapped_size = 0;
+	long cache_size, lock_cache = 0;
 	struct vfio_batch batch;
 	long npage;
 	int ret = 0;
@@ -1522,11 +1540,29 @@ static int vfio_pin_map_dma_chunk(unsigned long start_vaddr,
 	vfio_batch_init(&batch);
 
 	while (unmapped_size) {
+		if (lock_cache == 0) {
+			cache_size = min_t(long, unmapped_size >> PAGE_SHIFT,
+					   LOCKED_VM_CACHE_PAGES);
+			ret = vfio_lock_acct(dma, cache_size, false);
+			/*
+			 * More locked_vm is cached than might be used, so
+			 * don't fail on -ENOMEM, i.e. exceeding RLIMIT_MEMLOCK.
+			 */
+			if (ret) {
+				if (ret != -ENOMEM) {
+					vfio_batch_unpin(&batch, dma);
+					break;
+				}
+				cache_size = 0;
+			}
+			lock_cache = cache_size;
+		}
+
 		/* Pin a contiguous chunk of memory */
 		npage = vfio_pin_pages_remote(dma, start_vaddr + mapped_size,
 					      unmapped_size >> PAGE_SHIFT,
 					      &pfn, args->limit, &batch,
-					      args->mm);
+					      args->mm, &lock_cache);
 		if (npage <= 0) {
 			WARN_ON(!npage);
 			ret = (int)npage;
@@ -1548,6 +1584,7 @@ static int vfio_pin_map_dma_chunk(unsigned long start_vaddr,
 	}
 
 	vfio_batch_fini(&batch);
+	vfio_lock_acct(dma, -lock_cache, false);
 
 	/*
 	 * Undo the successfully completed part of this chunk now.  padata will
@@ -1771,6 +1808,7 @@ static int vfio_iommu_replay(struct vfio_iommu *iommu,
 	struct rb_node *n;
 	unsigned long limit = rlimit(RLIMIT_MEMLOCK) >> PAGE_SHIFT;
 	int ret;
+	long lock_cache = 0;
 
 	ret = vfio_wait_all_valid(iommu);
 	if (ret < 0)
@@ -1832,7 +1870,8 @@ static int vfio_iommu_replay(struct vfio_iommu *iommu,
 							      n >> PAGE_SHIFT,
 							      &pfn, limit,
 							      &batch,
-							      current->mm);
+							      current->mm,
+							      &lock_cache);
 				if (npage <= 0) {
 					WARN_ON(!npage);
 					ret = (int)npage;
-- 
2.34.1


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

* [RFC 09/16] padata: Use kthreads in do_multithreaded
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (7 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 10/16] padata: Helpers should respect main thread's CPU affinity Daniel Jordan
                   ` (7 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Unbound kworkers will soon not be ideal for multithreaded jobs because
helpers will inherit the resource controls of the main thread, but
changing these controls (e.g. CPU affinity) might conflict with those
that kworkers already have in place.  While the changes are only
temporary, it seems like a layering violation to mess with kworkers this
way, and undoing the settings might fail or add latency for future
works.

Use kthreads instead, which have none of these issues.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 kernel/padata.c | 47 ++++++++++++++++++++++-------------------------
 1 file changed, 22 insertions(+), 25 deletions(-)

diff --git a/kernel/padata.c b/kernel/padata.c
index b458deb17121..00509c83e356 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -29,6 +29,7 @@
 #include <linux/cpumask.h>
 #include <linux/err.h>
 #include <linux/cpu.h>
+#include <linux/kthread.h>
 #include <linux/list_sort.h>
 #include <linux/padata.h>
 #include <linux/mutex.h>
@@ -37,8 +38,6 @@
 #include <linux/sysfs.h>
 #include <linux/rcupdate.h>
 
-#define	PADATA_WORK_ONSTACK	1	/* Work's memory is on stack */
-
 struct padata_work {
 	struct work_struct	pw_work;
 	struct list_head	pw_list;  /* padata_free_works linkage */
@@ -70,7 +69,6 @@ struct padata_mt_job_state {
 };
 
 static void padata_free_pd(struct parallel_data *pd);
-static void padata_mt_helper(struct work_struct *work);
 
 static int padata_index_to_cpu(struct parallel_data *pd, int cpu_index)
 {
@@ -108,17 +106,7 @@ static struct padata_work *padata_work_alloc(void)
 	return pw;
 }
 
-static void padata_work_init(struct padata_work *pw, work_func_t work_fn,
-			     void *data, int flags)
-{
-	if (flags & PADATA_WORK_ONSTACK)
-		INIT_WORK_ONSTACK(&pw->pw_work, work_fn);
-	else
-		INIT_WORK(&pw->pw_work, work_fn);
-	pw->pw_data = data;
-}
-
-static int padata_work_alloc_mt(int nworks, void *data, struct list_head *head)
+static int padata_work_alloc_mt(int nworks, struct list_head *head)
 {
 	int i;
 
@@ -129,7 +117,6 @@ static int padata_work_alloc_mt(int nworks, void *data, struct list_head *head)
 
 		if (!pw)
 			break;
-		padata_work_init(pw, padata_mt_helper, data, 0);
 		list_add(&pw->pw_list, head);
 	}
 	spin_unlock(&padata_works_lock);
@@ -234,7 +221,8 @@ int padata_do_parallel(struct padata_shell *ps,
 	rcu_read_unlock_bh();
 
 	if (pw) {
-		padata_work_init(pw, padata_parallel_worker, padata, 0);
+		INIT_WORK(&pw->pw_work, padata_parallel_worker);
+		pw->pw_data = padata;
 		queue_work(pinst->parallel_wq, &pw->pw_work);
 	} else {
 		/* Maximum works limit exceeded, run in the current task. */
@@ -449,9 +437,9 @@ static int padata_setup_cpumasks(struct padata_instance *pinst)
 	return err;
 }
 
-static void padata_mt_helper(struct work_struct *w)
+static int padata_mt_helper(void *__pw)
 {
-	struct padata_work *pw = container_of(w, struct padata_work, pw_work);
+	struct padata_work *pw = __pw;
 	struct padata_mt_job_state *ps = pw->pw_data;
 	struct padata_mt_job *job = ps->job;
 	bool done;
@@ -500,6 +488,8 @@ static void padata_mt_helper(struct work_struct *w)
 
 	if (done)
 		complete(&ps->completion);
+
+	return 0;
 }
 
 static int padata_error_cmp(void *unused, const struct list_head *a,
@@ -593,7 +583,7 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	lockdep_init_map(&ps.lockdep_map, map_name, key, 0);
 	INIT_LIST_HEAD(&ps.failed_works);
 	ps.job		  = job;
-	ps.nworks	  = padata_work_alloc_mt(nworks, &ps, &works);
+	ps.nworks	  = padata_work_alloc_mt(nworks, &works);
 	ps.nworks_fini	  = 0;
 	ps.error	  = 0;
 	ps.position	  = job->start;
@@ -612,13 +602,21 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	lock_map_acquire(&ps.lockdep_map);
 	lock_map_release(&ps.lockdep_map);
 
-	list_for_each_entry(pw, &works, pw_list)
-		queue_work(system_unbound_wq, &pw->pw_work);
+	list_for_each_entry(pw, &works, pw_list) {
+		struct task_struct *task;
+
+		pw->pw_data = &ps;
+		task = kthread_create(padata_mt_helper, pw, "padata");
+		if (IS_ERR(task))
+			--ps.nworks;
+		else
+			wake_up_process(task);
+	}
 
-	/* Use the current thread, which saves starting a workqueue worker. */
-	padata_work_init(&my_work, padata_mt_helper, &ps, PADATA_WORK_ONSTACK);
+	/* Use the current task, which saves starting a kthread. */
+	my_work.pw_data = &ps;
 	INIT_LIST_HEAD(&my_work.pw_list);
-	padata_mt_helper(&my_work.pw_work);
+	padata_mt_helper(&my_work);
 
 	/* Wait for all the helpers to finish. */
 	wait_for_completion(&ps.completion);
@@ -626,7 +624,6 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	if (ps.error && job->undo_fn)
 		padata_undo(&ps, &works, &my_work);
 
-	destroy_work_on_stack(&my_work.pw_work);
 	padata_works_free(&works);
 	return ps.error;
 }
-- 
2.34.1


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

* [RFC 10/16] padata: Helpers should respect main thread's CPU affinity
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (8 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 09/16] padata: Use kthreads in do_multithreaded Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 11/16] padata: Cap helpers started to online CPUs Daniel Jordan
                   ` (6 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Helper threads should run only on the CPUs allowed by the main thread to
honor its CPU affinity.  Similarly, cap the number of helpers started to
the number of CPUs allowed by the main thread's cpumask to avoid
flooding that subset of CPUs.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 kernel/padata.c | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)

diff --git a/kernel/padata.c b/kernel/padata.c
index 00509c83e356..0f4002ed1518 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -571,6 +571,7 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	/* Ensure at least one thread when size < min_chunk. */
 	nworks = max(job->size / job->min_chunk, 1ul);
 	nworks = min(nworks, job->max_threads);
+	nworks = min(nworks, current->nr_cpus_allowed);
 
 	if (nworks == 1) {
 		/* Single thread, no coordination needed, cut to the chase. */
@@ -607,10 +608,12 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 
 		pw->pw_data = &ps;
 		task = kthread_create(padata_mt_helper, pw, "padata");
-		if (IS_ERR(task))
+		if (IS_ERR(task)) {
 			--ps.nworks;
-		else
+		} else {
+			kthread_bind_mask(task, current->cpus_ptr);
 			wake_up_process(task);
+		}
 	}
 
 	/* Use the current task, which saves starting a kthread. */
-- 
2.34.1


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

* [RFC 11/16] padata: Cap helpers started to online CPUs
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (9 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 10/16] padata: Helpers should respect main thread's CPU affinity Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus() Daniel Jordan
                   ` (5 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

padata can start num_possible_cpus() helpers, but this is too many
considering that every job's main thread participates and there may
be fewer online than possible CPUs.

Limit overall concurrency, including main thread(s), to
num_online_cpus() with the padata_works_inuse counter to prevent
CPU-intensive threads flooding the system in case of concurrent jobs.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 kernel/padata.c | 24 ++++++++++++++++++------
 1 file changed, 18 insertions(+), 6 deletions(-)

diff --git a/kernel/padata.c b/kernel/padata.c
index 0f4002ed1518..e27988d3e9ed 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -50,6 +50,7 @@ struct padata_work {
 
 static DEFINE_SPINLOCK(padata_works_lock);
 static struct padata_work *padata_works;
+static unsigned int padata_works_inuse;
 static LIST_HEAD(padata_free_works);
 
 struct padata_mt_job_state {
@@ -98,11 +99,16 @@ static struct padata_work *padata_work_alloc(void)
 
 	lockdep_assert_held(&padata_works_lock);
 
-	if (list_empty(&padata_free_works))
-		return NULL;	/* No more work items allowed to be queued. */
+	/* Are more work items allowed to be queued? */
+	if (padata_works_inuse >= num_online_cpus())
+		return NULL;
+
+	if (WARN_ON_ONCE(list_empty(&padata_free_works)))
+		return NULL;
 
 	pw = list_first_entry(&padata_free_works, struct padata_work, pw_list);
 	list_del(&pw->pw_list);
+	++padata_works_inuse;
 	return pw;
 }
 
@@ -111,7 +117,11 @@ static int padata_work_alloc_mt(int nworks, struct list_head *head)
 	int i;
 
 	spin_lock(&padata_works_lock);
-	/* Start at 1 because the current task participates in the job. */
+	/*
+	 * Increment inuse and start iterating at 1 to account for the main
+	 * thread participating in the job with its stack-allocated work.
+	 */
+	++padata_works_inuse;
 	for (i = 1; i < nworks; ++i) {
 		struct padata_work *pw = padata_work_alloc();
 
@@ -128,20 +138,22 @@ static void padata_work_free(struct padata_work *pw)
 {
 	lockdep_assert_held(&padata_works_lock);
 	list_add(&pw->pw_list, &padata_free_works);
+	WARN_ON_ONCE(!padata_works_inuse);
+	--padata_works_inuse;
 }
 
 static void padata_works_free(struct list_head *works)
 {
 	struct padata_work *cur, *next;
 
-	if (list_empty(works))
-		return;
-
 	spin_lock(&padata_works_lock);
 	list_for_each_entry_safe(cur, next, works, pw_list) {
 		list_del(&cur->pw_list);
 		padata_work_free(cur);
 	}
+	/* To account for the main thread finishing its part of the job. */
+	WARN_ON_ONCE(!padata_works_inuse);
+	--padata_works_inuse;
 	spin_unlock(&padata_works_lock);
 }
 
-- 
2.34.1


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

* [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus()
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (10 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 11/16] padata: Cap helpers started to online CPUs Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  6:06   ` kernel test robot
  2022-01-06 11:31   ` kernel test robot
  2022-01-06  0:46 ` [RFC 13/16] padata: Run helper threads at MAX_NICE Daniel Jordan
                   ` (4 subsequent siblings)
  16 siblings, 2 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Helpers are currently not bound by the main thread's CFS bandwidth
limits because they're kernel threads that run on the root runqueues, so
a multithreaded job could cause a task group to consume more quota than
it's configured for.

As a starting point for helpers honoring these limits, restrict a job to
only as many helpers as there are CPUs allowed by these limits.  Helpers
are generally CPU-bound, so starting more helpers than this would likely
exceed the group's entire quota.  Max CFS bandwidth CPUs are calculated
conservatively with integer division (quota / period).

This restriction ignores other tasks in the group that might also be
consuming quota, so it doesn't strictly prevent a group from exceeding
its limits.  However, this may be the right tradeoff between simplicity
and absolutely correct resource control, given that VFIO page pinning
typically happens during guest initialization when there's not much
other CPU activity in the group.  There's also a prototype for an
absolutely correct approach later in the series should that be
preferred.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 include/linux/sched/cgroup.h | 21 +++++++++++++++++++++
 kernel/padata.c              | 15 +++++++++++++++
 kernel/sched/core.c          | 19 +++++++++++++++++++
 3 files changed, 55 insertions(+)
 create mode 100644 include/linux/sched/cgroup.h

diff --git a/include/linux/sched/cgroup.h b/include/linux/sched/cgroup.h
new file mode 100644
index 000000000000..f89d92e9e015
--- /dev/null
+++ b/include/linux/sched/cgroup.h
@@ -0,0 +1,21 @@
+/* SPDX-License-Identifier: GPL-2.0 */
+#ifndef _LINUX_SCHED_CGROUP_H
+#define _LINUX_SCHED_CGROUP_H
+
+#include <linux/cgroup-defs.h>
+#include <linux/cpumask.h>
+
+#ifdef CONFIG_CFS_BANDWIDTH
+
+int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css);
+
+#else /* CONFIG_CFS_BANDWIDTH */
+
+static inline int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css)
+{
+	return nr_cpu_ids;
+}
+
+#endif /* CONFIG_CFS_BANDWIDTH */
+
+#endif /* _LINUX_SCHED_CGROUP_H */
diff --git a/kernel/padata.c b/kernel/padata.c
index e27988d3e9ed..ef6589a6b665 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -24,6 +24,7 @@
  * 51 Franklin St - Fifth Floor, Boston, MA 02110-1301 USA.
  */
 
+#include <linux/cgroup.h>
 #include <linux/completion.h>
 #include <linux/export.h>
 #include <linux/cpumask.h>
@@ -34,6 +35,7 @@
 #include <linux/padata.h>
 #include <linux/mutex.h>
 #include <linux/sched.h>
+#include <linux/sched/cgroup.h>
 #include <linux/slab.h>
 #include <linux/sysfs.h>
 #include <linux/rcupdate.h>
@@ -572,6 +574,7 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 {
 	/* In case threads finish at different times. */
 	static const unsigned long load_balance_factor = 4;
+	struct cgroup_subsys_state *cpu_css;
 	struct padata_work my_work, *pw;
 	struct padata_mt_job_state ps;
 	LIST_HEAD(works);
@@ -585,6 +588,18 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	nworks = min(nworks, job->max_threads);
 	nworks = min(nworks, current->nr_cpus_allowed);
 
+#ifdef CONFIG_CGROUP_SCHED
+	/*
+	 * Cap threads at the max number of CPUs current's CFS bandwidth
+	 * settings allow.  Keep it simple, don't try to keep this value up to
+	 * date.  The ifdef guards cpu_cgrp_id.
+	 */
+	rcu_read_lock();
+	cpu_css = task_css(current, cpu_cgrp_id);
+	nworks = min(nworks, max_cfs_bandwidth_cpus(cpu_css));
+	rcu_read_unlock();
+#endif
+
 	if (nworks == 1) {
 		/* Single thread, no coordination needed, cut to the chase. */
 		return job->thread_fn(job->start, job->start + job->size,
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index f3b27c6c5153..848c9fec8006 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -10021,6 +10021,25 @@ static long tg_get_cfs_burst(struct task_group *tg)
 	return burst_us;
 }
 
+/* Returns the max whole number of CPUs that @css's bandwidth settings allow. */
+int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css)
+{
+	struct task_group *tg = css_tg(css);
+	u64 quota_us, period_us;
+
+	if (tg == &root_task_group)
+		return nr_cpu_ids;
+
+	quota_us = tg_get_cfs_quota(tg);
+
+	if (quota_us == RUNTIME_INF)
+		return nr_cpu_ids;
+
+	period_us = tg_get_cfs_period(tg);
+
+	return quota_us / period_us;
+}
+
 static s64 cpu_cfs_quota_read_s64(struct cgroup_subsys_state *css,
 				  struct cftype *cft)
 {
-- 
2.34.1


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

* [RFC 13/16] padata: Run helper threads at MAX_NICE
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (11 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus() Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 14/16] padata: Nice helper threads one by one to prevent starvation Daniel Jordan
                   ` (3 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

Optimistic parallelization can go wrong if too many helpers are started
on a busy system.  They can unfairly degrade the performance of other
tasks, so they should be sensitive to current CPU utilization[1].

Achieve this by running helpers at MAX_NICE so that their CPU time is
proportional to idle CPU time.  The main thread, however, runs at its
original priority so that it can make progress on a heavily loaded
system, as it would if padata were not in the picture.

Here are two test cases in which a padata and a non-padata workload
compete for the same CPUs to show that normal priority (i.e. nice=0)
padata helpers cause the non-padata workload to run more slowly, whereas
MAX_NICE padata helpers don't.

Notes:
  - Each case was run using 8 CPUs on a large two-socket server, with a
    cpumask allowing all test threads to run anywhere within the 8.
  - The non-padata workload used 7 threads and the padata workload used 8
    threads to evaluate how much padata helpers, rather than the main padata
    thread, disturbed the non-padata workload.
  - The non-padata workload was started after the padata workload and run
    for less time to maximize the chances that the non-padata workload would
    be disturbed.
  - Runtimes in seconds.

Case 1: Synthetic, worst-case CPU contention

    padata_test - a tight loop doing integer multiplication to max out on CPU;
                  used for testing only, does not appear in this series
    stress-ng   - cpu stressor ("-c --cpu-method ackerman --cpu-ops 1200");

                 stress-ng
                     alone  (stdev)   max_nice  (stdev)   normal_prio  (stdev)
                  ------------------------------------------------------------
    padata_test                          96.87  ( 1.09)         90.81  ( 0.29)
    stress-ng        43.04  ( 0.00)      43.58  ( 0.01)         75.86  ( 0.39)

MAX_NICE helpers make a significant difference compared to normal
priority helpers, with stress-ng taking 76% longer to finish when
competing with normal priority padata threads than when run by itself,
but only 1% longer when run with MAX_NICE helpers.  The 1% comes from
the small amount of CPU time MAX_NICE threads are given despite their
low priority.

Case 2: Real-world CPU contention

    padata_vfio - VFIO page pin a 175G kvm guest
    usemem      - faults in 25G of anonymous THP per thread, PAGE_SIZE stride;
                  used to mimic the page clearing that dominates in padata_vfio
                  so that usemem competes for the same system resources

                    usemem
                     alone  (stdev)   max_nice  (stdev)   normal_prio  (stdev)
                  ------------------------------------------------------------
    padata_vfio                          14.74  ( 0.04)          9.93  ( 0.09)
        usemem       10.45  ( 0.04)      10.75  ( 0.04)         14.14  ( 0.07)

Here the effect is similar, just not as pronounced.  The usemem threads
take 35% longer to finish with normal priority padata threads than when
run alone, but only 3% longer when MAX_NICE is used.

[1] lkml.kernel.org/r/20171206143509.GG7515@dhcp22.suse.cz

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 kernel/padata.c | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/kernel/padata.c b/kernel/padata.c
index ef6589a6b665..83e86724b3e1 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -638,7 +638,10 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 		if (IS_ERR(task)) {
 			--ps.nworks;
 		} else {
+			/* Helper threads shouldn't disturb other workloads. */
+			set_user_nice(task, MAX_NICE);
 			kthread_bind_mask(task, current->cpus_ptr);
+
 			wake_up_process(task);
 		}
 	}
-- 
2.34.1


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

* [RFC 14/16] padata: Nice helper threads one by one to prevent starvation
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (12 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 13/16] padata: Run helper threads at MAX_NICE Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
                   ` (2 subsequent siblings)
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

With padata helper threads running at MAX_NICE, it's possible for one or
more of them to begin chunks of the job and then have their CPU time
constrained by higher priority threads.  The main padata thread, running
at normal priority, may finish all available chunks of the job and then
wait on the MAX_NICE helpers to finish the last in-progress chunks, for
longer than it would have if no helpers were used.

Avoid this by having the main thread assign its priority to each
unfinished helper one at a time so that on a heavily loaded system,
exactly one thread in a given padata call is running at the main thread's
priority.  At least one thread to ensure forward progress, and at most
one thread to limit excessive multithreading.

Here are tests like the ones for MAX_NICE, run on the same two-socket
server, but with a couple of differences:
 - The non-padata workload uses 8 CPUs instead of 7 to compete with the
   main padata thread as well as the padata helpers, so that when the main
   thread finishes, its CPU is completely occupied by the non-padata
   workload, meaning MAX_NICE helpers can't run as often.
 - The non-padata workload starts before the padata workload, rather
   than after, to maximize the chance that it interferes with helpers.

Runtimes in seconds.

Case 1: Synthetic, worst-case CPU contention

  padata_test - a tight loop doing integer multiplication to max out CPU;
                used for testing only, does not appear in this series
  stress-ng   - cpu stressor ("-c 8 --cpu-method ackermann --cpu-ops 1200");

            8_padata_thrs          8_padata_thrs
                 w/o_nice  (stdev)     with_nice  (stdev)  1_padata_thr  (stdev)
             ------------------------------------------------------------------
  padata_test       41.98  ( 0.22)         25.15  ( 2.98)        30.40  ( 0.61)
  stress-ng         44.79  ( 1.11)         46.37  ( 0.69)        53.29  ( 1.91)

Without nicing, padata_test finishes just after stress-ng does because
stress-ng needs to free up CPUs for the helpers to finish (padata_test
shows a shorter runtime than stress-ng because padata_test was started
later).  Nicing lets padata_test finish 40% sooner, and running the same
amount of work in padata_test with 1 thread instead of 8 takes longer
than "with_nice" because MAX_NICE threads still get some CPU time, and
the effect over 8 threads adds up.

stress-ng's total runtime gets a little longer going from no nicing to
nicing because each niced padata thread takes more CPU time than before
when the helpers were starved.

Competing against just one padata thread, stress-ng's reported walltime
goes up because that single thread interferes with fewer stress-ng
threads, but with more impact, causing a greater spread in the time it
takes for individual stress-ng threads to finish.  Averages of the
per-thread stress-ng times from "with_nice" to "1_padata_thr" come out
roughly the same, though, 43.81 and 43.89 respectively.  So the total
runtime of stress-ng across all threads is unaffected, but the time
stress-ng takes to finish running its threads completely actually
improves by spreading the padata_test work over more threads.

Case 2: Real-world CPU contention

  padata_vfio - VFIO page pin a 32G kvm guest
  usemem      - faults in 86G of anonymous THP per thread, PAGE_SIZE stride;
                used to mimic the page clearing that dominates in padata_vfio
                so that usemem competes for the same system resources

            8_padata_thrs          8_padata_thrs
                 w/o_nice  (stdev)     with_nice  (stdev)  1_padata_thr (stdev)
             ------------------------------------------------------------------
  padata_vfio       18.59  ( 0.19)         14.62  ( 2.03)        16.24  ( 0.90)
      usemem        47.54  ( 0.89)         48.18  ( 0.77)        49.70  ( 1.20)

These results are similar to case 1's, though the differences between
times are not quite as pronounced because padata_vfio ran shorter
compared to usemem.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 kernel/padata.c | 106 +++++++++++++++++++++++++++++++++---------------
 1 file changed, 73 insertions(+), 33 deletions(-)

diff --git a/kernel/padata.c b/kernel/padata.c
index 83e86724b3e1..52f670a5d6d9 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -40,10 +40,17 @@
 #include <linux/sysfs.h>
 #include <linux/rcupdate.h>
 
+enum padata_work_flags {
+	PADATA_WORK_FINISHED	= 1,
+	PADATA_WORK_UNDO	= 2,
+};
+
 struct padata_work {
 	struct work_struct	pw_work;
 	struct list_head	pw_list;  /* padata_free_works linkage */
+	enum padata_work_flags	pw_flags;
 	void			*pw_data;
+	struct task_struct	*pw_task;
 	/* holds job units from padata_mt_job::start to pw_error_start */
 	unsigned long		pw_error_offset;
 	unsigned long		pw_error_start;
@@ -58,9 +65,8 @@ static LIST_HEAD(padata_free_works);
 struct padata_mt_job_state {
 	spinlock_t		lock;
 	struct completion	completion;
+	struct task_struct	*niced_task;
 	struct padata_mt_job	*job;
-	int			nworks;
-	int			nworks_fini;
 	int			error; /* first error from thread_fn */
 	unsigned long		chunk_size;
 	unsigned long		position;
@@ -451,12 +457,44 @@ static int padata_setup_cpumasks(struct padata_instance *pinst)
 	return err;
 }
 
+static void padata_wait_for_helpers(struct padata_mt_job_state *ps,
+				    struct list_head *unfinished_works,
+				    struct list_head *finished_works)
+{
+	struct padata_work *work;
+
+	if (list_empty(unfinished_works))
+		return;
+
+	spin_lock(&ps->lock);
+	while (!list_empty(unfinished_works)) {
+		work = list_first_entry(unfinished_works, struct padata_work,
+					pw_list);
+		if (!(work->pw_flags & PADATA_WORK_FINISHED)) {
+			set_user_nice(work->pw_task, task_nice(current));
+			ps->niced_task = work->pw_task;
+			spin_unlock(&ps->lock);
+
+			wait_for_completion(&ps->completion);
+
+			spin_lock(&ps->lock);
+			WARN_ON_ONCE(!(work->pw_flags & PADATA_WORK_FINISHED));
+		}
+		/*
+		 * Leave works used in padata_undo() on ps->failed_works.
+		 * padata_undo() will move them to finished_works.
+		 */
+		if (!(work->pw_flags & PADATA_WORK_UNDO))
+			list_move(&work->pw_list, finished_works);
+	}
+	spin_unlock(&ps->lock);
+}
+
 static int padata_mt_helper(void *__pw)
 {
 	struct padata_work *pw = __pw;
 	struct padata_mt_job_state *ps = pw->pw_data;
 	struct padata_mt_job *job = ps->job;
-	bool done;
 
 	spin_lock(&ps->lock);
 
@@ -488,6 +526,7 @@ static int padata_mt_helper(void *__pw)
 				ps->error = ret;
 			/* Save information about where the job failed. */
 			if (job->undo_fn) {
+				pw->pw_flags |= PADATA_WORK_UNDO;
 				list_move(&pw->pw_list, &ps->failed_works);
 				pw->pw_error_start = position;
 				pw->pw_error_offset = position_offset;
@@ -496,12 +535,10 @@ static int padata_mt_helper(void *__pw)
 		}
 	}
 
-	++ps->nworks_fini;
-	done = (ps->nworks_fini == ps->nworks);
-	spin_unlock(&ps->lock);
-
-	if (done)
+	pw->pw_flags |= PADATA_WORK_FINISHED;
+	if (ps->niced_task == current)
 		complete(&ps->completion);
+	spin_unlock(&ps->lock);
 
 	return 0;
 }
@@ -520,7 +557,7 @@ static int padata_error_cmp(void *unused, const struct list_head *a,
 }
 
 static void padata_undo(struct padata_mt_job_state *ps,
-			struct list_head *works_list,
+			struct list_head *finished_works,
 			struct padata_work *stack_work)
 {
 	struct list_head *failed_works = &ps->failed_works;
@@ -548,11 +585,12 @@ static void padata_undo(struct padata_mt_job_state *ps,
 
 		if (failed_work) {
 			undo_pos = failed_work->pw_error_end;
-			/* main thread's stack_work stays off works_list */
+			/* main thread's stack_work stays off finished_works */
 			if (failed_work == stack_work)
 				list_del(&failed_work->pw_list);
 			else
-				list_move(&failed_work->pw_list, works_list);
+				list_move(&failed_work->pw_list,
+					  finished_works);
 		} else {
 			undo_pos = undo_end;
 		}
@@ -577,16 +615,17 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	struct cgroup_subsys_state *cpu_css;
 	struct padata_work my_work, *pw;
 	struct padata_mt_job_state ps;
-	LIST_HEAD(works);
-	int nworks;
+	LIST_HEAD(unfinished_works);
+	LIST_HEAD(finished_works);
+	int nworks, req;
 
 	if (job->size == 0)
 		return 0;
 
 	/* Ensure at least one thread when size < min_chunk. */
-	nworks = max(job->size / job->min_chunk, 1ul);
-	nworks = min(nworks, job->max_threads);
-	nworks = min(nworks, current->nr_cpus_allowed);
+	req = max(job->size / job->min_chunk, 1ul);
+	req = min(req, job->max_threads);
+	req = min(req, current->nr_cpus_allowed);
 
 #ifdef CONFIG_CGROUP_SCHED
 	/*
@@ -596,23 +635,23 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	 */
 	rcu_read_lock();
 	cpu_css = task_css(current, cpu_cgrp_id);
-	nworks = min(nworks, max_cfs_bandwidth_cpus(cpu_css));
+	req = min(req, max_cfs_bandwidth_cpus(cpu_css));
 	rcu_read_unlock();
 #endif
 
-	if (nworks == 1) {
+	if (req == 1) {
 		/* Single thread, no coordination needed, cut to the chase. */
 		return job->thread_fn(job->start, job->start + job->size,
 				      job->fn_arg);
 	}
 
+	nworks = padata_work_alloc_mt(req, &unfinished_works);
+
 	spin_lock_init(&ps.lock);
 	init_completion(&ps.completion);
 	lockdep_init_map(&ps.lockdep_map, map_name, key, 0);
 	INIT_LIST_HEAD(&ps.failed_works);
 	ps.job		  = job;
-	ps.nworks	  = padata_work_alloc_mt(nworks, &works);
-	ps.nworks_fini	  = 0;
 	ps.error	  = 0;
 	ps.position	  = job->start;
 	ps.remaining_size = job->size;
@@ -623,41 +662,42 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	 * increasing the number of chunks, guarantee at least the minimum
 	 * chunk size from the caller, and honor the caller's alignment.
 	 */
-	ps.chunk_size = job->size / (ps.nworks * load_balance_factor);
+	ps.chunk_size = job->size / (nworks * load_balance_factor);
 	ps.chunk_size = max(ps.chunk_size, job->min_chunk);
 	ps.chunk_size = roundup(ps.chunk_size, job->align);
 
 	lock_map_acquire(&ps.lockdep_map);
 	lock_map_release(&ps.lockdep_map);
 
-	list_for_each_entry(pw, &works, pw_list) {
-		struct task_struct *task;
-
+	list_for_each_entry(pw, &unfinished_works, pw_list) {
 		pw->pw_data = &ps;
-		task = kthread_create(padata_mt_helper, pw, "padata");
-		if (IS_ERR(task)) {
-			--ps.nworks;
+		pw->pw_task = kthread_create(padata_mt_helper, pw, "padata");
+		if (IS_ERR(pw->pw_task)) {
+			pw->pw_flags = PADATA_WORK_FINISHED;
 		} else {
 			/* Helper threads shouldn't disturb other workloads. */
-			set_user_nice(task, MAX_NICE);
-			kthread_bind_mask(task, current->cpus_ptr);
+			set_user_nice(pw->pw_task, MAX_NICE);
+
+			pw->pw_flags = 0;
+			kthread_bind_mask(pw->pw_task, current->cpus_ptr);
 
-			wake_up_process(task);
+			wake_up_process(pw->pw_task);
 		}
 	}
 
 	/* Use the current task, which saves starting a kthread. */
 	my_work.pw_data = &ps;
+	my_work.pw_flags = 0;
 	INIT_LIST_HEAD(&my_work.pw_list);
 	padata_mt_helper(&my_work);
 
 	/* Wait for all the helpers to finish. */
-	wait_for_completion(&ps.completion);
+	padata_wait_for_helpers(&ps, &unfinished_works, &finished_works);
 
 	if (ps.error && job->undo_fn)
-		padata_undo(&ps, &works, &my_work);
+		padata_undo(&ps, &finished_works, &my_work);
 
-	padata_works_free(&works);
+	padata_works_free(&finished_works);
 	return ps.error;
 }
 
-- 
2.34.1


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

* [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (13 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 14/16] padata: Nice helper threads one by one to prevent starvation Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  6:47   ` kernel test robot
                     ` (3 more replies)
  2022-01-06  0:46 ` [RFC 16/16] sched/fair: Consider kthread debt in cputime Daniel Jordan
  2022-01-06  1:13 ` [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Jason Gunthorpe
  16 siblings, 4 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

As before, helpers in multithreaded jobs don't honor the main thread's
CFS bandwidth limits, which could lead to the group exceeding its quota.

Fix it by having helpers remote charge their CPU time to the main
thread's task group.  A helper calls a pair of new interfaces
cpu_cgroup_remote_begin() and cpu_cgroup_remote_charge() (see function
header comments) to achieve this.

This is just supposed to start a discussion, so it's pretty simple.
Once a kthread has finished a remote charging period with
cpu_cgroup_remote_charge(), its runtime is subtracted from the target
task group's runtime (cfs_bandwidth::runtime) and any remainder is saved
as debt (cfs_bandwidth::debt) to pay off in later periods.

Remote charging tasks aren't throttled when the group reaches its quota,
and a task group doesn't run at all until its debt is completely paid,
but these shortcomings can be addressed if the approach ends up being
taken.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 include/linux/sched.h        |  2 +
 include/linux/sched/cgroup.h | 16 ++++++
 kernel/padata.c              | 26 +++++++---
 kernel/sched/core.c          | 39 +++++++++++++++
 kernel/sched/fair.c          | 94 +++++++++++++++++++++++++++++++++++-
 kernel/sched/sched.h         |  5 ++
 6 files changed, 174 insertions(+), 8 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index ec8d07d88641..cc04367d4458 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -487,6 +487,8 @@ struct sched_entity {
 	struct cfs_rq			*my_q;
 	/* cached value of my_q->h_nr_running */
 	unsigned long			runnable_weight;
+	/* sum_exec_runtime at the start of the remote charging period */
+	u64				remote_runtime_begin;
 #endif
 
 #ifdef CONFIG_SMP
diff --git a/include/linux/sched/cgroup.h b/include/linux/sched/cgroup.h
index f89d92e9e015..cb3b7941149f 100644
--- a/include/linux/sched/cgroup.h
+++ b/include/linux/sched/cgroup.h
@@ -5,6 +5,22 @@
 #include <linux/cgroup-defs.h>
 #include <linux/cpumask.h>
 
+#ifdef CONFIG_FAIR_GROUP_SCHED
+
+void cpu_cgroup_remote_begin(struct task_struct *p,
+			     struct cgroup_subsys_state *css);
+void cpu_cgroup_remote_charge(struct task_struct *p,
+			      struct cgroup_subsys_state *css);
+
+#else /* CONFIG_FAIR_GROUP_SCHED */
+
+static inline void cpu_cgroup_remote_begin(struct task_struct *p,
+					   struct cgroup_subsys_state *css) {}
+static inline void cpu_cgroup_remote_charge(struct task_struct *p,
+					    struct cgroup_subsys_state *css) {}
+
+#endif /* CONFIG_FAIR_GROUP_SCHED */
+
 #ifdef CONFIG_CFS_BANDWIDTH
 
 int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css);
diff --git a/kernel/padata.c b/kernel/padata.c
index 52f670a5d6d9..d595f11c2fdd 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -43,6 +43,7 @@
 enum padata_work_flags {
 	PADATA_WORK_FINISHED	= 1,
 	PADATA_WORK_UNDO	= 2,
+	PADATA_WORK_MAIN_THR	= 4,
 };
 
 struct padata_work {
@@ -75,6 +76,7 @@ struct padata_mt_job_state {
 #ifdef CONFIG_LOCKDEP
 	struct lockdep_map	lockdep_map;
 #endif
+	struct cgroup_subsys_state *cpu_css;
 };
 
 static void padata_free_pd(struct parallel_data *pd);
@@ -495,6 +497,10 @@ static int padata_mt_helper(void *__pw)
 	struct padata_work *pw = __pw;
 	struct padata_mt_job_state *ps = pw->pw_data;
 	struct padata_mt_job *job = ps->job;
+	bool is_main = pw->pw_flags & PADATA_WORK_MAIN_THR;
+
+	if (!is_main)
+		cpu_cgroup_remote_begin(current, ps->cpu_css);
 
 	spin_lock(&ps->lock);
 
@@ -518,6 +524,10 @@ static int padata_mt_helper(void *__pw)
 		ret = job->thread_fn(position, end, job->fn_arg);
 
 		lock_map_release(&ps->lockdep_map);
+
+		if (!is_main)
+			cpu_cgroup_remote_charge(current, ps->cpu_css);
+
 		spin_lock(&ps->lock);
 
 		if (ret) {
@@ -612,7 +622,6 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 {
 	/* In case threads finish at different times. */
 	static const unsigned long load_balance_factor = 4;
-	struct cgroup_subsys_state *cpu_css;
 	struct padata_work my_work, *pw;
 	struct padata_mt_job_state ps;
 	LIST_HEAD(unfinished_works);
@@ -628,18 +637,20 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 	req = min(req, current->nr_cpus_allowed);
 
 #ifdef CONFIG_CGROUP_SCHED
+	ps.cpu_css = task_get_css(current, cpu_cgrp_id);
+
 	/*
 	 * Cap threads at the max number of CPUs current's CFS bandwidth
 	 * settings allow.  Keep it simple, don't try to keep this value up to
 	 * date.  The ifdef guards cpu_cgrp_id.
 	 */
-	rcu_read_lock();
-	cpu_css = task_css(current, cpu_cgrp_id);
-	req = min(req, max_cfs_bandwidth_cpus(cpu_css));
-	rcu_read_unlock();
+	req = min(req, max_cfs_bandwidth_cpus(ps.cpu_css));
 #endif
 
 	if (req == 1) {
+#ifdef CONFIG_CGROUP_SCHED
+		css_put(ps.cpu_css);
+#endif
 		/* Single thread, no coordination needed, cut to the chase. */
 		return job->thread_fn(job->start, job->start + job->size,
 				      job->fn_arg);
@@ -687,12 +698,15 @@ int padata_do_multithreaded_job(struct padata_mt_job *job,
 
 	/* Use the current task, which saves starting a kthread. */
 	my_work.pw_data = &ps;
-	my_work.pw_flags = 0;
+	my_work.pw_flags = PADATA_WORK_MAIN_THR;
 	INIT_LIST_HEAD(&my_work.pw_list);
 	padata_mt_helper(&my_work);
 
 	/* Wait for all the helpers to finish. */
 	padata_wait_for_helpers(&ps, &unfinished_works, &finished_works);
+#ifdef CONFIG_CGROUP_SCHED
+	css_put(ps.cpu_css);
+#endif
 
 	if (ps.error && job->undo_fn)
 		padata_undo(&ps, &finished_works, &my_work);
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 848c9fec8006..a5e24b6bd7e0 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -9913,6 +9913,7 @@ static int tg_set_cfs_bandwidth(struct task_group *tg, u64 period, u64 quota,
 	cfs_b->period = ns_to_ktime(period);
 	cfs_b->quota = quota;
 	cfs_b->burst = burst;
+	cfs_b->debt = 0;
 
 	__refill_cfs_bandwidth_runtime(cfs_b);
 
@@ -10181,6 +10182,44 @@ static int cpu_cfs_stat_show(struct seq_file *sf, void *v)
 	return 0;
 }
 #endif /* CONFIG_CFS_BANDWIDTH */
+
+/**
+ * cpu_cgroup_remote_begin - begin charging p's CPU usage to a remote css
+ * @p: the kernel thread whose CPU usage should be accounted
+ * @css: the css to which the CPU usage should be accounted
+ *
+ * Begin charging a kernel thread's CPU usage to a remote (non-root) task group
+ * to account CPU time that the kernel thread spends working on behalf of the
+ * group.  Pair with at least one subsequent call to cpu_cgroup_remote_charge()
+ * to complete the charge.
+ *
+ * Supports CFS bandwidth and cgroup2 CPU accounting stats but not weight-based
+ * control for now.
+ */
+void cpu_cgroup_remote_begin(struct task_struct *p,
+			     struct cgroup_subsys_state *css)
+{
+	if (p->sched_class == &fair_sched_class)
+		cpu_cgroup_remote_begin_fair(p, css_tg(css));
+}
+
+/**
+ * cpu_cgroup_remote_charge - account p's CPU usage to a remote css
+ * @p: the kernel thread whose CPU usage should be accounted
+ * @css: the css to which the CPU usage should be accounted
+ *
+ * Account a kernel thread's CPU usage to a remote (non-root) task group.  Pair
+ * with a previous call to cpu_cgroup_remote_begin() with the same @p and @css.
+ * This may be invoked multiple times after the initial
+ * cpu_cgroup_remote_begin() to account additional CPU usage.
+ */
+void cpu_cgroup_remote_charge(struct task_struct *p,
+			      struct cgroup_subsys_state *css)
+{
+	if (p->sched_class == &fair_sched_class)
+		cpu_cgroup_remote_charge_fair(p, css_tg(css));
+}
+
 #endif /* CONFIG_FAIR_GROUP_SCHED */
 
 #ifdef CONFIG_RT_GROUP_SCHED
diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 44c452072a1b..3c2d7f245c68 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -4655,10 +4655,19 @@ static inline u64 sched_cfs_bandwidth_slice(void)
  */
 void __refill_cfs_bandwidth_runtime(struct cfs_bandwidth *cfs_b)
 {
-	if (unlikely(cfs_b->quota == RUNTIME_INF))
+	u64 quota = cfs_b->quota;
+	u64 payment;
+
+	if (unlikely(quota == RUNTIME_INF))
 		return;
 
-	cfs_b->runtime += cfs_b->quota;
+	if (cfs_b->debt) {
+		payment = min(quota, cfs_b->debt);
+		cfs_b->debt -= payment;
+		quota -= payment;
+	}
+
+	cfs_b->runtime += quota;
 	cfs_b->runtime = min(cfs_b->runtime, cfs_b->quota + cfs_b->burst);
 }
 
@@ -5406,6 +5415,32 @@ static void __maybe_unused unthrottle_offline_cfs_rqs(struct rq *rq)
 	rcu_read_unlock();
 }
 
+static void incur_cfs_debt(struct rq *rq, struct sched_entity *se,
+			   struct task_group *tg, u64 debt)
+{
+	if (!cfs_bandwidth_used())
+		return;
+
+	while (tg != &root_task_group) {
+		struct cfs_rq *cfs_rq = tg->cfs_rq[cpu_of(rq)];
+
+		if (cfs_rq->runtime_enabled) {
+			struct cfs_bandwidth *cfs_b = &tg->cfs_bandwidth;
+			u64 payment;
+
+			raw_spin_lock(&cfs_b->lock);
+
+			payment = min(cfs_b->runtime, debt);
+			cfs_b->runtime -= payment;
+			cfs_b->debt += debt - payment;
+
+			raw_spin_unlock(&cfs_b->lock);
+		}
+
+		tg = tg->parent;
+	}
+}
+
 #else /* CONFIG_CFS_BANDWIDTH */
 
 static inline bool cfs_bandwidth_used(void)
@@ -5448,6 +5483,8 @@ static inline struct cfs_bandwidth *tg_cfs_bandwidth(struct task_group *tg)
 static inline void destroy_cfs_bandwidth(struct cfs_bandwidth *cfs_b) {}
 static inline void update_runtime_enabled(struct rq *rq) {}
 static inline void unthrottle_offline_cfs_rqs(struct rq *rq) {}
+static inline void incur_cfs_debt(struct rq *rq, struct sched_entity *se,
+				  struct task_group *tg, u64 debt) {}
 
 #endif /* CONFIG_CFS_BANDWIDTH */
 
@@ -11452,6 +11489,59 @@ int sched_group_set_shares(struct task_group *tg, unsigned long shares)
 	mutex_unlock(&shares_mutex);
 	return 0;
 }
+
+#define INCUR_DEBT		1
+
+static void cpu_cgroup_remote(struct task_struct *p, struct task_group *tg,
+			      int flags)
+{
+	struct sched_entity *se = &p->se;
+	struct cfs_rq *cfs_rq;
+	struct rq_flags rf;
+	struct rq *rq;
+
+	/*
+	 * User tasks might change task groups between calls to this function,
+	 * which isn't handled for now, so disallow them.
+	 */
+	if (!(p->flags & PF_KTHREAD))
+		return;
+
+	/* kthreads already run in the root, so no need for remote charging. */
+	if (tg == &root_task_group)
+		return;
+
+	rq = task_rq_lock(p, &rf);
+	update_rq_clock(rq);
+
+	cfs_rq = cfs_rq_of(se);
+	update_curr(cfs_rq);
+
+	if (flags & INCUR_DEBT) {
+		u64 debt = se->sum_exec_runtime - se->remote_runtime_begin;
+
+		if (unlikely((s64)debt <= 0))
+			goto out;
+
+		incur_cfs_debt(rq, se, tg, debt);
+	}
+
+out:
+	se->remote_runtime_begin = se->sum_exec_runtime;
+
+	task_rq_unlock(rq, p, &rf);
+}
+
+void cpu_cgroup_remote_begin_fair(struct task_struct *p, struct task_group *tg)
+{
+	cpu_cgroup_remote(p, tg, 0);
+}
+
+void cpu_cgroup_remote_charge_fair(struct task_struct *p, struct task_group *tg)
+{
+	cpu_cgroup_remote(p, tg, INCUR_DEBT);
+}
+
 #else /* CONFIG_FAIR_GROUP_SCHED */
 
 void free_fair_sched_group(struct task_group *tg) { }
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index ddefb0419d7a..75dd6f89e295 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -367,6 +367,7 @@ struct cfs_bandwidth {
 	u64			quota;
 	u64			runtime;
 	u64			burst;
+	u64			debt;
 	s64			hierarchical_quota;
 
 	u8			idle;
@@ -472,6 +473,10 @@ extern void free_fair_sched_group(struct task_group *tg);
 extern int alloc_fair_sched_group(struct task_group *tg, struct task_group *parent);
 extern void online_fair_sched_group(struct task_group *tg);
 extern void unregister_fair_sched_group(struct task_group *tg);
+extern void cpu_cgroup_remote_begin_fair(struct task_struct *p,
+					 struct task_group *tg);
+extern void cpu_cgroup_remote_charge_fair(struct task_struct *p,
+					  struct task_group *tg);
 extern void init_tg_cfs_entry(struct task_group *tg, struct cfs_rq *cfs_rq,
 			struct sched_entity *se, int cpu,
 			struct sched_entity *parent);
-- 
2.34.1


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

* [RFC 16/16] sched/fair: Consider kthread debt in cputime
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (14 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
@ 2022-01-06  0:46 ` Daniel Jordan
  2022-01-06  1:13 ` [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Jason Gunthorpe
  16 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  0:46 UTC (permalink / raw)
  To: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Peter Zijlstra, Steffen Klassert, Steve Sistare, Tejun Heo,
	Tim Chen, Vincent Guittot
  Cc: linux-mm, kvm, linux-kernel, linux-crypto, Daniel Jordan

kthreads that charge their CPU time to a remote task group should be
accounted for in cgroup's cputime statistics.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 kernel/sched/fair.c | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 3c2d7f245c68..b3ebb34c475b 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -11524,6 +11524,11 @@ static void cpu_cgroup_remote(struct task_struct *p, struct task_group *tg,
 			goto out;
 
 		incur_cfs_debt(rq, se, tg, debt);
+
+		/* cputime accounting is only supported in cgroup2. */
+		__cgroup_account_cputime(tg->css.cgroup, debt);
+		__cgroup_account_cputime_field(tg->css.cgroup, CPUTIME_SYSTEM,
+					       debt);
 	}
 
 out:
-- 
2.34.1


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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-06  0:46 ` [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention Daniel Jordan
@ 2022-01-06  0:53   ` Jason Gunthorpe
  2022-01-06  1:17     ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-06  0:53 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 07:46:48PM -0500, Daniel Jordan wrote:
> padata threads hold mmap_lock as reader for the majority of their
> runtime in order to call pin_user_pages_remote(), but they also
> periodically take mmap_lock as writer for short periods to adjust
> mm->locked_vm, hurting parallelism.
> 
> Alleviate the write-side contention with a per-thread cache of locked_vm
> which allows taking mmap_lock as writer far less frequently.
> 
> Failure to refill the cache due to insufficient locked_vm will not cause
> the entire pinning operation to error out.  This avoids spurious failure
> in case some pinned pages aren't accounted to locked_vm.
> 
> Cache size is limited to provide some protection in the unlikely event
> of a concurrent locked_vm accounting operation in the same address space
> needlessly failing in case the cache takes more locked_vm than it needs.

Why not just do the pinned page accounting once at the start? Why does
it have to be done incrementally?

Jason

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

* Re: [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
  2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
                   ` (15 preceding siblings ...)
  2022-01-06  0:46 ` [RFC 16/16] sched/fair: Consider kthread debt in cputime Daniel Jordan
@ 2022-01-06  1:13 ` Jason Gunthorpe
  2022-01-07  3:03   ` Daniel Jordan
  16 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-06  1:13 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 07:46:40PM -0500, Daniel Jordan wrote:

> Get ready to parallelize.  In particular, pinning can fail, so make jobs
> undo-able.
> 
>      5  vfio/type1: Pass mm to vfio_pin_pages_remote()
>      6  vfio/type1: Refactor dma map removal
>      7  vfio/type1: Parallelize vfio_pin_map_dma()
>      8  vfio/type1: Cache locked_vm to ease mmap_lock contention

In some ways this kind of seems like overkill, why not just have
userspace break the guest VA into chunks and call map in parallel?
Similar to how it already does the prealloc in parallel?

This is a simpler kernel job of optimizing locking to allow
concurrency.

It is also not good that this inserts arbitary cuts in the IOVA
address space, that will cause iommu_map() to be called with smaller
npages, and could result in a long term inefficiency in the iommu.

I don't know how the kernel can combat this without prior knowledge of
the likely physical memory layout (eg is the VM using 1G huge pages or
something)..

Personally I'd rather see the results from Matthew's work to allow GUP
to work on folios efficiently before reaching to this extreme.

The results you got of only 1.2x improvement don't seem so
compelling. Based on the unpin work I fully expect that folio
optimized GUP will do much better than that with single threaded..

Jason

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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-06  0:53   ` Jason Gunthorpe
@ 2022-01-06  1:17     ` Daniel Jordan
  2022-01-06 12:34       ` Jason Gunthorpe
  0 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-06  1:17 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 08:53:39PM -0400, Jason Gunthorpe wrote:
> On Wed, Jan 05, 2022 at 07:46:48PM -0500, Daniel Jordan wrote:
> > padata threads hold mmap_lock as reader for the majority of their
> > runtime in order to call pin_user_pages_remote(), but they also
> > periodically take mmap_lock as writer for short periods to adjust
> > mm->locked_vm, hurting parallelism.
> > 
> > Alleviate the write-side contention with a per-thread cache of locked_vm
> > which allows taking mmap_lock as writer far less frequently.
> > 
> > Failure to refill the cache due to insufficient locked_vm will not cause
> > the entire pinning operation to error out.  This avoids spurious failure
> > in case some pinned pages aren't accounted to locked_vm.
> > 
> > Cache size is limited to provide some protection in the unlikely event
> > of a concurrent locked_vm accounting operation in the same address space
> > needlessly failing in case the cache takes more locked_vm than it needs.
> 
> Why not just do the pinned page accounting once at the start? Why does
> it have to be done incrementally?

Yeah, good question.  I tried doing it that way recently and it did
improve performance a bit, but I thought it wasn't enough of a gain to
justify how it overaccounted by the size of the entire pin.

If the concurrent accounting I worried about above isn't really a
concern, though, I can reconsider this.

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

* Re: [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus()
  2022-01-06  0:46 ` [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus() Daniel Jordan
@ 2022-01-06  6:06   ` kernel test robot
  2022-01-06 11:31   ` kernel test robot
  1 sibling, 0 replies; 47+ messages in thread
From: kernel test robot @ 2022-01-06  6:06 UTC (permalink / raw)
  To: kbuild-all

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

Hi Daniel,

[FYI, it's a private test report for your RFC patch.]
[auto build test WARNING on 7d2a07b769330c34b4deabeed939325c77a7ec2f]

url:    https://github.com/0day-ci/linux/commits/Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
base:   7d2a07b769330c34b4deabeed939325c77a7ec2f
config: arc-allyesconfig (https://download.01.org/0day-ci/archive/20220106/202201061409.pRp9ULXe-lkp(a)intel.com/config)
compiler: arceb-elf-gcc (GCC) 11.2.0
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/44b2a69264d6505d97cd115e781a10437a6dc90d
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
        git checkout 44b2a69264d6505d97cd115e781a10437a6dc90d
        # save the config file to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=gcc-11.2.0 make.cross O=build_dir ARCH=arc SHELL=/bin/bash kernel/sched/

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All warnings (new ones prefixed by >>):

   kernel/sched/core.c:3216:6: warning: no previous prototype for 'sched_set_stop_task' [-Wmissing-prototypes]
    3216 | void sched_set_stop_task(int cpu, struct task_struct *stop)
         |      ^~~~~~~~~~~~~~~~~~~
>> kernel/sched/core.c:10025:5: warning: no previous prototype for 'max_cfs_bandwidth_cpus' [-Wmissing-prototypes]
   10025 | int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css)
         |     ^~~~~~~~~~~~~~~~~~~~~~
   In file included from include/linux/perf_event.h:25,
                    from include/linux/trace_events.h:10,
                    from include/trace/trace_events.h:21,
                    from include/trace/define_trace.h:102,
                    from include/trace/events/sched.h:738,
                    from kernel/sched/core.c:10:
   arch/arc/include/asm/perf_event.h:126:27: warning: 'arc_pmu_cache_map' defined but not used [-Wunused-const-variable=]
     126 | static const unsigned int arc_pmu_cache_map[C(MAX)][C(OP_MAX)][C(RESULT_MAX)] = {
         |                           ^~~~~~~~~~~~~~~~~
   arch/arc/include/asm/perf_event.h:91:27: warning: 'arc_pmu_ev_hw_map' defined but not used [-Wunused-const-variable=]
      91 | static const char * const arc_pmu_ev_hw_map[] = {
         |                           ^~~~~~~~~~~~~~~~~


vim +/max_cfs_bandwidth_cpus +10025 kernel/sched/core.c

 10023	
 10024	/* Returns the max whole number of CPUs that @css's bandwidth settings allow. */
 10025	int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css)
 10026	{
 10027		struct task_group *tg = css_tg(css);
 10028		u64 quota_us, period_us;
 10029	
 10030		if (tg == &root_task_group)
 10031			return nr_cpu_ids;
 10032	
 10033		quota_us = tg_get_cfs_quota(tg);
 10034	
 10035		if (quota_us == RUNTIME_INF)
 10036			return nr_cpu_ids;
 10037	
 10038		period_us = tg_get_cfs_period(tg);
 10039	
 10040		return quota_us / period_us;
 10041	}
 10042	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
@ 2022-01-06  6:47   ` kernel test robot
  2022-01-06  6:47     ` kernel test robot
                     ` (2 subsequent siblings)
  3 siblings, 0 replies; 47+ messages in thread
From: kernel test robot @ 2022-01-06  6:47 UTC (permalink / raw)
  To: kbuild-all

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

Hi Daniel,

[FYI, it's a private test report for your RFC patch.]
[auto build test WARNING on 7d2a07b769330c34b4deabeed939325c77a7ec2f]

url:    https://github.com/0day-ci/linux/commits/Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
base:   7d2a07b769330c34b4deabeed939325c77a7ec2f
config: arc-allyesconfig (https://download.01.org/0day-ci/archive/20220106/202201061450.OId0pr1K-lkp(a)intel.com/config)
compiler: arceb-elf-gcc (GCC) 11.2.0
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/4c1eb3c284a7aac84110e319700d89e847dd0874
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
        git checkout 4c1eb3c284a7aac84110e319700d89e847dd0874
        # save the config file to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=gcc-11.2.0 make.cross O=build_dir ARCH=arc SHELL=/bin/bash kernel/sched/

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All warnings (new ones prefixed by >>):

   kernel/sched/core.c:3216:6: warning: no previous prototype for 'sched_set_stop_task' [-Wmissing-prototypes]
    3216 | void sched_set_stop_task(int cpu, struct task_struct *stop)
         |      ^~~~~~~~~~~~~~~~~~~
   kernel/sched/core.c:10026:5: warning: no previous prototype for 'max_cfs_bandwidth_cpus' [-Wmissing-prototypes]
   10026 | int max_cfs_bandwidth_cpus(struct cgroup_subsys_state *css)
         |     ^~~~~~~~~~~~~~~~~~~~~~
>> kernel/sched/core.c:10199:6: warning: no previous prototype for 'cpu_cgroup_remote_begin' [-Wmissing-prototypes]
   10199 | void cpu_cgroup_remote_begin(struct task_struct *p,
         |      ^~~~~~~~~~~~~~~~~~~~~~~
>> kernel/sched/core.c:10216:6: warning: no previous prototype for 'cpu_cgroup_remote_charge' [-Wmissing-prototypes]
   10216 | void cpu_cgroup_remote_charge(struct task_struct *p,
         |      ^~~~~~~~~~~~~~~~~~~~~~~~
   In file included from include/linux/perf_event.h:25,
                    from include/linux/trace_events.h:10,
                    from include/trace/trace_events.h:21,
                    from include/trace/define_trace.h:102,
                    from include/trace/events/sched.h:738,
                    from kernel/sched/core.c:10:
   arch/arc/include/asm/perf_event.h:126:27: warning: 'arc_pmu_cache_map' defined but not used [-Wunused-const-variable=]
     126 | static const unsigned int arc_pmu_cache_map[C(MAX)][C(OP_MAX)][C(RESULT_MAX)] = {
         |                           ^~~~~~~~~~~~~~~~~
   arch/arc/include/asm/perf_event.h:91:27: warning: 'arc_pmu_ev_hw_map' defined but not used [-Wunused-const-variable=]
      91 | static const char * const arc_pmu_ev_hw_map[] = {
         |                           ^~~~~~~~~~~~~~~~~


vim +/cpu_cgroup_remote_begin +10199 kernel/sched/core.c

 10185	
 10186	/**
 10187	 * cpu_cgroup_remote_begin - begin charging p's CPU usage to a remote css
 10188	 * @p: the kernel thread whose CPU usage should be accounted
 10189	 * @css: the css to which the CPU usage should be accounted
 10190	 *
 10191	 * Begin charging a kernel thread's CPU usage to a remote (non-root) task group
 10192	 * to account CPU time that the kernel thread spends working on behalf of the
 10193	 * group.  Pair with at least one subsequent call to cpu_cgroup_remote_charge()
 10194	 * to complete the charge.
 10195	 *
 10196	 * Supports CFS bandwidth and cgroup2 CPU accounting stats but not weight-based
 10197	 * control for now.
 10198	 */
 10199	void cpu_cgroup_remote_begin(struct task_struct *p,
 10200				     struct cgroup_subsys_state *css)
 10201	{
 10202		if (p->sched_class == &fair_sched_class)
 10203			cpu_cgroup_remote_begin_fair(p, css_tg(css));
 10204	}
 10205	
 10206	/**
 10207	 * cpu_cgroup_remote_charge - account p's CPU usage to a remote css
 10208	 * @p: the kernel thread whose CPU usage should be accounted
 10209	 * @css: the css to which the CPU usage should be accounted
 10210	 *
 10211	 * Account a kernel thread's CPU usage to a remote (non-root) task group.  Pair
 10212	 * with a previous call to cpu_cgroup_remote_begin() with the same @p and @css.
 10213	 * This may be invoked multiple times after the initial
 10214	 * cpu_cgroup_remote_begin() to account additional CPU usage.
 10215	 */
 10216	void cpu_cgroup_remote_charge(struct task_struct *p,
 10217				      struct cgroup_subsys_state *css)
 10218	{
 10219		if (p->sched_class == &fair_sched_class)
 10220			cpu_cgroup_remote_charge_fair(p, css_tg(css));
 10221	}
 10222	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
@ 2022-01-06  6:47     ` kernel test robot
  2022-01-06  6:47     ` kernel test robot
                       ` (2 subsequent siblings)
  3 siblings, 0 replies; 47+ messages in thread
From: kernel test robot @ 2022-01-06  6:47 UTC (permalink / raw)
  To: Daniel Jordan; +Cc: llvm, kbuild-all

Hi Daniel,

[FYI, it's a private test report for your RFC patch.]
[auto build test WARNING on 7d2a07b769330c34b4deabeed939325c77a7ec2f]

url:    https://github.com/0day-ci/linux/commits/Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
base:   7d2a07b769330c34b4deabeed939325c77a7ec2f
config: hexagon-buildonly-randconfig-r001-20220105 (https://download.01.org/0day-ci/archive/20220106/202201061412.xko6OiVF-lkp@intel.com/config)
compiler: clang version 14.0.0 (https://github.com/llvm/llvm-project ca7ffe09dc6e525109e3cd570cc5182ce568be13)
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/4c1eb3c284a7aac84110e319700d89e847dd0874
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
        git checkout 4c1eb3c284a7aac84110e319700d89e847dd0874
        # save the config file to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=clang make.cross W=1 O=build_dir ARCH=hexagon SHELL=/bin/bash kernel/sched/

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All warnings (new ones prefixed by >>):

   kernel/sched/core.c:3282:13: warning: variable 'rq' set but not used [-Wunused-but-set-variable]
           struct rq *rq;
                      ^
>> kernel/sched/core.c:10199:6: warning: no previous prototype for function 'cpu_cgroup_remote_begin' [-Wmissing-prototypes]
   void cpu_cgroup_remote_begin(struct task_struct *p,
        ^
   kernel/sched/core.c:10199:1: note: declare 'static' if the function is not intended to be used outside of this translation unit
   void cpu_cgroup_remote_begin(struct task_struct *p,
   ^
   static 
>> kernel/sched/core.c:10216:6: warning: no previous prototype for function 'cpu_cgroup_remote_charge' [-Wmissing-prototypes]
   void cpu_cgroup_remote_charge(struct task_struct *p,
        ^
   kernel/sched/core.c:10216:1: note: declare 'static' if the function is not intended to be used outside of this translation unit
   void cpu_cgroup_remote_charge(struct task_struct *p,
   ^
   static 
   kernel/sched/core.c:3272:20: warning: unused function 'rq_has_pinned_tasks' [-Wunused-function]
   static inline bool rq_has_pinned_tasks(struct rq *rq)
                      ^
   kernel/sched/core.c:5123:20: warning: unused function 'sched_tick_start' [-Wunused-function]
   static inline void sched_tick_start(int cpu) { }
                      ^
   kernel/sched/core.c:5124:20: warning: unused function 'sched_tick_stop' [-Wunused-function]
   static inline void sched_tick_stop(int cpu) { }
                      ^
   kernel/sched/core.c:5853:20: warning: unused function 'sched_core_cpu_starting' [-Wunused-function]
   static inline void sched_core_cpu_starting(unsigned int cpu) {}
                      ^
   kernel/sched/core.c:5854:20: warning: unused function 'sched_core_cpu_deactivate' [-Wunused-function]
   static inline void sched_core_cpu_deactivate(unsigned int cpu) {}
                      ^
   kernel/sched/core.c:5855:20: warning: unused function 'sched_core_cpu_dying' [-Wunused-function]
   static inline void sched_core_cpu_dying(unsigned int cpu) {}
                      ^
   9 warnings generated.


vim +/cpu_cgroup_remote_begin +10199 kernel/sched/core.c

 10185	
 10186	/**
 10187	 * cpu_cgroup_remote_begin - begin charging p's CPU usage to a remote css
 10188	 * @p: the kernel thread whose CPU usage should be accounted
 10189	 * @css: the css to which the CPU usage should be accounted
 10190	 *
 10191	 * Begin charging a kernel thread's CPU usage to a remote (non-root) task group
 10192	 * to account CPU time that the kernel thread spends working on behalf of the
 10193	 * group.  Pair with at least one subsequent call to cpu_cgroup_remote_charge()
 10194	 * to complete the charge.
 10195	 *
 10196	 * Supports CFS bandwidth and cgroup2 CPU accounting stats but not weight-based
 10197	 * control for now.
 10198	 */
 10199	void cpu_cgroup_remote_begin(struct task_struct *p,
 10200				     struct cgroup_subsys_state *css)
 10201	{
 10202		if (p->sched_class == &fair_sched_class)
 10203			cpu_cgroup_remote_begin_fair(p, css_tg(css));
 10204	}
 10205	
 10206	/**
 10207	 * cpu_cgroup_remote_charge - account p's CPU usage to a remote css
 10208	 * @p: the kernel thread whose CPU usage should be accounted
 10209	 * @css: the css to which the CPU usage should be accounted
 10210	 *
 10211	 * Account a kernel thread's CPU usage to a remote (non-root) task group.  Pair
 10212	 * with a previous call to cpu_cgroup_remote_begin() with the same @p and @css.
 10213	 * This may be invoked multiple times after the initial
 10214	 * cpu_cgroup_remote_begin() to account additional CPU usage.
 10215	 */
 10216	void cpu_cgroup_remote_charge(struct task_struct *p,
 10217				      struct cgroup_subsys_state *css)
 10218	{
 10219		if (p->sched_class == &fair_sched_class)
 10220			cpu_cgroup_remote_charge_fair(p, css_tg(css));
 10221	}
 10222	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all@lists.01.org

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
@ 2022-01-06  6:47     ` kernel test robot
  0 siblings, 0 replies; 47+ messages in thread
From: kernel test robot @ 2022-01-06  6:47 UTC (permalink / raw)
  To: kbuild-all

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

Hi Daniel,

[FYI, it's a private test report for your RFC patch.]
[auto build test WARNING on 7d2a07b769330c34b4deabeed939325c77a7ec2f]

url:    https://github.com/0day-ci/linux/commits/Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
base:   7d2a07b769330c34b4deabeed939325c77a7ec2f
config: hexagon-buildonly-randconfig-r001-20220105 (https://download.01.org/0day-ci/archive/20220106/202201061412.xko6OiVF-lkp(a)intel.com/config)
compiler: clang version 14.0.0 (https://github.com/llvm/llvm-project ca7ffe09dc6e525109e3cd570cc5182ce568be13)
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/4c1eb3c284a7aac84110e319700d89e847dd0874
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
        git checkout 4c1eb3c284a7aac84110e319700d89e847dd0874
        # save the config file to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=clang make.cross W=1 O=build_dir ARCH=hexagon SHELL=/bin/bash kernel/sched/

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All warnings (new ones prefixed by >>):

   kernel/sched/core.c:3282:13: warning: variable 'rq' set but not used [-Wunused-but-set-variable]
           struct rq *rq;
                      ^
>> kernel/sched/core.c:10199:6: warning: no previous prototype for function 'cpu_cgroup_remote_begin' [-Wmissing-prototypes]
   void cpu_cgroup_remote_begin(struct task_struct *p,
        ^
   kernel/sched/core.c:10199:1: note: declare 'static' if the function is not intended to be used outside of this translation unit
   void cpu_cgroup_remote_begin(struct task_struct *p,
   ^
   static 
>> kernel/sched/core.c:10216:6: warning: no previous prototype for function 'cpu_cgroup_remote_charge' [-Wmissing-prototypes]
   void cpu_cgroup_remote_charge(struct task_struct *p,
        ^
   kernel/sched/core.c:10216:1: note: declare 'static' if the function is not intended to be used outside of this translation unit
   void cpu_cgroup_remote_charge(struct task_struct *p,
   ^
   static 
   kernel/sched/core.c:3272:20: warning: unused function 'rq_has_pinned_tasks' [-Wunused-function]
   static inline bool rq_has_pinned_tasks(struct rq *rq)
                      ^
   kernel/sched/core.c:5123:20: warning: unused function 'sched_tick_start' [-Wunused-function]
   static inline void sched_tick_start(int cpu) { }
                      ^
   kernel/sched/core.c:5124:20: warning: unused function 'sched_tick_stop' [-Wunused-function]
   static inline void sched_tick_stop(int cpu) { }
                      ^
   kernel/sched/core.c:5853:20: warning: unused function 'sched_core_cpu_starting' [-Wunused-function]
   static inline void sched_core_cpu_starting(unsigned int cpu) {}
                      ^
   kernel/sched/core.c:5854:20: warning: unused function 'sched_core_cpu_deactivate' [-Wunused-function]
   static inline void sched_core_cpu_deactivate(unsigned int cpu) {}
                      ^
   kernel/sched/core.c:5855:20: warning: unused function 'sched_core_cpu_dying' [-Wunused-function]
   static inline void sched_core_cpu_dying(unsigned int cpu) {}
                      ^
   9 warnings generated.


vim +/cpu_cgroup_remote_begin +10199 kernel/sched/core.c

 10185	
 10186	/**
 10187	 * cpu_cgroup_remote_begin - begin charging p's CPU usage to a remote css
 10188	 * @p: the kernel thread whose CPU usage should be accounted
 10189	 * @css: the css to which the CPU usage should be accounted
 10190	 *
 10191	 * Begin charging a kernel thread's CPU usage to a remote (non-root) task group
 10192	 * to account CPU time that the kernel thread spends working on behalf of the
 10193	 * group.  Pair with at least one subsequent call to cpu_cgroup_remote_charge()
 10194	 * to complete the charge.
 10195	 *
 10196	 * Supports CFS bandwidth and cgroup2 CPU accounting stats but not weight-based
 10197	 * control for now.
 10198	 */
 10199	void cpu_cgroup_remote_begin(struct task_struct *p,
 10200				     struct cgroup_subsys_state *css)
 10201	{
 10202		if (p->sched_class == &fair_sched_class)
 10203			cpu_cgroup_remote_begin_fair(p, css_tg(css));
 10204	}
 10205	
 10206	/**
 10207	 * cpu_cgroup_remote_charge - account p's CPU usage to a remote css
 10208	 * @p: the kernel thread whose CPU usage should be accounted
 10209	 * @css: the css to which the CPU usage should be accounted
 10210	 *
 10211	 * Account a kernel thread's CPU usage to a remote (non-root) task group.  Pair
 10212	 * with a previous call to cpu_cgroup_remote_begin() with the same @p and @css.
 10213	 * This may be invoked multiple times after the initial
 10214	 * cpu_cgroup_remote_begin() to account additional CPU usage.
 10215	 */
 10216	void cpu_cgroup_remote_charge(struct task_struct *p,
 10217				      struct cgroup_subsys_state *css)
 10218	{
 10219		if (p->sched_class == &fair_sched_class)
 10220			cpu_cgroup_remote_charge_fair(p, css_tg(css));
 10221	}
 10222	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

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

* Re: [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus()
  2022-01-06  0:46 ` [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus() Daniel Jordan
  2022-01-06  6:06   ` kernel test robot
@ 2022-01-06 11:31   ` kernel test robot
  1 sibling, 0 replies; 47+ messages in thread
From: kernel test robot @ 2022-01-06 11:31 UTC (permalink / raw)
  To: kbuild-all

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

Hi Daniel,

[FYI, it's a private test report for your RFC patch.]
[auto build test ERROR on 7d2a07b769330c34b4deabeed939325c77a7ec2f]

url:    https://github.com/0day-ci/linux/commits/Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
base:   7d2a07b769330c34b4deabeed939325c77a7ec2f
config: m68k-allmodconfig (https://download.01.org/0day-ci/archive/20220106/202201061938.J2A5yLyf-lkp(a)intel.com/config)
compiler: m68k-linux-gcc (GCC) 11.2.0
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/44b2a69264d6505d97cd115e781a10437a6dc90d
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Daniel-Jordan/padata-vfio-sched-Multithreaded-VFIO-page-pinning/20220106-084934
        git checkout 44b2a69264d6505d97cd115e781a10437a6dc90d
        # save the config file to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=gcc-11.2.0 make.cross O=build_dir ARCH=m68k SHELL=/bin/bash

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All errors (new ones prefixed by >>):

   m68k-linux-ld: kernel/sched/core.o: in function `max_cfs_bandwidth_cpus':
>> (.text+0x56fc): undefined reference to `__udivdi3'

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-06  1:17     ` Daniel Jordan
@ 2022-01-06 12:34       ` Jason Gunthorpe
  2022-01-06 21:05         ` Alex Williamson
  0 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-06 12:34 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 08:17:08PM -0500, Daniel Jordan wrote:
> On Wed, Jan 05, 2022 at 08:53:39PM -0400, Jason Gunthorpe wrote:
> > On Wed, Jan 05, 2022 at 07:46:48PM -0500, Daniel Jordan wrote:
> > > padata threads hold mmap_lock as reader for the majority of their
> > > runtime in order to call pin_user_pages_remote(), but they also
> > > periodically take mmap_lock as writer for short periods to adjust
> > > mm->locked_vm, hurting parallelism.
> > > 
> > > Alleviate the write-side contention with a per-thread cache of locked_vm
> > > which allows taking mmap_lock as writer far less frequently.
> > > 
> > > Failure to refill the cache due to insufficient locked_vm will not cause
> > > the entire pinning operation to error out.  This avoids spurious failure
> > > in case some pinned pages aren't accounted to locked_vm.
> > > 
> > > Cache size is limited to provide some protection in the unlikely event
> > > of a concurrent locked_vm accounting operation in the same address space
> > > needlessly failing in case the cache takes more locked_vm than it needs.
> > 
> > Why not just do the pinned page accounting once at the start? Why does
> > it have to be done incrementally?
> 
> Yeah, good question.  I tried doing it that way recently and it did
> improve performance a bit, but I thought it wasn't enough of a gain to
> justify how it overaccounted by the size of the entire pin.

Why would it over account?

Jason

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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-06 12:34       ` Jason Gunthorpe
@ 2022-01-06 21:05         ` Alex Williamson
  2022-01-07  0:19           ` Jason Gunthorpe
  0 siblings, 1 reply; 47+ messages in thread
From: Alex Williamson @ 2022-01-06 21:05 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Daniel Jordan, Alexander Duyck, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Thu, 6 Jan 2022 08:34:56 -0400
Jason Gunthorpe <jgg@nvidia.com> wrote:

> On Wed, Jan 05, 2022 at 08:17:08PM -0500, Daniel Jordan wrote:
> > On Wed, Jan 05, 2022 at 08:53:39PM -0400, Jason Gunthorpe wrote:  
> > > On Wed, Jan 05, 2022 at 07:46:48PM -0500, Daniel Jordan wrote:  
> > > > padata threads hold mmap_lock as reader for the majority of their
> > > > runtime in order to call pin_user_pages_remote(), but they also
> > > > periodically take mmap_lock as writer for short periods to adjust
> > > > mm->locked_vm, hurting parallelism.
> > > > 
> > > > Alleviate the write-side contention with a per-thread cache of locked_vm
> > > > which allows taking mmap_lock as writer far less frequently.
> > > > 
> > > > Failure to refill the cache due to insufficient locked_vm will not cause
> > > > the entire pinning operation to error out.  This avoids spurious failure
> > > > in case some pinned pages aren't accounted to locked_vm.
> > > > 
> > > > Cache size is limited to provide some protection in the unlikely event
> > > > of a concurrent locked_vm accounting operation in the same address space
> > > > needlessly failing in case the cache takes more locked_vm than it needs.  
> > > 
> > > Why not just do the pinned page accounting once at the start? Why does
> > > it have to be done incrementally?  
> > 
> > Yeah, good question.  I tried doing it that way recently and it did
> > improve performance a bit, but I thought it wasn't enough of a gain to
> > justify how it overaccounted by the size of the entire pin.  
> 
> Why would it over account?

We'd be guessing that the entire virtual address mapping counts against
locked memory limits, but it might include PFNMAP pages or pages that
are already account via the page pinning interface that mdev devices
use.  At that point we're risking that the user isn't concurrently
doing something else that could fail as a result of pre-accounting and
fixup later schemes like this.  Thanks,

Alex


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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-06 21:05         ` Alex Williamson
@ 2022-01-07  0:19           ` Jason Gunthorpe
  2022-01-07  3:06             ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-07  0:19 UTC (permalink / raw)
  To: Alex Williamson
  Cc: Daniel Jordan, Alexander Duyck, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Thu, Jan 06, 2022 at 02:05:27PM -0700, Alex Williamson wrote:
> > > Yeah, good question.  I tried doing it that way recently and it did
> > > improve performance a bit, but I thought it wasn't enough of a gain to
> > > justify how it overaccounted by the size of the entire pin.  
> > 
> > Why would it over account?
> 
> We'd be guessing that the entire virtual address mapping counts against
> locked memory limits, but it might include PFNMAP pages or pages that
> are already account via the page pinning interface that mdev devices
> use.  At that point we're risking that the user isn't concurrently
> doing something else that could fail as a result of pre-accounting and
> fixup later schemes like this.  Thanks,

At least in iommufd I'm planning to keep the P2P ranges seperated from
the normal page ranges. For user space compat we'd have to scan over
the VA range looking for special VMAs. I expect in most cases there
are few VMAs..

Computing the # pages pinned by mdevs requires a interval tree scan,
in Daniel's target scenario the intervals will be empty so this costs
nothing.

At least it seems like it is not an insurmountable problem if it makes
an appreciable difference..

After seeing Daniels's patches I've been wondering if the pin step in
iommufd's draft could be parallized on a per-map basis without too
much trouble. It might give Daniel a way to do a quick approach
comparison..

Jason

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

* Re: [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
  2022-01-06  1:13 ` [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Jason Gunthorpe
@ 2022-01-07  3:03   ` Daniel Jordan
  2022-01-07 17:12     ` Jason Gunthorpe
  0 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-07  3:03 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 09:13:06PM -0400, Jason Gunthorpe wrote:
> On Wed, Jan 05, 2022 at 07:46:40PM -0500, Daniel Jordan wrote:
> 
> > Get ready to parallelize.  In particular, pinning can fail, so make jobs
> > undo-able.
> > 
> >      5  vfio/type1: Pass mm to vfio_pin_pages_remote()
> >      6  vfio/type1: Refactor dma map removal
> >      7  vfio/type1: Parallelize vfio_pin_map_dma()
> >      8  vfio/type1: Cache locked_vm to ease mmap_lock contention
> 
> In some ways this kind of seems like overkill, why not just have
> userspace break the guest VA into chunks and call map in parallel?
> Similar to how it already does the prealloc in parallel?
>
> This is a simpler kernel job of optimizing locking to allow
> concurrency.

I didn't consider doing it that way, and am not seeing a fundamental
reason it wouldn't work right off the bat.

At a glance, I think pinning would need to be moved out from under
vfio's iommu->lock.  I haven't checked to see how hard it would be, but
that plus the locking optimizations might end up being about the same
amount of complexity as the multithreading in the vfio driver, and doing
this in the kernel would speed things up for all vfio users without
having to duplicate the parallelism in userspace.

But yes, agreed, the lock optimization could definitely be split out and
used in a different approach.

> It is also not good that this inserts arbitary cuts in the IOVA
> address space, that will cause iommu_map() to be called with smaller
> npages, and could result in a long term inefficiency in the iommu.
> 
> I don't know how the kernel can combat this without prior knowledge of
> the likely physical memory layout (eg is the VM using 1G huge pages or
> something)..

The cuts aren't arbitrary, padata controls where they happen.  This is
optimizing for big memory ranges, so why isn't it enough that padata
breaks up the work along a big enough page-aligned chunk?  The vfio
driver does one iommu mapping per physically contiguous range, and I
don't think those will be large enough to be affected using such a chunk
size.  If cuts in per-thread ranges are an issue, I *think* userspace
has the same problem?

> Personally I'd rather see the results from Matthew's work to allow GUP
> to work on folios efficiently before reaching to this extreme.
> 
> The results you got of only 1.2x improvement don't seem so
> compelling.

I know you understand, but just to be clear for everyone, that 1.2x is
the overall improvement to qemu init from multithreaded pinning alone
when prefaulting is done in both base and test.

Pinning itself, the only thing being optimized, improves 8.5x in that
experiment, bringing the time from 1.8 seconds to .2 seconds.  That's a
significant savings IMHO

> Based on the unpin work I fully expect that folio
> optimized GUP will do much better than that with single threaded..

Yeah, I'm curious to see how folio will do as well.  And there are some
very nice, efficiently gained speedups in the unpin work.  Changes like
that benefit all gup users, too, as you've pointed out before.

But, I'm skeptical that singlethreaded optimization alone will remove
the bottleneck with the enormous memory sizes we use.  For instance,
scaling up the times from the unpin results with both optimizations (the
IB specific one too, which would need to be done for vfio), a 1T guest
would still take almost 2 seconds to pin/unpin.  

If people feel strongly that we should try optimizing other ways first,
ok, but I think these are complementary approaches.  I'm coming at this
problem this way because this is fundamentally a memory-intensive
operation where more bandwidth can help, and there are other kernel
paths we and others want this infrastructure for.

In any case, thanks a lot for the super quick feedback!

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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-07  0:19           ` Jason Gunthorpe
@ 2022-01-07  3:06             ` Daniel Jordan
  2022-01-07 15:18               ` Jason Gunthorpe
  0 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-07  3:06 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Alex Williamson, Alexander Duyck, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Thu, Jan 06, 2022 at 08:19:45PM -0400, Jason Gunthorpe wrote:
> On Thu, Jan 06, 2022 at 02:05:27PM -0700, Alex Williamson wrote:
> > > > Yeah, good question.  I tried doing it that way recently and it did
> > > > improve performance a bit, but I thought it wasn't enough of a gain to
> > > > justify how it overaccounted by the size of the entire pin.  
> > > 
> > > Why would it over account?
> > 
> > We'd be guessing that the entire virtual address mapping counts against
> > locked memory limits, but it might include PFNMAP pages or pages that
> > are already account via the page pinning interface that mdev devices
> > use.  At that point we're risking that the user isn't concurrently
> > doing something else that could fail as a result of pre-accounting and
> > fixup later schemes like this.  Thanks,

Yes, that's exactly what I was thinking.

> At least in iommufd I'm planning to keep the P2P ranges seperated from
> the normal page ranges. For user space compat we'd have to scan over
> the VA range looking for special VMAs. I expect in most cases there
> are few VMAs..
> 
> Computing the # pages pinned by mdevs requires a interval tree scan,
> in Daniel's target scenario the intervals will be empty so this costs
> nothing.
> 
> At least it seems like it is not an insurmountable problem if it makes
> an appreciable difference..

Ok, I can think more about this.

> After seeing Daniels's patches I've been wondering if the pin step in
> iommufd's draft could be parallized on a per-map basis without too
> much trouble. It might give Daniel a way to do a quick approach
> comparison..

Sorry, comparison between what?  I can take a look at iommufd tomorrow
though and see if your comment makes more sense.

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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-07  3:06             ` Daniel Jordan
@ 2022-01-07 15:18               ` Jason Gunthorpe
  2022-01-07 16:39                 ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-07 15:18 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alex Williamson, Alexander Duyck, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Thu, Jan 06, 2022 at 10:06:42PM -0500, Daniel Jordan wrote:

> > At least it seems like it is not an insurmountable problem if it makes
> > an appreciable difference..
> 
> Ok, I can think more about this.

Unfortunately iommufd is not quite ready yet, otherwise I might
suggest just focus on that not type 1 optimizations. Depends on your
timeframe I suppose.

> > After seeing Daniels's patches I've been wondering if the pin step in
> > iommufd's draft could be parallized on a per-map basis without too
> > much trouble. It might give Daniel a way to do a quick approach
> > comparison..
> 
> Sorry, comparison between what?  I can take a look at iommufd tomorrow
> though and see if your comment makes more sense.

I think it might be easier to change the iommufd locking than the
type1 locking to allow kernel-side parallel map ioctls. It is already
almost properly locked for this right now, just the iopt lock covers a
little bit too much.

It could give some idea what kind of performance user managed
concurrency gives vs kernel auto threading.

Jason

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

* Re: [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention
  2022-01-07 15:18               ` Jason Gunthorpe
@ 2022-01-07 16:39                 ` Daniel Jordan
  0 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-07 16:39 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Alex Williamson, Alexander Duyck, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Fri, Jan 07, 2022 at 11:18:07AM -0400, Jason Gunthorpe wrote:
> On Thu, Jan 06, 2022 at 10:06:42PM -0500, Daniel Jordan wrote:
> 
> > > At least it seems like it is not an insurmountable problem if it makes
> > > an appreciable difference..
> > 
> > Ok, I can think more about this.
> 
> Unfortunately iommufd is not quite ready yet, otherwise I might
> suggest just focus on that not type 1 optimizations. Depends on your
> timeframe I suppose.

Ok, I see.  Well, sooner the better I guess, we've been carrying changes
for this a while.

> > > After seeing Daniels's patches I've been wondering if the pin step in
> > > iommufd's draft could be parallized on a per-map basis without too
> > > much trouble. It might give Daniel a way to do a quick approach
> > > comparison..
> > 
> > Sorry, comparison between what?  I can take a look at iommufd tomorrow
> > though and see if your comment makes more sense.
> 
> I think it might be easier to change the iommufd locking than the
> type1 locking to allow kernel-side parallel map ioctls. It is already
> almost properly locked for this right now, just the iopt lock covers a
> little bit too much.
> 
> It could give some idea what kind of performance user managed
> concurrency gives vs kernel auto threading.

Aha, I see, thanks!

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

* Re: [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
  2022-01-07  3:03   ` Daniel Jordan
@ 2022-01-07 17:12     ` Jason Gunthorpe
  2022-01-10 22:27       ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-07 17:12 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto


> > It is also not good that this inserts arbitary cuts in the IOVA
> > address space, that will cause iommu_map() to be called with smaller
> > npages, and could result in a long term inefficiency in the iommu.
> > 
> > I don't know how the kernel can combat this without prior knowledge of
> > the likely physical memory layout (eg is the VM using 1G huge pages or
> > something)..
>
> The cuts aren't arbitrary, padata controls where they happen.  

Well, they are, you picked a PMD alignment if I recall.

If hugetlbfs is using PUD pages then this is the wrong alignment,
right?

I suppose it could compute the cuts differently to try to maximize
alignment at the cutpoints.. 

> size.  If cuts in per-thread ranges are an issue, I *think* userspace
> has the same problem?

Userspace should know what it has done, if it is using hugetlbfs it
knows how big the pages are.

> > The results you got of only 1.2x improvement don't seem so
> > compelling.
> 
> I know you understand, but just to be clear for everyone, that 1.2x is
> the overall improvement to qemu init from multithreaded pinning alone
> when prefaulting is done in both base and test.

Yes

> Pinning itself, the only thing being optimized, improves 8.5x in that
> experiment, bringing the time from 1.8 seconds to .2 seconds.  That's a
> significant savings IMHO

And here is where I suspect we'd get similar results from folio's
based on the unpin performance uplift we already saw.

As long as PUP doesn't have to COW its work is largely proportional to
the number of struct pages it processes, so we should be expecting an
upper limit of 512x gains on the PUP alone with foliation. This is in
line with what we saw with the prior unpin work.

The other optimization that would help a lot here is to use
pin_user_pages_fast(), something like:

  if (current->mm != remote_mm)
     mmap_lock()
     pin_user_pages_remote(..)
     mmap_unlock()
  else
     pin_user_pages_fast(..)

But you can't get that gain with kernel-size parallization, right?

(I haven't dug into if gup_fast relies on current due to IPIs or not,
maybe pin_user_pages_remote_fast can exist?)

> But, I'm skeptical that singlethreaded optimization alone will remove
> the bottleneck with the enormous memory sizes we use.  

I think you can get the 1.2x at least.

> scaling up the times from the unpin results with both optimizations (the
> IB specific one too, which would need to be done for vfio), 

Oh, I did the IB one already in iommufd...

> a 1T guest would still take almost 2 seconds to pin/unpin.

Single threaded? Isn't that excellent and completely dwarfed by the
populate overhead?

> If people feel strongly that we should try optimizing other ways first,
> ok, but I think these are complementary approaches.  I'm coming at this
> problem this way because this is fundamentally a memory-intensive
> operation where more bandwidth can help, and there are other kernel
> paths we and others want this infrastructure for.

At least here I would like to see an apples to apples at least before
we have this complexity. Full user threading vs kernel auto threading.

Saying multithreaded kernel gets 8x over single threaded userspace is
nice, but sort of irrelevant because we can have multithreaded
userspace, right?

Jason

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

* Re: [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
  2022-01-07 17:12     ` Jason Gunthorpe
@ 2022-01-10 22:27       ` Daniel Jordan
  2022-01-11  0:17         ` Jason Gunthorpe
  0 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-10 22:27 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Fri, Jan 07, 2022 at 01:12:48PM -0400, Jason Gunthorpe wrote:
> > The cuts aren't arbitrary, padata controls where they happen.  
> 
> Well, they are, you picked a PMD alignment if I recall.
> 
> If hugetlbfs is using PUD pages then this is the wrong alignment,
> right?
> 
> I suppose it could compute the cuts differently to try to maximize
> alignment at the cutpoints.. 

Yes, this is what I was suggesting, increase the alignment.

> > size.  If cuts in per-thread ranges are an issue, I *think* userspace
> > has the same problem?
> 
> Userspace should know what it has done, if it is using hugetlbfs it
> knows how big the pages are.

Right, what I mean is both user and kernel threads can end up splitting
a physically contiguous range of pages, however large the page size.

> > Pinning itself, the only thing being optimized, improves 8.5x in that
> > experiment, bringing the time from 1.8 seconds to .2 seconds.  That's a
> > significant savings IMHO
> 
> And here is where I suspect we'd get similar results from folio's
> based on the unpin performance uplift we already saw.
> 
> As long as PUP doesn't have to COW its work is largely proportional to
> the number of struct pages it processes, so we should be expecting an
> upper limit of 512x gains on the PUP alone with foliation.
>
> This is in line with what we saw with the prior unpin work.

"in line with what we saw"  Not following.  The unpin work had two
optimizations, I think, 4.5x and 3.5x which together give 16x.  Why is
that in line with the potential gains from pup?

Overall I see what you're saying, just curious what you meant here.

> The other optimization that would help a lot here is to use
> pin_user_pages_fast(), something like:
> 
>   if (current->mm != remote_mm)
>      mmap_lock()
>      pin_user_pages_remote(..)
>      mmap_unlock()
>   else
>      pin_user_pages_fast(..)
> 
> But you can't get that gain with kernel-size parallization, right?
> 
> (I haven't dug into if gup_fast relies on current due to IPIs or not,
> maybe pin_user_pages_remote_fast can exist?)

Yeah, not sure.  I'll have a look.

> > But, I'm skeptical that singlethreaded optimization alone will remove
> > the bottleneck with the enormous memory sizes we use.  
> 
> I think you can get the 1.2x at least.
> 
> > scaling up the times from the unpin results with both optimizations (the
> > IB specific one too, which would need to be done for vfio), 
> 
> Oh, I did the IB one already in iommufd...

Ahead of the curve!

> > a 1T guest would still take almost 2 seconds to pin/unpin.
> 
> Single threaded?

Yes.

> Isn't that excellent

Depends on who you ask, I guess.

> and completely dwarfed by the populate overhead?

Well yes, but here we all are optimizing gup anyway :-)

> > If people feel strongly that we should try optimizing other ways first,
> > ok, but I think these are complementary approaches.  I'm coming at this
> > problem this way because this is fundamentally a memory-intensive
> > operation where more bandwidth can help, and there are other kernel
> > paths we and others want this infrastructure for.
> 
> At least here I would like to see an apples to apples at least before
> we have this complexity. Full user threading vs kernel auto threading.
> 
> Saying multithreaded kernel gets 8x over single threaded userspace is
> nice, but sort of irrelevant because we can have multithreaded
> userspace, right?

One of my assumptions was that doing this in the kernel would benefit
all vfio users, avoiding duplicating the same sort of multithreading
logic across applications, including ones that didn't prefault.  Calling
it irrelevant seems a bit strong.  Parallelizing in either layer has its
upsides and downsides.

My assumption going into this series was that multithreading VFIO page
pinning in the kernel was a viable way forward given the positive
feedback I got from the VFIO maintainer last time I posted this, which
was admittedly a while ago, and I've since been focused on the other
parts of this series rather than what's been happening in the mm lately.
Anyway, your arguments are reasonable, so I'll go take a look at some of
these optimizations and see where I get.

Daniel

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

* Re: [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
  2022-01-10 22:27       ` Daniel Jordan
@ 2022-01-11  0:17         ` Jason Gunthorpe
  2022-01-11 16:20           ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Jason Gunthorpe @ 2022-01-11  0:17 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Mon, Jan 10, 2022 at 05:27:25PM -0500, Daniel Jordan wrote:

> > > Pinning itself, the only thing being optimized, improves 8.5x in that
> > > experiment, bringing the time from 1.8 seconds to .2 seconds.  That's a
> > > significant savings IMHO
> > 
> > And here is where I suspect we'd get similar results from folio's
> > based on the unpin performance uplift we already saw.
> > 
> > As long as PUP doesn't have to COW its work is largely proportional to
> > the number of struct pages it processes, so we should be expecting an
> > upper limit of 512x gains on the PUP alone with foliation.
> >
> > This is in line with what we saw with the prior unpin work.
> 
> "in line with what we saw"  Not following.  The unpin work had two
> optimizations, I think, 4.5x and 3.5x which together give 16x.  Why is
> that in line with the potential gains from pup?

It is the same basic issue, doing extra work, dirtying extra memory..

> > and completely dwarfed by the populate overhead?
> 
> Well yes, but here we all are optimizing gup anyway :-)

Well, I assume because we can user thread the populate, so I'd user
thread the gup too..

> One of my assumptions was that doing this in the kernel would benefit
> all vfio users, avoiding duplicating the same sort of multithreading
> logic across applications, including ones that didn't prefault.

I don't know of other users that use such huge memory sizes this would
matter, besides a VMM..

> My assumption going into this series was that multithreading VFIO page
> pinning in the kernel was a viable way forward given the positive
> feedback I got from the VFIO maintainer last time I posted this, which
> was admittedly a while ago, and I've since been focused on the other
> parts of this series rather than what's been happening in the mm lately.
> Anyway, your arguments are reasonable, so I'll go take a look at some of
> these optimizations and see where I get.

Well, it is not *unreasonable* it just doesn't seem compelling to me
yet.

Especially since we are not anywhere close to the limit of single
threaded performance. Aside from GUP, the whole way we transfer the
physical pages into the iommu is just begging for optimizations
eg Matthew's struct phyr needs to be an input and output at the iommu
layer to make this code really happy.

How much time do we burn messing around in redundant iommu layer
locking because everything is page at a time?

Jason

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
  2022-01-06  6:47   ` kernel test robot
  2022-01-06  6:47     ` kernel test robot
@ 2022-01-11 11:58   ` Peter Zijlstra
  2022-01-11 16:29     ` Daniel Jordan
  2022-01-14  9:31   ` Peter Zijlstra
  3 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-11 11:58 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 07:46:55PM -0500, Daniel Jordan wrote:
> As before, helpers in multithreaded jobs don't honor the main thread's
> CFS bandwidth limits, which could lead to the group exceeding its quota.
> 
> Fix it by having helpers remote charge their CPU time to the main
> thread's task group.  A helper calls a pair of new interfaces
> cpu_cgroup_remote_begin() and cpu_cgroup_remote_charge() (see function
> header comments) to achieve this.
> 
> This is just supposed to start a discussion, so it's pretty simple.
> Once a kthread has finished a remote charging period with
> cpu_cgroup_remote_charge(), its runtime is subtracted from the target
> task group's runtime (cfs_bandwidth::runtime) and any remainder is saved
> as debt (cfs_bandwidth::debt) to pay off in later periods.
> 
> Remote charging tasks aren't throttled when the group reaches its quota,
> and a task group doesn't run at all until its debt is completely paid,
> but these shortcomings can be addressed if the approach ends up being
> taken.
> 

*groan*... and not a single word on why it wouldn't be much better to
simply move the task into the relevant cgroup..

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

* Re: [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning
  2022-01-11  0:17         ` Jason Gunthorpe
@ 2022-01-11 16:20           ` Daniel Jordan
  0 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-11 16:20 UTC (permalink / raw)
  To: Jason Gunthorpe
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Johannes Weiner, Josh Triplett,
	Michal Hocko, Nico Pache, Pasha Tatashin, Peter Zijlstra,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Mon, Jan 10, 2022 at 08:17:51PM -0400, Jason Gunthorpe wrote:
> On Mon, Jan 10, 2022 at 05:27:25PM -0500, Daniel Jordan wrote:
> 
> > > > Pinning itself, the only thing being optimized, improves 8.5x in that
> > > > experiment, bringing the time from 1.8 seconds to .2 seconds.  That's a
> > > > significant savings IMHO
> > > 
> > > And here is where I suspect we'd get similar results from folio's
> > > based on the unpin performance uplift we already saw.
> > > 
> > > As long as PUP doesn't have to COW its work is largely proportional to
> > > the number of struct pages it processes, so we should be expecting an
> > > upper limit of 512x gains on the PUP alone with foliation.
> > >
> > > This is in line with what we saw with the prior unpin work.
> > 
> > "in line with what we saw"  Not following.  The unpin work had two
> > optimizations, I think, 4.5x and 3.5x which together give 16x.  Why is
> > that in line with the potential gains from pup?
> 
> It is the same basic issue, doing extra work, dirtying extra memory..

Ok, gotcha.

> I don't know of other users that use such huge memory sizes this would
> matter, besides a VMM..

Right, all the VMMs out there that use vfio.

> > My assumption going into this series was that multithreading VFIO page
> > pinning in the kernel was a viable way forward given the positive
> > feedback I got from the VFIO maintainer last time I posted this, which
> > was admittedly a while ago, and I've since been focused on the other
> > parts of this series rather than what's been happening in the mm lately.
> > Anyway, your arguments are reasonable, so I'll go take a look at some of
> > these optimizations and see where I get.
> 
> Well, it is not *unreasonable* it just doesn't seem compelling to me
> yet.
> 
> Especially since we are not anywhere close to the limit of single
> threaded performance. Aside from GUP, the whole way we transfer the
> physical pages into the iommu is just begging for optimizations
> eg Matthew's struct phyr needs to be an input and output at the iommu
> layer to make this code really happy.

/nods/  There are other ways forward.  As I say, I'll take a look.

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-11 11:58   ` Peter Zijlstra
@ 2022-01-11 16:29     ` Daniel Jordan
  2022-01-12 20:18       ` Tejun Heo
  0 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-11 16:29 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Tue, Jan 11, 2022 at 12:58:53PM +0100, Peter Zijlstra wrote:
> On Wed, Jan 05, 2022 at 07:46:55PM -0500, Daniel Jordan wrote:
> > As before, helpers in multithreaded jobs don't honor the main thread's
> > CFS bandwidth limits, which could lead to the group exceeding its quota.
> > 
> > Fix it by having helpers remote charge their CPU time to the main
> > thread's task group.  A helper calls a pair of new interfaces
> > cpu_cgroup_remote_begin() and cpu_cgroup_remote_charge() (see function
> > header comments) to achieve this.
> > 
> > This is just supposed to start a discussion, so it's pretty simple.
> > Once a kthread has finished a remote charging period with
> > cpu_cgroup_remote_charge(), its runtime is subtracted from the target
> > task group's runtime (cfs_bandwidth::runtime) and any remainder is saved
> > as debt (cfs_bandwidth::debt) to pay off in later periods.
> > 
> > Remote charging tasks aren't throttled when the group reaches its quota,
> > and a task group doesn't run at all until its debt is completely paid,
> > but these shortcomings can be addressed if the approach ends up being
> > taken.
> > 
> 
> *groan*... and not a single word on why it wouldn't be much better to
> simply move the task into the relevant cgroup..

Yes, the cover letter talks about that, I'll quote the relevant part
here.

---

    15  sched/fair: Account kthread runtime debt for CFS bandwidth
    16  sched/fair: Consider kthread debt in cputime

A prototype for remote charging in CFS bandwidth and cpu.stat, described more
in the next section.  It's debatable whether these last two are required for
this series.  Patch 12 caps the number of helper threads started according to
the max effective CPUs allowed by the quota and period of the main thread's
task group.  In practice, I think this hits the sweet spot between complexity
and respecting CFS bandwidth limits so that patch 15 might just be dropped.
For instance, when running qemu with a vfio device, the restriction from patch
12 was enough to avoid the helpers breaching CFS bandwidth limits.  That leaves
patch 16, which on its own seems overkill for all the hunks it would require
from patch 15, so it could be dropped too.

Patch 12 isn't airtight, though, since other tasks running in the task group
alongside the main thread and helpers could still result in overage.  So,
patches 15-16 give an idea of what absolutely correct accounting in the CPU
controller might look like in case there are real situations that want it.


Remote Charging in the CPU Controller
-------------------------------------

CPU-intensive kthreads aren't generally accounted in the CPU controller, so
they escape settings such as weight and bandwidth when they do work on behalf
of a task group.

This problem arises with multithreaded jobs, but is also an issue in other
places.  CPU activity from async memory reclaim (kswapd, cswapd?[5]) should be
accounted to the cgroup that the memory belongs to, and similarly CPU activity
from net rx should be accounted to the task groups that correspond to the
packets being received.  There are also vague complaints from Android[6].

Each use case has its own requirements[7].  In padata and reclaim, the task
group to account to is known ahead of time, but net rx has to spend cycles
processing a packet before its destination task group is known, so any solution
should be able to work without knowing the task group in advance.  Furthermore,
the CPU controller shouldn't throttle reclaim or net rx in real time since both
are doing high priority work.  These make approaches that run kthreads directly
in a task group, like cgroup-aware workqueues[8] or a kernel path for
CLONE_INTO_CGROUP, infeasible.  Running kthreads directly in cgroups also has a
downside for padata because helpers' MAX_NICE priority is "shadowed" by the
priority of the group entities they're running under.

The proposed solution of remote charging can accrue debt to a task group to be
paid off or forgiven later, addressing all these issues.  A kthread calls the
interface

    void cpu_cgroup_remote_begin(struct task_struct *p,
                                 struct cgroup_subsys_state *css);

to begin remote charging to @css, causing @p's current sum_exec_runtime to be
updated and saved.  The @css arg isn't required and can be removed later to
facilitate the unknown cgroup case mentioned above.  Then the kthread calls
another interface

    void cpu_cgroup_remote_charge(struct task_struct *p,
                                  struct cgroup_subsys_state *css);

to account the sum_exec_runtime that @p has used since the first call.
Internally, a new field cfs_bandwidth::debt is added to keep track of unpaid
debt that's only used when the debt exceeds the quota in the current period.

Weight-based control isn't implemented for now since padata helpers run at
MAX_NICE and so always yield to anything higher priority, meaning they would
rarely compete with other task groups.

[ We have another use case to use remote charging for implementing
  CFS bandwidth control across multiple machines.  This is an entirely
  different topic that deserves its own thread. ]


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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-11 16:29     ` Daniel Jordan
@ 2022-01-12 20:18       ` Tejun Heo
  2022-01-13 21:08         ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Tejun Heo @ 2022-01-12 20:18 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Peter Zijlstra, Alexander Duyck, Alex Williamson, Andrew Morton,
	Ben Segall, Cornelia Huck, Dan Williams, Dave Hansen,
	Dietmar Eggemann, Herbert Xu, Ingo Molnar, Jason Gunthorpe,
	Johannes Weiner, Josh Triplett, Michal Hocko, Nico Pache,
	Pasha Tatashin, Steffen Klassert, Steve Sistare, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

Hello,

On Tue, Jan 11, 2022 at 11:29:50AM -0500, Daniel Jordan wrote:
...
> This problem arises with multithreaded jobs, but is also an issue in other
> places.  CPU activity from async memory reclaim (kswapd, cswapd?[5]) should be
> accounted to the cgroup that the memory belongs to, and similarly CPU activity
> from net rx should be accounted to the task groups that correspond to the
> packets being received.  There are also vague complaints from Android[6].

These are pretty big holes in CPU cycle accounting right now and I think
spend-first-and-backcharge is the right solution for most of them given
experiences from other controllers. That said,

> Each use case has its own requirements[7].  In padata and reclaim, the task
> group to account to is known ahead of time, but net rx has to spend cycles
> processing a packet before its destination task group is known, so any solution
> should be able to work without knowing the task group in advance.  Furthermore,
> the CPU controller shouldn't throttle reclaim or net rx in real time since both
> are doing high priority work.  These make approaches that run kthreads directly
> in a task group, like cgroup-aware workqueues[8] or a kernel path for
> CLONE_INTO_CGROUP, infeasible.  Running kthreads directly in cgroups also has a
> downside for padata because helpers' MAX_NICE priority is "shadowed" by the
> priority of the group entities they're running under.
> 
> The proposed solution of remote charging can accrue debt to a task group to be
> paid off or forgiven later, addressing all these issues.  A kthread calls the
> interface
> 
>     void cpu_cgroup_remote_begin(struct task_struct *p,
>                                  struct cgroup_subsys_state *css);
> 
> to begin remote charging to @css, causing @p's current sum_exec_runtime to be
> updated and saved.  The @css arg isn't required and can be removed later to
> facilitate the unknown cgroup case mentioned above.  Then the kthread calls
> another interface
> 
>     void cpu_cgroup_remote_charge(struct task_struct *p,
>                                   struct cgroup_subsys_state *css);
> 
> to account the sum_exec_runtime that @p has used since the first call.
> Internally, a new field cfs_bandwidth::debt is added to keep track of unpaid
> debt that's only used when the debt exceeds the quota in the current period.
> 
> Weight-based control isn't implemented for now since padata helpers run at
> MAX_NICE and so always yield to anything higher priority, meaning they would
> rarely compete with other task groups.

If we're gonna do this, let's please do it right and make weight based
control work too. Otherwise, its usefulness is pretty limited.

Thanks.

-- 
tejun

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-12 20:18       ` Tejun Heo
@ 2022-01-13 21:08         ` Daniel Jordan
  2022-01-13 21:11           ` Daniel Jordan
  0 siblings, 1 reply; 47+ messages in thread
From: Daniel Jordan @ 2022-01-13 21:08 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Peter Zijlstra, Alexander Duyck, Alex Williamson, Andrew Morton,
	Ben Segall, Cornelia Huck, Dan Williams, Dave Hansen,
	Dietmar Eggemann, Herbert Xu, Ingo Molnar, Jason Gunthorpe,
	Johannes Weiner, Josh Triplett, Michal Hocko, Nico Pache,
	Pasha Tatashin, Steffen Klassert, Steve Sistare, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 12, 2022 at 10:18:16AM -1000, Tejun Heo wrote:
> Hello,

Hi, Tejun.

> On Tue, Jan 11, 2022 at 11:29:50AM -0500, Daniel Jordan wrote:
> ...
> > This problem arises with multithreaded jobs, but is also an issue in other
> > places.  CPU activity from async memory reclaim (kswapd, cswapd?[5]) should be
> > accounted to the cgroup that the memory belongs to, and similarly CPU activity
> > from net rx should be accounted to the task groups that correspond to the
> > packets being received.  There are also vague complaints from Android[6].
> 
> These are pretty big holes in CPU cycle accounting right now and I think
> spend-first-and-backcharge is the right solution for most of them given
> experiences from other controllers. That said,
> 
> > Each use case has its own requirements[7].  In padata and reclaim, the task
> > group to account to is known ahead of time, but net rx has to spend cycles
> > processing a packet before its destination task group is known, so any solution
> > should be able to work without knowing the task group in advance.  Furthermore,
> > the CPU controller shouldn't throttle reclaim or net rx in real time since both
> > are doing high priority work.  These make approaches that run kthreads directly
> > in a task group, like cgroup-aware workqueues[8] or a kernel path for
> > CLONE_INTO_CGROUP, infeasible.  Running kthreads directly in cgroups also has a
> > downside for padata because helpers' MAX_NICE priority is "shadowed" by the
> > priority of the group entities they're running under.
> > 
> > The proposed solution of remote charging can accrue debt to a task group to be
> > paid off or forgiven later, addressing all these issues.  A kthread calls the
> > interface
> > 
> >     void cpu_cgroup_remote_begin(struct task_struct *p,
> >                                  struct cgroup_subsys_state *css);
> > 
> > to begin remote charging to @css, causing @p's current sum_exec_runtime to be
> > updated and saved.  The @css arg isn't required and can be removed later to
> > facilitate the unknown cgroup case mentioned above.  Then the kthread calls
> > another interface
> > 
> >     void cpu_cgroup_remote_charge(struct task_struct *p,
> >                                   struct cgroup_subsys_state *css);
> > 
> > to account the sum_exec_runtime that @p has used since the first call.
> > Internally, a new field cfs_bandwidth::debt is added to keep track of unpaid
> > debt that's only used when the debt exceeds the quota in the current period.
> > 
> > Weight-based control isn't implemented for now since padata helpers run at
> > MAX_NICE and so always yield to anything higher priority, meaning they would
> > rarely compete with other task groups.
> 
> If we're gonna do this, let's please do it right and make weight based
> control work too. Otherwise, its usefulness is pretty limited.

Ok, understood.

Doing it as presented is an incremental step and all that's required for
this.  I figured weight could be added later with the first user that
actually needs it.

I did prototype weight too, though, just to see if it was all gonna work
together, so given how the discussion elsewhere in the thread is going,
I might respin the scheduler part of this with another use case and
weight-based control included.

I got this far, do the interface and CFS skeleton seem sane?  Both are
basically unchanged with weight-based control included, the weight parts
are just more code on top.

Thanks for looking.

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-13 21:08         ` Daniel Jordan
@ 2022-01-13 21:11           ` Daniel Jordan
  0 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-13 21:11 UTC (permalink / raw)
  To: Tejun Heo
  Cc: Peter Zijlstra, Alexander Duyck, Alex Williamson, Andrew Morton,
	Ben Segall, Cornelia Huck, Dan Williams, Dave Hansen,
	Dietmar Eggemann, Herbert Xu, Ingo Molnar, Jason Gunthorpe,
	Johannes Weiner, Josh Triplett, Michal Hocko, Nico Pache,
	Pasha Tatashin, Steffen Klassert, Steve Sistare, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Thu, Jan 13, 2022 at 04:08:57PM -0500, Daniel Jordan wrote:
> On Wed, Jan 12, 2022 at 10:18:16AM -1000, Tejun Heo wrote:
> > If we're gonna do this, let's please do it right and make weight based
> > control work too. Otherwise, its usefulness is pretty limited.
> 
> Ok, understood.
> 
> Doing it as presented is an incremental step and all that's required for
> this.  I figured weight could be added later with the first user that
> actually needs it.
> 
> I did prototype weight too, though, just to see if it was all gonna work
> together, so given how the discussion elsewhere in the thread is going,
> I might respin the scheduler part of this with another use case and
> weight-based control included.
> 
> I got this far, do the interface and CFS skeleton seem sane?  Both are

s/CFS/CFS bandwidth/

> basically unchanged with weight-based control included, the weight parts
> are just more code on top.
> 
> Thanks for looking.

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
                     ` (2 preceding siblings ...)
  2022-01-11 11:58   ` Peter Zijlstra
@ 2022-01-14  9:31   ` Peter Zijlstra
  2022-01-14  9:40     ` Peter Zijlstra
                       ` (2 more replies)
  3 siblings, 3 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-14  9:31 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Wed, Jan 05, 2022 at 07:46:55PM -0500, Daniel Jordan wrote:
> As before, helpers in multithreaded jobs don't honor the main thread's
> CFS bandwidth limits, which could lead to the group exceeding its quota.
> 
> Fix it by having helpers remote charge their CPU time to the main
> thread's task group.  A helper calls a pair of new interfaces
> cpu_cgroup_remote_begin() and cpu_cgroup_remote_charge() (see function
> header comments) to achieve this.
> 
> This is just supposed to start a discussion, so it's pretty simple.
> Once a kthread has finished a remote charging period with
> cpu_cgroup_remote_charge(), its runtime is subtracted from the target
> task group's runtime (cfs_bandwidth::runtime) and any remainder is saved
> as debt (cfs_bandwidth::debt) to pay off in later periods.
> 
> Remote charging tasks aren't throttled when the group reaches its quota,
> and a task group doesn't run at all until its debt is completely paid,
> but these shortcomings can be addressed if the approach ends up being
> taken.

> diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
> index 44c452072a1b..3c2d7f245c68 100644
> --- a/kernel/sched/fair.c
> +++ b/kernel/sched/fair.c
> @@ -4655,10 +4655,19 @@ static inline u64 sched_cfs_bandwidth_slice(void)
>   */
>  void __refill_cfs_bandwidth_runtime(struct cfs_bandwidth *cfs_b)
>  {
> -	if (unlikely(cfs_b->quota == RUNTIME_INF))
> +	u64 quota = cfs_b->quota;
> +	u64 payment;
> +
> +	if (unlikely(quota == RUNTIME_INF))
>  		return;
>  
> -	cfs_b->runtime += cfs_b->quota;
> +	if (cfs_b->debt) {
> +		payment = min(quota, cfs_b->debt);
> +		cfs_b->debt -= payment;
> +		quota -= payment;
> +	}
> +
> +	cfs_b->runtime += quota;
>  	cfs_b->runtime = min(cfs_b->runtime, cfs_b->quota + cfs_b->burst);
>  }

It might be easier to make cfs_bandwidth::runtime an s64 and make it go
negative.

> @@ -5406,6 +5415,32 @@ static void __maybe_unused unthrottle_offline_cfs_rqs(struct rq *rq)
>  	rcu_read_unlock();
>  }
>  
> +static void incur_cfs_debt(struct rq *rq, struct sched_entity *se,
> +			   struct task_group *tg, u64 debt)
> +{
> +	if (!cfs_bandwidth_used())
> +		return;
> +
> +	while (tg != &root_task_group) {
> +		struct cfs_rq *cfs_rq = tg->cfs_rq[cpu_of(rq)];
> +
> +		if (cfs_rq->runtime_enabled) {
> +			struct cfs_bandwidth *cfs_b = &tg->cfs_bandwidth;
> +			u64 payment;
> +
> +			raw_spin_lock(&cfs_b->lock);
> +
> +			payment = min(cfs_b->runtime, debt);
> +			cfs_b->runtime -= payment;

At this point it might hit 0 (or go negative if/when you do the above)
and you'll need to throttle the group.

> +			cfs_b->debt += debt - payment;
> +
> +			raw_spin_unlock(&cfs_b->lock);
> +		}
> +
> +		tg = tg->parent;
> +	}
> +}

So part of the problem I have with this is that these external things
can consume all the bandwidth and basically indefinitely starve the
group.

This is doulby so if you're going to account things like softirq network
processing.

Also, why does the whole charging API have a task argument? It either is
current or NULL in case of things like softirq, neither really make
sense as an argument.

Also, by virtue of this being a start-stop annotation interface, the
accrued time might be arbitrarily large and arbitrarily delayed. I'm not
sure that's sensible.

For tasks it might be better to mark the task and have the tick DTRT
instead of later trying to 'migrate' the time.

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-14  9:31   ` Peter Zijlstra
@ 2022-01-14  9:40     ` Peter Zijlstra
  2022-01-14 16:38       ` Tejun Heo
  2022-01-18 17:40       ` Daniel Jordan
  2022-01-14 16:30     ` Tejun Heo
  2022-01-18 17:32     ` Daniel Jordan
  2 siblings, 2 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-14  9:40 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Fri, Jan 14, 2022 at 10:31:55AM +0100, Peter Zijlstra wrote:

> Also, by virtue of this being a start-stop annotation interface, the
> accrued time might be arbitrarily large and arbitrarily delayed. I'm not
> sure that's sensible.
> 
> For tasks it might be better to mark the task and have the tick DTRT
> instead of later trying to 'migrate' the time.

Which is then very close to simply sticking the task into the right
cgroup for a limited duration.

You could do a special case sched_move_task(), that takes a css argument
instead of using the current task_css. Then for cgroups it looks like
nothing changes, but the scheduler will DTRT and act like it is in the
target cgroup. Then at the end, simply move it back to task_css.

This obviously doesn't work for SoftIRQ accounting, but that is
'special' anyway. Softirq stuff is not otherwise under scheduler
control and has preemption disabled.

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-14  9:31   ` Peter Zijlstra
  2022-01-14  9:40     ` Peter Zijlstra
@ 2022-01-14 16:30     ` Tejun Heo
  2022-01-18 17:32     ` Daniel Jordan
  2 siblings, 0 replies; 47+ messages in thread
From: Tejun Heo @ 2022-01-14 16:30 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Daniel Jordan, Alexander Duyck, Alex Williamson, Andrew Morton,
	Ben Segall, Cornelia Huck, Dan Williams, Dave Hansen,
	Dietmar Eggemann, Herbert Xu, Ingo Molnar, Jason Gunthorpe,
	Johannes Weiner, Josh Triplett, Michal Hocko, Nico Pache,
	Pasha Tatashin, Steffen Klassert, Steve Sistare, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

Hello,

On Fri, Jan 14, 2022 at 10:31:55AM +0100, Peter Zijlstra wrote:
> So part of the problem I have with this is that these external things
> can consume all the bandwidth and basically indefinitely starve the
> group.
> 
> This is doulby so if you're going to account things like softirq network
> processing.

So, anything which is accounted this way should slow down / stall the
originator so that backcharges can't run away. In many cases, these
connections are already there - e.g. if you keep charging socket rx buffers
to a cgroup, the processes in the cgroup will slow down and the backpressure
on the network socket will slow down the incoming packets. Sometimes, the
backpressure propagation needs to be added explicitly - e.g. filesystem
metadata writes can run away because something can keeping on issuing
metadata updates without getting throttled neither on memory or io side. So,
for situations like that, we need to add an explicit mechanism to throttle
the originator asynchronously.

Thanks.

-- 
tejun

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-14  9:40     ` Peter Zijlstra
@ 2022-01-14 16:38       ` Tejun Heo
  2022-01-18 17:40       ` Daniel Jordan
  1 sibling, 0 replies; 47+ messages in thread
From: Tejun Heo @ 2022-01-14 16:38 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Daniel Jordan, Alexander Duyck, Alex Williamson, Andrew Morton,
	Ben Segall, Cornelia Huck, Dan Williams, Dave Hansen,
	Dietmar Eggemann, Herbert Xu, Ingo Molnar, Jason Gunthorpe,
	Johannes Weiner, Josh Triplett, Michal Hocko, Nico Pache,
	Pasha Tatashin, Steffen Klassert, Steve Sistare, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

Hello,

On Fri, Jan 14, 2022 at 10:40:06AM +0100, Peter Zijlstra wrote:
> You could do a special case sched_move_task(), that takes a css argument
> instead of using the current task_css. Then for cgroups it looks like
> nothing changes, but the scheduler will DTRT and act like it is in the
> target cgroup. Then at the end, simply move it back to task_css.
> 
> This obviously doesn't work for SoftIRQ accounting, but that is
> 'special' anyway. Softirq stuff is not otherwise under scheduler
> control and has preemption disabled.

So, if this particular use case doesn't fit the backcharge model (I'm not
sure yet). I'd much prefer it to maintain dynamic per-cgroup helper threads
than move tasks around dynamically. Nothing else is using migration this way
and we don't even need migration for seeding cgroups w/ CLONE_INTO_CGROUP.
In the future, this should allow further optimizations and likely
simplifications. It'd suck to have an odd exception usage.

Thanks.

-- 
tejun

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-14  9:31   ` Peter Zijlstra
  2022-01-14  9:40     ` Peter Zijlstra
  2022-01-14 16:30     ` Tejun Heo
@ 2022-01-18 17:32     ` Daniel Jordan
  2 siblings, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-18 17:32 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Fri, Jan 14, 2022 at 10:31:55AM +0100, Peter Zijlstra wrote:
> On Wed, Jan 05, 2022 at 07:46:55PM -0500, Daniel Jordan wrote:
> > diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
> > index 44c452072a1b..3c2d7f245c68 100644
> > --- a/kernel/sched/fair.c
> > +++ b/kernel/sched/fair.c
> > @@ -4655,10 +4655,19 @@ static inline u64 sched_cfs_bandwidth_slice(void)
> >   */
> >  void __refill_cfs_bandwidth_runtime(struct cfs_bandwidth *cfs_b)
> >  {
> > -	if (unlikely(cfs_b->quota == RUNTIME_INF))
> > +	u64 quota = cfs_b->quota;
> > +	u64 payment;
> > +
> > +	if (unlikely(quota == RUNTIME_INF))
> >  		return;
> >  
> > -	cfs_b->runtime += cfs_b->quota;
> > +	if (cfs_b->debt) {
> > +		payment = min(quota, cfs_b->debt);
> > +		cfs_b->debt -= payment;
> > +		quota -= payment;
> > +	}
> > +
> > +	cfs_b->runtime += quota;
> >  	cfs_b->runtime = min(cfs_b->runtime, cfs_b->quota + cfs_b->burst);
> >  }
> 
> It might be easier to make cfs_bandwidth::runtime an s64 and make it go
> negative.

Yep, nice, no need for a new field in cfs_bandwidth.

> > @@ -5406,6 +5415,32 @@ static void __maybe_unused unthrottle_offline_cfs_rqs(struct rq *rq)
> >  	rcu_read_unlock();
> >  }
> >  
> > +static void incur_cfs_debt(struct rq *rq, struct sched_entity *se,
> > +			   struct task_group *tg, u64 debt)
> > +{
> > +	if (!cfs_bandwidth_used())
> > +		return;
> > +
> > +	while (tg != &root_task_group) {
> > +		struct cfs_rq *cfs_rq = tg->cfs_rq[cpu_of(rq)];
> > +
> > +		if (cfs_rq->runtime_enabled) {
> > +			struct cfs_bandwidth *cfs_b = &tg->cfs_bandwidth;
> > +			u64 payment;
> > +
> > +			raw_spin_lock(&cfs_b->lock);
> > +
> > +			payment = min(cfs_b->runtime, debt);
> > +			cfs_b->runtime -= payment;
> 
> At this point it might hit 0 (or go negative if/when you do the above)
> and you'll need to throttle the group.

I might not be following you, but there could be cfs_rq's with local
runtime_remaining, so even if it goes 0 or negative, the group might
still have quota left and so shouldn't be throttled right away.

I was thinking the throttling would happen as normal, when a cfs_rq ran
out of runtime_remaining and failed to refill it from
cfs_bandwidth::runtime.

> > +			cfs_b->debt += debt - payment;
> > +
> > +			raw_spin_unlock(&cfs_b->lock);
> > +		}
> > +
> > +		tg = tg->parent;
> > +	}
> > +}
> 
> So part of the problem I have with this is that these external things
> can consume all the bandwidth and basically indefinitely starve the
> group.
> 
> This is doulby so if you're going to account things like softirq network
> processing.

Yes.  As Tejun points out, I'll make sure remote charging doesn't run
away.

> Also, why does the whole charging API have a task argument? It either is
> current or NULL in case of things like softirq, neither really make
> sense as an argument.

@task distinguishes between NULL for softirq and current for everybody
else.

It's possible to detect this difference internally though, if that's
what you're saying, so @task can go away.

> Also, by virtue of this being a start-stop annotation interface, the
> accrued time might be arbitrarily large and arbitrarily delayed. I'm not
> sure that's sensible.

Yes, that is a risk.  With start-stop, users need to be careful to
account often enough and have a "reasonable" upper bound on period
length, however that's defined.  Multithreaded jobs are probably the
worst offender since these threads charge a sizable amount at once
compared to the other use cases.

> For tasks it might be better to mark the task and have the tick DTRT
> instead of later trying to 'migrate' the time.

Ok, I'll try that.  The start-stop approach keeps remote charging from
adding overhead in the tick for non-remote-charging things, far and away
the common case, but I'll see how expensive the tick-based approach is.

Can hide it behind a static branch for systems not using the cpu
contoller.

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

* Re: [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth
  2022-01-14  9:40     ` Peter Zijlstra
  2022-01-14 16:38       ` Tejun Heo
@ 2022-01-18 17:40       ` Daniel Jordan
  1 sibling, 0 replies; 47+ messages in thread
From: Daniel Jordan @ 2022-01-18 17:40 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Alexander Duyck, Alex Williamson, Andrew Morton, Ben Segall,
	Cornelia Huck, Dan Williams, Dave Hansen, Dietmar Eggemann,
	Herbert Xu, Ingo Molnar, Jason Gunthorpe, Johannes Weiner,
	Josh Triplett, Michal Hocko, Nico Pache, Pasha Tatashin,
	Steffen Klassert, Steve Sistare, Tejun Heo, Tim Chen,
	Vincent Guittot, linux-mm, kvm, linux-kernel, linux-crypto

On Fri, Jan 14, 2022 at 10:40:06AM +0100, Peter Zijlstra wrote:
> On Fri, Jan 14, 2022 at 10:31:55AM +0100, Peter Zijlstra wrote:
> 
> > Also, by virtue of this being a start-stop annotation interface, the
> > accrued time might be arbitrarily large and arbitrarily delayed. I'm not
> > sure that's sensible.
> > 
> > For tasks it might be better to mark the task and have the tick DTRT
> > instead of later trying to 'migrate' the time.
> 
> Which is then very close to simply sticking the task into the right
> cgroup for a limited duration.
> 
> You could do a special case sched_move_task(), that takes a css argument
> instead of using the current task_css. Then for cgroups it looks like
> nothing changes, but the scheduler will DTRT and act like it is in the
> target cgroup. Then at the end, simply move it back to task_css.

Yes, that's one of the things I tried.  Less new code in the scheduler
this way.

> This obviously doesn't work for SoftIRQ accounting, but that is
> 'special' anyway. Softirq stuff is not otherwise under scheduler
> control and has preemption disabled.

It also doesn't work for memory reclaim since that shouldn't be
throttled in real time.  Reclaim and softirq seem to demand something
that doesn't move tasks onto runqueues, that's external to avoid getting
pushed off cpu.

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

end of thread, other threads:[~2022-01-18 17:40 UTC | newest]

Thread overview: 47+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-01-06  0:46 [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Daniel Jordan
2022-01-06  0:46 ` [RFC 01/16] padata: Remove __init from multithreading functions Daniel Jordan
2022-01-06  0:46 ` [RFC 02/16] padata: Return first error from a job Daniel Jordan
2022-01-06  0:46 ` [RFC 03/16] padata: Add undo support Daniel Jordan
2022-01-06  0:46 ` [RFC 04/16] padata: Detect deadlocks between main and helper threads Daniel Jordan
2022-01-06  0:46 ` [RFC 05/16] vfio/type1: Pass mm to vfio_pin_pages_remote() Daniel Jordan
2022-01-06  0:46 ` [RFC 06/16] vfio/type1: Refactor dma map removal Daniel Jordan
2022-01-06  0:46 ` [RFC 07/16] vfio/type1: Parallelize vfio_pin_map_dma() Daniel Jordan
2022-01-06  0:46 ` [RFC 08/16] vfio/type1: Cache locked_vm to ease mmap_lock contention Daniel Jordan
2022-01-06  0:53   ` Jason Gunthorpe
2022-01-06  1:17     ` Daniel Jordan
2022-01-06 12:34       ` Jason Gunthorpe
2022-01-06 21:05         ` Alex Williamson
2022-01-07  0:19           ` Jason Gunthorpe
2022-01-07  3:06             ` Daniel Jordan
2022-01-07 15:18               ` Jason Gunthorpe
2022-01-07 16:39                 ` Daniel Jordan
2022-01-06  0:46 ` [RFC 09/16] padata: Use kthreads in do_multithreaded Daniel Jordan
2022-01-06  0:46 ` [RFC 10/16] padata: Helpers should respect main thread's CPU affinity Daniel Jordan
2022-01-06  0:46 ` [RFC 11/16] padata: Cap helpers started to online CPUs Daniel Jordan
2022-01-06  0:46 ` [RFC 12/16] sched, padata: Bound max threads with max_cfs_bandwidth_cpus() Daniel Jordan
2022-01-06  6:06   ` kernel test robot
2022-01-06 11:31   ` kernel test robot
2022-01-06  0:46 ` [RFC 13/16] padata: Run helper threads at MAX_NICE Daniel Jordan
2022-01-06  0:46 ` [RFC 14/16] padata: Nice helper threads one by one to prevent starvation Daniel Jordan
2022-01-06  0:46 ` [RFC 15/16] sched/fair: Account kthread runtime debt for CFS bandwidth Daniel Jordan
2022-01-06  6:47   ` kernel test robot
2022-01-06  6:47   ` kernel test robot
2022-01-06  6:47     ` kernel test robot
2022-01-11 11:58   ` Peter Zijlstra
2022-01-11 16:29     ` Daniel Jordan
2022-01-12 20:18       ` Tejun Heo
2022-01-13 21:08         ` Daniel Jordan
2022-01-13 21:11           ` Daniel Jordan
2022-01-14  9:31   ` Peter Zijlstra
2022-01-14  9:40     ` Peter Zijlstra
2022-01-14 16:38       ` Tejun Heo
2022-01-18 17:40       ` Daniel Jordan
2022-01-14 16:30     ` Tejun Heo
2022-01-18 17:32     ` Daniel Jordan
2022-01-06  0:46 ` [RFC 16/16] sched/fair: Consider kthread debt in cputime Daniel Jordan
2022-01-06  1:13 ` [RFC 00/16] padata, vfio, sched: Multithreaded VFIO page pinning Jason Gunthorpe
2022-01-07  3:03   ` Daniel Jordan
2022-01-07 17:12     ` Jason Gunthorpe
2022-01-10 22:27       ` Daniel Jordan
2022-01-11  0:17         ` Jason Gunthorpe
2022-01-11 16:20           ` Daniel Jordan

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.