linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v3 0/8] padata: parallelize deferred page init
@ 2020-05-27 17:36 Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 1/8] padata: remove exit routine Daniel Jordan
                   ` (7 more replies)
  0 siblings, 8 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

Thanks to Alex for his continued review and Josh for running v2!  Please
continue to review and test, and acks for the padata parts would be
appreciated.

Daniel

--

Deferred struct page init is a bottleneck in kernel boot--the biggest
for us and probably others.  Optimizing it maximizes availability for
large-memory systems and allows spinning up short-lived VMs as needed
without having to leave them running.  It also benefits bare metal
machines hosting VMs that are sensitive to downtime.  In projects such
as VMM Fast Restart[1], where guest state is preserved across kexec
reboot, it helps prevent application and network timeouts in the guests.

So, multithread deferred init to take full advantage of system memory
bandwidth.

Extend padata, a framework that handles many parallel singlethreaded
jobs, to handle multithreaded jobs as well by adding support for
splitting up the work evenly, specifying a minimum amount of work that's
appropriate for one helper thread to do, load balancing between helpers,
and coordinating them.  More documentation in patches 4 and 8.

This series is the first step in a project to address other memory
proportional bottlenecks in the kernel such as pmem struct page init,
vfio page pinning, hugetlb fallocate, and munmap.  Deferred page init
doesn't require concurrency limits, resource control, or priority
adjustments like these other users will because it happens during boot
when the system is otherwise idle and waiting for page init to finish.

This has been run on a variety of x86 systems and speeds up kernel boot
by 4% to 49%, saving up to 1.6 out of 4 seconds.  Patch 6 has more
numbers.

The powerpc and s390 lists are included in case they want to give this a
try, they had enabled this feature when it was configured per arch.

Series based on v5.7-rc7 plus these three from mmotm

  mm-call-touch_nmi_watchdog-on-max-order-boundaries-in-deferred-init.patch
  mm-initialize-deferred-pages-with-interrupts-enabled.patch
  mm-call-cond_resched-from-deferred_init_memmap.patch

and it's available here:

  git://oss.oracle.com/git/linux-dmjordan.git padata-mt-definit-v3
  https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=shortlog;h=refs/heads/padata-mt-definit-v3

and the future users and related features are available as
work-in-progress:

  git://oss.oracle.com/git/linux-dmjordan.git padata-mt-wip-v0.5
  https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=shortlog;h=refs/heads/padata-mt-wip-v0.5

v3:
 - Remove nr_pages accounting as suggested by Alex, adding a new patch
 - Align deferred init ranges up not down, simplify surrounding code (Alex)
 - Add Josh's T-b's from v2 (Josh's T-b's for v1 lost in rebase, apologies!)
 - Move padata.h include up in init/main.c to reduce patch collisions (Andrew)
 - Slightly reword Documentation patch
 - Rebase on v5.7-rc7 and retest

v2:
 - Improve the problem statement (Andrew, Josh, Pavel)
 - Add T-b's to unchanged patches (Josh)
 - Fully initialize max-order blocks to avoid buddy issues (Alex)
 - Parallelize on section-aligned boundaries to avoid potential
   false sharing (Alex)
 - Return the maximum thread count from a function that architectures
   can override, with the generic version returning 1 (current
   behavior).  Override for x86 since that's the only arch this series
   has been tested on so far.  Other archs can test with more threads
   by dropping patch 6.
 - Rebase to v5.7-rc6, rerun tests

RFC v4 [2] -> v1:
 - merged with padata (Peter)
 - got rid of the 'task' nomenclature (Peter, Jon)

future work branch:
 - made lockdep-aware (Jason, Peter)
 - adjust workqueue worker priority with renice_or_cancel() (Tejun)
 - fixed undo problem in VFIO (Alex)

The remaining feedback, mainly resource control awareness (cgroup etc),
is TODO for later series.

[1] https://static.sched.com/hosted_files/kvmforum2019/66/VMM-fast-restart_kvmforum2019.pdf
    https://www.youtube.com/watch?v=pBsHnf93tcQ
    https://lore.kernel.org/linux-mm/1588812129-8596-1-git-send-email-anthony.yznaga@oracle.com/

[2] https://lore.kernel.org/linux-mm/20181105165558.11698-1-daniel.m.jordan@oracle.com/

Daniel Jordan (8):
  padata: remove exit routine
  padata: initialize earlier
  padata: allocate work structures for parallel jobs from a pool
  padata: add basic support for multithreaded jobs
  mm: don't track number of pages during deferred initialization
  mm: parallelize deferred_init_memmap()
  mm: make deferred init's max threads arch-specific
  padata: document multithreaded jobs

 Documentation/core-api/padata.rst |  41 +++--
 arch/x86/mm/init_64.c             |  12 ++
 include/linux/memblock.h          |   3 +
 include/linux/padata.h            |  43 ++++-
 init/main.c                       |   2 +
 kernel/padata.c                   | 277 ++++++++++++++++++++++++------
 mm/Kconfig                        |   6 +-
 mm/page_alloc.c                   |  59 +++++--
 8 files changed, 361 insertions(+), 82 deletions(-)


base-commit: 9cb1fd0efd195590b828b9b865421ad345a4a145
prerequisite-patch-id: 4ad522141e1119a325a9799dad2bd982fbac8b7c
prerequisite-patch-id: 169273327e56f5461101a71dfbd6b4cfd4570cf0
prerequisite-patch-id: 0f34692c8a9673d4c4f6a3545cf8ec3a2abf8620
-- 
2.26.2



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

* [PATCH v3 1/8] padata: remove exit routine
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 2/8] padata: initialize earlier Daniel Jordan
                   ` (6 subsequent siblings)
  7 siblings, 0 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

padata_driver_exit() is unnecessary because padata isn't built as a
module and doesn't exit.

padata's init routine will soon allocate memory, so getting rid of the
exit function now avoids pointless code to free it.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 kernel/padata.c | 6 ------
 1 file changed, 6 deletions(-)

diff --git a/kernel/padata.c b/kernel/padata.c
index a6afa12fb75ee..835919c745266 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -1072,10 +1072,4 @@ static __init int padata_driver_init(void)
 }
 module_init(padata_driver_init);
 
-static __exit void padata_driver_exit(void)
-{
-	cpuhp_remove_multi_state(CPUHP_PADATA_DEAD);
-	cpuhp_remove_multi_state(hp_online);
-}
-module_exit(padata_driver_exit);
 #endif
-- 
2.26.2



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

* [PATCH v3 2/8] padata: initialize earlier
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 1/8] padata: remove exit routine Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 3/8] padata: allocate work structures for parallel jobs from a pool Daniel Jordan
                   ` (5 subsequent siblings)
  7 siblings, 0 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

padata will soon initialize the system's struct pages in parallel, so it
needs to be ready by page_alloc_init_late().

The error return from padata_driver_init() triggers an initcall warning,
so add a warning to padata_init() to avoid silent failure.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 include/linux/padata.h |  6 ++++++
 init/main.c            |  2 ++
 kernel/padata.c        | 17 ++++++++---------
 3 files changed, 16 insertions(+), 9 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index a0d8b41850b25..476ecfa41f363 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -164,6 +164,12 @@ struct padata_instance {
 #define	PADATA_INVALID	4
 };
 
+#ifdef CONFIG_PADATA
+extern void __init padata_init(void);
+#else
+static inline void __init padata_init(void) {}
+#endif
+
 extern struct padata_instance *padata_alloc_possible(const char *name);
 extern void padata_free(struct padata_instance *pinst);
 extern struct padata_shell *padata_alloc_shell(struct padata_instance *pinst);
diff --git a/init/main.c b/init/main.c
index 03371976d3872..df32f67214d23 100644
--- a/init/main.c
+++ b/init/main.c
@@ -63,6 +63,7 @@
 #include <linux/debugobjects.h>
 #include <linux/lockdep.h>
 #include <linux/kmemleak.h>
+#include <linux/padata.h>
 #include <linux/pid_namespace.h>
 #include <linux/device/driver.h>
 #include <linux/kthread.h>
@@ -1482,6 +1483,7 @@ static noinline void __init kernel_init_freeable(void)
 	smp_init();
 	sched_init_smp();
 
+	padata_init();
 	page_alloc_init_late();
 	/* Initialize page ext after all struct pages are initialized. */
 	page_ext_init();
diff --git a/kernel/padata.c b/kernel/padata.c
index 835919c745266..6f709bc0fc413 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -31,7 +31,6 @@
 #include <linux/slab.h>
 #include <linux/sysfs.h>
 #include <linux/rcupdate.h>
-#include <linux/module.h>
 
 #define MAX_OBJ_NUM 1000
 
@@ -1050,26 +1049,26 @@ void padata_free_shell(struct padata_shell *ps)
 }
 EXPORT_SYMBOL(padata_free_shell);
 
-#ifdef CONFIG_HOTPLUG_CPU
-
-static __init int padata_driver_init(void)
+void __init padata_init(void)
 {
+#ifdef CONFIG_HOTPLUG_CPU
 	int ret;
 
 	ret = cpuhp_setup_state_multi(CPUHP_AP_ONLINE_DYN, "padata:online",
 				      padata_cpu_online, NULL);
 	if (ret < 0)
-		return ret;
+		goto err;
 	hp_online = ret;
 
 	ret = cpuhp_setup_state_multi(CPUHP_PADATA_DEAD, "padata:dead",
 				      NULL, padata_cpu_dead);
 	if (ret < 0) {
 		cpuhp_remove_multi_state(hp_online);
-		return ret;
+		goto err;
 	}
-	return 0;
-}
-module_init(padata_driver_init);
 
+	return;
+err:
+	pr_warn("padata: initialization failed\n");
 #endif
+}
-- 
2.26.2



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

* [PATCH v3 3/8] padata: allocate work structures for parallel jobs from a pool
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 1/8] padata: remove exit routine Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 2/8] padata: initialize earlier Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 4/8] padata: add basic support for multithreaded jobs Daniel Jordan
                   ` (4 subsequent siblings)
  7 siblings, 0 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

