linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 0/7] padata: parallelize deferred page init
@ 2020-04-30 20:11 Daniel Jordan
  2020-04-30 20:11 ` [PATCH 1/7] padata: remove exit routine Daniel Jordan
                   ` (8 more replies)
  0 siblings, 9 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	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, and this series is the
first step.

It extends 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 7.

The first user is deferred struct page init, a large bottleneck in
kernel boot--actually the largest for us and likely others too.  This
path doesn't require concurrency limits, resource control, or priority
adjustments like future users will (vfio, hugetlb fallocate, munmap)
because it happens during boot when the system is otherwise idle and
waiting on page init to finish.

This has been tested on a variety of x86 systems and speeds up kernel
boot by 6% to 49% by making deferred init 63% to 91% faster.  Patch 6
has detailed numbers.  Test results from other systems appreciated.

This series is based on v5.6 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

All of the above can be found in this branch:

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

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

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

Thanks to everyone who commented on the last version of this[0],
including Alex Williamson, Jason Gunthorpe, Jonathan Corbet, Michal
Hocko, Pavel Machek, Peter Zijlstra, Randy Dunlap, Robert Elliott, Tejun
Heo, and Zi Yan.

RFC v4 -> padata 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.

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

Daniel Jordan (7):
  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: move zone iterator outside of deferred_init_maxorder()
  mm: parallelize deferred_init_memmap()
  padata: document multithreaded jobs

 Documentation/core-api/padata.rst |  41 +++--
 include/linux/padata.h            |  43 ++++-
 init/main.c                       |   2 +
 kernel/padata.c                   | 277 ++++++++++++++++++++++++------
 mm/Kconfig                        |   6 +-
 mm/page_alloc.c                   | 118 ++++++-------
 6 files changed, 355 insertions(+), 132 deletions(-)


base-commit: 7111951b8d4973bda27ff663f2cf18b663d15b48
prerequisite-patch-id: 4ad522141e1119a325a9799dad2bd982fbac8b7c
prerequisite-patch-id: 169273327e56f5461101a71dfbd6b4cfd4570cf0
prerequisite-patch-id: 0f34692c8a9673d4c4f6a3545cf8ec3a2abf8620
-- 
2.26.2



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

* [PATCH 1/7] padata: remove exit routine
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-04-30 20:11 ` [PATCH 2/7] padata: initialize earlier Daniel Jordan
                   ` (7 subsequent siblings)
  8 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	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>
---
 kernel/padata.c | 6 ------
 1 file changed, 6 deletions(-)

diff --git a/kernel/padata.c b/kernel/padata.c
index 72777c10bb9cb..36a8e98741bb3 100644
--- a/kernel/padata.c
+++ b/kernel/padata.c
@@ -1071,10 +1071,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] 35+ messages in thread

* [PATCH 2/7] padata: initialize earlier
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
  2020-04-30 20:11 ` [PATCH 1/7] padata: remove exit routine Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-04-30 20:11 ` [PATCH 3/7] padata: allocate work structures for parallel jobs from a pool Daniel Jordan
                   ` (6 subsequent siblings)
  8 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	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>
---
 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 ee4947af823f3..5451a80e43016 100644
--- a/init/main.c
+++ b/init/main.c
@@ -94,6 +94,7 @@
 #include <linux/rodata_test.h>
 #include <linux/jump_label.h>
 #include <linux/mem_encrypt.h>
+#include <linux/padata.h>
 
 #include <asm/io.h>
 #include <asm/bugs.h>
@@ -1438,6 +1439,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 36a8e98741bb3..b05cd30f8905b 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
 
@@ -1049,26 +1048,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] 35+ messages in thread

* [PATCH 3/7] padata: allocate work structures for parallel jobs from a pool
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
  2020-04-30 20:11 ` [PATCH 1/7] padata: remove exit routine Daniel Jordan
  2020-04-30 20:11 ` [PATCH 2/7] padata: initialize earlier Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-04-30 20:11 ` [PATCH 4/7] padata: add basic support for multithreaded jobs Daniel Jordan
                   ` (5 subsequent siblings)
  8 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	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>
---
 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 b05cd30f8905b..edd3ff551e262 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);
@@ -1050,6 +1071,7 @@ EXPORT_SYMBOL(padata_free_shell);
 
 void __init padata_init(void)
 {
+	unsigned int i, possible_cpus;
 #ifdef CONFIG_HOTPLUG_CPU
 	int ret;
 
@@ -1061,13 +1083,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] 35+ messages in thread

* [PATCH 4/7] padata: add basic support for multithreaded jobs
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
                   ` (2 preceding siblings ...)
  2020-04-30 20:11 ` [PATCH 3/7] padata: allocate work structures for parallel jobs from a pool Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-04-30 20:11 ` [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder() Daniel Jordan
                   ` (4 subsequent siblings)
  8 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	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>
---
 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 edd3ff551e262..ccb617d37677a 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] 35+ messages in thread

* [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
                   ` (3 preceding siblings ...)
  2020-04-30 20:11 ` [PATCH 4/7] padata: add basic support for multithreaded jobs Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-04-30 21:43   ` Alexander Duyck
  2020-04-30 20:11 ` [PATCH 6/7] mm: parallelize deferred_init_memmap() Daniel Jordan
                   ` (3 subsequent siblings)
  8 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	Daniel Jordan

padata will soon divide up pfn ranges between threads when parallelizing
deferred init, and deferred_init_maxorder() complicates that by using an
opaque index in addition to start and end pfns.  Move the index outside
the function to make splitting the job easier, and simplify the code
while at it.

deferred_init_maxorder() now always iterates within a single pfn range
instead of potentially multiple ranges, and advances start_pfn to the
end of that range instead of the max-order block so partial pfn ranges
in the block aren't skipped in a later iteration.  The section alignment
check in deferred_grow_zone() is removed as well since this alignment is
no longer guaranteed.  It's not clear what value the alignment provided
originally.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 mm/page_alloc.c | 88 +++++++++++++++----------------------------------
 1 file changed, 27 insertions(+), 61 deletions(-)

diff --git a/mm/page_alloc.c b/mm/page_alloc.c
index 68669d3a5a665..990514d8f0d94 100644
--- a/mm/page_alloc.c
+++ b/mm/page_alloc.c
@@ -1708,55 +1708,23 @@ deferred_init_mem_pfn_range_in_zone(u64 *i, struct zone *zone,
 }
 
 /*
- * Initialize and free pages. We do it in two loops: first we initialize
- * struct page, then free to buddy allocator, because while we are
- * freeing pages we can access pages that are ahead (computing buddy
- * page in __free_one_page()).
- *
- * In order to try and keep some memory in the cache we have the loop
- * broken along max page order boundaries. This way we will not cause
- * any issues with the buddy page computation.
+ * Initialize the struct pages and then free them to the buddy allocator at
+ * most a max order block at a time because while we are freeing pages we can
+ * access pages that are ahead (computing buddy page in __free_one_page()).
+ * It's also cache friendly.
  */
 static unsigned long __init
-deferred_init_maxorder(u64 *i, struct zone *zone, unsigned long *start_pfn,
-		       unsigned long *end_pfn)
+deferred_init_maxorder(struct zone *zone, unsigned long *start_pfn,
+		       unsigned long end_pfn)
 {
-	unsigned long mo_pfn = ALIGN(*start_pfn + 1, MAX_ORDER_NR_PAGES);
-	unsigned long spfn = *start_pfn, epfn = *end_pfn;
-	unsigned long nr_pages = 0;
-	u64 j = *i;
-
-	/* First we loop through and initialize the page values */
-	for_each_free_mem_pfn_range_in_zone_from(j, zone, start_pfn, end_pfn) {
-		unsigned long t;
-
-		if (mo_pfn <= *start_pfn)
-			break;
-
-		t = min(mo_pfn, *end_pfn);
-		nr_pages += deferred_init_pages(zone, *start_pfn, t);
-
-		if (mo_pfn < *end_pfn) {
-			*start_pfn = mo_pfn;
-			break;
-		}
-	}
-
-	/* Reset values and now loop through freeing pages as needed */
-	swap(j, *i);
-
-	for_each_free_mem_pfn_range_in_zone_from(j, zone, &spfn, &epfn) {
-		unsigned long t;
-
-		if (mo_pfn <= spfn)
-			break;
+	unsigned long nr_pages, pfn;
 
-		t = min(mo_pfn, epfn);
-		deferred_free_pages(spfn, t);
+	pfn = ALIGN(*start_pfn + 1, MAX_ORDER_NR_PAGES);
+	pfn = min(pfn, end_pfn);
 
-		if (mo_pfn <= epfn)
-			break;
-	}
+	nr_pages = deferred_init_pages(zone, *start_pfn, pfn);
+	deferred_free_pages(*start_pfn, pfn);
+	*start_pfn = pfn;
 
 	return nr_pages;
 }
@@ -1814,9 +1782,11 @@ static int __init deferred_init_memmap(void *data)
 	 * that we can avoid introducing any issues with the buddy
 	 * allocator.
 	 */
-	while (spfn < epfn) {
-		nr_pages += deferred_init_maxorder(&i, zone, &spfn, &epfn);
-		cond_resched();
+	for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
+		while (spfn < epfn) {
+			nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
+			cond_resched();
+		}
 	}
 zone_empty:
 	/* Sanity check that the next zone really is unpopulated */
@@ -1883,22 +1853,18 @@ deferred_grow_zone(struct zone *zone, unsigned int order)
 	 * that we can avoid introducing any issues with the buddy
 	 * allocator.
 	 */
-	while (spfn < epfn) {
-		/* update our first deferred PFN for this section */
-		first_deferred_pfn = spfn;
-
-		nr_pages += deferred_init_maxorder(&i, zone, &spfn, &epfn);
-		touch_nmi_watchdog();
-
-		/* We should only stop along section boundaries */
-		if ((first_deferred_pfn ^ spfn) < PAGES_PER_SECTION)
-			continue;
-
-		/* If our quota has been met we can stop here */
-		if (nr_pages >= nr_pages_needed)
-			break;
+	for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
+		while (spfn < epfn) {
+			nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
+			touch_nmi_watchdog();
+
+			/* If our quota has been met we can stop here */
+			if (nr_pages >= nr_pages_needed)
+				goto out;
+		}
 	}
 
+out:
 	pgdat->first_deferred_pfn = spfn;
 	pgdat_resize_unlock(pgdat, &flags);
 
-- 
2.26.2



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

* [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
                   ` (4 preceding siblings ...)
  2020-04-30 20:11 ` [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder() Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-05-04 22:33   ` Alexander Duyck
  2020-04-30 20:11 ` [PATCH 7/7] padata: document multithreaded jobs Daniel Jordan
                   ` (2 subsequent siblings)
  8 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	Daniel Jordan

Deferred struct page init uses one thread per node, which is a
significant bottleneck at boot for big machines--often the largest.
Parallelize to reduce system downtime.

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 MAX_ORDER_NR_PAGES-aligned ranges to avoid
accessing uninitialized buddy pages, so set the job's alignment
accordingly.

The minimum chunk size is also MAX_ORDER_NR_PAGES because there was
benefit to using multiple threads even on relatively small memory (1G)
systems.

    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
                   ------------------------    ------------------------
                   speedup  time_ms (stdev)    speedup  time_ms (stdev)
         base           --   4056.7 (  5.5)         --   1763.3 (  4.2)
         test        39.9%   2436.7 (  2.1)      91.8%    144.3 (  5.9)

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

                   kernel boot                 deferred init
                   ------------------------    ------------------------
                   speedup  time_ms (stdev)    speedup  time_ms (stdev)
         base           --   1957.3 ( 14.0)         --   1093.7 ( 12.9)
         test        49.1%    996.0 (  7.2)      88.4%    127.3 (  5.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
                   ------------------------    ------------------------
                   speedup  time_ms (stdev)    speedup  time_ms (stdev)
         base           --   1666.0 (  3.5)         --    618.0 (  3.5)
         test        31.3%   1145.3 (  1.5)      85.6%     89.0 (  1.7)

    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
                   ------------------------    ------------------------
                   speedup  time_ms (stdev)    speedup  time_ms (stdev)
         base           --   1029.7 ( 42.3)         --    253.7 (  3.1)
         test        23.3%    789.3 ( 15.0)      76.3%     60.0 (  5.6)

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
                   ------------------------    ------------------------
                   speedup  time_ms (stdev)    speedup  time_ms (stdev)
         base           --    757.7 ( 17.1)         --     57.0 (  0.0)
         test         6.2%    710.3 ( 15.0)      63.2%     21.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
                   ------------------------    ------------------------
                   speedup  time_ms (stdev)    speedup  time_ms (stdev)
         base           --    656.3 (  7.1)         --     57.3 (  1.5)
         test         8.6%    599.7 (  5.9)      62.8%     21.3 (  1.2)

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 mm/Kconfig      |  6 +++---
 mm/page_alloc.c | 46 ++++++++++++++++++++++++++++++++++++++--------
 2 files changed, 41 insertions(+), 11 deletions(-)

diff --git a/mm/Kconfig b/mm/Kconfig
index ab80933be65ff..e5007206c7601 100644
--- a/mm/Kconfig
+++ b/mm/Kconfig
@@ -622,13 +622,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 990514d8f0d94..96d6d0d920c27 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>
@@ -1729,6 +1730,25 @@ deferred_init_maxorder(struct zone *zone, unsigned long *start_pfn,
 	return nr_pages;
 }
 
+struct def_init_args {
+	struct zone *zone;
+	atomic_long_t nr_pages;
+};
+
+static void __init deferred_init_memmap_chunk(unsigned long spfn,
+					      unsigned long epfn, void *arg)
+{
+	struct def_init_args *args = arg;
+	unsigned long nr_pages = 0;
+
+	while (spfn < epfn) {
+		nr_pages += deferred_init_maxorder(args->zone, &spfn, epfn);
+		cond_resched();
+	}
+
+	atomic_long_add(nr_pages, &args->nr_pages);
+}
+
 /* Initialise remaining memory on a node */
 static int __init deferred_init_memmap(void *data)
 {
@@ -1738,7 +1758,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 */
@@ -1778,15 +1798,25 @@ 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);
+
 	for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
-		while (spfn < epfn) {
-			nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
-			cond_resched();
-		}
+		struct def_init_args args = { zone, ATOMIC_LONG_INIT(0) };
+		struct padata_mt_job job = {
+			.thread_fn   = deferred_init_memmap_chunk,
+			.fn_arg      = &args,
+			.start       = spfn,
+			.size	     = epfn - spfn,
+			.align	     = MAX_ORDER_NR_PAGES,
+			.min_chunk   = MAX_ORDER_NR_PAGES,
+			.max_threads = max_threads,
+		};
+
+		padata_do_multithreaded(&job);
+		nr_pages += atomic_long_read(&args.nr_pages);
 	}
 zone_empty:
 	/* Sanity check that the next zone really is unpopulated */
-- 
2.26.2



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

* [PATCH 7/7] padata: document multithreaded jobs
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
                   ` (5 preceding siblings ...)
  2020-04-30 20:11 ` [PATCH 6/7] mm: parallelize deferred_init_memmap() Daniel Jordan
@ 2020-04-30 20:11 ` Daniel Jordan
  2020-04-30 21:31 ` [PATCH 0/7] padata: parallelize deferred page init Andrew Morton
  2020-05-01  1:09 ` Josh Triplett
  8 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-04-30 20:11 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, linux-kernel,
	Daniel Jordan

Add Documentation for multithreaded jobs.

Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
---
 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..b7e047af993e8 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: April 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 parallel 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 a stack-allocated structure that wraps the required data.  Last, call
+padata_do_multithreaded(), which will return once the job is finished.
+
 Interface
 =========
 
-- 
2.26.2



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

* Re: [PATCH 0/7] padata: parallelize deferred page init
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
                   ` (6 preceding siblings ...)
  2020-04-30 20:11 ` [PATCH 7/7] padata: document multithreaded jobs Daniel Jordan
@ 2020-04-30 21:31 ` Andrew Morton
  2020-04-30 21:40   ` Pavel Tatashin
  2020-05-01  0:50   ` Josh Triplett
  2020-05-01  1:09 ` Josh Triplett
  8 siblings, 2 replies; 35+ messages in thread
From: Andrew Morton @ 2020-04-30 21:31 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: 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,
	Shile Zhang, Tejun Heo, Zi Yan, linux-crypto, linux-mm,
	linux-kernel

On Thu, 30 Apr 2020 16:11:18 -0400 Daniel Jordan <daniel.m.jordan@oracle.com> wrote:

> 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, and this series is the
> first step.
> 
> It extends 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 7.
> 
> The first user is deferred struct page init, a large bottleneck in
> kernel boot--actually the largest for us and likely others too.  This
> path doesn't require concurrency limits, resource control, or priority
> adjustments like future users will (vfio, hugetlb fallocate, munmap)
> because it happens during boot when the system is otherwise idle and
> waiting on page init to finish.
> 
> This has been tested on a variety of x86 systems and speeds up kernel
> boot by 6% to 49% by making deferred init 63% to 91% faster.

How long is this up-to-91% in seconds?  If it's 91% of a millisecond
then not impressed.  If it's 91% of two weeks then better :)

Relatedly, how important is boot time on these large machines anyway? 
They presumably have lengthy uptimes so boot time is relatively
unimportant?

IOW, can you please explain more fully why this patchset is valuable to
our users?


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

* Re: [PATCH 0/7] padata: parallelize deferred page init
  2020-04-30 21:31 ` [PATCH 0/7] padata: parallelize deferred page init Andrew Morton
@ 2020-04-30 21:40   ` Pavel Tatashin
  2020-05-01  2:40     ` Daniel Jordan
  2020-05-01  0:50   ` Josh Triplett
  1 sibling, 1 reply; 35+ messages in thread
From: Pavel Tatashin @ 2020-04-30 21:40 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Daniel Jordan, 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, Peter Zijlstra, Randy Dunlap,
	Shile Zhang, Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On Thu, Apr 30, 2020 at 5:31 PM Andrew Morton <akpm@linux-foundation.org> wrote:
>
> On Thu, 30 Apr 2020 16:11:18 -0400 Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
>
> > 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, and this series is the
> > first step.
> >
> > It extends 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 7.
> >
> > The first user is deferred struct page init, a large bottleneck in
> > kernel boot--actually the largest for us and likely others too.  This
> > path doesn't require concurrency limits, resource control, or priority
> > adjustments like future users will (vfio, hugetlb fallocate, munmap)
> > because it happens during boot when the system is otherwise idle and
> > waiting on page init to finish.
> >
> > This has been tested on a variety of x86 systems and speeds up kernel
> > boot by 6% to 49% by making deferred init 63% to 91% faster.
>
> How long is this up-to-91% in seconds?  If it's 91% of a millisecond
> then not impressed.  If it's 91% of two weeks then better :)
>
> Relatedly, how important is boot time on these large machines anyway?
> They presumably have lengthy uptimes so boot time is relatively
> unimportant?

Large machines indeed have a lengthy uptime, but they also can host a
large number of VMs meaning that downtime of the host increases the
downtime of VMs in cloud environments. Some VMs might be very sensible
to downtime: game servers, traders, etc.

>
> IOW, can you please explain more fully why this patchset is valuable to
> our users?


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-04-30 20:11 ` [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder() Daniel Jordan
@ 2020-04-30 21:43   ` Alexander Duyck
  2020-05-01  2:45     ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-04-30 21:43 UTC (permalink / raw)
  To: Daniel Jordan, Andrew Morton, Herbert Xu, Steffen Klassert
  Cc: Alex Williamson, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel

On 4/30/2020 1:11 PM, Daniel Jordan wrote:
> padata will soon divide up pfn ranges between threads when parallelizing
> deferred init, and deferred_init_maxorder() complicates that by using an
> opaque index in addition to start and end pfns.  Move the index outside
> the function to make splitting the job easier, and simplify the code
> while at it.
> 
> deferred_init_maxorder() now always iterates within a single pfn range
> instead of potentially multiple ranges, and advances start_pfn to the
> end of that range instead of the max-order block so partial pfn ranges
> in the block aren't skipped in a later iteration.  The section alignment
> check in deferred_grow_zone() is removed as well since this alignment is
> no longer guaranteed.  It's not clear what value the alignment provided
> originally.
> 
> Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>

So part of the reason for splitting it up along section aligned 
boundaries was because we already had an existing functionality in 
deferred_grow_zone that was going in and pulling out a section aligned 
chunk and processing it to prepare enough memory for other threads to 
keep running. I suspect that the section alignment was done because 
normally I believe that is also the alignment for memory onlining.

With this already breaking things up over multiple threads how does this 
work with deferred_grow_zone? Which thread is it trying to allocate from 
if it needs to allocate some memory for itself?

Also what is to prevent a worker from stop deferred_grow_zone from 
bailing out in the middle of a max order page block if there is a hole 
in the middle of the block?

> ---
>   mm/page_alloc.c | 88 +++++++++++++++----------------------------------
>   1 file changed, 27 insertions(+), 61 deletions(-)
> 
> diff --git a/mm/page_alloc.c b/mm/page_alloc.c
> index 68669d3a5a665..990514d8f0d94 100644
> --- a/mm/page_alloc.c
> +++ b/mm/page_alloc.c
> @@ -1708,55 +1708,23 @@ deferred_init_mem_pfn_range_in_zone(u64 *i, struct zone *zone,
>   }
>   
>   /*
> - * Initialize and free pages. We do it in two loops: first we initialize
> - * struct page, then free to buddy allocator, because while we are
> - * freeing pages we can access pages that are ahead (computing buddy
> - * page in __free_one_page()).
> - *
> - * In order to try and keep some memory in the cache we have the loop
> - * broken along max page order boundaries. This way we will not cause
> - * any issues with the buddy page computation.
> + * Initialize the struct pages and then free them to the buddy allocator at
> + * most a max order block at a time because while we are freeing pages we can
> + * access pages that are ahead (computing buddy page in __free_one_page()).
> + * It's also cache friendly.
>    */
>   static unsigned long __init
> -deferred_init_maxorder(u64 *i, struct zone *zone, unsigned long *start_pfn,
> -		       unsigned long *end_pfn)
> +deferred_init_maxorder(struct zone *zone, unsigned long *start_pfn,
> +		       unsigned long end_pfn)
>   {
> -	unsigned long mo_pfn = ALIGN(*start_pfn + 1, MAX_ORDER_NR_PAGES);
> -	unsigned long spfn = *start_pfn, epfn = *end_pfn;
> -	unsigned long nr_pages = 0;
> -	u64 j = *i;
> -
> -	/* First we loop through and initialize the page values */
> -	for_each_free_mem_pfn_range_in_zone_from(j, zone, start_pfn, end_pfn) {
> -		unsigned long t;
> -
> -		if (mo_pfn <= *start_pfn)
> -			break;
> -
> -		t = min(mo_pfn, *end_pfn);
> -		nr_pages += deferred_init_pages(zone, *start_pfn, t);
> -
> -		if (mo_pfn < *end_pfn) {
> -			*start_pfn = mo_pfn;
> -			break;
> -		}
> -	}
> -
> -	/* Reset values and now loop through freeing pages as needed */
> -	swap(j, *i);
> -
> -	for_each_free_mem_pfn_range_in_zone_from(j, zone, &spfn, &epfn) {
> -		unsigned long t;
> -
> -		if (mo_pfn <= spfn)
> -			break;
> +	unsigned long nr_pages, pfn;
>   
> -		t = min(mo_pfn, epfn);
> -		deferred_free_pages(spfn, t);
> +	pfn = ALIGN(*start_pfn + 1, MAX_ORDER_NR_PAGES);
> +	pfn = min(pfn, end_pfn);
>   
> -		if (mo_pfn <= epfn)
> -			break;
> -	}
> +	nr_pages = deferred_init_pages(zone, *start_pfn, pfn);
> +	deferred_free_pages(*start_pfn, pfn);
> +	*start_pfn = pfn;
>   
>   	return nr_pages;
>   }
> @@ -1814,9 +1782,11 @@ static int __init deferred_init_memmap(void *data)
>   	 * that we can avoid introducing any issues with the buddy
>   	 * allocator.
>   	 */
> -	while (spfn < epfn) {
> -		nr_pages += deferred_init_maxorder(&i, zone, &spfn, &epfn);
> -		cond_resched();
> +	for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
> +		while (spfn < epfn) {
> +			nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
> +			cond_resched();
> +		}
>   	}
>   zone_empty:
>   	/* Sanity check that the next zone really is unpopulated */
> @@ -1883,22 +1853,18 @@ deferred_grow_zone(struct zone *zone, unsigned int order)
>   	 * that we can avoid introducing any issues with the buddy
>   	 * allocator.
>   	 */
> -	while (spfn < epfn) {
> -		/* update our first deferred PFN for this section */
> -		first_deferred_pfn = spfn;
> -
> -		nr_pages += deferred_init_maxorder(&i, zone, &spfn, &epfn);
> -		touch_nmi_watchdog();
> -
> -		/* We should only stop along section boundaries */
> -		if ((first_deferred_pfn ^ spfn) < PAGES_PER_SECTION)
> -			continue;
> -
> -		/* If our quota has been met we can stop here */
> -		if (nr_pages >= nr_pages_needed)
> -			break;
> +	for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
> +		while (spfn < epfn) {
> +			nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
> +			touch_nmi_watchdog();
> +
> +			/* If our quota has been met we can stop here */
> +			if (nr_pages >= nr_pages_needed)
> +				goto out;
> +		}
>   	}
>   
> +out:
>   	pgdat->first_deferred_pfn = spfn;
>   	pgdat_resize_unlock(pgdat, &flags);
>   
> 


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

* Re: [PATCH 0/7] padata: parallelize deferred page init
  2020-04-30 21:31 ` [PATCH 0/7] padata: parallelize deferred page init Andrew Morton
  2020-04-30 21:40   ` Pavel Tatashin
@ 2020-05-01  0:50   ` Josh Triplett
  1 sibling, 0 replies; 35+ messages in thread
From: Josh Triplett @ 2020-05-01  0:50 UTC (permalink / raw)
  To: Andrew Morton
  Cc: Daniel Jordan, Herbert Xu, Steffen Klassert, Alex Williamson,
	Alexander Duyck, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Kirill Tkhai, Michal Hocko,
	Pavel Machek, Pavel Tatashin, Peter Zijlstra, Randy Dunlap,
	Shile Zhang, Tejun Heo, Zi Yan, linux-crypto, linux-mm,
	linux-kernel

On Thu, Apr 30, 2020 at 02:31:31PM -0700, Andrew Morton wrote:
> On Thu, 30 Apr 2020 16:11:18 -0400 Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
> > 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, and this series is the
> > first step.
> > 
> > It extends 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 7.
> > 
> > The first user is deferred struct page init, a large bottleneck in
> > kernel boot--actually the largest for us and likely others too.  This
> > path doesn't require concurrency limits, resource control, or priority
> > adjustments like future users will (vfio, hugetlb fallocate, munmap)
> > because it happens during boot when the system is otherwise idle and
> > waiting on page init to finish.
> > 
> > This has been tested on a variety of x86 systems and speeds up kernel
> > boot by 6% to 49% by making deferred init 63% to 91% faster.
> 
> How long is this up-to-91% in seconds?  If it's 91% of a millisecond
> then not impressed.  If it's 91% of two weeks then better :)

Some test results on a system with 96 CPUs and 192GB of memory:

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.227868] node 0 initialised, 23398907 pages in 28ms
[    0.230019] node 1 initialised, 24189223 pages in 28ms
...
[    0.361069] Run /sbin/init as init process

That makes a huge difference; memory initialization is the largest
remaining component of boot time.

> Relatedly, how important is boot time on these large machines anyway? 
> They presumably have lengthy uptimes so boot time is relatively
> unimportant?

Cloud systems and other virtual machines may have a huge amount of
memory but not necessarily run for a long time; on such systems, boot
time becomes critically important. Reducing boot time on cloud systems
and VMs makes the difference between "leave running to reduce latency"
and "just boot up when needed".

- Josh Triplett


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

* Re: [PATCH 0/7] padata: parallelize deferred page init
  2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
                   ` (7 preceding siblings ...)
  2020-04-30 21:31 ` [PATCH 0/7] padata: parallelize deferred page init Andrew Morton
@ 2020-05-01  1:09 ` Josh Triplett
  2020-05-01  2:48   ` Daniel Jordan
  8 siblings, 1 reply; 35+ messages in thread
From: Josh Triplett @ 2020-05-01  1:09 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, Kirill Tkhai, Michal Hocko,
	Pavel Machek, Pavel Tatashin, Peter Zijlstra, Randy Dunlap,
	Shile Zhang, Tejun Heo, Zi Yan, linux-crypto, linux-mm,
	linux-kernel

On Thu, Apr 30, 2020 at 04:11:18PM -0400, Daniel Jordan wrote:
> 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, and this series is the
> first step.
> 
> It extends 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 7.
> 
> The first user is deferred struct page init, a large bottleneck in
> kernel boot--actually the largest for us and likely others too.  This
> path doesn't require concurrency limits, resource control, or priority
> adjustments like future users will (vfio, hugetlb fallocate, munmap)
> because it happens during boot when the system is otherwise idle and
> waiting on page init to finish.
> 
> This has been tested on a variety of x86 systems and speeds up kernel
> boot by 6% to 49% by making deferred init 63% to 91% faster.  Patch 6
> has detailed numbers.  Test results from other systems appreciated.
> 
> This series is based on v5.6 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
> 
> All of the above can be found in this branch:
> 
>   git://oss.oracle.com/git/linux-dmjordan.git padata-mt-definit-v1
>   https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=shortlog;h=refs/heads/padata-mt-definit-v1

For the series (and the three prerequisite patches):

Tested-by: Josh Triplett <josh@joshtriplett.org>

Thank you for writing this, and thank you for working towards
upstreaming it!


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

* Re: [PATCH 0/7] padata: parallelize deferred page init
  2020-04-30 21:40   ` Pavel Tatashin
@ 2020-05-01  2:40     ` Daniel Jordan
  0 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-05-01  2:40 UTC (permalink / raw)
  To: Pavel Tatashin
  Cc: Andrew Morton, Daniel Jordan, 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,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Thu, Apr 30, 2020 at 05:40:59PM -0400, Pavel Tatashin wrote:
> On Thu, Apr 30, 2020 at 5:31 PM Andrew Morton <akpm@linux-foundation.org> wrote:
> > On Thu, 30 Apr 2020 16:11:18 -0400 Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
> >
> > > 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, and this series is the
> > > first step.
> > >
> > > It extends 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 7.
> > >
> > > The first user is deferred struct page init, a large bottleneck in
> > > kernel boot--actually the largest for us and likely others too.  This
> > > path doesn't require concurrency limits, resource control, or priority
> > > adjustments like future users will (vfio, hugetlb fallocate, munmap)
> > > because it happens during boot when the system is otherwise idle and
> > > waiting on page init to finish.
> > >
> > > This has been tested on a variety of x86 systems and speeds up kernel
> > > boot by 6% to 49% by making deferred init 63% to 91% faster.
> >
> > How long is this up-to-91% in seconds?  If it's 91% of a millisecond
> > then not impressed.  If it's 91% of two weeks then better :)

The largest system I could test had 384G per node and saved 1.5 out of 4
seconds.

> > Relatedly, how important is boot time on these large machines anyway?
> > They presumably have lengthy uptimes so boot time is relatively
> > unimportant?
> 
> Large machines indeed have a lengthy uptime, but they also can host a
> large number of VMs meaning that downtime of the host increases the
> downtime of VMs in cloud environments. Some VMs might be very sensible
> to downtime: game servers, traders, etc.
>
> > IOW, can you please explain more fully why this patchset is valuable to
> > our users?

I'll let the users speak for themselves, but I have a similar use case to Pavel
of limiting the downtime of VMs running on these large systems, and spinning up
instances as fast as possible is also desirable for our cloud users.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-04-30 21:43   ` Alexander Duyck
@ 2020-05-01  2:45     ` Daniel Jordan
  2020-05-04 22:10       ` Alexander Duyck
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-01  2:45 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, linux-kernel

Hi Alex,

On Thu, Apr 30, 2020 at 02:43:28PM -0700, Alexander Duyck wrote:
> On 4/30/2020 1:11 PM, Daniel Jordan wrote:
> > padata will soon divide up pfn ranges between threads when parallelizing
> > deferred init, and deferred_init_maxorder() complicates that by using an
> > opaque index in addition to start and end pfns.  Move the index outside
> > the function to make splitting the job easier, and simplify the code
> > while at it.
> > 
> > deferred_init_maxorder() now always iterates within a single pfn range
> > instead of potentially multiple ranges, and advances start_pfn to the
> > end of that range instead of the max-order block so partial pfn ranges
> > in the block aren't skipped in a later iteration.  The section alignment
> > check in deferred_grow_zone() is removed as well since this alignment is
> > no longer guaranteed.  It's not clear what value the alignment provided
> > originally.
> > 
> > Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
> 
> So part of the reason for splitting it up along section aligned boundaries
> was because we already had an existing functionality in deferred_grow_zone
> that was going in and pulling out a section aligned chunk and processing it
> to prepare enough memory for other threads to keep running. I suspect that
> the section alignment was done because normally I believe that is also the
> alignment for memory onlining.

I think Pavel added that functionality, maybe he could confirm.

My impression was that the reason deferred_grow_zone aligned the requested
order up to a section was to make enough memory available to avoid being called
on every allocation.

> With this already breaking things up over multiple threads how does this
> work with deferred_grow_zone? Which thread is it trying to allocate from if
> it needs to allocate some memory for itself?

I may not be following your question, but deferred_grow_zone doesn't allocate
memory during the multithreading in deferred_init_memmap because the latter
sets first_deferred_pfn so that deferred_grow_zone bails early.

> Also what is to prevent a worker from stop deferred_grow_zone from bailing
> out in the middle of a max order page block if there is a hole in the middle
> of the block?

deferred_grow_zone remains singlethreaded.  It could stop in the middle of a
max order block, but it can't run concurrently with deferred_init_memmap, as
per above, so if deferred_init_memmap were to init 'n free the remaining part
of the block, the previous portion would have already been initialized.


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

* Re: [PATCH 0/7] padata: parallelize deferred page init
  2020-05-01  1:09 ` Josh Triplett
@ 2020-05-01  2:48   ` Daniel Jordan
  0 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-05-01  2:48 UTC (permalink / raw)
  To: Josh Triplett
  Cc: Daniel Jordan, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, linux-kernel

On Thu, Apr 30, 2020 at 06:09:35PM -0700, Josh Triplett wrote:
> On Thu, Apr 30, 2020 at 04:11:18PM -0400, Daniel Jordan wrote:
> > 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, and this series is the
> > first step.
> > 
> > It extends 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 7.
> > 
> > The first user is deferred struct page init, a large bottleneck in
> > kernel boot--actually the largest for us and likely others too.  This
> > path doesn't require concurrency limits, resource control, or priority
> > adjustments like future users will (vfio, hugetlb fallocate, munmap)
> > because it happens during boot when the system is otherwise idle and
> > waiting on page init to finish.
> > 
> > This has been tested on a variety of x86 systems and speeds up kernel
> > boot by 6% to 49% by making deferred init 63% to 91% faster.  Patch 6
> > has detailed numbers.  Test results from other systems appreciated.
> > 
> > This series is based on v5.6 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
> > 
> > All of the above can be found in this branch:
> > 
> >   git://oss.oracle.com/git/linux-dmjordan.git padata-mt-definit-v1
> >   https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=shortlog;h=refs/heads/padata-mt-definit-v1
> 
> For the series (and the three prerequisite patches):
> 
> Tested-by: Josh Triplett <josh@joshtriplett.org>

Appreciate the runs, Josh, thanks.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-01  2:45     ` Daniel Jordan
@ 2020-05-04 22:10       ` Alexander Duyck
  2020-05-05  0:54         ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-04 22:10 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, LKML

On Thu, Apr 30, 2020 at 7:45 PM Daniel Jordan
<daniel.m.jordan@oracle.com> wrote:
>
> Hi Alex,
>
> On Thu, Apr 30, 2020 at 02:43:28PM -0700, Alexander Duyck wrote:
> > On 4/30/2020 1:11 PM, Daniel Jordan wrote:
> > > padata will soon divide up pfn ranges between threads when parallelizing
> > > deferred init, and deferred_init_maxorder() complicates that by using an
> > > opaque index in addition to start and end pfns.  Move the index outside
> > > the function to make splitting the job easier, and simplify the code
> > > while at it.
> > >
> > > deferred_init_maxorder() now always iterates within a single pfn range
> > > instead of potentially multiple ranges, and advances start_pfn to the
> > > end of that range instead of the max-order block so partial pfn ranges
> > > in the block aren't skipped in a later iteration.  The section alignment
> > > check in deferred_grow_zone() is removed as well since this alignment is
> > > no longer guaranteed.  It's not clear what value the alignment provided
> > > originally.
> > >
> > > Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
> >
> > So part of the reason for splitting it up along section aligned boundaries
> > was because we already had an existing functionality in deferred_grow_zone
> > that was going in and pulling out a section aligned chunk and processing it
> > to prepare enough memory for other threads to keep running. I suspect that
> > the section alignment was done because normally I believe that is also the
> > alignment for memory onlining.
>
> I think Pavel added that functionality, maybe he could confirm.
>
> My impression was that the reason deferred_grow_zone aligned the requested
> order up to a section was to make enough memory available to avoid being called
> on every allocation.
>
> > With this already breaking things up over multiple threads how does this
> > work with deferred_grow_zone? Which thread is it trying to allocate from if
> > it needs to allocate some memory for itself?
>
> I may not be following your question, but deferred_grow_zone doesn't allocate
> memory during the multithreading in deferred_init_memmap because the latter
> sets first_deferred_pfn so that deferred_grow_zone bails early.

It has been a while since I looked at this code so I forgot that
deferred_grow_zone is essentially blocked out once we start the
per-node init.

> > Also what is to prevent a worker from stop deferred_grow_zone from bailing
> > out in the middle of a max order page block if there is a hole in the middle
> > of the block?
>
> deferred_grow_zone remains singlethreaded.  It could stop in the middle of a
> max order block, but it can't run concurrently with deferred_init_memmap, as
> per above, so if deferred_init_memmap were to init 'n free the remaining part
> of the block, the previous portion would have already been initialized.

So we cannot stop in the middle of a max order block. That shouldn't
be possible as part of the issue is that the buddy allocator will
attempt to access the buddy for the page which could cause issues if
it tries to merge the page with one that is not initialized. So if
your code supports that then it is definitely broken. That was one of
the reasons for all of the variable weirdness in
deferred_init_maxorder. I was going through and making certain that
while we were initializing the range we were freeing the pages in
MAX_ORDER aligned blocks and skipping over whatever reserved blocks
were there. Basically it was handling the case where a single
MAX_ORDER block could span multiple ranges.

On x86 this was all pretty straightforward and I don't believe we
needed the code, but I seem to recall there were some other
architectures that had more complex memory layouts at the time and
that was one of the reasons why I had to be careful to wait until I
had processed the full MAX_ORDER block before I could start freeing
the pages, otherwise it would start triggering memory corruptions.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-04-30 20:11 ` [PATCH 6/7] mm: parallelize deferred_init_memmap() Daniel Jordan
@ 2020-05-04 22:33   ` Alexander Duyck
  2020-05-04 23:38     ` Josh Triplett
  2020-05-05  1:26     ` Daniel Jordan
  0 siblings, 2 replies; 35+ messages in thread
From: Alexander Duyck @ 2020-05-04 22:33 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, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, LKML

On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
<daniel.m.jordan@oracle.com> wrote:
>
> Deferred struct page init uses one thread per node, which is a
> significant bottleneck at boot for big machines--often the largest.
> Parallelize to reduce system downtime.
>
> 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 MAX_ORDER_NR_PAGES-aligned ranges to avoid
> accessing uninitialized buddy pages, so set the job's alignment
> accordingly.
>
> The minimum chunk size is also MAX_ORDER_NR_PAGES because there was
> benefit to using multiple threads even on relatively small memory (1G)
> systems.
>
>     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
>                    ------------------------    ------------------------
>                    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>          base           --   4056.7 (  5.5)         --   1763.3 (  4.2)
>          test        39.9%   2436.7 (  2.1)      91.8%    144.3 (  5.9)
>
>     Intel(R) Xeon(R) CPU E5-2699C v4 @ 2.20GHz (Broadwell, bare metal)
>       1 node * 16 cores * 2 threads = 32 CPUs
>       192G/node = 192G memory
>
>                    kernel boot                 deferred init
>                    ------------------------    ------------------------
>                    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>          base           --   1957.3 ( 14.0)         --   1093.7 ( 12.9)
>          test        49.1%    996.0 (  7.2)      88.4%    127.3 (  5.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
>                    ------------------------    ------------------------
>                    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>          base           --   1666.0 (  3.5)         --    618.0 (  3.5)
>          test        31.3%   1145.3 (  1.5)      85.6%     89.0 (  1.7)
>
>     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
>                    ------------------------    ------------------------
>                    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>          base           --   1029.7 ( 42.3)         --    253.7 (  3.1)
>          test        23.3%    789.3 ( 15.0)      76.3%     60.0 (  5.6)
>
> 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
>                    ------------------------    ------------------------
>                    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>          base           --    757.7 ( 17.1)         --     57.0 (  0.0)
>          test         6.2%    710.3 ( 15.0)      63.2%     21.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
>                    ------------------------    ------------------------
>                    speedup  time_ms (stdev)    speedup  time_ms (stdev)
>          base           --    656.3 (  7.1)         --     57.3 (  1.5)
>          test         8.6%    599.7 (  5.9)      62.8%     21.3 (  1.2)
>
> Signed-off-by: Daniel Jordan <daniel.m.jordan@oracle.com>
> ---
>  mm/Kconfig      |  6 +++---
>  mm/page_alloc.c | 46 ++++++++++++++++++++++++++++++++++++++--------
>  2 files changed, 41 insertions(+), 11 deletions(-)
>
> diff --git a/mm/Kconfig b/mm/Kconfig
> index ab80933be65ff..e5007206c7601 100644
> --- a/mm/Kconfig
> +++ b/mm/Kconfig
> @@ -622,13 +622,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 990514d8f0d94..96d6d0d920c27 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>
> @@ -1729,6 +1730,25 @@ deferred_init_maxorder(struct zone *zone, unsigned long *start_pfn,
>         return nr_pages;
>  }
>
> +struct def_init_args {
> +       struct zone *zone;
> +       atomic_long_t nr_pages;
> +};
> +
> +static void __init deferred_init_memmap_chunk(unsigned long spfn,
> +                                             unsigned long epfn, void *arg)
> +{
> +       struct def_init_args *args = arg;
> +       unsigned long nr_pages = 0;
> +
> +       while (spfn < epfn) {
> +               nr_pages += deferred_init_maxorder(args->zone, &spfn, epfn);
> +               cond_resched();
> +       }
> +
> +       atomic_long_add(nr_pages, &args->nr_pages);
> +}
> +
>  /* Initialise remaining memory on a node */
>  static int __init deferred_init_memmap(void *data)
>  {
> @@ -1738,7 +1758,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 */
> @@ -1778,15 +1798,25 @@ 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.
>          */

I would be curious about your data. That isn't what I have seen in the
past. Typically only up to about 8 or 10 CPUs gives you any benefit,
beyond that I was usually cache/memory bandwidth bound.

> +       max_threads = max(cpumask_weight(cpumask), 1u);
> +

We will need to gather data on if having a ton of threads works for
all architectures. For x86 I think we are freeing back pages in
pageblock_order sized chunks so we only have to touch them once in
initialize and then free the two pageblock_order chunks into the buddy
allocator.

>         for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
> -               while (spfn < epfn) {
> -                       nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
> -                       cond_resched();
> -               }
> +               struct def_init_args args = { zone, ATOMIC_LONG_INIT(0) };
> +               struct padata_mt_job job = {
> +                       .thread_fn   = deferred_init_memmap_chunk,
> +                       .fn_arg      = &args,
> +                       .start       = spfn,
> +                       .size        = epfn - spfn,
> +                       .align       = MAX_ORDER_NR_PAGES,
> +                       .min_chunk   = MAX_ORDER_NR_PAGES,
> +                       .max_threads = max_threads,
> +               };
> +
> +               padata_do_multithreaded(&job);
> +               nr_pages += atomic_long_read(&args.nr_pages);
>         }
>  zone_empty:
>         /* Sanity check that the next zone really is unpopulated */

Okay so looking at this I can see why you wanted to structure the
other patch the way you did. However I am not sure that is the best
way to go about doing it. It might make more sense to go through and
accumulate sections. If you hit the end of a range and the start of
the next range is in another section, then you split it as a new job,
otherwise I would just accumulate it into the current job. You then
could section align the work and be more or less guaranteed that each
worker thread should be generating finished work products, and not
incomplete max order pages.

That solution would work with the existing code as well since you
could basically just compare the start pfn coming out of the
deferred_init_maxorder versus the end of the chunk to determine if you
should exit or not.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-04 22:33   ` Alexander Duyck
@ 2020-05-04 23:38     ` Josh Triplett
  2020-05-05  0:40       ` Alexander Duyck
  2020-05-05  1:26     ` Daniel Jordan
  1 sibling, 1 reply; 35+ messages in thread
From: Josh Triplett @ 2020-05-04 23:38 UTC (permalink / raw)
  To: Alexander Duyck, Daniel Jordan
  Cc: Andrew Morton, Herbert Xu, Steffen Klassert, Alex Williamson,
	Alexander Duyck, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Kirill Tkhai, Michal Hocko,
	Pavel Machek, Pavel Tatashin, Peter Zijlstra, Randy Dunlap,
	Shile Zhang, Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On May 4, 2020 3:33:58 PM PDT, Alexander Duyck <alexander.duyck@gmail.com> wrote:
>On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
><daniel.m.jordan@oracle.com> wrote:
>>         /*
>> -        * 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.
>>          */
>
>I would be curious about your data. That isn't what I have seen in the
>past. Typically only up to about 8 or 10 CPUs gives you any benefit,
>beyond that I was usually cache/memory bandwidth bound.

I've found pretty much linear performance up to memory bandwidth, and on the systems I was testing, I didn't saturate memory bandwidth until about the full number of physical cores. From number of cores up to number of threads, the performance stayed about flat; it didn't get any better or worse.

- Josh


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-04 23:38     ` Josh Triplett
@ 2020-05-05  0:40       ` Alexander Duyck
  2020-05-05  1:48         ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-05  0:40 UTC (permalink / raw)
  To: Josh Triplett
  Cc: Daniel Jordan, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Mon, May 4, 2020 at 4:44 PM Josh Triplett <josh@joshtriplett.org> wrote:
>
> On May 4, 2020 3:33:58 PM PDT, Alexander Duyck <alexander.duyck@gmail.com> wrote:
> >On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
> ><daniel.m.jordan@oracle.com> wrote:
> >>         /*
> >> -        * 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.
> >>          */
> >
> >I would be curious about your data. That isn't what I have seen in the
> >past. Typically only up to about 8 or 10 CPUs gives you any benefit,
> >beyond that I was usually cache/memory bandwidth bound.
>
> I've found pretty much linear performance up to memory bandwidth, and on the systems I was testing, I didn't saturate memory bandwidth until about the full number of physical cores. From number of cores up to number of threads, the performance stayed about flat; it didn't get any better or worse.

That doesn't sound right though based on the numbers you provided. The
system you had was 192GB spread over 2 nodes with 48thread/24core per
node, correct? Your numbers went from ~290ms to ~28ms so a 10x
decrease, that doesn't sound linear when you spread the work over 24
cores to get there. I agree that the numbers largely stay flat once
you hit the peak, I have seen similar behavior when I was working on
the deferred init code previously. One concern I have though is that
we may end up seeing better performance with a subset of cores instead
of running all of the cores/threads, especially if features such as
turbo come into play. In addition we are talking x86 only so far. I
would be interested in seeing if this has benefits or not for other
architectures.

Also what is the penalty that is being paid in order to break up the
work before-hand and set it up for the parallel work? I would be
interested in seeing what the cost is on a system with fewer cores per
node, maybe even down to 1. That would tell us how much additional
overhead is being added to set things up to run in parallel. If I get
a chance tomorrow I might try applying the patches and doing some
testing myself.

Thanks.

- Alex


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-04 22:10       ` Alexander Duyck
@ 2020-05-05  0:54         ` Daniel Jordan
  2020-05-05 15:27           ` Alexander Duyck
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-05  0:54 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Alexander Duyck, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On Mon, May 04, 2020 at 03:10:46PM -0700, Alexander Duyck wrote:
> So we cannot stop in the middle of a max order block. That shouldn't
> be possible as part of the issue is that the buddy allocator will
> attempt to access the buddy for the page which could cause issues if
> it tries to merge the page with one that is not initialized. So if
> your code supports that then it is definitely broken. That was one of
> the reasons for all of the variable weirdness in
> deferred_init_maxorder. I was going through and making certain that
> while we were initializing the range we were freeing the pages in
> MAX_ORDER aligned blocks and skipping over whatever reserved blocks
> were there. Basically it was handling the case where a single
> MAX_ORDER block could span multiple ranges.
> 
> On x86 this was all pretty straightforward and I don't believe we
> needed the code, but I seem to recall there were some other
> architectures that had more complex memory layouts at the time and
> that was one of the reasons why I had to be careful to wait until I
> had processed the full MAX_ORDER block before I could start freeing
> the pages, otherwise it would start triggering memory corruptions.

Yes, thanks, I missed the case where deferred_grow_zone could stop
mid-max-order-block.

Maybe it's better to leave deferred_init_maxorder alone and adapt the
multithreading to the existing implementation.  That'd mean dealing with the
pesky opaque index somehow, so deferred_init_mem_pfn_range_in_zone() could be
generalized to find it in the thread function based on the start/end range, or
it could be maintained as part of the range that padata passes to the thread
function.

Or, keep this patch but make sure deferred_grow_zone stops on a
max-order-aligned boundary.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-04 22:33   ` Alexander Duyck
  2020-05-04 23:38     ` Josh Triplett
@ 2020-05-05  1:26     ` Daniel Jordan
  1 sibling, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-05-05  1:26 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, 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, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On Mon, May 04, 2020 at 03:33:58PM -0700, Alexander Duyck wrote:
> On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
> > @@ -1778,15 +1798,25 @@ 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.
> >          */
> 
> I would be curious about your data. That isn't what I have seen in the
> past. Typically only up to about 8 or 10 CPUs gives you any benefit,
> beyond that I was usually cache/memory bandwidth bound.

I was surprised too!  For most of its development, this set had an interface to
get the number of cores on the theory that this was about where the bandwidth
got saturated, but the data showed otherwise.

There were diminishing returns, but they were more apparent on Haswell than
Skylake for instance.  I'll post some more data later in the thread where you
guys are talking about it.

> 
> > +       max_threads = max(cpumask_weight(cpumask), 1u);
> > +
> 
> We will need to gather data on if having a ton of threads works for
> all architectures.

Agreed.  I'll rope in some of the arch lists in the next version and include
the debugging knob to vary the thread count.

> For x86 I think we are freeing back pages in
> pageblock_order sized chunks so we only have to touch them once in
> initialize and then free the two pageblock_order chunks into the buddy
> allocator.
> 
> >         for_each_free_mem_pfn_range_in_zone_from(i, zone, &spfn, &epfn) {
> > -               while (spfn < epfn) {
> > -                       nr_pages += deferred_init_maxorder(zone, &spfn, epfn);
> > -                       cond_resched();
> > -               }
> > +               struct def_init_args args = { zone, ATOMIC_LONG_INIT(0) };
> > +               struct padata_mt_job job = {
> > +                       .thread_fn   = deferred_init_memmap_chunk,
> > +                       .fn_arg      = &args,
> > +                       .start       = spfn,
> > +                       .size        = epfn - spfn,
> > +                       .align       = MAX_ORDER_NR_PAGES,
> > +                       .min_chunk   = MAX_ORDER_NR_PAGES,
> > +                       .max_threads = max_threads,
> > +               };
> > +
> > +               padata_do_multithreaded(&job);
> > +               nr_pages += atomic_long_read(&args.nr_pages);
> >         }
> >  zone_empty:
> >         /* Sanity check that the next zone really is unpopulated */
> 
> Okay so looking at this I can see why you wanted to structure the
> other patch the way you did. However I am not sure that is the best
> way to go about doing it. It might make more sense to go through and
> accumulate sections. If you hit the end of a range and the start of
> the next range is in another section, then you split it as a new job,
> otherwise I would just accumulate it into the current job. You then
> could section align the work and be more or less guaranteed that each
> worker thread should be generating finished work products, and not
> incomplete max order pages.

This guarantee holds now with the max-order alignment passed to padata, so I
don't see what more doing it on section boundaries buys us.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-05  0:40       ` Alexander Duyck
@ 2020-05-05  1:48         ` Daniel Jordan
  2020-05-05  2:09           ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-05  1:48 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Josh Triplett, Daniel Jordan, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Alexander Duyck, Dan Williams,
	Dave Hansen, David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Mon, May 04, 2020 at 05:40:19PM -0700, Alexander Duyck wrote:
> On Mon, May 4, 2020 at 4:44 PM Josh Triplett <josh@joshtriplett.org> wrote:
> >
> > On May 4, 2020 3:33:58 PM PDT, Alexander Duyck <alexander.duyck@gmail.com> wrote:
> > >On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
> > ><daniel.m.jordan@oracle.com> wrote:
> > >>         /*
> > >> -        * 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.
> > >>          */
> > >
> > >I would be curious about your data. That isn't what I have seen in the
> > >past. Typically only up to about 8 or 10 CPUs gives you any benefit,
> > >beyond that I was usually cache/memory bandwidth bound.

On Skylake it took more than 8 or 10 CPUs, though on other machines the benefit
of using all versus half or 3/4 of the CPUs is less significant.

Given that the rest of the system is idle at this point, my main concern is
whether other archs regress past a certain thread count.


    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)         --   4056.7 (  5.5)         --   1763.3 (  4.2)
          (  1)      -2.3%   4153.3 (  2.5)      -5.3%   1861.7 (  5.5)
      12% (  6)      53.8%   2637.7 ( 38.7)     408.7%    346.7 ( 37.5)
      25% ( 13)      62.4%   2497.3 ( 38.5)     739.7%    210.0 ( 41.8)
      37% ( 19)      63.8%   2477.0 ( 19.0)     851.4%    185.3 ( 21.5)
      50% ( 26)      64.1%   2471.7 ( 21.4)     881.4%    179.7 ( 25.8)
      75% ( 39)      65.2%   2455.7 ( 33.2)     990.7%    161.7 ( 29.3)
     100% ( 52)      66.5%   2436.7 (  2.1)    1121.7%    144.3 (  5.9)


    Intel(R) Xeon(R) CPU E5-2699C v4 @ 2.20GHz (Broadwell, bare metal)
      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)         --   1957.3 ( 14.0)         --   1093.7 ( 12.9)
          (  1)       1.4%   1930.7 ( 10.0)       3.8%   1053.3 (  7.6)
      12% (  4)      70.0%   1151.7 (  9.0)     292.5%    278.7 (  0.6)
      25% (  8)      86.2%   1051.0 (  7.8)     514.4%    178.0 (  2.6)
      37% ( 12)      95.1%   1003.3 (  7.6)     672.0%    141.7 (  3.8)
      50% ( 16)      93.0%   1014.3 ( 20.0)     720.2%    133.3 (  3.2)
      75% ( 24)      97.8%    989.3 (  6.7)     765.7%    126.3 (  1.5)
     100% ( 32)      96.5%    996.0 (  7.2)     758.9%    127.3 (  5.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)         --   1666.0 (  3.5)         --    618.0 (  3.5)
          (  1)       1.0%   1649.7 (  1.5)       3.0%    600.0 (  1.0)
      12% (  4)      34.9%   1234.7 ( 21.4)     237.7%    183.0 ( 22.5)
      25% (  9)      42.0%   1173.0 ( 10.0)     417.9%    119.3 (  9.6)
      37% ( 13)      44.4%   1153.7 ( 17.0)     524.2%     99.0 ( 15.6)
      50% ( 18)      44.8%   1150.3 ( 15.5)     534.9%     97.3 ( 16.2)
      75% ( 27)      44.8%   1150.3 (  2.5)     550.5%     95.0 (  5.6)
     100% ( 36)      45.5%   1145.3 (  1.5)     594.4%     89.0 (  1.7)
    

    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.7 ( 42.3)         --    253.7 (  3.1)
          (  1)       3.4%    995.3 ( 21.4)       4.5%    242.7 (  5.5)
      12% (  2)      16.3%    885.7 ( 24.4)      86.5%    136.0 (  5.2)
      25% (  4)      23.3%    835.0 ( 21.5)     195.0%     86.0 (  1.7)
      37% (  6)      28.0%    804.7 ( 15.7)     249.1%     72.7 (  2.1)
      50% (  8)      26.3%    815.3 ( 11.7)     290.3%     65.0 (  3.5)
      75% ( 12)      30.7%    787.7 (  2.1)     284.3%     66.0 (  3.6)
     100% ( 16)      30.4%    789.3 ( 15.0)     322.8%     60.0 (  5.6)
    
    
    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)         --    757.7 ( 17.1)         --     57.0 (  0.0)
      25% (  1)      -1.0%    765.3 (  5.5)       3.6%     55.0 (  0.0)
      50% (  2)       4.9%    722.3 ( 21.5)      74.5%     32.7 (  4.6)
      75% (  3)       3.8%    729.7 (  4.9)     119.2%     26.0 (  0.0)
     100% (  4)       6.7%    710.3 ( 15.0)     171.4%     21.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)         --    656.3 (  7.1)         --     57.3 (  1.5)
      25% (  1)       1.8%    644.7 (  3.1)       0.6%     57.0 (  0.0)
      50% (  2)       7.0%    613.7 (  5.1)      68.6%     34.0 (  5.3)
      75% (  3)       7.4%    611.3 (  6.7)     135.6%     24.3 (  0.6)
     100% (  4)       9.4%    599.7 (  5.9)     168.8%     21.3 (  1.2)


> > I've found pretty much linear performance up to memory bandwidth, and on the systems I was testing, I didn't saturate memory bandwidth until about the full number of physical cores. From number of cores up to number of threads, the performance stayed about flat; it didn't get any better or worse.
> 
> That doesn't sound right though based on the numbers you provided. The
> system you had was 192GB spread over 2 nodes with 48thread/24core per
> node, correct? Your numbers went from ~290ms to ~28ms so a 10x
> decrease, that doesn't sound linear when you spread the work over 24
> cores to get there. I agree that the numbers largely stay flat once
> you hit the peak, I have seen similar behavior when I was working on
> the deferred init code previously. One concern I have though is that
> we may end up seeing better performance with a subset of cores instead
> of running all of the cores/threads, especially if features such as
> turbo come into play. In addition we are talking x86 only so far. I
> would be interested in seeing if this has benefits or not for other
> architectures.
> 
> Also what is the penalty that is being paid in order to break up the
> work before-hand and set it up for the parallel work? I would be
> interested in seeing what the cost is on a system with fewer cores per
> node, maybe even down to 1. That would tell us how much additional
> overhead is being added to set things up to run in parallel.

The numbers above have the 1-thread case.  It seems close to the noise.

> If I get
> a chance tomorrow I might try applying the patches and doing some
> testing myself.

If you end up doing that, you might find this helpful:
    https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=patch;h=afc72bf8478b95a1d6d174c269ff3693c60630e0
    
and maybe this:
    https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=patch;h=dff6537eab281e5a9917682c4adf9059c0574223

Thanks for looking this over.

[ By the way, I'm going to be out Tuesday but back the rest of the week. ]


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-05  1:48         ` Daniel Jordan
@ 2020-05-05  2:09           ` Daniel Jordan
  2020-05-05 14:55             ` Alexander Duyck
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-05  2:09 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Josh Triplett, Daniel Jordan, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Alexander Duyck, Dan Williams,
	Dave Hansen, David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Mon, May 04, 2020 at 09:48:44PM -0400, Daniel Jordan wrote:
> On Mon, May 04, 2020 at 05:40:19PM -0700, Alexander Duyck wrote:
> > On Mon, May 4, 2020 at 4:44 PM Josh Triplett <josh@joshtriplett.org> wrote:
> > >
> > > On May 4, 2020 3:33:58 PM PDT, Alexander Duyck <alexander.duyck@gmail.com> wrote:
> > > >On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
> > > ><daniel.m.jordan@oracle.com> wrote:
> > > >>         /*
> > > >> -        * 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.
> > > >>          */
> > > >
> > > >I would be curious about your data. That isn't what I have seen in the
> > > >past. Typically only up to about 8 or 10 CPUs gives you any benefit,
> > > >beyond that I was usually cache/memory bandwidth bound.
> 
> On Skylake it took more than 8 or 10 CPUs, though on other machines the benefit
> of using all versus half or 3/4 of the CPUs is less significant.
> 
> Given that the rest of the system is idle at this point, my main concern is
> whether other archs regress past a certain thread count.

Reposting the data to be consistent with the way the percentages are reported
in the changelog.


    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)         --   4056.7 (  5.5)         --   1763.3 (  4.2)
       2% (  1)      -2.4%   4153.3 (  2.5)      -5.6%   1861.7 (  5.5)
      12% (  6)      35.0%   2637.7 ( 38.7)      80.3%    346.7 ( 37.5)
      25% ( 13)      38.4%   2497.3 ( 38.5)      88.1%    210.0 ( 41.8)
      37% ( 19)      38.9%   2477.0 ( 19.0)      89.5%    185.3 ( 21.5)
      50% ( 26)      39.1%   2471.7 ( 21.4)      89.8%    179.7 ( 25.8)
      75% ( 39)      39.5%   2455.7 ( 33.2)      90.8%    161.7 ( 29.3)
     100% ( 52)      39.9%   2436.7 (  2.1)      91.8%    144.3 (  5.9)
    
    
    Intel(R) Xeon(R) CPU E5-2699C v4 @ 2.20GHz (Broadwell, bare metal)
      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)         --   1957.3 ( 14.0)         --   1093.7 ( 12.9)
       3% (  1)       1.4%   1930.7 ( 10.0)       3.7%   1053.3 (  7.6)
      12% (  4)      41.2%   1151.7 (  9.0)      74.5%    278.7 (  0.6)
      25% (  8)      46.3%   1051.0 (  7.8)      83.7%    178.0 (  2.6)
      38% ( 12)      48.7%   1003.3 (  7.6)      87.0%    141.7 (  3.8)
      50% ( 16)      48.2%   1014.3 ( 20.0)      87.8%    133.3 (  3.2)
      75% ( 24)      49.5%    989.3 (  6.7)      88.4%    126.3 (  1.5)
     100% ( 32)      49.1%    996.0 (  7.2)      88.4%    127.3 (  5.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)         --   1666.0 (  3.5)         --    618.0 (  3.5)
       3% (  1)       1.0%   1649.7 (  1.5)       2.9%    600.0 (  1.0)
      11% (  4)      25.9%   1234.7 ( 21.4)      70.4%    183.0 ( 22.5)
      25% (  9)      29.6%   1173.0 ( 10.0)      80.7%    119.3 (  9.6)
      36% ( 13)      30.8%   1153.7 ( 17.0)      84.0%     99.0 ( 15.6)
      50% ( 18)      31.0%   1150.3 ( 15.5)      84.3%     97.3 ( 16.2)
      75% ( 27)      31.0%   1150.3 (  2.5)      84.6%     95.0 (  5.6)
     100% ( 36)      31.3%   1145.3 (  1.5)      85.6%     89.0 (  1.7)
    
    
    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.7 ( 42.3)         --    253.7 (  3.1)
       6% (  1)       3.3%    995.3 ( 21.4)       4.3%    242.7 (  5.5)
      12% (  2)      14.0%    885.7 ( 24.4)      46.4%    136.0 (  5.2)
      25% (  4)      18.9%    835.0 ( 21.5)      66.1%     86.0 (  1.7)
      38% (  6)      21.9%    804.7 ( 15.7)      71.4%     72.7 (  2.1)
      50% (  8)      20.8%    815.3 ( 11.7)      74.4%     65.0 (  3.5)
      75% ( 12)      23.5%    787.7 (  2.1)      74.0%     66.0 (  3.6)
     100% ( 16)      23.3%    789.3 ( 15.0)      76.3%     60.0 (  5.6)
    
    
    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)         --    757.7 ( 17.1)         --     57.0 (  0.0)
      25% (  1)      -1.0%    765.3 (  5.5)       3.5%     55.0 (  0.0)
      50% (  2)       4.7%    722.3 ( 21.5)      42.7%     32.7 (  4.6)
      75% (  3)       3.7%    729.7 (  4.9)      54.4%     26.0 (  0.0)
     100% (  4)       6.2%    710.3 ( 15.0)      63.2%     21.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)         --    656.3 (  7.1)         --     57.3 (  1.5)
      25% (  1)       1.8%    644.7 (  3.1)       0.6%     57.0 (  0.0)
      50% (  2)       6.5%    613.7 (  5.1)      40.7%     34.0 (  5.3)
      75% (  3)       6.9%    611.3 (  6.7)      57.6%     24.3 (  0.6)
     100% (  4)       8.6%    599.7 (  5.9)      62.8%     21.3 (  1.2)