padata allocates per-CPU, per-instance work structs for parallel jobs.
A do_parallel call assigns a job to a sequence number and hashes the
number to a CPU, where the job will eventually run using the
corresponding work.

This approach fit with how padata used to bind a job to each CPU
round-robin, makes less sense after commit bfde23ce200e6 ("padata:
unbind parallel jobs from specific CPUs") because a work isn't bound to
a particular CPU anymore, and isn't needed at all for multithreaded jobs
because they don't have sequence numbers.

Replace the per-CPU works with a preallocated pool, which allows sharing
them between existing padata users and the upcoming multithreaded user.
The pool will also facilitate setting NUMA-aware concurrency limits with
later users.

The pool is sized according to the number of possible CPUs.  With this
limit, MAX_OBJ_NUM no longer makes sense, so remove it.

If the global pool is exhausted, a parallel job is run in the current
task instead to throttle a system trying to do too much in parallel.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 include/linux/padata.h |   8 +--
 kernel/padata.c        | 118 +++++++++++++++++++++++++++--------------
 2 files changed, 78 insertions(+), 48 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index 476ecfa41f363..3bfa503503ac5 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -24,7 +24,6 @@
  * @list: List entry, to attach to the padata lists.
  * @pd: Pointer to the internal control structure.
  * @cb_cpu: Callback cpu for serializatioon.
- * @cpu: Cpu for parallelization.
  * @seq_nr: Sequence number of the parallelized data object.
  * @info: Used to pass information from the parallel to the serial function.
  * @parallel: Parallel execution function.
@@ -34,7 +33,6 @@ struct padata_priv {
 	struct list_head	list;
 	struct parallel_data	*pd;
 	int			cb_cpu;
-	int			cpu;
 	unsigned int		seq_nr;
 	int			info;
 	void                    (*parallel)(struct padata_priv *padata);
@@ -68,15 +66,11 @@ struct padata_serial_queue {
 /**
  * struct padata_parallel_queue - The percpu padata parallel queue
  *
- * @parallel: List to wait for parallelization.
  * @reorder: List to wait for reordering after parallel processing.
- * @work: work struct for parallelization.
  * @num_obj: Number of objects that are processed by this cpu.
  */
 struct padata_parallel_queue {
-       struct padata_list    parallel;
        struct padata_list    reorder;
-       struct work_struct    work;
        atomic_t              num_obj;
 };
 
@@ -111,7 +105,7 @@ struct parallel_data {
 	struct padata_parallel_queue	__percpu *pqueue;
 	struct padata_serial_queue	__percpu *squeue;
 	atomic_t			refcnt;
-	atomic_t			seq_nr;
+	unsigned int			seq_nr;
 	unsigned int			processed;
 	int				cpu;
 	struct padata_cpumask		cpumask;
diff --git a/kernel/padata.c b/kernel/padata.c
index 6f709bc0fc413..78ff9aa529204 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -32,7 +32,15 @@
 #include <linux/sysfs.h>
 #include <linux/rcupdate.h>
 
-#define MAX_OBJ_NUM 1000
+struct padata_work {
+	struct work_struct	pw_work;
+	struct list_head	pw_list;  /* padata_free_works linkage */
+	void			*pw_data;
+};
+
+static DEFINE_SPINLOCK(padata_works_lock);
+static struct padata_work *padata_works;
+static LIST_HEAD(padata_free_works);
 
 static void padata_free_pd(struct parallel_data *pd);
 
@@ -58,30 +66,44 @@ static int padata_cpu_hash(struct parallel_data *pd, unsigned int seq_nr)
 	return padata_index_to_cpu(pd, cpu_index);
 }
 
-static void padata_parallel_worker(struct work_struct *parallel_work)
+static struct padata_work *padata_work_alloc(void)
 {
-	struct padata_parallel_queue *pqueue;
-	LIST_HEAD(local_list);
+	struct padata_work *pw;
 
-	local_bh_disable();
-	pqueue = container_of(parallel_work,
-			      struct padata_parallel_queue, work);
+	lockdep_assert_held(&padata_works_lock);
 
-	spin_lock(&pqueue->parallel.lock);
-	list_replace_init(&pqueue->parallel.list, &local_list);
-	spin_unlock(&pqueue->parallel.lock);
+	if (list_empty(&padata_free_works))
+		return NULL;	/* No more work items allowed to be queued. */
 
-	while (!list_empty(&local_list)) {
-		struct padata_priv *padata;
+	pw = list_first_entry(&padata_free_works, struct padata_work, pw_list);
+	list_del(&pw->pw_list);
+	return pw;
+}
 
-		padata = list_entry(local_list.next,
-				    struct padata_priv, list);
+static void padata_work_init(struct padata_work *pw, work_func_t work_fn,
+			     void *data)
+{
+	INIT_WORK(&pw->pw_work, work_fn);
+	pw->pw_data = data;
+}
 
-		list_del_init(&padata->list);
+static void padata_work_free(struct padata_work *pw)
+{
+	lockdep_assert_held(&padata_works_lock);
+	list_add(&pw->pw_list, &padata_free_works);
+}
 
-		padata->parallel(padata);
-	}
+static void padata_parallel_worker(struct work_struct *parallel_work)
+{
+	struct padata_work *pw = container_of(parallel_work, struct padata_work,
+					      pw_work);
+	struct padata_priv *padata = pw->pw_data;
 
+	local_bh_disable();
+	padata->parallel(padata);
+	spin_lock(&padata_works_lock);
+	padata_work_free(pw);
+	spin_unlock(&padata_works_lock);
 	local_bh_enable();
 }
 
@@ -105,9 +127,9 @@ int padata_do_parallel(struct padata_shell *ps,
 		       struct padata_priv *padata, int *cb_cpu)
 {
 	struct padata_instance *pinst = ps->pinst;
-	int i, cpu, cpu_index, target_cpu, err;
-	struct padata_parallel_queue *queue;
+	int i, cpu, cpu_index, err;
 	struct parallel_data *pd;
+	struct padata_work *pw;
 
 	rcu_read_lock_bh();
 
@@ -135,25 +157,25 @@ int padata_do_parallel(struct padata_shell *ps,
 	if ((pinst->flags & PADATA_RESET))
 		goto out;
 
-	if (atomic_read(&pd->refcnt) >= MAX_OBJ_NUM)
-		goto out;
-
-	err = 0;
 	atomic_inc(&pd->refcnt);
 	padata->pd = pd;
 	padata->cb_cpu = *cb_cpu;
 
-	padata->seq_nr = atomic_inc_return(&pd->seq_nr);
-	target_cpu = padata_cpu_hash(pd, padata->seq_nr);
-	padata->cpu = target_cpu;
-	queue = per_cpu_ptr(pd->pqueue, target_cpu);
-
-	spin_lock(&queue->parallel.lock);
-	list_add_tail(&padata->list, &queue->parallel.list);
-	spin_unlock(&queue->parallel.lock);
+	rcu_read_unlock_bh();
 
-	queue_work(pinst->parallel_wq, &queue->work);
+	spin_lock(&padata_works_lock);
+	padata->seq_nr = ++pd->seq_nr;
+	pw = padata_work_alloc();
+	spin_unlock(&padata_works_lock);
+	if (pw) {
+		padata_work_init(pw, padata_parallel_worker, padata);
+		queue_work(pinst->parallel_wq, &pw->pw_work);
+	} else {
+		/* Maximum works limit exceeded, run in the current task. */
+		padata->parallel(padata);
+	}
 
+	return 0;
 out:
 	rcu_read_unlock_bh();
 
@@ -324,8 +346,9 @@ static void padata_serial_worker(struct work_struct *serial_work)
 void padata_do_serial(struct padata_priv *padata)
 {
 	struct parallel_data *pd = padata->pd;
+	int hashed_cpu = padata_cpu_hash(pd, padata->seq_nr);
 	struct padata_parallel_queue *pqueue = per_cpu_ptr(pd->pqueue,
-							   padata->cpu);
+							   hashed_cpu);
 	struct padata_priv *cur;
 
 	spin_lock(&pqueue->reorder.lock);
@@ -416,8 +439,6 @@ static void padata_init_pqueues(struct parallel_data *pd)
 		pqueue = per_cpu_ptr(pd->pqueue, cpu);
 
 		__padata_list_init(&pqueue->reorder);
-		__padata_list_init(&pqueue->parallel);
-		INIT_WORK(&pqueue->work, padata_parallel_worker);
 		atomic_set(&pqueue->num_obj, 0);
 	}
 }
@@ -451,7 +472,7 @@ static struct parallel_data *padata_alloc_pd(struct padata_shell *ps)
 
 	padata_init_pqueues(pd);
 	padata_init_squeues(pd);
-	atomic_set(&pd->seq_nr, -1);
+	pd->seq_nr = -1;
 	atomic_set(&pd->refcnt, 1);
 	spin_lock_init(&pd->lock);
 	pd->cpu = cpumask_first(pd->cpumask.pcpu);
@@ -1051,6 +1072,7 @@ EXPORT_SYMBOL(padata_free_shell);
 
 void __init padata_init(void)
 {
+	unsigned int i, possible_cpus;
 #ifdef CONFIG_HOTPLUG_CPU
 	int ret;
 
@@ -1062,13 +1084,27 @@ void __init padata_init(void)
 
 	ret = cpuhp_setup_state_multi(CPUHP_PADATA_DEAD, "padata:dead",
 				      NULL, padata_cpu_dead);
-	if (ret < 0) {
-		cpuhp_remove_multi_state(hp_online);
-		goto err;
-	}
+	if (ret < 0)
+		goto remove_online_state;
+#endif
+
+	possible_cpus = num_possible_cpus();
+	padata_works = kmalloc_array(possible_cpus, sizeof(struct padata_work),
+				     GFP_KERNEL);
+	if (!padata_works)
+		goto remove_dead_state;
+
+	for (i = 0; i < possible_cpus; ++i)
+		list_add(&padata_works[i].pw_list, &padata_free_works);
 
 	return;
+
+remove_dead_state:
+#ifdef CONFIG_HOTPLUG_CPU
+	cpuhp_remove_multi_state(CPUHP_PADATA_DEAD);
+remove_online_state:
+	cpuhp_remove_multi_state(hp_online);
 err:
-	pr_warn("padata: initialization failed\n");
 #endif
+	pr_warn("padata: initialization failed\n");
 }
-- 
2.26.2



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

* [PATCH v3 4/8] padata: add basic support for multithreaded jobs
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
                   ` (2 preceding siblings ...)
  2020-05-27 17:36 ` [PATCH v3 3/8] padata: allocate work structures for parallel jobs from a pool Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 5/8] mm: don't track number of pages during deferred initialization Daniel Jordan
                   ` (3 subsequent siblings)
  7 siblings, 0 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

Sometimes the kernel doesn't take full advantage of system memory
bandwidth, leading to a single CPU spending excessive time in
initialization paths where the data scales with memory size.

Multithreading naturally addresses this problem.

Extend padata, a framework that handles many parallel yet singlethreaded
jobs, to also handle multithreaded jobs by adding support for splitting
up the work evenly, specifying a minimum amount of work that's
appropriate for one helper thread to do, load balancing between helpers,
and coordinating them.

This is inspired by work from Pavel Tatashin and Steve Sistare.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 include/linux/padata.h |  29 ++++++++
 kernel/padata.c        | 152 ++++++++++++++++++++++++++++++++++++++++-
 2 files changed, 178 insertions(+), 3 deletions(-)

diff --git a/include/linux/padata.h b/include/linux/padata.h
index 3bfa503503ac5..b0affa466a841 100644
--- a/include/linux/padata.h
+++ b/include/linux/padata.h
@@ -4,6 +4,9 @@
  *
  * Copyright (C) 2008, 2009 secunet Security Networks AG
  * Copyright (C) 2008, 2009 Steffen Klassert <steffen.klassert@secunet.com>
+ *
+ * Copyright (c) 2020 Oracle and/or its affiliates.
+ * Author: Daniel Jordan <daniel.m.jordan@oracle.com>
  */
 
 #ifndef PADATA_H
@@ -130,6 +133,31 @@ struct padata_shell {
 	struct list_head		list;
 };
 
+/**
+ * struct padata_mt_job - represents one multithreaded job
+ *
+ * @thread_fn: Called for each chunk of work that a padata thread does.
+ * @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).
+ * @align: Ranges passed to the thread function fall on this boundary, with the
+ *         possible exceptions of the beginning and end of the job.
+ * @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.
+ * @max_threads: Max threads to use for the job, actual number may be less
+ *               depending on task size and minimum chunk size.
+ */
+struct padata_mt_job {
+	void (*thread_fn)(unsigned long start, unsigned long end, void *arg);
+	void			*fn_arg;
+	unsigned long		start;
+	unsigned long		size;
+	unsigned long		align;
+	unsigned long		min_chunk;
+	int			max_threads;
+};
+
 /**
  * struct padata_instance - The overall control structure.
  *
@@ -171,6 +199,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 int padata_set_cpumask(struct padata_instance *pinst, int cpumask_type,
 			      cpumask_var_t cpumask);
 extern int padata_start(struct padata_instance *pinst);
diff --git a/kernel/padata.c b/kernel/padata.c
index 78ff9aa529204..e78f57d9aef90 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -7,6 +7,9 @@
  * Copyright (C) 2008, 2009 secunet Security Networks AG
  * Copyright (C) 2008, 2009 Steffen Klassert <steffen.klassert@secunet.com>
  *
+ * Copyright (c) 2020 Oracle and/or its affiliates.
+ * Author: Daniel Jordan <daniel.m.jordan@oracle.com>
+ *
  * This program is free software; you can redistribute it and/or modify it
  * under the terms and conditions of the GNU General Public License,
  * version 2, as published by the Free Software Foundation.
@@ -21,6 +24,7 @@
  * 51 Franklin St - Fifth Floor, Boston, MA 02110-1301 USA.
  */
 
+#include <linux/completion.h>
 #include <linux/export.h>
 #include <linux/cpumask.h>
 #include <linux/err.h>
@@ -32,6 +36,8 @@
 #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 */
@@ -42,7 +48,17 @@ static DEFINE_SPINLOCK(padata_works_lock);
 static struct padata_work *padata_works;
 static LIST_HEAD(padata_free_works);
 
+struct padata_mt_job_state {
+	spinlock_t		lock;
+	struct completion	completion;
+	struct padata_mt_job	*job;
+	int			nworks;
+	int			nworks_fini;
+	unsigned long		chunk_size;
+};
+
 static void padata_free_pd(struct parallel_data *pd);
+static void __init padata_mt_helper(struct work_struct *work);
 
 static int padata_index_to_cpu(struct parallel_data *pd, int cpu_index)
 {
@@ -81,18 +97,56 @@ static struct padata_work *padata_work_alloc(void)
 }
 
 static void padata_work_init(struct padata_work *pw, work_func_t work_fn,
-			     void *data)
+			     void *data, int flags)
 {
-	INIT_WORK(&pw->pw_work, work_fn);
+	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 __init padata_work_alloc_mt(int nworks, void *data,
+				       struct list_head *head)
+{
+	int i;
+
+	spin_lock(&padata_works_lock);
+	/* Start at 1 because the current task participates in the job. */
+	for (i = 1; i < nworks; ++i) {
+		struct padata_work *pw = padata_work_alloc();
+
+		if (!pw)
+			break;
+		padata_work_init(pw, padata_mt_helper, data, 0);
+		list_add(&pw->pw_list, head);
+	}
+	spin_unlock(&padata_works_lock);
+
+	return i;
+}
+
 static void padata_work_free(struct padata_work *pw)
 {
 	lockdep_assert_held(&padata_works_lock);
 	list_add(&pw->pw_list, &padata_free_works);
 }
 
+static void __init 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);
+	}
+	spin_unlock(&padata_works_lock);
+}
+
 static void padata_parallel_worker(struct work_struct *parallel_work)
 {
 	struct padata_work *pw = container_of(parallel_work, struct padata_work,
@@ -168,7 +222,7 @@ int padata_do_parallel(struct padata_shell *ps,
 	pw = padata_work_alloc();
 	spin_unlock(&padata_works_lock);
 	if (pw) {
-		padata_work_init(pw, padata_parallel_worker, padata);
+		padata_work_init(pw, padata_parallel_worker, padata, 0);
 		queue_work(pinst->parallel_wq, &pw->pw_work);
 	} else {
 		/* Maximum works limit exceeded, run in the current task. */
@@ -409,6 +463,98 @@ static int pd_setup_cpumasks(struct parallel_data *pd,
 	return err;
 }
 
+static void __init 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;
+	struct padata_mt_job *job = ps->job;
+	bool done;
+
+	spin_lock(&ps->lock);
+
+	while (job->size > 0) {
+		unsigned long start, size, end;
+
+		start = job->start;
+		/* 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;
+
+		job->start = end;
+		job->size -= size;
+
+		spin_unlock(&ps->lock);
+		job->thread_fn(start, end, job->fn_arg);
+		spin_lock(&ps->lock);
+	}
+
+	++ps->nworks_fini;
+	done = (ps->nworks_fini == ps->nworks);
+	spin_unlock(&ps->lock);
+
+	if (done)
+		complete(&ps->completion);
+}
+
+/**
+ * padata_do_multithreaded - run a multithreaded job
+ * @job: Description of the job.
+ *
+ * See the definition of struct padata_mt_job for more details.
+ */
+void __init padata_do_multithreaded(struct padata_mt_job *job)
+{
+	/* In case threads finish at different times. */
+	static const unsigned long load_balance_factor = 4;
+	struct padata_work my_work, *pw;
+	struct padata_mt_job_state ps;
+	LIST_HEAD(works);
+	int nworks;
+
+	if (job->size == 0)
+		return;
+
+	/* Ensure at least one thread when size < min_chunk. */
+	nworks = max(job->size / job->min_chunk, 1ul);
+	nworks = min(nworks, job->max_threads);
+
+	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;
+	}
+
+	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;
+
+	/*
+	 * Chunk size is the amount of work a helper does per call to the
+	 * thread function.  Load balance large jobs between threads by
+	 * 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 = max(ps.chunk_size, job->min_chunk);
+	ps.chunk_size = roundup(ps.chunk_size, job->align);
+
+	list_for_each_entry(pw, &works, pw_list)
+		queue_work(system_unbound_wq, &pw->pw_work);
+
+	/* Use the current thread, which saves starting a workqueue worker. */
+	padata_work_init(&my_work, padata_mt_helper, &ps, PADATA_WORK_ONSTACK);
+	padata_mt_helper(&my_work.pw_work);
+
+	/* Wait for all the helpers to finish. */
+	wait_for_completion(&ps.completion);
+
+	destroy_work_on_stack(&my_work.pw_work);
+	padata_works_free(&works);
+}
+
 static void __padata_list_init(struct padata_list *pd_list)
 {
 	INIT_LIST_HEAD(&pd_list->list);
-- 
2.26.2



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

* [PATCH v3 5/8] mm: don't track number of pages during deferred initialization
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
                   ` (3 preceding siblings ...)
  2020-05-27 17:36 ` [PATCH v3 4/8] padata: add basic support for multithreaded jobs Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 21:18   ` Alexander Duyck
  2020-05-27 17:36 ` [PATCH v3 6/8] mm: parallelize deferred_init_memmap() Daniel Jordan
                   ` (2 subsequent siblings)
  7 siblings, 1 reply; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

Deferred page init used to report the number of pages initialized:

  node 0 initialised, 32439114 pages in 97ms

Tracking this makes the code more complicated when using multiple
threads.  Given that the statistic probably has limited value,
especially since a zone grows on demand so that the page count can vary,
just remove it.

The boot message now looks like

  node 0 deferred pages initialised in 97ms

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Suggested-by: Alexander Duyck <alexander.h.duyck@linux.intel.com>
---
 mm/page_alloc.c | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)

diff --git a/mm/page_alloc.c b/mm/page_alloc.c
index d0c0d9364aa6d..d64f3027fdfa6 100644
--- a/mm/page_alloc.c
+++ b/mm/page_alloc.c
@@ -1819,7 +1819,7 @@ static int __init deferred_init_memmap(void *data)
 {
 	pg_data_t *pgdat = data;
 	const struct cpumask *cpumask = cpumask_of_node(pgdat->node_id);
-	unsigned long spfn = 0, epfn = 0, nr_pages = 0;
+	unsigned long spfn = 0, epfn = 0;
 	unsigned long first_init_pfn, flags;
 	unsigned long start = jiffies;
 	struct zone *zone;
@@ -1868,15 +1868,15 @@ static int __init deferred_init_memmap(void *data)
 	 * allocator.
 	 */
 	while (spfn < epfn) {
-		nr_pages += deferred_init_maxorder(&i, zone, &spfn, &epfn);
+		deferred_init_maxorder(&i, zone, &spfn, &epfn);
 		cond_resched();
 	}
 zone_empty:
 	/* Sanity check that the next zone really is unpopulated */
 	WARN_ON(++zid < MAX_NR_ZONES && populated_zone(++zone));
 
-	pr_info("node %d initialised, %lu pages in %ums\n",
-		pgdat->node_id,	nr_pages, jiffies_to_msecs(jiffies - start));
+	pr_info("node %d deferred pages initialised in %ums\n",
+		pgdat->node_id, jiffies_to_msecs(jiffies - start));
 
 	pgdat_init_report_one_done();
 	return 0;
-- 
2.26.2



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

* [PATCH v3 6/8] mm: parallelize deferred_init_memmap()
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
                   ` (4 preceding siblings ...)
  2020-05-27 17:36 ` [PATCH v3 5/8] mm: don't track number of pages during deferred initialization Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 21:27   ` Alexander Duyck
  2020-05-27 17:36 ` [PATCH v3 7/8] mm: make deferred init's max threads arch-specific Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 8/8] padata: document multithreaded jobs Daniel Jordan
  7 siblings, 1 reply; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

Deferred struct page init is a significant bottleneck in kernel boot.
Optimizing it maximizes availability for large-memory systems and allows
spinning up short-lived VMs as needed without having to leave them
running.  It also benefits bare metal machines hosting VMs that are
sensitive to downtime.  In projects such as VMM Fast Restart[1], where
guest state is preserved across kexec reboot, it helps prevent
application and network timeouts in the guests.

Multithread to take full advantage of system memory bandwidth.

The maximum number of threads is capped at the number of CPUs on the
node because speedups always improve with additional threads on every
system tested, and at this phase of boot, the system is otherwise idle
and waiting on page init to finish.

Helper threads operate on section-aligned ranges to both avoid false
sharing when setting the pageblock's migrate type and to avoid accessing
uninitialized buddy pages, though max order alignment is enough for the
latter.

The minimum chunk size is also a section.  There was benefit to using
multiple threads even on relatively small memory (1G) systems, and this
is the smallest size that the alignment allows.

The time (milliseconds) is the slowest node to initialize since boot
blocks until all nodes finish.  intel_pstate is loaded in active mode
without hwp and with turbo enabled, and intel_idle is active as well.

    Intel(R) Xeon(R) Platinum 8167M CPU @ 2.00GHz (Skylake, bare metal)
      2 nodes * 26 cores * 2 threads = 104 CPUs
      384G/node = 768G memory

                   kernel boot                 deferred init
                   ------------------------    ------------------------
    node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
          (  0)         --   4089.7 (  8.1)         --   1785.7 (  7.6)
       2% (  1)       1.7%   4019.3 (  1.5)       3.8%   1717.7 ( 11.8)
      12% (  6)      34.9%   2662.7 (  2.9)      79.9%    359.3 (  0.6)
      25% ( 13)      39.9%   2459.0 (  3.6)      91.2%    157.0 (  0.0)
      37% ( 19)      39.2%   2485.0 ( 29.7)      90.4%    172.0 ( 28.6)
      50% ( 26)      39.3%   2482.7 ( 25.7)      90.3%    173.7 ( 30.0)
      75% ( 39)      39.0%   2495.7 (  5.5)      89.4%    190.0 (  1.0)
     100% ( 52)      40.2%   2443.7 (  3.8)      92.3%    138.0 (  1.0)

    Intel(R) Xeon(R) CPU E5-2699C v4 @ 2.20GHz (Broadwell, kvm guest)
      1 node * 16 cores * 2 threads = 32 CPUs
      192G/node = 192G memory

                   kernel boot                 deferred init
                   ------------------------    ------------------------
    node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
          (  0)         --   1988.7 (  9.6)         --   1096.0 ( 11.5)
       3% (  1)       1.1%   1967.0 ( 17.6)       0.3%   1092.7 ( 11.0)
      12% (  4)      41.1%   1170.3 ( 14.2)      73.8%    287.0 (  3.6)
      25% (  8)      47.1%   1052.7 ( 21.9)      83.9%    177.0 ( 13.5)
      38% ( 12)      48.9%   1016.3 ( 12.1)      86.8%    144.7 (  1.5)
      50% ( 16)      48.9%   1015.7 (  8.1)      87.8%    134.0 (  4.4)
      75% ( 24)      49.1%   1012.3 (  3.1)      88.1%    130.3 (  2.3)
     100% ( 32)      49.5%   1004.0 (  5.3)      88.5%    125.7 (  2.1)

    Intel(R) Xeon(R) CPU E5-2699 v3 @ 2.30GHz (Haswell, bare metal)
      2 nodes * 18 cores * 2 threads = 72 CPUs
      128G/node = 256G memory

                   kernel boot                 deferred init
                   ------------------------    ------------------------
    node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
          (  0)         --   1680.0 (  4.6)         --    627.0 (  4.0)
       3% (  1)       0.3%   1675.7 (  4.5)      -0.2%    628.0 (  3.6)
      11% (  4)      25.6%   1250.7 (  2.1)      67.9%    201.0 (  0.0)
      25% (  9)      30.7%   1164.0 ( 17.3)      81.8%    114.3 ( 17.7)
      36% ( 13)      31.4%   1152.7 ( 10.8)      84.0%    100.3 ( 17.9)
      50% ( 18)      31.5%   1150.7 (  9.3)      83.9%    101.0 ( 14.1)
      75% ( 27)      31.7%   1148.0 (  5.6)      84.5%     97.3 (  6.4)
     100% ( 36)      32.0%   1142.3 (  4.0)      85.6%     90.0 (  1.0)

    AMD EPYC 7551 32-Core Processor (Zen, kvm guest)
      1 node * 8 cores * 2 threads = 16 CPUs
      64G/node = 64G memory

                   kernel boot                 deferred init
                   ------------------------    ------------------------
    node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
          (  0)         --   1029.3 ( 25.1)         --    240.7 (  1.5)
       6% (  1)      -0.6%   1036.0 (  7.8)      -2.2%    246.0 (  0.0)
      12% (  2)      11.8%    907.7 (  8.6)      44.7%    133.0 (  1.0)
      25% (  4)      13.9%    886.0 ( 10.6)      62.6%     90.0 (  6.0)
      38% (  6)      17.8%    845.7 ( 14.2)      69.1%     74.3 (  3.8)
      50% (  8)      16.8%    856.0 ( 22.1)      72.9%     65.3 (  5.7)
      75% ( 12)      15.4%    871.0 ( 29.2)      79.8%     48.7 (  7.4)
     100% ( 16)      21.0%    813.7 ( 21.0)      80.5%     47.0 (  5.2)

Server-oriented distros that enable deferred page init sometimes run in
small VMs, and they still benefit even though the fraction of boot time
saved is smaller:

    AMD EPYC 7551 32-Core Processor (Zen, kvm guest)
      1 node * 2 cores * 2 threads = 4 CPUs
      16G/node = 16G memory

                   kernel boot                 deferred init
                   ------------------------    ------------------------
    node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
          (  0)         --    716.0 ( 14.0)         --     49.7 (  0.6)
      25% (  1)       1.8%    703.0 (  5.3)      -4.0%     51.7 (  0.6)
      50% (  2)       1.6%    704.7 (  1.2)      43.0%     28.3 (  0.6)
      75% (  3)       2.7%    696.7 ( 13.1)      49.7%     25.0 (  0.0)
     100% (  4)       4.1%    687.0 ( 10.4)      55.7%     22.0 (  0.0)

    Intel(R) Xeon(R) CPU E5-2699 v3 @ 2.30GHz (Haswell, kvm guest)
      1 node * 2 cores * 2 threads = 4 CPUs
      14G/node = 14G memory

                   kernel boot                 deferred init
                   ------------------------    ------------------------
    node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
          (  0)         --    787.7 (  6.4)         --    122.3 (  0.6)
      25% (  1)       0.2%    786.3 ( 10.8)      -2.5%    125.3 (  2.1)
      50% (  2)       5.9%    741.0 ( 13.9)      37.6%     76.3 ( 19.7)
      75% (  3)       8.3%    722.0 ( 19.0)      49.9%     61.3 (  3.2)
     100% (  4)       9.3%    714.7 (  9.5)      56.4%     53.3 (  1.5)

On Josh's 96-CPU and 192G memory system:

    Without this patch series:
    [    0.487132] node 0 initialised, 23398907 pages in 292ms
    [    0.499132] node 1 initialised, 24189223 pages in 304ms
    ...
    [    0.629376] Run /sbin/init as init process

    With this patch series:
    [    0.231435] node 1 initialised, 24189223 pages in 32ms
    [    0.236718] node 0 initialised, 23398907 pages in 36ms

[1] https://static.sched.com/hosted_files/kvmforum2019/66/VMM-fast-restart_kvmforum2019.pdf

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 mm/Kconfig      |  6 +++---
 mm/page_alloc.c | 46 ++++++++++++++++++++++++++++++++++++++++------
 2 files changed, 43 insertions(+), 9 deletions(-)

diff --git a/mm/Kconfig b/mm/Kconfig
index c1acc34c1c358..04c1da3f9f44c 100644
--- a/mm/Kconfig
+++ b/mm/Kconfig
@@ -750,13 +750,13 @@ config DEFERRED_STRUCT_PAGE_INIT
 	depends on SPARSEMEM
 	depends on !NEED_PER_CPU_KM
 	depends on 64BIT
+	select PADATA
 	help
 	  Ordinarily all struct pages are initialised during early boot in a
 	  single thread. On very large machines this can take a considerable
 	  amount of time. If this option is set, large machines will bring up
-	  a subset of memmap at boot and then initialise the rest in parallel
-	  by starting one-off "pgdatinitX" kernel thread for each node X. This
-	  has a potential performance impact on processes running early in the
+	  a subset of memmap at boot and then initialise the rest in parallel.
+	  This has a potential performance impact on tasks running early in the
 	  lifetime of the system until these kthreads finish the
 	  initialisation.
 
diff --git a/mm/page_alloc.c b/mm/page_alloc.c
index d64f3027fdfa6..1d47016849531 100644
--- a/mm/page_alloc.c
+++ b/mm/page_alloc.c
@@ -68,6 +68,7 @@
 #include <linux/lockdep.h>
 #include <linux/nmi.h>
 #include <linux/psi.h>
+#include <linux/padata.h>
 
 #include <asm/sections.h>
 #include <asm/tlbflush.h>
@@ -1814,6 +1815,26 @@ deferred_init_maxorder(u64 *i, struct zone *zone, unsigned long *start_pfn,
 	return nr_pages;
 }
 
+static void __init
+deferred_init_memmap_chunk(unsigned long start_pfn, unsigned long end_pfn,
+			   void *arg)
+{
+	unsigned long spfn, epfn;
+	struct zone *zone = arg;
+	u64 i;
+
+	deferred_init_mem_pfn_range_in_zone(&i, zone, &spfn, &epfn, start_pfn);
+
+	/*
+	 * Initialize and free pages in MAX_ORDER sized increments so that we
+	 * can avoid introducing any issues with the buddy allocator.
+	 */
+	while (spfn < end_pfn) {
+		deferred_init_maxorder(&i, zone, &spfn, &epfn);
+		cond_resched();
+	}
+}
+
 /* Initialise remaining memory on a node */
 static int __init deferred_init_memmap(void *data)
 {
@@ -1823,7 +1844,7 @@ static int __init deferred_init_memmap(void *data)
 	unsigned long first_init_pfn, flags;
 	unsigned long start = jiffies;
 	struct zone *zone;
-	int zid;
+	int zid, max_threads;
 	u64 i;
 
 	/* Bind memory initialisation thread to a local node if possible */
@@ -1863,13 +1884,26 @@ static int __init deferred_init_memmap(void *data)
 		goto zone_empty;
 
 	/*
-	 * Initialize and free pages in MAX_ORDER sized increments so
-	 * that we can avoid introducing any issues with the buddy
-	 * allocator.
+	 * More CPUs always led to greater speedups on tested systems, up to
+	 * all the nodes' CPUs.  Use all since the system is otherwise idle now.
 	 */
+	max_threads = max(cpumask_weight(cpumask), 1u);
+
 	while (spfn < epfn) {
-		deferred_init_maxorder(&i, zone, &spfn, &epfn);
-		cond_resched();
+		unsigned long epfn_align = ALIGN(epfn, PAGES_PER_SECTION);
+		struct padata_mt_job job = {
+			.thread_fn   = deferred_init_memmap_chunk,
+			.fn_arg      = zone,
+			.start       = spfn,
+			.size        = epfn_align - spfn,
+			.align       = PAGES_PER_SECTION,
+			.min_chunk   = PAGES_PER_SECTION,
+			.max_threads = max_threads,
+		};
+
+		padata_do_multithreaded(&job);
+		deferred_init_mem_pfn_range_in_zone(&i, zone, &spfn, &epfn,
+						    epfn_align);
 	}
 zone_empty:
 	/* Sanity check that the next zone really is unpopulated */
-- 
2.26.2



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

* [PATCH v3 7/8] mm: make deferred init's max threads arch-specific
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
                   ` (5 preceding siblings ...)
  2020-05-27 17:36 ` [PATCH v3 6/8] mm: parallelize deferred_init_memmap() Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  2020-05-27 17:36 ` [PATCH v3 8/8] padata: document multithreaded jobs Daniel Jordan
  7 siblings, 0 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

Using padata during deferred init has only been tested on x86, so for
now limit it to this architecture.

If another arch wants this, it can find the max thread limit that's best
for it and override deferred_page_init_max_threads().

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 arch/x86/mm/init_64.c    | 12 ++++++++++++
 include/linux/memblock.h |  3 +++
 mm/page_alloc.c          | 13 ++++++++-----
 3 files changed, 23 insertions(+), 5 deletions(-)

diff --git a/arch/x86/mm/init_64.c b/arch/x86/mm/init_64.c
index 8b5f73f5e207c..2d749ec12ea8a 100644
--- a/arch/x86/mm/init_64.c
+++ b/arch/x86/mm/init_64.c
@@ -1260,6 +1260,18 @@ void __init mem_init(void)
 	mem_init_print_info(NULL);
 }
 
+#ifdef CONFIG_DEFERRED_STRUCT_PAGE_INIT
+int __init deferred_page_init_max_threads(const struct cpumask *node_cpumask)
+{
+	/*
+	 * More CPUs always led to greater speedups on tested systems, up to
+	 * all the nodes' CPUs.  Use all since the system is otherwise idle
+	 * now.
+	 */
+	return max_t(int, cpumask_weight(node_cpumask), 1);
+}
+#endif
+
 int kernel_set_to_readonly;
 
 void mark_rodata_ro(void)
diff --git a/include/linux/memblock.h b/include/linux/memblock.h
index 6bc37a731d27b..2b289df44194f 100644
--- a/include/linux/memblock.h
+++ b/include/linux/memblock.h
@@ -275,6 +275,9 @@ void __next_mem_pfn_range_in_zone(u64 *idx, struct zone *zone,
 #define for_each_free_mem_pfn_range_in_zone_from(i, zone, p_start, p_end) \
 	for (; i != U64_MAX;					  \
 	     __next_mem_pfn_range_in_zone(&i, zone, p_start, p_end))
+
+int __init deferred_page_init_max_threads(const struct cpumask *node_cpumask);
+
 #endif /* CONFIG_DEFERRED_STRUCT_PAGE_INIT */
 
 /**
diff --git a/mm/page_alloc.c b/mm/page_alloc.c
index 1d47016849531..329fd1a809c59 100644
--- a/mm/page_alloc.c
+++ b/mm/page_alloc.c
@@ -1835,6 +1835,13 @@ deferred_init_memmap_chunk(unsigned long start_pfn, unsigned long end_pfn,
 	}
 }
 
+/* An arch may override for more concurrency. */
+__weak int __init
+deferred_page_init_max_threads(const struct cpumask *node_cpumask)
+{
+	return 1;
+}
+
 /* Initialise remaining memory on a node */
 static int __init deferred_init_memmap(void *data)
 {
@@ -1883,11 +1890,7 @@ static int __init deferred_init_memmap(void *data)
 						 first_init_pfn))
 		goto zone_empty;
 
-	/*
-	 * More CPUs always led to greater speedups on tested systems, up to
-	 * all the nodes' CPUs.  Use all since the system is otherwise idle now.
-	 */
-	max_threads = max(cpumask_weight(cpumask), 1u);
+	max_threads = deferred_page_init_max_threads(cpumask);
 
 	while (spfn < epfn) {
 		unsigned long epfn_align = ALIGN(epfn, PAGES_PER_SECTION);
-- 
2.26.2



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

* [PATCH v3 8/8] padata: document multithreaded jobs
  2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
                   ` (6 preceding siblings ...)
  2020-05-27 17:36 ` [PATCH v3 7/8] mm: make deferred init's max threads arch-specific Daniel Jordan
@ 2020-05-27 17:36 ` Daniel Jordan
  7 siblings, 0 replies; 11+ messages in thread
From: Daniel Jordan @ 2020-05-27 17:36 UTC (permalink / raw)
  To: Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Robert Elliott,
	Shile Zhang, Steven Sistare, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel, linux-s390, linuxppc-dev, Daniel Jordan

Add Documentation for multithreaded jobs.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
Tested-by: Josh Triplett <josh@joshtriplett.org>
---
 Documentation/core-api/padata.rst | 41 +++++++++++++++++++++++--------
 1 file changed, 31 insertions(+), 10 deletions(-)

diff --git a/Documentation/core-api/padata.rst b/Documentation/core-api/padata.rst
index 9a24c111781d9..0830e5b0e8211 100644
--- a/Documentation/core-api/padata.rst
+++ b/Documentation/core-api/padata.rst
@@ -4,23 +4,26 @@
 The padata parallel execution mechanism
 =======================================
 
-:Date: December 2019
+:Date: May 2020
 
 Padata is a mechanism by which the kernel can farm jobs out to be done in
-parallel on multiple CPUs while retaining their ordering.  It was developed for
-use with the IPsec code, which needs to be able to perform encryption and
-decryption on large numbers of packets without reordering those packets.  The
-crypto developers made a point of writing padata in a sufficiently general
-fashion that it could be put to other uses as well.
+parallel on multiple CPUs while optionally retaining their ordering.
 
-Usage
-=====
+It was originally developed for IPsec, which needs to perform encryption and
+decryption on large numbers of packets without reordering those packets.  This
+is currently the sole consumer of padata's serialized job support.
+
+Padata also supports multithreaded jobs, splitting up the job evenly while load
+balancing and coordinating between threads.
+
+Running Serialized Jobs
+=======================
 
 Initializing
 ------------
 
-The first step in using padata is to set up a padata_instance structure for
-overall control of how jobs are to be run::
+The first step in using padata to run serialized jobs is to set up a
+padata_instance structure for overall control of how jobs are to be run::
 
     #include <linux/padata.h>
 
@@ -162,6 +165,24 @@ functions that correspond to the allocation in reverse::
 It is the user's responsibility to ensure all outstanding jobs are complete
 before any of the above are called.
 
+Running Multithreaded Jobs
+==========================
+
+A multithreaded job has a main thread and zero or more helper threads, with the
+main thread participating in the job and then waiting until all helpers have
+finished.  padata splits the job into units called chunks, where a chunk is a
+piece of the job that one thread completes in one call to the thread function.
+
+A user has to do three things to run a multithreaded job.  First, describe the
+job by defining a padata_mt_job structure, which is explained in the Interface
+section.  This includes a pointer to the thread function, which padata will
+call each time it assigns a job chunk to a thread.  Then, define the thread
+function, which accepts three arguments, ``start``, ``end``, and ``arg``, where
+the first two delimit the range that the thread operates on and the last is a
+pointer to the job's shared state, if any.  Prepare the shared state, which is
+typically allocated on the main thread's stack.  Last, call
+padata_do_multithreaded(), which will return once the job is finished.
+
 Interface
 =========
 
-- 
2.26.2



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

* Re: [PATCH v3 5/8] mm: don't track number of pages during deferred initialization
  2020-05-27 17:36 ` [PATCH v3 5/8] mm: don't track number of pages during deferred initialization Daniel Jordan
@ 2020-05-27 21:18   ` Alexander Duyck
  0 siblings, 0 replies; 11+ messages in thread
From: Alexander Duyck @ 2020-05-27 21:18 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Andrew Morton, Herbert Xu, Steffen Klassert, Alex Williamson,
	Alexander Duyck, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Robert Elliott, Shile Zhang, Steven Sistare,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML, linux-s390,
	open list:LINUX FOR POWERPC (32-BIT AND 64-BIT)

On Wed, May 27, 2020 at 10:37 AM Daniel Jordan
<daniel.m.jordan@oracle.com> wrote:
>
> Deferred page init used to report the number of pages initialized:
>
>   node 0 initialised, 32439114 pages in 97ms
>
> Tracking this makes the code more complicated when using multiple
> threads.  Given that the statistic probably has limited value,
> especially since a zone grows on demand so that the page count can vary,
> just remove it.
>
> The boot message now looks like
>
>   node 0 deferred pages initialised in 97ms
>
> Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
> Suggested-by: Alexander Duyck <alexander.h.duyck@linux.intel.com>

This looks good to me.

Reviewed-by: Alexander Duyck <alexander.h.duyck@linux.intel.com>

> ---
>  mm/page_alloc.c | 8 ++++----
>  1 file changed, 4 insertions(+), 4 deletions(-)
>
> diff --git a/mm/page_alloc.c b/mm/page_alloc.c
> index d0c0d9364aa6d..d64f3027fdfa6 100644
> --- a/mm/page_alloc.c
> +++ b/mm/page_alloc.c
> @@ -1819,7 +1819,7 @@ static int __init deferred_init_memmap(void *data)
>  {
>         pg_data_t *pgdat = data;
>         const struct cpumask *cpumask = cpumask_of_node(pgdat->node_id);
> -       unsigned long spfn = 0, epfn = 0, nr_pages = 0;
> +       unsigned long spfn = 0, epfn = 0;
>         unsigned long first_init_pfn, flags;
>         unsigned long start = jiffies;
>         struct zone *zone;
> @@ -1868,15 +1868,15 @@ static int __init deferred_init_memmap(void *data)
>          * allocator.
>          */
>         while (spfn < epfn) {
> -               nr_pages += deferred_init_maxorder(&i, zone, &spfn, &epfn);
> +               deferred_init_maxorder(&i, zone, &spfn, &epfn);
>                 cond_resched();
>         }
>  zone_empty:
>         /* Sanity check that the next zone really is unpopulated */
>         WARN_ON(++zid < MAX_NR_ZONES && populated_zone(++zone));
>
> -       pr_info("node %d initialised, %lu pages in %ums\n",
> -               pgdat->node_id, nr_pages, jiffies_to_msecs(jiffies - start));
> +       pr_info("node %d deferred pages initialised in %ums\n",
> +               pgdat->node_id, jiffies_to_msecs(jiffies - start));
>
>         pgdat_init_report_one_done();
>         return 0;
> --
> 2.26.2
>
>


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

* Re: [PATCH v3 6/8] mm: parallelize deferred_init_memmap()
  2020-05-27 17:36 ` [PATCH v3 6/8] mm: parallelize deferred_init_memmap() Daniel Jordan
@ 2020-05-27 21:27   ` Alexander Duyck
  0 siblings, 0 replies; 11+ messages in thread
From: Alexander Duyck @ 2020-05-27 21:27 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Andrew Morton, Herbert Xu, Steffen Klassert, Alex Williamson,
	Alexander Duyck, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Robert Elliott, Shile Zhang, Steven Sistare,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML, linux-s390,
	open list:LINUX FOR POWERPC (32-BIT AND 64-BIT)

On Wed, May 27, 2020 at 10:37 AM Daniel Jordan
<daniel.m.jordan@oracle.com> wrote:
>
> Deferred struct page init is a significant bottleneck in kernel boot.
> Optimizing it maximizes availability for large-memory systems and allows
> spinning up short-lived VMs as needed without having to leave them
> running.  It also benefits bare metal machines hosting VMs that are
> sensitive to downtime.  In projects such as VMM Fast Restart[1], where
> guest state is preserved across kexec reboot, it helps prevent
> application and network timeouts in the guests.
>
> Multithread to take full advantage of system memory bandwidth.
>
> The maximum number of threads is capped at the number of CPUs on the
> node because speedups always improve with additional threads on every
> system tested, and at this phase of boot, the system is otherwise idle
> and waiting on page init to finish.
>
> Helper threads operate on section-aligned ranges to both avoid false
> sharing when setting the pageblock's migrate type and to avoid accessing
> uninitialized buddy pages, though max order alignment is enough for the
> latter.
>
> The minimum chunk size is also a section.  There was benefit to using
> multiple threads even on relatively small memory (1G) systems, and this
> is the smallest size that the alignment allows.
>
> The time (milliseconds) is the slowest node to initialize since boot
> blocks until all nodes finish.  intel_pstate is loaded in active mode
> without hwp and with turbo enabled, and intel_idle is active as well.
>
>     Intel(R) Xeon(R) Platinum 8167M CPU @ 2.00GHz (Skylake, bare metal)
>       2 nodes * 26 cores * 2 threads = 104 CPUs
>       384G/node = 768G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>     node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>           (  0)         --   4089.7 (  8.1)         --   1785.7 (  7.6)
>        2% (  1)       1.7%   4019.3 (  1.5)       3.8%   1717.7 ( 11.8)
>       12% (  6)      34.9%   2662.7 (  2.9)      79.9%    359.3 (  0.6)
>       25% ( 13)      39.9%   2459.0 (  3.6)      91.2%    157.0 (  0.0)
>       37% ( 19)      39.2%   2485.0 ( 29.7)      90.4%    172.0 ( 28.6)
>       50% ( 26)      39.3%   2482.7 ( 25.7)      90.3%    173.7 ( 30.0)
>       75% ( 39)      39.0%   2495.7 (  5.5)      89.4%    190.0 (  1.0)
>      100% ( 52)      40.2%   2443.7 (  3.8)      92.3%    138.0 (  1.0)
>
>     Intel(R) Xeon(R) CPU E5-2699C v4 @ 2.20GHz (Broadwell, kvm guest)
>       1 node * 16 cores * 2 threads = 32 CPUs
>       192G/node = 192G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>     node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>           (  0)         --   1988.7 (  9.6)         --   1096.0 ( 11.5)
>        3% (  1)       1.1%   1967.0 ( 17.6)       0.3%   1092.7 ( 11.0)
>       12% (  4)      41.1%   1170.3 ( 14.2)      73.8%    287.0 (  3.6)
>       25% (  8)      47.1%   1052.7 ( 21.9)      83.9%    177.0 ( 13.5)
>       38% ( 12)      48.9%   1016.3 ( 12.1)      86.8%    144.7 (  1.5)
>       50% ( 16)      48.9%   1015.7 (  8.1)      87.8%    134.0 (  4.4)
>       75% ( 24)      49.1%   1012.3 (  3.1)      88.1%    130.3 (  2.3)
>      100% ( 32)      49.5%   1004.0 (  5.3)      88.5%    125.7 (  2.1)
>
>     Intel(R) Xeon(R) CPU E5-2699 v3 @ 2.30GHz (Haswell, bare metal)
>       2 nodes * 18 cores * 2 threads = 72 CPUs
>       128G/node = 256G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>     node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>           (  0)         --   1680.0 (  4.6)         --    627.0 (  4.0)
>        3% (  1)       0.3%   1675.7 (  4.5)      -0.2%    628.0 (  3.6)
>       11% (  4)      25.6%   1250.7 (  2.1)      67.9%    201.0 (  0.0)
>       25% (  9)      30.7%   1164.0 ( 17.3)      81.8%    114.3 ( 17.7)
>       36% ( 13)      31.4%   1152.7 ( 10.8)      84.0%    100.3 ( 17.9)
>       50% ( 18)      31.5%   1150.7 (  9.3)      83.9%    101.0 ( 14.1)
>       75% ( 27)      31.7%   1148.0 (  5.6)      84.5%     97.3 (  6.4)
>      100% ( 36)      32.0%   1142.3 (  4.0)      85.6%     90.0 (  1.0)
>
>     AMD EPYC 7551 32-Core Processor (Zen, kvm guest)
>       1 node * 8 cores * 2 threads = 16 CPUs
>       64G/node = 64G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>     node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>           (  0)         --   1029.3 ( 25.1)         --    240.7 (  1.5)
>        6% (  1)      -0.6%   1036.0 (  7.8)      -2.2%    246.0 (  0.0)
>       12% (  2)      11.8%    907.7 (  8.6)      44.7%    133.0 (  1.0)
>       25% (  4)      13.9%    886.0 ( 10.6)      62.6%     90.0 (  6.0)
>       38% (  6)      17.8%    845.7 ( 14.2)      69.1%     74.3 (  3.8)
>       50% (  8)      16.8%    856.0 ( 22.1)      72.9%     65.3 (  5.7)
>       75% ( 12)      15.4%    871.0 ( 29.2)      79.8%     48.7 (  7.4)
>      100% ( 16)      21.0%    813.7 ( 21.0)      80.5%     47.0 (  5.2)
>
> Server-oriented distros that enable deferred page init sometimes run in
> small VMs, and they still benefit even though the fraction of boot time
> saved is smaller:
>
>     AMD EPYC 7551 32-Core Processor (Zen, kvm guest)
>       1 node * 2 cores * 2 threads = 4 CPUs
>       16G/node = 16G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>     node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>           (  0)         --    716.0 ( 14.0)         --     49.7 (  0.6)
>       25% (  1)       1.8%    703.0 (  5.3)      -4.0%     51.7 (  0.6)
>       50% (  2)       1.6%    704.7 (  1.2)      43.0%     28.3 (  0.6)
>       75% (  3)       2.7%    696.7 ( 13.1)      49.7%     25.0 (  0.0)
>      100% (  4)       4.1%    687.0 ( 10.4)      55.7%     22.0 (  0.0)
>
>     Intel(R) Xeon(R) CPU E5-2699 v3 @ 2.30GHz (Haswell, kvm guest)
>       1 node * 2 cores * 2 threads = 4 CPUs
>       14G/node = 14G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>     node% (thr)    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>           (  0)         --    787.7 (  6.4)         --    122.3 (  0.6)
>       25% (  1)       0.2%    786.3 ( 10.8)      -2.5%    125.3 (  2.1)
>       50% (  2)       5.9%    741.0 ( 13.9)      37.6%     76.3 ( 19.7)
>       75% (  3)       8.3%    722.0 ( 19.0)      49.9%     61.3 (  3.2)
>      100% (  4)       9.3%    714.7 (  9.5)      56.4%     53.3 (  1.5)
>
> On Josh's 96-CPU and 192G memory system:
>
>     Without this patch series:
>     [    0.487132] node 0 initialised, 23398907 pages in 292ms
>     [    0.499132] node 1 initialised, 24189223 pages in 304ms
>     ...
>     [    0.629376] Run /sbin/init as init process
>
>     With this patch series:
>     [    0.231435] node 1 initialised, 24189223 pages in 32ms
>     [    0.236718] node 0 initialised, 23398907 pages in 36ms
>
> [1] https://static.sched.com/hosted_files/kvmforum2019/66/VMM-fast-restart_kvmforum2019.pdf
>
> Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
> Tested-by: Josh Triplett <josh@joshtriplett.org>
> ---
>  mm/Kconfig      |  6 +++---
>  mm/page_alloc.c | 46 ++++++++++++++++++++++++++++++++++++++++------
>  2 files changed, 43 insertions(+), 9 deletions(-)
>
> diff --git a/mm/Kconfig b/mm/Kconfig
> index c1acc34c1c358..04c1da3f9f44c 100644
> --- a/mm/Kconfig
> +++ b/mm/Kconfig
> @@ -750,13 +750,13 @@ config DEFERRED_STRUCT_PAGE_INIT
>         depends on SPARSEMEM
>         depends on !NEED_PER_CPU_KM
>         depends on 64BIT
> +       select PADATA
>         help
>           Ordinarily all struct pages are initialised during early boot in a
>           single thread. On very large machines this can take a considerable
>           amount of time. If this option is set, large machines will bring up
> -         a subset of memmap at boot and then initialise the rest in parallel
> -         by starting one-off "pgdatinitX" kernel thread for each node X. This
> -         has a potential performance impact on processes running early in the
> +         a subset of memmap at boot and then initialise the rest in parallel.
> +         This has a potential performance impact on tasks running early in the
>           lifetime of the system until these kthreads finish the
>           initialisation.
>
> diff --git a/mm/page_alloc.c b/mm/page_alloc.c
> index d64f3027fdfa6..1d47016849531 100644
> --- a/mm/page_alloc.c
> +++ b/mm/page_alloc.c
> @@ -68,6 +68,7 @@
>  #include <linux/lockdep.h>
>  #include <linux/nmi.h>
>  #include <linux/psi.h>
> +#include <linux/padata.h>
>
>  #include <asm/sections.h>
>  #include <asm/tlbflush.h>
> @@ -1814,6 +1815,26 @@ deferred_init_maxorder(u64 *i, struct zone *zone, unsigned long *start_pfn,
>         return nr_pages;
>  }
>
> +static void __init
> +deferred_init_memmap_chunk(unsigned long start_pfn, unsigned long end_pfn,
> +                          void *arg)
> +{
> +       unsigned long spfn, epfn;
> +       struct zone *zone = arg;
> +       u64 i;
> +
> +       deferred_init_mem_pfn_range_in_zone(&i, zone, &spfn, &epfn, start_pfn);
> +
> +       /*
> +        * Initialize and free pages in MAX_ORDER sized increments so that we
> +        * can avoid introducing any issues with the buddy allocator.
> +        */
> +       while (spfn < end_pfn) {
> +               deferred_init_maxorder(&i, zone, &spfn, &epfn);
> +               cond_resched();
> +       }
> +}
> +
>  /* Initialise remaining memory on a node */
>  static int __init deferred_init_memmap(void *data)
>  {
> @@ -1823,7 +1844,7 @@ static int __init deferred_init_memmap(void *data)
>         unsigned long first_init_pfn, flags;
>         unsigned long start = jiffies;
>         struct zone *zone;
> -       int zid;
> +       int zid, max_threads;
>         u64 i;
>
>         /* Bind memory initialisation thread to a local node if possible */
> @@ -1863,13 +1884,26 @@ static int __init deferred_init_memmap(void *data)
>                 goto zone_empty;
>
>         /*
> -        * Initialize and free pages in MAX_ORDER sized increments so
> -        * that we can avoid introducing any issues with the buddy
> -        * allocator.
> +        * More CPUs always led to greater speedups on tested systems, up to
> +        * all the nodes' CPUs.  Use all since the system is otherwise idle now.
>          */
> +       max_threads = max(cpumask_weight(cpumask), 1u);
> +
>         while (spfn < epfn) {
> -               deferred_init_maxorder(&i, zone, &spfn, &epfn);
> -               cond_resched();
> +               unsigned long epfn_align = ALIGN(epfn, PAGES_PER_SECTION);
> +               struct padata_mt_job job = {
> +                       .thread_fn   = deferred_init_memmap_chunk,
> +                       .fn_arg      = zone,
> +                       .start       = spfn,
> +                       .size        = epfn_align - spfn,
> +                       .align       = PAGES_PER_SECTION,
> +                       .min_chunk   = PAGES_PER_SECTION,
> +                       .max_threads = max_threads,
> +               };
> +
> +               padata_do_multithreaded(&job);
> +               deferred_init_mem_pfn_range_in_zone(&i, zone, &spfn, &epfn,
> +                                                   epfn_align);
>         }
>  zone_empty:
>         /* Sanity check that the next zone really is unpopulated */

So I am not a huge fan of using deferred_init_mem_pfn_range_in zone
simply for the fact that we end up essentially discarding the i value
and will have to walk the list repeatedly. However I don't think the
overhead will be that great as I suspect there aren't going to be
systems with that many ranges. So this is probably fine.

Reviewed-by: Alexander Duyck <alexander.h.duyck@linux.intel.com>


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

end of thread, other threads:[~2020-05-27 21:27 UTC | newest]

Thread overview: 11+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-05-27 17:36 [PATCH v3 0/8] padata: parallelize deferred page init Daniel Jordan
2020-05-27 17:36 ` [PATCH v3 1/8] padata: remove exit routine Daniel Jordan
2020-05-27 17:36 ` [PATCH v3 2/8] padata: initialize earlier Daniel Jordan
2020-05-27 17:36 ` [PATCH v3 3/8] padata: allocate work structures for parallel jobs from a pool Daniel Jordan
2020-05-27 17:36 ` [PATCH v3 4/8] padata: add basic support for multithreaded jobs Daniel Jordan
2020-05-27 17:36 ` [PATCH v3 5/8] mm: don't track number of pages during deferred initialization Daniel Jordan
2020-05-27 21:18   ` Alexander Duyck
2020-05-27 17:36 ` [PATCH v3 6/8] mm: parallelize deferred_init_memmap() Daniel Jordan
2020-05-27 21:27   ` Alexander Duyck
2020-05-27 17:36 ` [PATCH v3 7/8] mm: make deferred init's max threads arch-specific Daniel Jordan
2020-05-27 17:36 ` [PATCH v3 8/8] padata: document multithreaded jobs Daniel Jordan

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