> > > I've found pretty much linear performance up to memory bandwidth, and on the systems I was testing, I didn't saturate memory bandwidth until about the full number of physical cores. From number of cores up to number of threads, the performance stayed about flat; it didn't get any better or worse.
> > 
> > That doesn't sound right though based on the numbers you provided. The
> > system you had was 192GB spread over 2 nodes with 48thread/24core per
> > node, correct? Your numbers went from ~290ms to ~28ms so a 10x
> > decrease, that doesn't sound linear when you spread the work over 24
> > cores to get there. I agree that the numbers largely stay flat once
> > you hit the peak, I have seen similar behavior when I was working on
> > the deferred init code previously. One concern I have though is that
> > we may end up seeing better performance with a subset of cores instead
> > of running all of the cores/threads, especially if features such as
> > turbo come into play. In addition we are talking x86 only so far. I
> > would be interested in seeing if this has benefits or not for other
> > architectures.
> > 
> > Also what is the penalty that is being paid in order to break up the
> > work before-hand and set it up for the parallel work? I would be
> > interested in seeing what the cost is on a system with fewer cores per
> > node, maybe even down to 1. That would tell us how much additional
> > overhead is being added to set things up to run in parallel.
> 
> The numbers above have the 1-thread case.  It seems close to the noise.
> 
> > If I get
> > a chance tomorrow I might try applying the patches and doing some
> > testing myself.
> 
> If you end up doing that, you might find this helpful:
>     https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=patch;h=afc72bf8478b95a1d6d174c269ff3693c60630e0
>     
> and maybe this:
>     https://oss.oracle.com/git/gitweb.cgi?p=linux-dmjordan.git;a=patch;h=dff6537eab281e5a9917682c4adf9059c0574223
> 
> Thanks for looking this over.
> 
> [ By the way, I'm going to be out Tuesday but back the rest of the week. ]
> 


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-05  2:09           ` Daniel Jordan
@ 2020-05-05 14:55             ` Alexander Duyck
  2020-05-06 22:21               ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-05 14:55 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Josh Triplett, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Mon, May 4, 2020 at 7:11 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
>
> On Mon, May 04, 2020 at 09:48:44PM -0400, Daniel Jordan wrote:
> > On Mon, May 04, 2020 at 05:40:19PM -0700, Alexander Duyck wrote:
> > > On Mon, May 4, 2020 at 4:44 PM Josh Triplett <josh@joshtriplett.org> wrote:
> > > >
> > > > On May 4, 2020 3:33:58 PM PDT, Alexander Duyck <alexander.duyck@gmail.com> wrote:
> > > > >On Thu, Apr 30, 2020 at 1:12 PM Daniel Jordan
> > > > ><daniel.m.jordan@oracle.com> wrote:
> > > > >>         /*
> > > > >> -        * 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.
> > > > >>          */
> > > > >
> > > > >I would be curious about your data. That isn't what I have seen in the
> > > > >past. Typically only up to about 8 or 10 CPUs gives you any benefit,
> > > > >beyond that I was usually cache/memory bandwidth bound.
> >
> > On Skylake it took more than 8 or 10 CPUs, though on other machines the benefit
> > of using all versus half or 3/4 of the CPUs is less significant.
> >
> > Given that the rest of the system is idle at this point, my main concern is
> > whether other archs regress past a certain thread count.
>
> Reposting the data to be consistent with the way the percentages are reported
> in the changelog.
>
>
>     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)         --   4056.7 (  5.5)         --   1763.3 (  4.2)
>        2% (  1)      -2.4%   4153.3 (  2.5)      -5.6%   1861.7 (  5.5)
>       12% (  6)      35.0%   2637.7 ( 38.7)      80.3%    346.7 ( 37.5)
>       25% ( 13)      38.4%   2497.3 ( 38.5)      88.1%    210.0 ( 41.8)
>       37% ( 19)      38.9%   2477.0 ( 19.0)      89.5%    185.3 ( 21.5)
>       50% ( 26)      39.1%   2471.7 ( 21.4)      89.8%    179.7 ( 25.8)
>       75% ( 39)      39.5%   2455.7 ( 33.2)      90.8%    161.7 ( 29.3)
>      100% ( 52)      39.9%   2436.7 (  2.1)      91.8%    144.3 (  5.9)
>
>
>     Intel(R) Xeon(R) CPU E5-2699C v4 @ 2.20GHz (Broadwell, bare metal)
>       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)         --   1957.3 ( 14.0)         --   1093.7 ( 12.9)
>        3% (  1)       1.4%   1930.7 ( 10.0)       3.7%   1053.3 (  7.6)
>       12% (  4)      41.2%   1151.7 (  9.0)      74.5%    278.7 (  0.6)
>       25% (  8)      46.3%   1051.0 (  7.8)      83.7%    178.0 (  2.6)
>       38% ( 12)      48.7%   1003.3 (  7.6)      87.0%    141.7 (  3.8)
>       50% ( 16)      48.2%   1014.3 ( 20.0)      87.8%    133.3 (  3.2)
>       75% ( 24)      49.5%    989.3 (  6.7)      88.4%    126.3 (  1.5)
>      100% ( 32)      49.1%    996.0 (  7.2)      88.4%    127.3 (  5.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)         --   1666.0 (  3.5)         --    618.0 (  3.5)
>        3% (  1)       1.0%   1649.7 (  1.5)       2.9%    600.0 (  1.0)
>       11% (  4)      25.9%   1234.7 ( 21.4)      70.4%    183.0 ( 22.5)
>       25% (  9)      29.6%   1173.0 ( 10.0)      80.7%    119.3 (  9.6)
>       36% ( 13)      30.8%   1153.7 ( 17.0)      84.0%     99.0 ( 15.6)
>       50% ( 18)      31.0%   1150.3 ( 15.5)      84.3%     97.3 ( 16.2)
>       75% ( 27)      31.0%   1150.3 (  2.5)      84.6%     95.0 (  5.6)
>      100% ( 36)      31.3%   1145.3 (  1.5)      85.6%     89.0 (  1.7)
>
>
>     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.7 ( 42.3)         --    253.7 (  3.1)
>        6% (  1)       3.3%    995.3 ( 21.4)       4.3%    242.7 (  5.5)
>       12% (  2)      14.0%    885.7 ( 24.4)      46.4%    136.0 (  5.2)
>       25% (  4)      18.9%    835.0 ( 21.5)      66.1%     86.0 (  1.7)
>       38% (  6)      21.9%    804.7 ( 15.7)      71.4%     72.7 (  2.1)
>       50% (  8)      20.8%    815.3 ( 11.7)      74.4%     65.0 (  3.5)
>       75% ( 12)      23.5%    787.7 (  2.1)      74.0%     66.0 (  3.6)
>      100% ( 16)      23.3%    789.3 ( 15.0)      76.3%     60.0 (  5.6)
>
>
>     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)         --    757.7 ( 17.1)         --     57.0 (  0.0)
>       25% (  1)      -1.0%    765.3 (  5.5)       3.5%     55.0 (  0.0)
>       50% (  2)       4.7%    722.3 ( 21.5)      42.7%     32.7 (  4.6)
>       75% (  3)       3.7%    729.7 (  4.9)      54.4%     26.0 (  0.0)
>      100% (  4)       6.2%    710.3 ( 15.0)      63.2%     21.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)         --    656.3 (  7.1)         --     57.3 (  1.5)
>       25% (  1)       1.8%    644.7 (  3.1)       0.6%     57.0 (  0.0)
>       50% (  2)       6.5%    613.7 (  5.1)      40.7%     34.0 (  5.3)
>       75% (  3)       6.9%    611.3 (  6.7)      57.6%     24.3 (  0.6)
>      100% (  4)       8.6%    599.7 (  5.9)      62.8%     21.3 (  1.2)

One question about this data. What is the power management
configuration on the systems when you are running these tests? I'm
just curious if CPU frequency scaling, C states, and turbo are
enabled? I ask because that is what I have seen usually make the
difference in these kind of workloads as the throughput starts
dropping off as you start seeing the core frequency lower and more
cores become active.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-05  0:54         ` Daniel Jordan
@ 2020-05-05 15:27           ` Alexander Duyck
  2020-05-06 22:39             ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-05 15:27 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, LKML

On Mon, May 4, 2020 at 5:54 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
>
> On Mon, May 04, 2020 at 03:10:46PM -0700, Alexander Duyck wrote:
> > So we cannot stop in the middle of a max order block. That shouldn't
> > be possible as part of the issue is that the buddy allocator will
> > attempt to access the buddy for the page which could cause issues if
> > it tries to merge the page with one that is not initialized. So if
> > your code supports that then it is definitely broken. That was one of
> > the reasons for all of the variable weirdness in
> > deferred_init_maxorder. I was going through and making certain that
> > while we were initializing the range we were freeing the pages in
> > MAX_ORDER aligned blocks and skipping over whatever reserved blocks
> > were there. Basically it was handling the case where a single
> > MAX_ORDER block could span multiple ranges.
> >
> > On x86 this was all pretty straightforward and I don't believe we
> > needed the code, but I seem to recall there were some other
> > architectures that had more complex memory layouts at the time and
> > that was one of the reasons why I had to be careful to wait until I
> > had processed the full MAX_ORDER block before I could start freeing
> > the pages, otherwise it would start triggering memory corruptions.
>
> Yes, thanks, I missed the case where deferred_grow_zone could stop
> mid-max-order-block.

As it turns out that deferred_free_range will be setting the
migratetype for the page. In a sparse config the migratetype bits are
stored in the section bitmap. So to avoid cacheline bouncing it would
make sense to section align the tasks so that they only have one
thread touching one section rather than having the pageblock_flags
getting bounced between threads. It should also reduce the overhead
for having to parallelize the work in the first place since a section
is several times larger than a MAX_ORDER page and allows for more
batching of the work.

> Maybe it's better to leave deferred_init_maxorder alone and adapt the
> multithreading to the existing implementation.  That'd mean dealing with the
> pesky opaque index somehow, so deferred_init_mem_pfn_range_in_zone() could be
> generalized to find it in the thread function based on the start/end range, or
> it could be maintained as part of the range that padata passes to the thread
> function.

You may be better off just implementing your threads to operate like
deferred_grow_zone does. All your worker thread really needs then is
to know where to start performing the page initialization and then it
could go through and process an entire section worth of pages. The
other bit that would have to be changed is patch 6 so that you combine
any ranges that might span a single section instead of just splitting
the work up based on the ranges.

If you are referring to the mo_pfn you shouldn't even need to think
about it. All it is doing is guaranteeing you are processing at least
a full max order worth of pages. Without that the logic before was
either process a whole section, or just process all of memory
initializing it before it started freeing it. I found it made things
much more efficient to process only up to MAX_ORDER at a time as you
could squeeze that into the L2 cache for most x86 processors at least
and it reduced the memory bandwidth by quite a bit. If you update the
code to only provide section aligned/sized ranges of of PFNs to
initialize then it can pretty much be ignored since all it is doing is
defining the break point for single MAX_ORDER chunks which would be
smaller than a section anyway.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-05 14:55             ` Alexander Duyck
@ 2020-05-06 22:21               ` Daniel Jordan
  2020-05-06 22:36                 ` Alexander Duyck
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-06 22:21 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Josh Triplett, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Alexander Duyck, Dan Williams,
	Dave Hansen, David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Tue, May 05, 2020 at 07:55:43AM -0700, Alexander Duyck wrote:
> One question about this data. What is the power management
> configuration on the systems when you are running these tests? I'm
> just curious if CPU frequency scaling, C states, and turbo are
> enabled?

Yes, intel_pstate is loaded in active mode without hwp and with turbo enabled
(those power management docs are great by the way!) and intel_idle is in use
too.

> I ask because that is what I have seen usually make the
> difference in these kind of workloads as the throughput starts
> dropping off as you start seeing the core frequency lower and more
> cores become active.

If I follow, you're saying there's a chance performance would improve with the
above disabled, but how often would a system be configured that way?  Even if
it were faster, the machine is configured how it's configured, or am I missing
your point?


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-06 22:21               ` Daniel Jordan
@ 2020-05-06 22:36                 ` Alexander Duyck
  2020-05-06 22:43                   ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-06 22:36 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Josh Triplett, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Alexander Duyck, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Wed, May 6, 2020 at 3:21 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
>
> On Tue, May 05, 2020 at 07:55:43AM -0700, Alexander Duyck wrote:
> > One question about this data. What is the power management
> > configuration on the systems when you are running these tests? I'm
> > just curious if CPU frequency scaling, C states, and turbo are
> > enabled?
>
> Yes, intel_pstate is loaded in active mode without hwp and with turbo enabled
> (those power management docs are great by the way!) and intel_idle is in use
> too.
>
> > I ask because that is what I have seen usually make the
> > difference in these kind of workloads as the throughput starts
> > dropping off as you start seeing the core frequency lower and more
> > cores become active.
>
> If I follow, you're saying there's a chance performance would improve with the
> above disabled, but how often would a system be configured that way?  Even if
> it were faster, the machine is configured how it's configured, or am I missing
> your point?

I think you might be missing my point. What I was getting at is that I
know for performance testing sometimes C states and P states get
disabled in order to get consistent results between runs, it sounds
like you have them enabled though. I was just wondering if you had
disabled them or not. If they were disabled then you wouldn't get the
benefits of turbo and as such adding more cores wouldn't come at a
penalty, while with it enabled the first few cores should start to
slow down as they fell out of turbo mode. So it may be part of the
reason why you are only hitting about 10x at full core count.

As it stands I think your code may speed up a bit if you split the
work up based on section instead of max order. That would get rid of
any cache bouncing you may be doing on the pageblock flags and reduce
the overhead for splitting the work up into individual pieces since
each piece will be bigger.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-05 15:27           ` Alexander Duyck
@ 2020-05-06 22:39             ` Daniel Jordan
  2020-05-07 15:26               ` Alexander Duyck
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-06 22:39 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Alexander Duyck, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On Tue, May 05, 2020 at 08:27:52AM -0700, Alexander Duyck wrote:
> As it turns out that deferred_free_range will be setting the
> migratetype for the page. In a sparse config the migratetype bits are
> stored in the section bitmap. So to avoid cacheline bouncing it would
> make sense to section align the tasks so that they only have one
> thread touching one section rather than having the pageblock_flags
> getting bounced between threads.

That's a good point, I'll change the alignment.

I kicked off some runs on the Skylake bare metal system to check how this did
and the performance stayed the same, but see below.

> It should also reduce the overhead
> for having to parallelize the work in the first place since a section
> is several times larger than a MAX_ORDER page and allows for more
> batching of the work.

I think you may be assuming that threads work in MAX_ORDER batches, maybe
because that's the job's min_chunk, but padata works differently.  The
min_chunk is a lower bound that establishes the smallest amount of work that
makes sense for a thread to do in one go, so in this case it's useful to
prevent starting large numbers of threads to initialize a tiny amount of pages.

Internally padata uses total job size and min chunk to arrive at the chunk
size, which on big machines will be much larger than min_chunk.  The idea is
the chunk size should be large enough to minimize multithreading overhead but
small enough to permit load balancing between threads.

This is probably why the results didn't change much when aligning by section,
but that doesn't mean other systems won't benefit.

> > Maybe it's better to leave deferred_init_maxorder alone and adapt the
> > multithreading to the existing implementation.  That'd mean dealing with the
> > pesky opaque index somehow, so deferred_init_mem_pfn_range_in_zone() could be

I should have been explicit, was thinking of @i from
for_each_free_mem_pfn_range_in_zone_from() when mentioning the opaque index.

> > generalized to find it in the thread function based on the start/end range, or
> > it could be maintained as part of the range that padata passes to the thread
> > function.
> 
> You may be better off just implementing your threads to operate like
> deferred_grow_zone does. All your worker thread really needs then is
> to know where to start performing the page initialization and then it
> could go through and process an entire section worth of pages. The
> other bit that would have to be changed is patch 6 so that you combine
> any ranges that might span a single section instead of just splitting
> the work up based on the ranges.

How are you thinking of combining them?  I don't see a way to do it without
storing an arbitrary number of ranges somewhere for each thread.

> If you are referring to the mo_pfn you shouldn't even need to think
> about it.

(clarified "opaque index" above)

> All it is doing is guaranteeing you are processing at least
> a full max order worth of pages. Without that the logic before was
> either process a whole section, or just process all of memory
> initializing it before it started freeing it. I found it made things
> much more efficient to process only up to MAX_ORDER at a time as you
> could squeeze that into the L2 cache for most x86 processors at least
> and it reduced the memory bandwidth by quite a bit.

Yes, that was clever, we should keep doing it that way.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-06 22:36                 ` Alexander Duyck
@ 2020-05-06 22:43                   ` Daniel Jordan
  2020-05-06 23:01                     ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-06 22:43 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Josh Triplett, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Alexander Duyck, Dan Williams,
	Dave Hansen, David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Wed, May 06, 2020 at 03:36:54PM -0700, Alexander Duyck wrote:
> On Wed, May 6, 2020 at 3:21 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
> >
> > On Tue, May 05, 2020 at 07:55:43AM -0700, Alexander Duyck wrote:
> > > One question about this data. What is the power management
> > > configuration on the systems when you are running these tests? I'm
> > > just curious if CPU frequency scaling, C states, and turbo are
> > > enabled?
> >
> > Yes, intel_pstate is loaded in active mode without hwp and with turbo enabled
> > (those power management docs are great by the way!) and intel_idle is in use
> > too.
> >
> > > I ask because that is what I have seen usually make the
> > > difference in these kind of workloads as the throughput starts
> > > dropping off as you start seeing the core frequency lower and more
> > > cores become active.
> >
> > If I follow, you're saying there's a chance performance would improve with the
> > above disabled, but how often would a system be configured that way?  Even if
> > it were faster, the machine is configured how it's configured, or am I missing
> > your point?
> 
> I think you might be missing my point. What I was getting at is that I
> know for performance testing sometimes C states and P states get
> disabled in order to get consistent results between runs, it sounds
> like you have them enabled though. I was just wondering if you had
> disabled them or not. If they were disabled then you wouldn't get the
> benefits of turbo and as such adding more cores wouldn't come at a
> penalty, while with it enabled the first few cores should start to
> slow down as they fell out of turbo mode. So it may be part of the
> reason why you are only hitting about 10x at full core count.

All right, that makes way more sense.

> As it stands I think your code may speed up a bit if you split the
> work up based on section instead of max order. That would get rid of
> any cache bouncing you may be doing on the pageblock flags and reduce
> the overhead for splitting the work up into individual pieces since
> each piece will be bigger.

See my other mail.


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

* Re: [PATCH 6/7] mm: parallelize deferred_init_memmap()
  2020-05-06 22:43                   ` Daniel Jordan
@ 2020-05-06 23:01                     ` Daniel Jordan
  0 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-05-06 23:01 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Josh Triplett, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Alexander Duyck, Dan Williams,
	Dave Hansen, David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Kirill Tkhai, Michal Hocko, Pavel Machek, Pavel Tatashin,
	Peter Zijlstra, Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan,
	linux-crypto, linux-mm, LKML

On Wed, May 06, 2020 at 06:43:35PM -0400, Daniel Jordan wrote:
> On Wed, May 06, 2020 at 03:36:54PM -0700, Alexander Duyck wrote:
> > On Wed, May 6, 2020 at 3:21 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
> > >
> > > On Tue, May 05, 2020 at 07:55:43AM -0700, Alexander Duyck wrote:
> > > > One question about this data. What is the power management
> > > > configuration on the systems when you are running these tests? I'm
> > > > just curious if CPU frequency scaling, C states, and turbo are
> > > > enabled?
> > >
> > > Yes, intel_pstate is loaded in active mode without hwp and with turbo enabled
> > > (those power management docs are great by the way!) and intel_idle is in use
> > > too.
> > >
> > > > I ask because that is what I have seen usually make the
> > > > difference in these kind of workloads as the throughput starts
> > > > dropping off as you start seeing the core frequency lower and more
> > > > cores become active.
> > >
> > > If I follow, you're saying there's a chance performance would improve with the
> > > above disabled, but how often would a system be configured that way?  Even if
> > > it were faster, the machine is configured how it's configured, or am I missing
> > > your point?
> > 
> > I think you might be missing my point. What I was getting at is that I
> > know for performance testing sometimes C states and P states get
> > disabled in order to get consistent results between runs, it sounds
> > like you have them enabled though. I was just wondering if you had
> > disabled them or not. If they were disabled then you wouldn't get the
> > benefits of turbo and as such adding more cores wouldn't come at a
> > penalty, while with it enabled the first few cores should start to
> > slow down as they fell out of turbo mode. So it may be part of the
> > reason why you are only hitting about 10x at full core count.

I checked the memory bandwidth of the biggest system, the Skylake.  Couldn't
find official specs for it, all I could quickly find were stream results from a
blog post of ours that quoted a range of about 123-145 GB/s over both nodes
when compiling with gcc.  That's with all CPUs.

Again using all CPUs, multithreaded page init is doing 41 GiB/s per node
assuming it's just touching the 64 bytes of each page struct, so assuming
there's more memory traffic than just struct page, it seems another part of the
reason for only 10x is we're bottlenecked on memory bandwidth.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-06 22:39             ` Daniel Jordan
@ 2020-05-07 15:26               ` Alexander Duyck
  2020-05-07 20:20                 ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-07 15:26 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, LKML

On Wed, May 6, 2020 at 3:39 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
>
> On Tue, May 05, 2020 at 08:27:52AM -0700, Alexander Duyck wrote:
> > As it turns out that deferred_free_range will be setting the
> > migratetype for the page. In a sparse config the migratetype bits are
> > stored in the section bitmap. So to avoid cacheline bouncing it would
> > make sense to section align the tasks so that they only have one
> > thread touching one section rather than having the pageblock_flags
> > getting bounced between threads.
>
> That's a good point, I'll change the alignment.
>
> I kicked off some runs on the Skylake bare metal system to check how this did
> and the performance stayed the same, but see below.
>
> > It should also reduce the overhead
> > for having to parallelize the work in the first place since a section
> > is several times larger than a MAX_ORDER page and allows for more
> > batching of the work.
>
> I think you may be assuming that threads work in MAX_ORDER batches, maybe
> because that's the job's min_chunk, but padata works differently.  The
> min_chunk is a lower bound that establishes the smallest amount of work that
> makes sense for a thread to do in one go, so in this case it's useful to
> prevent starting large numbers of threads to initialize a tiny amount of pages.
>
> Internally padata uses total job size and min chunk to arrive at the chunk
> size, which on big machines will be much larger than min_chunk.  The idea is
> the chunk size should be large enough to minimize multithreading overhead but
> small enough to permit load balancing between threads.
>
> This is probably why the results didn't change much when aligning by section,
> but that doesn't mean other systems won't benefit.

Okay, that makes sense.

> > > Maybe it's better to leave deferred_init_maxorder alone and adapt the
> > > multithreading to the existing implementation.  That'd mean dealing with the
> > > pesky opaque index somehow, so deferred_init_mem_pfn_range_in_zone() could be
>
> I should have been explicit, was thinking of @i from
> () when mentioning the opaque index.

Okay, that makes sense. However in reality you don't need to split
that piece out. All you really are doing is splitting up the
first_init_pfn value over multiple threads so you just need to make
use of deferred_init_mem_pfn_range_in_zone() to initialize it.

> > > generalized to find it in the thread function based on the start/end range, or
> > > it could be maintained as part of the range that padata passes to the thread
> > > function.
> >
> > You may be better off just implementing your threads to operate like
> > deferred_grow_zone does. All your worker thread really needs then is
> > to know where to start performing the page initialization and then it
> > could go through and process an entire section worth of pages. The
> > other bit that would have to be changed is patch 6 so that you combine
> > any ranges that might span a single section instead of just splitting
> > the work up based on the ranges.
>
> How are you thinking of combining them?  I don't see a way to do it without
> storing an arbitrary number of ranges somewhere for each thread.

So when you are putting together your data you are storing a starting
value and a length. All you end up having to do is make certain that
the size + start pfn is section aligned. Then if you jump to a new
section you have the option of either adding to the size of your
current section or submitting the range and starting with a new start
pfn in a new section. All you are really doing is breaking up the
first_deferred_pfn over multiple sections. What I would do is section
align end_pfn, and then check the next range from the zone. If the
start_pfn of the next range is less than end_pfn you merge the two
ranges by just increasing the size, otherwise you could start a new
range.

The idea is that you just want to define what the valid range of PFNs
are, and if there are sizable holes you skip over them. You would
leave most of the lifting for identifying exactly what PFNs to
initialize to the pfn_range_in_zone iterators since they would all be
read-only accesses anyway.

> > If you are referring to the mo_pfn you shouldn't even need to think
> > about it.
>
> (clarified "opaque index" above)

Thanks.

> > All it is doing is guaranteeing you are processing at least
> > a full max order worth of pages. Without that the logic before was
> > either process a whole section, or just process all of memory
> > initializing it before it started freeing it. I found it made things
> > much more efficient to process only up to MAX_ORDER at a time as you
> > could squeeze that into the L2 cache for most x86 processors at least
> > and it reduced the memory bandwidth by quite a bit.
>
> Yes, that was clever, we should keep doing it that way.

Thanks.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-07 15:26               ` Alexander Duyck
@ 2020-05-07 20:20                 ` Daniel Jordan
  2020-05-07 21:18                   ` Alexander Duyck
  0 siblings, 1 reply; 35+ messages in thread
From: Daniel Jordan @ 2020-05-07 20:20 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Alexander Duyck, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On Thu, May 07, 2020 at 08:26:26AM -0700, Alexander Duyck wrote:
> On Wed, May 6, 2020 at 3:39 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
> > On Tue, May 05, 2020 at 08:27:52AM -0700, Alexander Duyck wrote:
> > > > Maybe it's better to leave deferred_init_maxorder alone and adapt the
> > > > multithreading to the existing implementation.  That'd mean dealing with the
> > > > pesky opaque index somehow, so deferred_init_mem_pfn_range_in_zone() could be
> >
> > I should have been explicit, was thinking of @i from
> > () when mentioning the opaque index.
> 
> Okay, that makes sense. However in reality you don't need to split
> that piece out. All you really are doing is splitting up the
> first_init_pfn value over multiple threads so you just need to make
> use of deferred_init_mem_pfn_range_in_zone() to initialize it.

Ok, I assume you mean that each thread should use
deferred_init_mem_pfn_range_in_zone.  Yes, that's what I meant when saying that
function could be generalized, though not sure we should opt for this.

> > > > generalized to find it in the thread function based on the start/end range, or
> > > > it could be maintained as part of the range that padata passes to the thread
> > > > function.
> > >
> > > You may be better off just implementing your threads to operate like
> > > deferred_grow_zone does. All your worker thread really needs then is
> > > to know where to start performing the page initialization and then it
> > > could go through and process an entire section worth of pages. The
> > > other bit that would have to be changed is patch 6 so that you combine
> > > any ranges that might span a single section instead of just splitting
> > > the work up based on the ranges.
> >
> > How are you thinking of combining them?  I don't see a way to do it without
> > storing an arbitrary number of ranges somewhere for each thread.
> 
> So when you are putting together your data you are storing a starting
> value and a length. All you end up having to do is make certain that
> the size + start pfn is section aligned. Then if you jump to a new
> section you have the option of either adding to the size of your
> current section or submitting the range and starting with a new start
> pfn in a new section. All you are really doing is breaking up the
> first_deferred_pfn over multiple sections. What I would do is section
> align end_pfn, and then check the next range from the zone. If the
> start_pfn of the next range is less than end_pfn you merge the two
> ranges by just increasing the size, otherwise you could start a new
> range.
> 
> The idea is that you just want to define what the valid range of PFNs
> are, and if there are sizable holes you skip over them. You would
> leave most of the lifting for identifying exactly what PFNs to
> initialize to the pfn_range_in_zone iterators since they would all be
> read-only accesses anyway.

Ok, I follow you.  My assumption is that there are generally few free pfn
ranges relative to the total number of pfns being initialized so that it's
efficient to parallelize over a single pfn range from the zone iterator.  On
the systems I tested, there were about 20 tiny ranges and one enormous range
per node so that firing off a job per range kept things simple without
affecting performance.  If that assumption holds, I'm not sure it's worth it to
merge ranges.

With the series as it stands plus leaving in the section alignment check in
deferred_grow_zone (which I think could be relaxed to a maxorder alignment
check) so it doesn't stop mid-max-order-block, threads simply deal with a
start/end range and deferred_init_maxorder becomes shorter and simpler too.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-07 20:20                 ` Daniel Jordan
@ 2020-05-07 21:18                   ` Alexander Duyck
  2020-05-07 22:15                     ` Daniel Jordan
  0 siblings, 1 reply; 35+ messages in thread
From: Alexander Duyck @ 2020-05-07 21:18 UTC (permalink / raw)
  To: Daniel Jordan
  Cc: Alexander Duyck, Andrew Morton, Herbert Xu, Steffen Klassert,
	Alex Williamson, Dan Williams, Dave Hansen, David Hildenbrand,
	Jason Gunthorpe, Jonathan Corbet, Josh Triplett, Kirill Tkhai,
	Michal Hocko, Pavel Machek, Pavel Tatashin, Peter Zijlstra,
	Randy Dunlap, Shile Zhang, Tejun Heo, Zi Yan, linux-crypto,
	linux-mm, LKML

On Thu, May 7, 2020 at 1:20 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
>
> On Thu, May 07, 2020 at 08:26:26AM -0700, Alexander Duyck wrote:
> > On Wed, May 6, 2020 at 3:39 PM Daniel Jordan <daniel.m.jordan@oracle.com> wrote:
> > > On Tue, May 05, 2020 at 08:27:52AM -0700, Alexander Duyck wrote:
> > > > > Maybe it's better to leave deferred_init_maxorder alone and adapt the
> > > > > multithreading to the existing implementation.  That'd mean dealing with the
> > > > > pesky opaque index somehow, so deferred_init_mem_pfn_range_in_zone() could be
> > >
> > > I should have been explicit, was thinking of @i from
> > > () when mentioning the opaque index.
> >
> > Okay, that makes sense. However in reality you don't need to split
> > that piece out. All you really are doing is splitting up the
> > first_init_pfn value over multiple threads so you just need to make
> > use of deferred_init_mem_pfn_range_in_zone() to initialize it.
>
> Ok, I assume you mean that each thread should use
> deferred_init_mem_pfn_range_in_zone.  Yes, that's what I meant when saying that
> function could be generalized, though not sure we should opt for this.

Yes that is what I meant.

> > > > > generalized to find it in the thread function based on the start/end range, or
> > > > > it could be maintained as part of the range that padata passes to the thread
> > > > > function.
> > > >
> > > > You may be better off just implementing your threads to operate like
> > > > deferred_grow_zone does. All your worker thread really needs then is
> > > > to know where to start performing the page initialization and then it
> > > > could go through and process an entire section worth of pages. The
> > > > other bit that would have to be changed is patch 6 so that you combine
> > > > any ranges that might span a single section instead of just splitting
> > > > the work up based on the ranges.
> > >
> > > How are you thinking of combining them?  I don't see a way to do it without
> > > storing an arbitrary number of ranges somewhere for each thread.
> >
> > So when you are putting together your data you are storing a starting
> > value and a length. All you end up having to do is make certain that
> > the size + start pfn is section aligned. Then if you jump to a new
> > section you have the option of either adding to the size of your
> > current section or submitting the range and starting with a new start
> > pfn in a new section. All you are really doing is breaking up the
> > first_deferred_pfn over multiple sections. What I would do is section
> > align end_pfn, and then check the next range from the zone. If the
> > start_pfn of the next range is less than end_pfn you merge the two
> > ranges by just increasing the size, otherwise you could start a new
> > range.
> >
> > The idea is that you just want to define what the valid range of PFNs
> > are, and if there are sizable holes you skip over them. You would
> > leave most of the lifting for identifying exactly what PFNs to
> > initialize to the pfn_range_in_zone iterators since they would all be
> > read-only accesses anyway.
>
> Ok, I follow you.  My assumption is that there are generally few free pfn
> ranges relative to the total number of pfns being initialized so that it's
> efficient to parallelize over a single pfn range from the zone iterator.  On
> the systems I tested, there were about 20 tiny ranges and one enormous range
> per node so that firing off a job per range kept things simple without
> affecting performance.  If that assumption holds, I'm not sure it's worth it to
> merge ranges.

The idea behind merging ranges it to address possible cases where a
range is broken up such that there is a hole in a max order block as a
result. By combining the ranges if they both span the same section we
can guarantee that the entire section will be initialized as a block
and not potentially have partially initialized sections floating
around. Without that mo_pfn logic I had in there I was getting panics
every so often when booting up one of my systems as I recall.

Also the iterator itself is cheap. It is basically just walking a
read-only list so it scales efficiently as well. One of the reasons
why I arranged the code the way I did is that it also allowed me to
get rid of an extra check in the code as the previous code was having
to verify if the pfn belonged to the node. That is all handled
directly through the for_each_free_mem_pfn_range_in_zone[_from] call
now.

> With the series as it stands plus leaving in the section alignment check in
> deferred_grow_zone (which I think could be relaxed to a maxorder alignment
> check) so it doesn't stop mid-max-order-block, threads simply deal with a
> start/end range and deferred_init_maxorder becomes shorter and simpler too.

I still think we are better off initializing complete sections since
the pageblock_flags are fully initialized that way as well. What
guarantee do you have that all of the memory ranges will be max order
aligned? The problem is we have to guarantee all pages are initialized
before we start freeing the pages in a max order page. If we just
process each block as-is I believe we can end up with some
architectures trying to access uninitialized memory in the buddy
allocator as a result. That is why the deferred_init_maxorder function
will walk through the iterator, using the _from version to avoid
unnecessary iteration, the first time initializing the pages it needs
to cross that max order boundary, and then again to free the max order
block of pages that have been initialized. The iterator itself is
farily cheap and only has to get you through the smaller ranges before
you end up at the one big range that it just kind of sits at while it
is working on getting it processed.


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

* Re: [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder()
  2020-05-07 21:18                   ` Alexander Duyck
@ 2020-05-07 22:15                     ` Daniel Jordan
  0 siblings, 0 replies; 35+ messages in thread
From: Daniel Jordan @ 2020-05-07 22:15 UTC (permalink / raw)
  To: Alexander Duyck
  Cc: Daniel Jordan, Alexander Duyck, Andrew Morton, Herbert Xu,
	Steffen Klassert, Alex Williamson, Dan Williams, Dave Hansen,
	David Hildenbrand, Jason Gunthorpe, Jonathan Corbet,
	Josh Triplett, Kirill Tkhai, Michal Hocko, Pavel Machek,
	Pavel Tatashin, Peter Zijlstra, Randy Dunlap, Shile Zhang,
	Tejun Heo, Zi Yan, linux-crypto, linux-mm, LKML

On Thu, May 07, 2020 at 02:18:42PM -0700, Alexander Duyck wrote:
> The idea behind merging ranges it to address possible cases where a
> range is broken up such that there is a hole in a max order block as a
> result.

Gah, yes, you're right, there could be multiple ranges in a max order block, so
the threads have to use the zone iterators to skip the holes.

> By combining the ranges if they both span the same section we
> can guarantee that the entire section will be initialized as a block
> and not potentially have partially initialized sections floating
> around. Without that mo_pfn logic I had in there I was getting panics
> every so often when booting up one of my systems as I recall.
> 
> Also the iterator itself is cheap. It is basically just walking a
> read-only list so it scales efficiently as well. One of the reasons

Agreed, it's not expensive, it's just gnarliness I was hoping to avoid, but
obviously it's not gonna work.

> why I arranged the code the way I did is that it also allowed me to
> get rid of an extra check in the code as the previous code was having
> to verify if the pfn belonged to the node. That is all handled
> directly through the for_each_free_mem_pfn_range_in_zone[_from] call
> now.
> 
> > With the series as it stands plus leaving in the section alignment check in
> > deferred_grow_zone (which I think could be relaxed to a maxorder alignment
> > check) so it doesn't stop mid-max-order-block, threads simply deal with a
> > start/end range and deferred_init_maxorder becomes shorter and simpler too.
> 
> I still think we are better off initializing complete sections since
> the pageblock_flags are fully initialized that way as well.

Fair enough.

> What
> guarantee do you have that all of the memory ranges will be max order
> aligned?

Sure, it's a problem with multiple ranges in a maxorder block, the rest
could've been handled.

> The problem is we have to guarantee all pages are initialized
> before we start freeing the pages in a max order page. If we just
> process each block as-is I believe we can end up with some
> architectures trying to access uninitialized memory in the buddy
> allocator as a result. That is why the deferred_init_maxorder function
> will walk through the iterator, using the _from version to avoid
> unnecessary iteration, the first time initializing the pages it needs
> to cross that max order boundary, and then again to free the max order
> block of pages that have been initialized. The iterator itself is
> farily cheap and only has to get you through the smaller ranges before
> you end up at the one big range that it just kind of sits at while it
> is working on getting it processed.

Right.


Ok, I think we're on the same page for the next version.  Thanks for the
thorough review!


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

end of thread, other threads:[~2020-05-07 22:15 UTC | newest]

Thread overview: 35+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-04-30 20:11 [PATCH 0/7] padata: parallelize deferred page init Daniel Jordan
2020-04-30 20:11 ` [PATCH 1/7] padata: remove exit routine Daniel Jordan
2020-04-30 20:11 ` [PATCH 2/7] padata: initialize earlier Daniel Jordan
2020-04-30 20:11 ` [PATCH 3/7] padata: allocate work structures for parallel jobs from a pool Daniel Jordan
2020-04-30 20:11 ` [PATCH 4/7] padata: add basic support for multithreaded jobs Daniel Jordan
2020-04-30 20:11 ` [PATCH 5/7] mm: move zone iterator outside of deferred_init_maxorder() Daniel Jordan
2020-04-30 21:43   ` Alexander Duyck
2020-05-01  2:45     ` Daniel Jordan
2020-05-04 22:10       ` Alexander Duyck
2020-05-05  0:54         ` Daniel Jordan
2020-05-05 15:27           ` Alexander Duyck
2020-05-06 22:39             ` Daniel Jordan
2020-05-07 15:26               ` Alexander Duyck
2020-05-07 20:20                 ` Daniel Jordan
2020-05-07 21:18                   ` Alexander Duyck
2020-05-07 22:15                     ` Daniel Jordan
2020-04-30 20:11 ` [PATCH 6/7] mm: parallelize deferred_init_memmap() Daniel Jordan
2020-05-04 22:33   ` Alexander Duyck
2020-05-04 23:38     ` Josh Triplett
2020-05-05  0:40       ` Alexander Duyck
2020-05-05  1:48         ` Daniel Jordan
2020-05-05  2:09           ` Daniel Jordan
2020-05-05 14:55             ` Alexander Duyck
2020-05-06 22:21               ` Daniel Jordan
2020-05-06 22:36                 ` Alexander Duyck
2020-05-06 22:43                   ` Daniel Jordan
2020-05-06 23:01                     ` Daniel Jordan
2020-05-05  1:26     ` Daniel Jordan
2020-04-30 20:11 ` [PATCH 7/7] padata: document multithreaded jobs Daniel Jordan
2020-04-30 21:31 ` [PATCH 0/7] padata: parallelize deferred page init Andrew Morton
2020-04-30 21:40   ` Pavel Tatashin
2020-05-01  2:40     ` Daniel Jordan
2020-05-01  0:50   ` Josh Triplett
2020-05-01  1:09 ` Josh Triplett
2020-05-01  2:48   ` 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).