rcu.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
@ 2019-08-14 16:04 Joel Fernandes (Google)
  2019-08-14 16:04 ` [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests Joel Fernandes (Google)
                   ` (2 more replies)
  0 siblings, 3 replies; 30+ messages in thread
From: Joel Fernandes (Google) @ 2019-08-14 16:04 UTC (permalink / raw)
  To: linux-kernel
  Cc: Joel Fernandes (Google),
	kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Paul E. McKenney, Rao Shoaib, rcu,
	Steven Rostedt

Recently a discussion about stability and performance of a system
involving a high rate of kfree_rcu() calls surfaced on the list [1]
which led to another discussion how to prepare for this situation.

This patch adds basic batching support for kfree_rcu(). It is "basic"
because we do none of the slab management, dynamic allocation, code
moving or any of the other things, some of which previous attempts did
[2]. These fancier improvements can be follow-up patches and there are
different ideas being discussed in those regards. This is an effort to
start simple, and build up from there. In the future, an extension to
use kfree_bulk and possibly per-slab batching could be done to further
improve performance due to cache-locality and slab-specific bulk free
optimizations. By using an array of pointers, the worker thread
processing the work would need to read lesser data since it does not
need to deal with large rcu_head(s) any longer.

Torture tests follow in the next patch and show improvements of around
5x reduction in number of  grace periods on a 16 CPU system. More
details and test data are in that patch.

There is an implication with rcu_barrier() with this patch. Since the
kfree_rcu() calls can be batched, and may not be handed yet to the RCU
machinery in fact, the monitor may not have even run yet to do the
queue_rcu_work(), there seems no easy way of implementing rcu_barrier()
to wait for those kfree_rcu()s that are already made. So this means a
kfree_rcu() followed by an rcu_barrier() does not imply that memory will
be freed once rcu_barrier() returns.

Another implication is higher active memory usage (although not
run-away..) until the kfree_rcu() flooding ends, in comparison to
without batching. More details about this are in the second patch which
adds an rcuperf test.

Finally, in the near future we will get rid of kfree_rcu() special casing
within RCU such as in rcu_do_batch and switch everything to just
batching. Currently we don't do that since timer subsystem is not yet up
and we cannot schedule the kfree_rcu() monitor as the timer subsystem's
lock are not initialized. That would also mean getting rid of
kfree_call_rcu_nobatch() entirely.

[1] http://lore.kernel.org/lkml/20190723035725-mutt-send-email-mst@kernel.org
[2] https://lkml.org/lkml/2017/12/19/824

Cc: kernel-team@android.com
Cc: kernel-team@lge.com
Co-developed-by: Byungchul Park <byungchul.park@lge.com>
Signed-off-by: Byungchul Park <byungchul.park@lge.com>
Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>

---
v3->v4: Some corrections by Paul.
	Used xchg in places to simplify code.

v2->v3: Just some code comment changes thanks to Byungchul.

RFCv1->PATCH v2: Removed limits on the ->head list, just let it grow.
                   Dropped KFREE_MAX_JIFFIES to HZ/50 from HZ/20 to reduce OOM occurrence.
                   Removed sleeps in rcuperf test, just using cond_resched()in loop.
                   Better code comments ;)

 include/linux/rcutiny.h |   5 ++
 include/linux/rcutree.h |   1 +
 kernel/rcu/tree.c       | 194 ++++++++++++++++++++++++++++++++++++++--
 3 files changed, 194 insertions(+), 6 deletions(-)

diff --git a/include/linux/rcutiny.h b/include/linux/rcutiny.h
index 8e727f57d814..383f2481750f 100644
--- a/include/linux/rcutiny.h
+++ b/include/linux/rcutiny.h
@@ -39,6 +39,11 @@ static inline void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
 	call_rcu(head, func);
 }
 
+static inline void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
+{
+	call_rcu(head, func);
+}
+
 void rcu_qs(void);
 
 static inline void rcu_softirq_qs(void)
diff --git a/include/linux/rcutree.h b/include/linux/rcutree.h
index 735601ac27d3..7e38b39ec634 100644
--- a/include/linux/rcutree.h
+++ b/include/linux/rcutree.h
@@ -34,6 +34,7 @@ static inline void rcu_virt_note_context_switch(int cpu)
 
 void synchronize_rcu_expedited(void);
 void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func);
+void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func);
 
 void rcu_barrier(void);
 bool rcu_eqs_special_set(int cpu);
diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
index a14e5fbbea46..1d1847cadea2 100644
--- a/kernel/rcu/tree.c
+++ b/kernel/rcu/tree.c
@@ -2593,17 +2593,185 @@ void call_rcu(struct rcu_head *head, rcu_callback_t func)
 }
 EXPORT_SYMBOL_GPL(call_rcu);
 
+
+/* Maximum number of jiffies to wait before draining a batch. */
+#define KFREE_DRAIN_JIFFIES (HZ / 50)
+
 /*
- * Queue an RCU callback for lazy invocation after a grace period.
- * This will likely be later named something like "call_rcu_lazy()",
- * but this change will require some way of tagging the lazy RCU
- * callbacks in the list of pending callbacks. Until then, this
- * function may only be called from __kfree_rcu().
+ * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
+ * kfree(s) is queued for freeing after a grace period, right away.
  */
-void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
+struct kfree_rcu_cpu {
+	/* The rcu_work node for queuing work with queue_rcu_work(). The work
+	 * is done after a grace period.
+	 */
+	struct rcu_work rcu_work;
+
+	/* The list of objects being queued in a batch but are not yet
+	 * scheduled to be freed.
+	 */
+	struct rcu_head *head;
+
+	/* The list of objects that have now left ->head and are queued for
+	 * freeing after a grace period.
+	 */
+	struct rcu_head *head_free;
+
+	/* Protect concurrent access to this structure. */
+	spinlock_t lock;
+
+	/* The delayed work that flushes ->head to ->head_free incase ->head
+	 * within KFREE_DRAIN_JIFFIES. In case flushing cannot be done if RCU
+	 * is busy, ->head just continues to grow and we retry flushing later.
+	 */
+	struct delayed_work monitor_work;
+	bool monitor_todo;	/* Is a delayed work pending execution? */
+};
+
+static DEFINE_PER_CPU(struct kfree_rcu_cpu, krc);
+
+/*
+ * This function is invoked in workqueue context after a grace period.
+ * It frees all the objects queued on ->head_free.
+ */
+static void kfree_rcu_work(struct work_struct *work)
+{
+	unsigned long flags;
+	struct rcu_head *head, *next;
+	struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
+					struct kfree_rcu_cpu, rcu_work);
+
+	spin_lock_irqsave(&krcp->lock, flags);
+	head = krcp->head_free;
+	krcp->head_free = NULL;
+	spin_unlock_irqrestore(&krcp->lock, flags);
+
+	/*
+	 * The head is detached and not referenced from anywhere, so lockless
+	 * access is Ok.
+	 */
+	for (; head; head = next) {
+		next = head->next;
+		/* Could be possible to optimize with kfree_bulk in future */
+		__rcu_reclaim(rcu_state.name, head);
+		cond_resched_tasks_rcu_qs();
+	}
+}
+
+/*
+ * Schedule the kfree batch RCU work to run in workqueue context after a GP.
+ *
+ * This function is invoked by kfree_rcu_monitor() when the KFREE_DRAIN_JIFFIES
+ * timeout has been reached.
+ */
+static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
+{
+	lockdep_assert_held(&krcp->lock);
+
+	/* If a previous RCU batch work is already in progress, we cannot queue
+	 * another one, just refuse the optimization and it will be retried
+	 * again in KFREE_DRAIN_JIFFIES time.
+	 */
+	if (krcp->head_free)
+		return false;
+
+	krcp->head_free = krcp->head;
+	krcp->head = NULL;
+	INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
+	queue_rcu_work(system_wq, &krcp->rcu_work);
+
+	return true;
+}
+
+static inline void kfree_rcu_drain_unlock(struct kfree_rcu_cpu *krcp,
+				   unsigned long flags)
+{
+	/* Flush ->head to ->head_free, all objects on ->head_free will be
+	 * kfree'd after a grace period.
+	 */
+	if (queue_kfree_rcu_work(krcp)) {
+		/* Success! Our job is done here. */
+		spin_unlock_irqrestore(&krcp->lock, flags);
+		return;
+	}
+
+	/* Previous batch that was queued to RCU did not get free yet, let us
+	 * try again soon.
+	 */
+	if (!xchg(&krcp->monitor_todo, true))
+		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
+	spin_unlock_irqrestore(&krcp->lock, flags);
+}
+
+/*
+ * This function is invoked after the KFREE_DRAIN_JIFFIES timeout has elapsed,
+ * and it drains the specified kfree_rcu_cpu structure's ->head list.
+ */
+static void kfree_rcu_monitor(struct work_struct *work)
+{
+	unsigned long flags;
+	struct kfree_rcu_cpu *krcp = container_of(work, struct kfree_rcu_cpu,
+						 monitor_work.work);
+
+	spin_lock_irqsave(&krcp->lock, flags);
+	if (xchg(&krcp->monitor_todo, false))
+		kfree_rcu_drain_unlock(krcp, flags);
+	else
+		spin_unlock_irqrestore(&krcp->lock, flags);
+}
+
+/*
+ * This version of kfree_call_rcu does not do batching of kfree_rcu() requests.
+ * Used only by rcuperf torture test for comparison with kfree_rcu_batch().
+ */
+void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
 {
 	__call_rcu(head, func, -1, 1);
 }
+EXPORT_SYMBOL_GPL(kfree_call_rcu_nobatch);
+
+/*
+ * Queue a request for lazy invocation of kfree() after a grace period.
+ *
+ * Each kfree_call_rcu() request is added to a batch. The batch will be drained
+ * every KFREE_DRAIN_JIFFIES number of jiffies. All the objects in the batch
+ * will be kfree'd in workqueue context. This allows us to:
+ *
+ * 1. Batch requests together to reduce the number of grace periods during
+ * heavy kfree_rcu() load.
+ *
+ * 2. In the future, makes it possible to use kfree_bulk() on a large number of
+ * kfree_rcu() requests thus reducing the per-object overhead of kfree() and
+ * also reducing cache misses.
+ */
+void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
+{
+	unsigned long flags;
+	struct kfree_rcu_cpu *krcp;
+
+	/* kfree_call_rcu() batching requires timers to be up. If the scheduler
+	 * is not yet up, just skip batching and do the non-batched version.
+	 */
+	if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
+		return kfree_call_rcu_nobatch(head, func);
+
+	head->func = func;
+
+	local_irq_save(flags);	/* For safely calling this_cpu_ptr(). */
+	krcp = this_cpu_ptr(&krc);
+	spin_lock(&krcp->lock);
+
+	/* Queue the kfree but don't yet schedule the batch. */
+	head->next = krcp->head;
+	krcp->head = head;
+
+	/* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
+	if (!xchg(&krcp->monitor_todo, true))
+		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
+
+	spin_unlock(&krcp->lock);
+	local_irq_restore(flags);
+}
 EXPORT_SYMBOL_GPL(kfree_call_rcu);
 
 /*
@@ -3455,10 +3623,24 @@ static void __init rcu_dump_rcu_node_tree(void)
 struct workqueue_struct *rcu_gp_wq;
 struct workqueue_struct *rcu_par_gp_wq;
 
+static void __init kfree_rcu_batch_init(void)
+{
+	int cpu;
+
+	for_each_possible_cpu(cpu) {
+		struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
+
+		spin_lock_init(&krcp->lock);
+		INIT_DELAYED_WORK(&krcp->monitor_work, kfree_rcu_monitor);
+	}
+}
+
 void __init rcu_init(void)
 {
 	int cpu;
 
+	kfree_rcu_batch_init();
+
 	rcu_early_boot_tests();
 
 	rcu_bootup_announce();
-- 
2.23.0.rc1.153.gdeed80330f-goog

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

* [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-14 16:04 [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Joel Fernandes (Google)
@ 2019-08-14 16:04 ` Joel Fernandes (Google)
  2019-08-14 22:58   ` Paul E. McKenney
  2019-08-16 16:43 ` [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Paul E. McKenney
  2019-09-18  9:58 ` Uladzislau Rezki
  2 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes (Google) @ 2019-08-14 16:04 UTC (permalink / raw)
  To: linux-kernel
  Cc: Joel Fernandes (Google),
	byungchul.park, Davidlohr Bueso, Josh Triplett, kernel-team,
	kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Paul E. McKenney, Rao Shoaib, rcu,
	Steven Rostedt

This test runs kfree_rcu in a loop to measure performance of the new
kfree_rcu batching functionality.

The following table shows results when booting with arguments:
rcuperf.kfree_loops=200000 rcuperf.kfree_alloc_num=1000 rcuperf.kfree_rcu_test=1

In addition, rcuperf.kfree_no_batch is used to toggle the batching of
kfree_rcu()s for a test run.

rcuperf.kfree_no_batch	GPs	time (seconds)
 0 (default)		1732	15.9
 1			9133 	14.5

Note that the results are the same for the case:
1. Patch is not applied and rcuperf.kfree_no_batch=0
2. Patch is applied     and rcuperf.kfree_no_batch=1

On a 16 CPU system with the above boot parameters, we see that the total
number of grace periods that elapse during the test drops from 9133 when
not batching to 1732 when batching (a 5X improvement). The kfree_rcu()
flood itself slows down a bit when batching, though, as shown. This is
likely due to rcuperf threads contending with the additional worker
threads that are now running both before (the monitor) and after (the
work done to kfree()) the grace period.

Note that the active memory consumption during the kfree_rcu() flood
does increase to around 300-400MB due to the batching (from around 50MB
without batching). However, this memory consumption is relatively
constant and is just an effect of the buffering. In other words, the
system is able to keep up with the kfree_rcu() load. The memory
consumption comes down to 200-300MB if KFREE_DRAIN_JIFFIES is
increased from HZ/50 to HZ/80.

Also, when running the test, please disable CONFIG_DEBUG_PREEMPT and
CONFIG_PROVE_RCU for realistic comparisons with/without batching.

Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>
---
 .../admin-guide/kernel-parameters.txt         |  17 ++
 kernel/rcu/rcuperf.c                          | 189 +++++++++++++++++-
 2 files changed, 198 insertions(+), 8 deletions(-)

diff --git a/Documentation/admin-guide/kernel-parameters.txt b/Documentation/admin-guide/kernel-parameters.txt
index 7ccd158b3894..a9156ca5de24 100644
--- a/Documentation/admin-guide/kernel-parameters.txt
+++ b/Documentation/admin-guide/kernel-parameters.txt
@@ -3895,6 +3895,23 @@
 			test until boot completes in order to avoid
 			interference.
 
+	rcuperf.kfree_rcu_test= [KNL]
+			Set to measure performance of kfree_rcu() flooding.
+
+	rcuperf.kfree_nthreads= [KNL]
+			The number of threads running loops of kfree_rcu().
+
+	rcuperf.kfree_alloc_num= [KNL]
+			Number of allocations and frees done in an iteration.
+
+	rcuperf.kfree_loops= [KNL]
+			Number of loops doing rcuperf.kfree_alloc_num number
+			of allocations and frees.
+
+	rcuperf.kfree_no_batch= [KNL]
+			Use the non-batching (slower) version of kfree_rcu.
+			This is useful for comparing with the batched version.
+
 	rcuperf.nreaders= [KNL]
 			Set number of RCU readers.  The value -1 selects
 			N, where N is the number of CPUs.  A value
diff --git a/kernel/rcu/rcuperf.c b/kernel/rcu/rcuperf.c
index 7a6890b23c5f..70d6ac19cbff 100644
--- a/kernel/rcu/rcuperf.c
+++ b/kernel/rcu/rcuperf.c
@@ -86,6 +86,7 @@ torture_param(bool, shutdown, RCUPERF_SHUTDOWN,
 	      "Shutdown at end of performance tests.");
 torture_param(int, verbose, 1, "Enable verbose debugging printk()s");
 torture_param(int, writer_holdoff, 0, "Holdoff (us) between GPs, zero to disable");
+torture_param(int, kfree_rcu_test, 0, "Do we run a kfree_rcu perf test?");
 
 static char *perf_type = "rcu";
 module_param(perf_type, charp, 0444);
@@ -105,8 +106,8 @@ static atomic_t n_rcu_perf_writer_finished;
 static wait_queue_head_t shutdown_wq;
 static u64 t_rcu_perf_writer_started;
 static u64 t_rcu_perf_writer_finished;
-static unsigned long b_rcu_perf_writer_started;
-static unsigned long b_rcu_perf_writer_finished;
+static unsigned long b_rcu_gp_test_started;
+static unsigned long b_rcu_gp_test_finished;
 static DEFINE_PER_CPU(atomic_t, n_async_inflight);
 
 static int rcu_perf_writer_state;
@@ -379,10 +380,10 @@ rcu_perf_writer(void *arg)
 	if (atomic_inc_return(&n_rcu_perf_writer_started) >= nrealwriters) {
 		t_rcu_perf_writer_started = t;
 		if (gp_exp) {
-			b_rcu_perf_writer_started =
+			b_rcu_gp_test_started =
 				cur_ops->exp_completed() / 2;
 		} else {
-			b_rcu_perf_writer_started = cur_ops->get_gp_seq();
+			b_rcu_gp_test_started = cur_ops->get_gp_seq();
 		}
 	}
 
@@ -435,10 +436,10 @@ rcu_perf_writer(void *arg)
 				PERFOUT_STRING("Test complete");
 				t_rcu_perf_writer_finished = t;
 				if (gp_exp) {
-					b_rcu_perf_writer_finished =
+					b_rcu_gp_test_finished =
 						cur_ops->exp_completed() / 2;
 				} else {
-					b_rcu_perf_writer_finished =
+					b_rcu_gp_test_finished =
 						cur_ops->get_gp_seq();
 				}
 				if (shutdown) {
@@ -523,8 +524,8 @@ rcu_perf_cleanup(void)
 			 t_rcu_perf_writer_finished -
 			 t_rcu_perf_writer_started,
 			 ngps,
-			 rcuperf_seq_diff(b_rcu_perf_writer_finished,
-					  b_rcu_perf_writer_started));
+			 rcuperf_seq_diff(b_rcu_gp_test_finished,
+					  b_rcu_gp_test_started));
 		for (i = 0; i < nrealwriters; i++) {
 			if (!writer_durations)
 				break;
@@ -592,6 +593,175 @@ rcu_perf_shutdown(void *arg)
 	return -EINVAL;
 }
 
+/*
+ * kfree_rcu performance tests: Start a kfree_rcu loop on all CPUs for number
+ * of iterations and measure total time and number of GP for all iterations to complete.
+ */
+
+torture_param(int, kfree_nthreads, -1, "Number of threads running loops of kfree_rcu().");
+torture_param(int, kfree_alloc_num, 8000, "Number of allocations and frees done in an iteration.");
+torture_param(int, kfree_loops, 10, "Number of loops doing kfree_alloc_num allocations and frees.");
+torture_param(int, kfree_no_batch, 0, "Use the non-batching (slower) version of kfree_rcu.");
+
+static struct task_struct **kfree_reader_tasks;
+static int kfree_nrealthreads;
+static atomic_t n_kfree_perf_thread_started;
+static atomic_t n_kfree_perf_thread_ended;
+
+struct kfree_obj {
+	char kfree_obj[8];
+	struct rcu_head rh;
+};
+
+static int
+kfree_perf_thread(void *arg)
+{
+	int i, loop = 0;
+	long me = (long)arg;
+	struct kfree_obj **alloc_ptrs;
+	u64 start_time, end_time;
+
+	VERBOSE_PERFOUT_STRING("kfree_perf_thread task started");
+	set_cpus_allowed_ptr(current, cpumask_of(me % nr_cpu_ids));
+	set_user_nice(current, MAX_NICE);
+
+	alloc_ptrs = (struct kfree_obj **)kmalloc(sizeof(struct kfree_obj *) * kfree_alloc_num,
+						  GFP_KERNEL);
+	if (!alloc_ptrs)
+		return -ENOMEM;
+
+	start_time = ktime_get_mono_fast_ns();
+
+	if (atomic_inc_return(&n_kfree_perf_thread_started) >= kfree_nrealthreads) {
+		if (gp_exp)
+			b_rcu_gp_test_started = cur_ops->exp_completed() / 2;
+		else
+			b_rcu_gp_test_started = cur_ops->get_gp_seq();
+	}
+
+	do {
+		for (i = 0; i < kfree_alloc_num; i++) {
+			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
+			if (!alloc_ptrs[i])
+				return -ENOMEM;
+		}
+
+		for (i = 0; i < kfree_alloc_num; i++) {
+			if (!kfree_no_batch) {
+				kfree_rcu(alloc_ptrs[i], rh);
+			} else {
+				rcu_callback_t cb;
+
+				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
+				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
+			}
+		}
+
+		cond_resched();
+	} while (!torture_must_stop() && ++loop < kfree_loops);
+
+	if (atomic_inc_return(&n_kfree_perf_thread_ended) >= kfree_nrealthreads) {
+		end_time = ktime_get_mono_fast_ns();
+
+		if (gp_exp)
+			b_rcu_gp_test_finished = cur_ops->exp_completed() / 2;
+		else
+			b_rcu_gp_test_finished = cur_ops->get_gp_seq();
+
+		pr_alert("Total time taken by all kfree'ers: %llu ns, loops: %d, batches: %ld\n",
+		       (unsigned long long)(end_time - start_time), kfree_loops,
+		       rcuperf_seq_diff(b_rcu_gp_test_finished, b_rcu_gp_test_started));
+		if (shutdown) {
+			smp_mb(); /* Assign before wake. */
+			wake_up(&shutdown_wq);
+		}
+	}
+
+	kfree(alloc_ptrs);
+	torture_kthread_stopping("kfree_perf_thread");
+	return 0;
+}
+
+static void
+kfree_perf_cleanup(void)
+{
+	int i;
+
+	if (torture_cleanup_begin())
+		return;
+
+	if (kfree_reader_tasks) {
+		for (i = 0; i < kfree_nrealthreads; i++)
+			torture_stop_kthread(kfree_perf_thread,
+					     kfree_reader_tasks[i]);
+		kfree(kfree_reader_tasks);
+	}
+
+	torture_cleanup_end();
+}
+
+/*
+ * shutdown kthread.  Just waits to be awakened, then shuts down system.
+ */
+static int
+kfree_perf_shutdown(void *arg)
+{
+	do {
+		wait_event(shutdown_wq,
+			   atomic_read(&n_kfree_perf_thread_ended) >=
+			   kfree_nrealthreads);
+	} while (atomic_read(&n_kfree_perf_thread_ended) < kfree_nrealthreads);
+
+	smp_mb(); /* Wake before output. */
+
+	kfree_perf_cleanup();
+	kernel_power_off();
+	return -EINVAL;
+}
+
+static int __init
+kfree_perf_init(void)
+{
+	long i;
+	int firsterr = 0;
+
+	kfree_nrealthreads = compute_real(kfree_nthreads);
+	/* Start up the kthreads. */
+	if (shutdown) {
+		init_waitqueue_head(&shutdown_wq);
+		firsterr = torture_create_kthread(kfree_perf_shutdown, NULL,
+						  shutdown_task);
+		if (firsterr)
+			goto unwind;
+		schedule_timeout_uninterruptible(1);
+	}
+
+	kfree_reader_tasks = kcalloc(kfree_nrealthreads, sizeof(kfree_reader_tasks[0]),
+			       GFP_KERNEL);
+	if (kfree_reader_tasks == NULL) {
+		firsterr = -ENOMEM;
+		goto unwind;
+	}
+
+	for (i = 0; i < kfree_nrealthreads; i++) {
+		firsterr = torture_create_kthread(kfree_perf_thread, (void *)i,
+						  kfree_reader_tasks[i]);
+		if (firsterr)
+			goto unwind;
+	}
+
+	while (atomic_read(&n_kfree_perf_thread_started) < kfree_nrealthreads)
+		schedule_timeout_uninterruptible(1);
+
+	torture_init_end();
+	return 0;
+
+unwind:
+	torture_init_end();
+	kfree_perf_cleanup();
+	return firsterr;
+}
+
 static int __init
 rcu_perf_init(void)
 {
@@ -624,6 +794,9 @@ rcu_perf_init(void)
 	if (cur_ops->init)
 		cur_ops->init();
 
+	if (kfree_rcu_test)
+		return kfree_perf_init();
+
 	nrealwriters = compute_real(nwriters);
 	nrealreaders = compute_real(nreaders);
 	atomic_set(&n_rcu_perf_reader_started, 0);
-- 
2.23.0.rc1.153.gdeed80330f-goog


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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-14 16:04 ` [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests Joel Fernandes (Google)
@ 2019-08-14 22:58   ` Paul E. McKenney
  2019-08-19 19:33     ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-14 22:58 UTC (permalink / raw)
  To: Joel Fernandes (Google)
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Wed, Aug 14, 2019 at 12:04:11PM -0400, Joel Fernandes (Google) wrote:
> This test runs kfree_rcu in a loop to measure performance of the new
> kfree_rcu batching functionality.

kfree_rcu().

> The following table shows results when booting with arguments:
> rcuperf.kfree_loops=200000 rcuperf.kfree_alloc_num=1000 rcuperf.kfree_rcu_test=1
> 
> In addition, rcuperf.kfree_no_batch is used to toggle the batching of
> kfree_rcu()s for a test run.
> 
> rcuperf.kfree_no_batch	GPs	time (seconds)
>  0 (default)		1732	15.9
>  1			9133 	14.5
> 
> Note that the results are the same for the case:
> 1. Patch is not applied and rcuperf.kfree_no_batch=0
> 2. Patch is applied     and rcuperf.kfree_no_batch=1
> 
> On a 16 CPU system with the above boot parameters, we see that the total
> number of grace periods that elapse during the test drops from 9133 when
> not batching to 1732 when batching (a 5X improvement). The kfree_rcu()
> flood itself slows down a bit when batching, though, as shown. This is
> likely due to rcuperf threads contending with the additional worker
> threads that are now running both before (the monitor) and after (the
> work done to kfree()) the grace period.

Another possibility is that the batching approach is resulting in a
greater number of objects waiting to be freed (noted below), and it
takes the extra 1.4 seconds to catch up.  How would you decide which
effect is the most important?  (Your path of least resistance is to
remove the speculation.)

> Note that the active memory consumption during the kfree_rcu() flood
> does increase to around 300-400MB due to the batching (from around 50MB
> without batching). However, this memory consumption is relatively
> constant and is just an effect of the buffering. In other words, the
> system is able to keep up with the kfree_rcu() load. The memory
> consumption comes down to 200-300MB if KFREE_DRAIN_JIFFIES is
> increased from HZ/50 to HZ/80.
> 
> Also, when running the test, please disable CONFIG_DEBUG_PREEMPT and
> CONFIG_PROVE_RCU for realistic comparisons with/without batching.
> 
> Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>

Looks pretty close, just a very few issues needing fixing below.

							Thanx, Paul

> ---
>  .../admin-guide/kernel-parameters.txt         |  17 ++
>  kernel/rcu/rcuperf.c                          | 189 +++++++++++++++++-
>  2 files changed, 198 insertions(+), 8 deletions(-)
> 
> diff --git a/Documentation/admin-guide/kernel-parameters.txt b/Documentation/admin-guide/kernel-parameters.txt
> index 7ccd158b3894..a9156ca5de24 100644
> --- a/Documentation/admin-guide/kernel-parameters.txt
> +++ b/Documentation/admin-guide/kernel-parameters.txt
> @@ -3895,6 +3895,23 @@
>  			test until boot completes in order to avoid
>  			interference.
>  
> +	rcuperf.kfree_rcu_test= [KNL]
> +			Set to measure performance of kfree_rcu() flooding.
> +
> +	rcuperf.kfree_nthreads= [KNL]
> +			The number of threads running loops of kfree_rcu().
> +
> +	rcuperf.kfree_alloc_num= [KNL]
> +			Number of allocations and frees done in an iteration.
> +
> +	rcuperf.kfree_loops= [KNL]
> +			Number of loops doing rcuperf.kfree_alloc_num number
> +			of allocations and frees.
> +
> +	rcuperf.kfree_no_batch= [KNL]
> +			Use the non-batching (slower) version of kfree_rcu.
> +			This is useful for comparing with the batched version.

I suggest s/slower/more efficient/ given that the batching takes more
wall-clock time than does the no-batching.

>  	rcuperf.nreaders= [KNL]
>  			Set number of RCU readers.  The value -1 selects
>  			N, where N is the number of CPUs.  A value
> diff --git a/kernel/rcu/rcuperf.c b/kernel/rcu/rcuperf.c
> index 7a6890b23c5f..70d6ac19cbff 100644
> --- a/kernel/rcu/rcuperf.c
> +++ b/kernel/rcu/rcuperf.c
> @@ -86,6 +86,7 @@ torture_param(bool, shutdown, RCUPERF_SHUTDOWN,
>  	      "Shutdown at end of performance tests.");
>  torture_param(int, verbose, 1, "Enable verbose debugging printk()s");
>  torture_param(int, writer_holdoff, 0, "Holdoff (us) between GPs, zero to disable");
> +torture_param(int, kfree_rcu_test, 0, "Do we run a kfree_rcu perf test?");
>  
>  static char *perf_type = "rcu";
>  module_param(perf_type, charp, 0444);
> @@ -105,8 +106,8 @@ static atomic_t n_rcu_perf_writer_finished;
>  static wait_queue_head_t shutdown_wq;
>  static u64 t_rcu_perf_writer_started;
>  static u64 t_rcu_perf_writer_finished;
> -static unsigned long b_rcu_perf_writer_started;
> -static unsigned long b_rcu_perf_writer_finished;
> +static unsigned long b_rcu_gp_test_started;
> +static unsigned long b_rcu_gp_test_finished;
>  static DEFINE_PER_CPU(atomic_t, n_async_inflight);
>  
>  static int rcu_perf_writer_state;
> @@ -379,10 +380,10 @@ rcu_perf_writer(void *arg)
>  	if (atomic_inc_return(&n_rcu_perf_writer_started) >= nrealwriters) {
>  		t_rcu_perf_writer_started = t;
>  		if (gp_exp) {
> -			b_rcu_perf_writer_started =
> +			b_rcu_gp_test_started =
>  				cur_ops->exp_completed() / 2;
>  		} else {
> -			b_rcu_perf_writer_started = cur_ops->get_gp_seq();
> +			b_rcu_gp_test_started = cur_ops->get_gp_seq();
>  		}
>  	}
>  
> @@ -435,10 +436,10 @@ rcu_perf_writer(void *arg)
>  				PERFOUT_STRING("Test complete");
>  				t_rcu_perf_writer_finished = t;
>  				if (gp_exp) {
> -					b_rcu_perf_writer_finished =
> +					b_rcu_gp_test_finished =
>  						cur_ops->exp_completed() / 2;
>  				} else {
> -					b_rcu_perf_writer_finished =
> +					b_rcu_gp_test_finished =
>  						cur_ops->get_gp_seq();
>  				}
>  				if (shutdown) {
> @@ -523,8 +524,8 @@ rcu_perf_cleanup(void)
>  			 t_rcu_perf_writer_finished -
>  			 t_rcu_perf_writer_started,
>  			 ngps,
> -			 rcuperf_seq_diff(b_rcu_perf_writer_finished,
> -					  b_rcu_perf_writer_started));
> +			 rcuperf_seq_diff(b_rcu_gp_test_finished,
> +					  b_rcu_gp_test_started));
>  		for (i = 0; i < nrealwriters; i++) {
>  			if (!writer_durations)
>  				break;
> @@ -592,6 +593,175 @@ rcu_perf_shutdown(void *arg)
>  	return -EINVAL;
>  }
>  
> +/*
> + * kfree_rcu performance tests: Start a kfree_rcu loop on all CPUs for number
> + * of iterations and measure total time and number of GP for all iterations to complete.
> + */
> +
> +torture_param(int, kfree_nthreads, -1, "Number of threads running loops of kfree_rcu().");
> +torture_param(int, kfree_alloc_num, 8000, "Number of allocations and frees done in an iteration.");
> +torture_param(int, kfree_loops, 10, "Number of loops doing kfree_alloc_num allocations and frees.");
> +torture_param(int, kfree_no_batch, 0, "Use the non-batching (slower) version of kfree_rcu.");
> +
> +static struct task_struct **kfree_reader_tasks;
> +static int kfree_nrealthreads;
> +static atomic_t n_kfree_perf_thread_started;
> +static atomic_t n_kfree_perf_thread_ended;
> +
> +struct kfree_obj {
> +	char kfree_obj[8];
> +	struct rcu_head rh;
> +};

(Aside from above, no need to change this part of the patch, at least not
that I know of at the moment.)

24 bytes on a 64-bit system, 16 on a 32-bit system.  So there might
have been 10 million extra objects awaiting free in the batching case
given the 400M-50M=350M excess for the batching approach.  If freeing
each object took about 100ns, that could account for the additional
wall-clock time for the batching approach.

> +
> +static int
> +kfree_perf_thread(void *arg)
> +{
> +	int i, loop = 0;
> +	long me = (long)arg;
> +	struct kfree_obj **alloc_ptrs;
> +	u64 start_time, end_time;
> +
> +	VERBOSE_PERFOUT_STRING("kfree_perf_thread task started");
> +	set_cpus_allowed_ptr(current, cpumask_of(me % nr_cpu_ids));

(No need for a change, another aside:  This assumes dense CPU numbering,
which will cause trouble at some point.  As you may have noticed from
the other similar code in rcuperf.c, I have been using the strategy of
waiting until a real problem shows up before fixing it.)

> +	set_user_nice(current, MAX_NICE);
> +
> +	alloc_ptrs = (struct kfree_obj **)kmalloc(sizeof(struct kfree_obj *) * kfree_alloc_num,
> +						  GFP_KERNEL);
> +	if (!alloc_ptrs)
> +		return -ENOMEM;
> +
> +	start_time = ktime_get_mono_fast_ns();
> +
> +	if (atomic_inc_return(&n_kfree_perf_thread_started) >= kfree_nrealthreads) {
> +		if (gp_exp)
> +			b_rcu_gp_test_started = cur_ops->exp_completed() / 2;
> +		else
> +			b_rcu_gp_test_started = cur_ops->get_gp_seq();
> +	}
> +
> +	do {
> +		for (i = 0; i < kfree_alloc_num; i++) {
> +			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> +			if (!alloc_ptrs[i])
> +				return -ENOMEM;
> +		}
> +
> +		for (i = 0; i < kfree_alloc_num; i++) {
> +			if (!kfree_no_batch) {
> +				kfree_rcu(alloc_ptrs[i], rh);
> +			} else {
> +				rcu_callback_t cb;
> +
> +				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> +				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> +			}
> +		}

The point of allocating a large batch and then kfree_rcu()ing them in a
loop is to defeat the per-CPU pool optimization?  Either way, a comment
would be very good!

> +
> +		cond_resched();
> +	} while (!torture_must_stop() && ++loop < kfree_loops);
> +
> +	if (atomic_inc_return(&n_kfree_perf_thread_ended) >= kfree_nrealthreads) {
> +		end_time = ktime_get_mono_fast_ns();
> +
> +		if (gp_exp)
> +			b_rcu_gp_test_finished = cur_ops->exp_completed() / 2;

Why not use a .gp_diff field similar to the way that rcutorture does?
(Yes, rcutorture ignores numbers of expedited grace periods, but the
GP sequence numbers now have the same formats in both cases.)

This can be a follow-on.

> +		else
> +			b_rcu_gp_test_finished = cur_ops->get_gp_seq();
> +
> +		pr_alert("Total time taken by all kfree'ers: %llu ns, loops: %d, batches: %ld\n",
> +		       (unsigned long long)(end_time - start_time), kfree_loops,
> +		       rcuperf_seq_diff(b_rcu_gp_test_finished, b_rcu_gp_test_started));
> +		if (shutdown) {
> +			smp_mb(); /* Assign before wake. */
> +			wake_up(&shutdown_wq);
> +		}
> +	}
> +
> +	kfree(alloc_ptrs);
> +	torture_kthread_stopping("kfree_perf_thread");
> +	return 0;
> +}
> +
> +static void
> +kfree_perf_cleanup(void)
> +{
> +	int i;
> +
> +	if (torture_cleanup_begin())
> +		return;
> +
> +	if (kfree_reader_tasks) {
> +		for (i = 0; i < kfree_nrealthreads; i++)
> +			torture_stop_kthread(kfree_perf_thread,
> +					     kfree_reader_tasks[i]);
> +		kfree(kfree_reader_tasks);
> +	}
> +
> +	torture_cleanup_end();
> +}
> +
> +/*
> + * shutdown kthread.  Just waits to be awakened, then shuts down system.
> + */
> +static int
> +kfree_perf_shutdown(void *arg)
> +{
> +	do {
> +		wait_event(shutdown_wq,
> +			   atomic_read(&n_kfree_perf_thread_ended) >=
> +			   kfree_nrealthreads);
> +	} while (atomic_read(&n_kfree_perf_thread_ended) < kfree_nrealthreads);
> +
> +	smp_mb(); /* Wake before output. */
> +
> +	kfree_perf_cleanup();
> +	kernel_power_off();
> +	return -EINVAL;
> +}
> +
> +static int __init
> +kfree_perf_init(void)
> +{
> +	long i;
> +	int firsterr = 0;
> +
> +	kfree_nrealthreads = compute_real(kfree_nthreads);
> +	/* Start up the kthreads. */
> +	if (shutdown) {
> +		init_waitqueue_head(&shutdown_wq);
> +		firsterr = torture_create_kthread(kfree_perf_shutdown, NULL,
> +						  shutdown_task);
> +		if (firsterr)
> +			goto unwind;
> +		schedule_timeout_uninterruptible(1);
> +	}
> +
> +	kfree_reader_tasks = kcalloc(kfree_nrealthreads, sizeof(kfree_reader_tasks[0]),
> +			       GFP_KERNEL);
> +	if (kfree_reader_tasks == NULL) {
> +		firsterr = -ENOMEM;
> +		goto unwind;
> +	}
> +
> +	for (i = 0; i < kfree_nrealthreads; i++) {
> +		firsterr = torture_create_kthread(kfree_perf_thread, (void *)i,
> +						  kfree_reader_tasks[i]);
> +		if (firsterr)
> +			goto unwind;
> +	}
> +
> +	while (atomic_read(&n_kfree_perf_thread_started) < kfree_nrealthreads)
> +		schedule_timeout_uninterruptible(1);
> +
> +	torture_init_end();
> +	return 0;
> +
> +unwind:
> +	torture_init_end();
> +	kfree_perf_cleanup();
> +	return firsterr;
> +}
> +
>  static int __init
>  rcu_perf_init(void)
>  {
> @@ -624,6 +794,9 @@ rcu_perf_init(void)
>  	if (cur_ops->init)
>  		cur_ops->init();
>  
> +	if (kfree_rcu_test)
> +		return kfree_perf_init();
> +
>  	nrealwriters = compute_real(nwriters);
>  	nrealreaders = compute_real(nreaders);
>  	atomic_set(&n_rcu_perf_reader_started, 0);
> -- 
> 2.23.0.rc1.153.gdeed80330f-goog
> 

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-14 16:04 [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Joel Fernandes (Google)
  2019-08-14 16:04 ` [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests Joel Fernandes (Google)
@ 2019-08-16 16:43 ` Paul E. McKenney
  2019-08-16 17:44   ` Joel Fernandes
  2019-09-18  9:58 ` Uladzislau Rezki
  2 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-16 16:43 UTC (permalink / raw)
  To: Joel Fernandes (Google)
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

Hello, Joel,

I reworked the commit log as follows, but was then unsuccessful in
working out which -rcu commit to apply it to.  Could you please
tell me what commit to apply this to?  (Once applied, git cherry-pick
is usually pretty good about handling minor conflicts.)

							Thanx, Paul

On Wed, Aug 14, 2019 at 12:04:10PM -0400, Joel Fernandes (Google) wrote:
> A recent discussion about stability and performance systems with
> kfree_rcu() flooding [1] led to another discussion how to better handle
> this situation.
> 
> This commit starts simple, adding only basic batching support for
> kfree_rcu(). It is "basic" because it does none of the slab management,
> dynamic allocation, or code movement carried out by a previous attempt
> [2].  These additional improvements can be implemented later as agreement
> is reached on these other issues.  For example, future work might
> increase cache locality by applying vector object lists, kfree_bulk(),
> or per-slab batching to further improve handling of kfree_rcu() floods.
> 
> Performance tests are provided in a latter commmit.  These tests show a
> 5x reduction in number of grace periods on a 16 CPU system, with minimal
> increase in kfree() latency.
> 
> Note that this commit prevents rcu_barrier() from waiting for the
> execution of the kfree() calls associated with prior kfree_rcu()
> invocations.  This should not be a problem, given that the resulting
> pending kfree() calls do not prevent module unloading or filesystem
> unmounting.  The reason rcu_barrier() no longer waits for the kfree()
> calls is that the kfree_rcu() requests are now batched, so that at
> any given time there might be kfree_rcu() requests that are not yet
> known to the core RCU machinery.  Furthermore, once a kfree_rcu()
> grace period has elapsed, the actual kfree() invocations happen in
> workqueue context.  So rcu_barrier() no longer waits for all of the
> prior requests, nor it does not wait for the workqueue handlers to
> start, let alone complete.  If there is ever a good reason to wait for
> the kfree() invocation corresponding to all prior kfree_rcu() calls,
> an approapriate kfree_rcu_barrier() can be constructed.  However, at
> the moment no reasonable use case is apparent.
> 
> This commit can result in increased memory footprint because the
> batching can increase the kfree_rcu()-to-kfree() latency.  Later
> commits will reduce this memory footprint.
> 
> Later commits will also remove the special handling of kfree_rcu() by
> __rcu_reclaim() within the RCU core.  This will require changes to
> rcuperf testing and to early boot handling of kfree_rcu().
> 
> [1] http://lore.kernel.org/lkml/20190723035725-mutt-send-email-mst@kernel.org
> [2] https://lkml.org/lkml/2017/12/19/824
> 
> Cc: kernel-team@android.com
> Cc: kernel-team@lge.com
> Co-developed-by: Byungchul Park <byungchul.park@lge.com>
> Signed-off-by: Byungchul Park <byungchul.park@lge.com>
> Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>
> 
> ---
> v3->v4: Some corrections by Paul.
> 	Used xchg in places to simplify code.
> 
> v2->v3: Just some code comment changes thanks to Byungchul.
> 
> RFCv1->PATCH v2: Removed limits on the ->head list, just let it grow.
>                    Dropped KFREE_MAX_JIFFIES to HZ/50 from HZ/20 to reduce OOM occurrence.
>                    Removed sleeps in rcuperf test, just using cond_resched()in loop.
>                    Better code comments ;)
> 
>  include/linux/rcutiny.h |   5 ++
>  include/linux/rcutree.h |   1 +
>  kernel/rcu/tree.c       | 194 ++++++++++++++++++++++++++++++++++++++--
>  3 files changed, 194 insertions(+), 6 deletions(-)
> 
> diff --git a/include/linux/rcutiny.h b/include/linux/rcutiny.h
> index 8e727f57d814..383f2481750f 100644
> --- a/include/linux/rcutiny.h
> +++ b/include/linux/rcutiny.h
> @@ -39,6 +39,11 @@ static inline void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
>  	call_rcu(head, func);
>  }
>  
> +static inline void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
> +{
> +	call_rcu(head, func);
> +}
> +
>  void rcu_qs(void);
>  
>  static inline void rcu_softirq_qs(void)
> diff --git a/include/linux/rcutree.h b/include/linux/rcutree.h
> index 735601ac27d3..7e38b39ec634 100644
> --- a/include/linux/rcutree.h
> +++ b/include/linux/rcutree.h
> @@ -34,6 +34,7 @@ static inline void rcu_virt_note_context_switch(int cpu)
>  
>  void synchronize_rcu_expedited(void);
>  void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func);
> +void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func);
>  
>  void rcu_barrier(void);
>  bool rcu_eqs_special_set(int cpu);
> diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> index a14e5fbbea46..1d1847cadea2 100644
> --- a/kernel/rcu/tree.c
> +++ b/kernel/rcu/tree.c
> @@ -2593,17 +2593,185 @@ void call_rcu(struct rcu_head *head, rcu_callback_t func)
>  }
>  EXPORT_SYMBOL_GPL(call_rcu);
>  
> +
> +/* Maximum number of jiffies to wait before draining a batch. */
> +#define KFREE_DRAIN_JIFFIES (HZ / 50)
> +
>  /*
> - * Queue an RCU callback for lazy invocation after a grace period.
> - * This will likely be later named something like "call_rcu_lazy()",
> - * but this change will require some way of tagging the lazy RCU
> - * callbacks in the list of pending callbacks. Until then, this
> - * function may only be called from __kfree_rcu().
> + * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> + * kfree(s) is queued for freeing after a grace period, right away.
>   */
> -void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> +struct kfree_rcu_cpu {
> +	/* The rcu_work node for queuing work with queue_rcu_work(). The work
> +	 * is done after a grace period.
> +	 */
> +	struct rcu_work rcu_work;
> +
> +	/* The list of objects being queued in a batch but are not yet
> +	 * scheduled to be freed.
> +	 */
> +	struct rcu_head *head;
> +
> +	/* The list of objects that have now left ->head and are queued for
> +	 * freeing after a grace period.
> +	 */
> +	struct rcu_head *head_free;
> +
> +	/* Protect concurrent access to this structure. */
> +	spinlock_t lock;
> +
> +	/* The delayed work that flushes ->head to ->head_free incase ->head
> +	 * within KFREE_DRAIN_JIFFIES. In case flushing cannot be done if RCU
> +	 * is busy, ->head just continues to grow and we retry flushing later.
> +	 */
> +	struct delayed_work monitor_work;
> +	bool monitor_todo;	/* Is a delayed work pending execution? */
> +};
> +
> +static DEFINE_PER_CPU(struct kfree_rcu_cpu, krc);
> +
> +/*
> + * This function is invoked in workqueue context after a grace period.
> + * It frees all the objects queued on ->head_free.
> + */
> +static void kfree_rcu_work(struct work_struct *work)
> +{
> +	unsigned long flags;
> +	struct rcu_head *head, *next;
> +	struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
> +					struct kfree_rcu_cpu, rcu_work);
> +
> +	spin_lock_irqsave(&krcp->lock, flags);
> +	head = krcp->head_free;
> +	krcp->head_free = NULL;
> +	spin_unlock_irqrestore(&krcp->lock, flags);
> +
> +	/*
> +	 * The head is detached and not referenced from anywhere, so lockless
> +	 * access is Ok.
> +	 */
> +	for (; head; head = next) {
> +		next = head->next;
> +		/* Could be possible to optimize with kfree_bulk in future */
> +		__rcu_reclaim(rcu_state.name, head);
> +		cond_resched_tasks_rcu_qs();
> +	}
> +}
> +
> +/*
> + * Schedule the kfree batch RCU work to run in workqueue context after a GP.
> + *
> + * This function is invoked by kfree_rcu_monitor() when the KFREE_DRAIN_JIFFIES
> + * timeout has been reached.
> + */
> +static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
> +{
> +	lockdep_assert_held(&krcp->lock);
> +
> +	/* If a previous RCU batch work is already in progress, we cannot queue
> +	 * another one, just refuse the optimization and it will be retried
> +	 * again in KFREE_DRAIN_JIFFIES time.
> +	 */
> +	if (krcp->head_free)
> +		return false;
> +
> +	krcp->head_free = krcp->head;
> +	krcp->head = NULL;
> +	INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
> +	queue_rcu_work(system_wq, &krcp->rcu_work);
> +
> +	return true;
> +}
> +
> +static inline void kfree_rcu_drain_unlock(struct kfree_rcu_cpu *krcp,
> +				   unsigned long flags)
> +{
> +	/* Flush ->head to ->head_free, all objects on ->head_free will be
> +	 * kfree'd after a grace period.
> +	 */
> +	if (queue_kfree_rcu_work(krcp)) {
> +		/* Success! Our job is done here. */
> +		spin_unlock_irqrestore(&krcp->lock, flags);
> +		return;
> +	}
> +
> +	/* Previous batch that was queued to RCU did not get free yet, let us
> +	 * try again soon.
> +	 */
> +	if (!xchg(&krcp->monitor_todo, true))
> +		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> +	spin_unlock_irqrestore(&krcp->lock, flags);
> +}
> +
> +/*
> + * This function is invoked after the KFREE_DRAIN_JIFFIES timeout has elapsed,
> + * and it drains the specified kfree_rcu_cpu structure's ->head list.
> + */
> +static void kfree_rcu_monitor(struct work_struct *work)
> +{
> +	unsigned long flags;
> +	struct kfree_rcu_cpu *krcp = container_of(work, struct kfree_rcu_cpu,
> +						 monitor_work.work);
> +
> +	spin_lock_irqsave(&krcp->lock, flags);
> +	if (xchg(&krcp->monitor_todo, false))
> +		kfree_rcu_drain_unlock(krcp, flags);
> +	else
> +		spin_unlock_irqrestore(&krcp->lock, flags);
> +}
> +
> +/*
> + * This version of kfree_call_rcu does not do batching of kfree_rcu() requests.
> + * Used only by rcuperf torture test for comparison with kfree_rcu_batch().
> + */
> +void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
>  {
>  	__call_rcu(head, func, -1, 1);
>  }
> +EXPORT_SYMBOL_GPL(kfree_call_rcu_nobatch);
> +
> +/*
> + * Queue a request for lazy invocation of kfree() after a grace period.
> + *
> + * Each kfree_call_rcu() request is added to a batch. The batch will be drained
> + * every KFREE_DRAIN_JIFFIES number of jiffies. All the objects in the batch
> + * will be kfree'd in workqueue context. This allows us to:
> + *
> + * 1. Batch requests together to reduce the number of grace periods during
> + * heavy kfree_rcu() load.
> + *
> + * 2. In the future, makes it possible to use kfree_bulk() on a large number of
> + * kfree_rcu() requests thus reducing the per-object overhead of kfree() and
> + * also reducing cache misses.
> + */
> +void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> +{
> +	unsigned long flags;
> +	struct kfree_rcu_cpu *krcp;
> +
> +	/* kfree_call_rcu() batching requires timers to be up. If the scheduler
> +	 * is not yet up, just skip batching and do the non-batched version.
> +	 */
> +	if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
> +		return kfree_call_rcu_nobatch(head, func);
> +
> +	head->func = func;
> +
> +	local_irq_save(flags);	/* For safely calling this_cpu_ptr(). */
> +	krcp = this_cpu_ptr(&krc);
> +	spin_lock(&krcp->lock);
> +
> +	/* Queue the kfree but don't yet schedule the batch. */
> +	head->next = krcp->head;
> +	krcp->head = head;
> +
> +	/* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
> +	if (!xchg(&krcp->monitor_todo, true))
> +		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> +
> +	spin_unlock(&krcp->lock);
> +	local_irq_restore(flags);
> +}
>  EXPORT_SYMBOL_GPL(kfree_call_rcu);
>  
>  /*
> @@ -3455,10 +3623,24 @@ static void __init rcu_dump_rcu_node_tree(void)
>  struct workqueue_struct *rcu_gp_wq;
>  struct workqueue_struct *rcu_par_gp_wq;
>  
> +static void __init kfree_rcu_batch_init(void)
> +{
> +	int cpu;
> +
> +	for_each_possible_cpu(cpu) {
> +		struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
> +
> +		spin_lock_init(&krcp->lock);
> +		INIT_DELAYED_WORK(&krcp->monitor_work, kfree_rcu_monitor);
> +	}
> +}
> +
>  void __init rcu_init(void)
>  {
>  	int cpu;
>  
> +	kfree_rcu_batch_init();
> +
>  	rcu_early_boot_tests();
>  
>  	rcu_bootup_announce();
> -- 
> 2.23.0.rc1.153.gdeed80330f-goog
> 

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-16 16:43 ` [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Paul E. McKenney
@ 2019-08-16 17:44   ` Joel Fernandes
  2019-08-16 19:16     ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-16 17:44 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Fri, Aug 16, 2019 at 09:43:30AM -0700, Paul E. McKenney wrote:
> Hello, Joel,
> 
> I reworked the commit log as follows, but was then unsuccessful in
> working out which -rcu commit to apply it to.  Could you please
> tell me what commit to apply this to?  (Once applied, git cherry-pick
> is usually pretty good about handling minor conflicts.)

It was originally based on v5.3-rc2

I was able to apply it just now to the rcu -dev branch and I pushed it here:
https://github.com/joelagnel/linux-kernel.git (branch paul-dev)

Let me know if any other issues, thanks for the change log rework!

thanks,

 - Joel

> 							Thanx, Paul
> 
> On Wed, Aug 14, 2019 at 12:04:10PM -0400, Joel Fernandes (Google) wrote:
> > A recent discussion about stability and performance systems with
> > kfree_rcu() flooding [1] led to another discussion how to better handle
> > this situation.
> > 
> > This commit starts simple, adding only basic batching support for
> > kfree_rcu(). It is "basic" because it does none of the slab management,
> > dynamic allocation, or code movement carried out by a previous attempt
> > [2].  These additional improvements can be implemented later as agreement
> > is reached on these other issues.  For example, future work might
> > increase cache locality by applying vector object lists, kfree_bulk(),
> > or per-slab batching to further improve handling of kfree_rcu() floods.
> > 
> > Performance tests are provided in a latter commmit.  These tests show a
> > 5x reduction in number of grace periods on a 16 CPU system, with minimal
> > increase in kfree() latency.
> > 
> > Note that this commit prevents rcu_barrier() from waiting for the
> > execution of the kfree() calls associated with prior kfree_rcu()
> > invocations.  This should not be a problem, given that the resulting
> > pending kfree() calls do not prevent module unloading or filesystem
> > unmounting.  The reason rcu_barrier() no longer waits for the kfree()
> > calls is that the kfree_rcu() requests are now batched, so that at
> > any given time there might be kfree_rcu() requests that are not yet
> > known to the core RCU machinery.  Furthermore, once a kfree_rcu()
> > grace period has elapsed, the actual kfree() invocations happen in
> > workqueue context.  So rcu_barrier() no longer waits for all of the
> > prior requests, nor it does not wait for the workqueue handlers to
> > start, let alone complete.  If there is ever a good reason to wait for
> > the kfree() invocation corresponding to all prior kfree_rcu() calls,
> > an approapriate kfree_rcu_barrier() can be constructed.  However, at
> > the moment no reasonable use case is apparent.
> > 
> > This commit can result in increased memory footprint because the
> > batching can increase the kfree_rcu()-to-kfree() latency.  Later
> > commits will reduce this memory footprint.
> > 
> > Later commits will also remove the special handling of kfree_rcu() by
> > __rcu_reclaim() within the RCU core.  This will require changes to
> > rcuperf testing and to early boot handling of kfree_rcu().
> > 
> > [1] http://lore.kernel.org/lkml/20190723035725-mutt-send-email-mst@kernel.org
> > [2] https://lkml.org/lkml/2017/12/19/824
> > 
> > Cc: kernel-team@android.com
> > Cc: kernel-team@lge.com
> > Co-developed-by: Byungchul Park <byungchul.park@lge.com>
> > Signed-off-by: Byungchul Park <byungchul.park@lge.com>
> > Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>
> > 
> > ---
> > v3->v4: Some corrections by Paul.
> > 	Used xchg in places to simplify code.
> > 
> > v2->v3: Just some code comment changes thanks to Byungchul.
> > 
> > RFCv1->PATCH v2: Removed limits on the ->head list, just let it grow.
> >                    Dropped KFREE_MAX_JIFFIES to HZ/50 from HZ/20 to reduce OOM occurrence.
> >                    Removed sleeps in rcuperf test, just using cond_resched()in loop.
> >                    Better code comments ;)
> > 
> >  include/linux/rcutiny.h |   5 ++
> >  include/linux/rcutree.h |   1 +
> >  kernel/rcu/tree.c       | 194 ++++++++++++++++++++++++++++++++++++++--
> >  3 files changed, 194 insertions(+), 6 deletions(-)
> > 
> > diff --git a/include/linux/rcutiny.h b/include/linux/rcutiny.h
> > index 8e727f57d814..383f2481750f 100644
> > --- a/include/linux/rcutiny.h
> > +++ b/include/linux/rcutiny.h
> > @@ -39,6 +39,11 @@ static inline void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> >  	call_rcu(head, func);
> >  }
> >  
> > +static inline void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
> > +{
> > +	call_rcu(head, func);
> > +}
> > +
> >  void rcu_qs(void);
> >  
> >  static inline void rcu_softirq_qs(void)
> > diff --git a/include/linux/rcutree.h b/include/linux/rcutree.h
> > index 735601ac27d3..7e38b39ec634 100644
> > --- a/include/linux/rcutree.h
> > +++ b/include/linux/rcutree.h
> > @@ -34,6 +34,7 @@ static inline void rcu_virt_note_context_switch(int cpu)
> >  
> >  void synchronize_rcu_expedited(void);
> >  void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func);
> > +void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func);
> >  
> >  void rcu_barrier(void);
> >  bool rcu_eqs_special_set(int cpu);
> > diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> > index a14e5fbbea46..1d1847cadea2 100644
> > --- a/kernel/rcu/tree.c
> > +++ b/kernel/rcu/tree.c
> > @@ -2593,17 +2593,185 @@ void call_rcu(struct rcu_head *head, rcu_callback_t func)
> >  }
> >  EXPORT_SYMBOL_GPL(call_rcu);
> >  
> > +
> > +/* Maximum number of jiffies to wait before draining a batch. */
> > +#define KFREE_DRAIN_JIFFIES (HZ / 50)
> > +
> >  /*
> > - * Queue an RCU callback for lazy invocation after a grace period.
> > - * This will likely be later named something like "call_rcu_lazy()",
> > - * but this change will require some way of tagging the lazy RCU
> > - * callbacks in the list of pending callbacks. Until then, this
> > - * function may only be called from __kfree_rcu().
> > + * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> > + * kfree(s) is queued for freeing after a grace period, right away.
> >   */
> > -void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> > +struct kfree_rcu_cpu {
> > +	/* The rcu_work node for queuing work with queue_rcu_work(). The work
> > +	 * is done after a grace period.
> > +	 */
> > +	struct rcu_work rcu_work;
> > +
> > +	/* The list of objects being queued in a batch but are not yet
> > +	 * scheduled to be freed.
> > +	 */
> > +	struct rcu_head *head;
> > +
> > +	/* The list of objects that have now left ->head and are queued for
> > +	 * freeing after a grace period.
> > +	 */
> > +	struct rcu_head *head_free;
> > +
> > +	/* Protect concurrent access to this structure. */
> > +	spinlock_t lock;
> > +
> > +	/* The delayed work that flushes ->head to ->head_free incase ->head
> > +	 * within KFREE_DRAIN_JIFFIES. In case flushing cannot be done if RCU
> > +	 * is busy, ->head just continues to grow and we retry flushing later.
> > +	 */
> > +	struct delayed_work monitor_work;
> > +	bool monitor_todo;	/* Is a delayed work pending execution? */
> > +};
> > +
> > +static DEFINE_PER_CPU(struct kfree_rcu_cpu, krc);
> > +
> > +/*
> > + * This function is invoked in workqueue context after a grace period.
> > + * It frees all the objects queued on ->head_free.
> > + */
> > +static void kfree_rcu_work(struct work_struct *work)
> > +{
> > +	unsigned long flags;
> > +	struct rcu_head *head, *next;
> > +	struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
> > +					struct kfree_rcu_cpu, rcu_work);
> > +
> > +	spin_lock_irqsave(&krcp->lock, flags);
> > +	head = krcp->head_free;
> > +	krcp->head_free = NULL;
> > +	spin_unlock_irqrestore(&krcp->lock, flags);
> > +
> > +	/*
> > +	 * The head is detached and not referenced from anywhere, so lockless
> > +	 * access is Ok.
> > +	 */
> > +	for (; head; head = next) {
> > +		next = head->next;
> > +		/* Could be possible to optimize with kfree_bulk in future */
> > +		__rcu_reclaim(rcu_state.name, head);
> > +		cond_resched_tasks_rcu_qs();
> > +	}
> > +}
> > +
> > +/*
> > + * Schedule the kfree batch RCU work to run in workqueue context after a GP.
> > + *
> > + * This function is invoked by kfree_rcu_monitor() when the KFREE_DRAIN_JIFFIES
> > + * timeout has been reached.
> > + */
> > +static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
> > +{
> > +	lockdep_assert_held(&krcp->lock);
> > +
> > +	/* If a previous RCU batch work is already in progress, we cannot queue
> > +	 * another one, just refuse the optimization and it will be retried
> > +	 * again in KFREE_DRAIN_JIFFIES time.
> > +	 */
> > +	if (krcp->head_free)
> > +		return false;
> > +
> > +	krcp->head_free = krcp->head;
> > +	krcp->head = NULL;
> > +	INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
> > +	queue_rcu_work(system_wq, &krcp->rcu_work);
> > +
> > +	return true;
> > +}
> > +
> > +static inline void kfree_rcu_drain_unlock(struct kfree_rcu_cpu *krcp,
> > +				   unsigned long flags)
> > +{
> > +	/* Flush ->head to ->head_free, all objects on ->head_free will be
> > +	 * kfree'd after a grace period.
> > +	 */
> > +	if (queue_kfree_rcu_work(krcp)) {
> > +		/* Success! Our job is done here. */
> > +		spin_unlock_irqrestore(&krcp->lock, flags);
> > +		return;
> > +	}
> > +
> > +	/* Previous batch that was queued to RCU did not get free yet, let us
> > +	 * try again soon.
> > +	 */
> > +	if (!xchg(&krcp->monitor_todo, true))
> > +		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> > +	spin_unlock_irqrestore(&krcp->lock, flags);
> > +}
> > +
> > +/*
> > + * This function is invoked after the KFREE_DRAIN_JIFFIES timeout has elapsed,
> > + * and it drains the specified kfree_rcu_cpu structure's ->head list.
> > + */
> > +static void kfree_rcu_monitor(struct work_struct *work)
> > +{
> > +	unsigned long flags;
> > +	struct kfree_rcu_cpu *krcp = container_of(work, struct kfree_rcu_cpu,
> > +						 monitor_work.work);
> > +
> > +	spin_lock_irqsave(&krcp->lock, flags);
> > +	if (xchg(&krcp->monitor_todo, false))
> > +		kfree_rcu_drain_unlock(krcp, flags);
> > +	else
> > +		spin_unlock_irqrestore(&krcp->lock, flags);
> > +}
> > +
> > +/*
> > + * This version of kfree_call_rcu does not do batching of kfree_rcu() requests.
> > + * Used only by rcuperf torture test for comparison with kfree_rcu_batch().
> > + */
> > +void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
> >  {
> >  	__call_rcu(head, func, -1, 1);
> >  }
> > +EXPORT_SYMBOL_GPL(kfree_call_rcu_nobatch);
> > +
> > +/*
> > + * Queue a request for lazy invocation of kfree() after a grace period.
> > + *
> > + * Each kfree_call_rcu() request is added to a batch. The batch will be drained
> > + * every KFREE_DRAIN_JIFFIES number of jiffies. All the objects in the batch
> > + * will be kfree'd in workqueue context. This allows us to:
> > + *
> > + * 1. Batch requests together to reduce the number of grace periods during
> > + * heavy kfree_rcu() load.
> > + *
> > + * 2. In the future, makes it possible to use kfree_bulk() on a large number of
> > + * kfree_rcu() requests thus reducing the per-object overhead of kfree() and
> > + * also reducing cache misses.
> > + */
> > +void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> > +{
> > +	unsigned long flags;
> > +	struct kfree_rcu_cpu *krcp;
> > +
> > +	/* kfree_call_rcu() batching requires timers to be up. If the scheduler
> > +	 * is not yet up, just skip batching and do the non-batched version.
> > +	 */
> > +	if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
> > +		return kfree_call_rcu_nobatch(head, func);
> > +
> > +	head->func = func;
> > +
> > +	local_irq_save(flags);	/* For safely calling this_cpu_ptr(). */
> > +	krcp = this_cpu_ptr(&krc);
> > +	spin_lock(&krcp->lock);
> > +
> > +	/* Queue the kfree but don't yet schedule the batch. */
> > +	head->next = krcp->head;
> > +	krcp->head = head;
> > +
> > +	/* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
> > +	if (!xchg(&krcp->monitor_todo, true))
> > +		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> > +
> > +	spin_unlock(&krcp->lock);
> > +	local_irq_restore(flags);
> > +}
> >  EXPORT_SYMBOL_GPL(kfree_call_rcu);
> >  
> >  /*
> > @@ -3455,10 +3623,24 @@ static void __init rcu_dump_rcu_node_tree(void)
> >  struct workqueue_struct *rcu_gp_wq;
> >  struct workqueue_struct *rcu_par_gp_wq;
> >  
> > +static void __init kfree_rcu_batch_init(void)
> > +{
> > +	int cpu;
> > +
> > +	for_each_possible_cpu(cpu) {
> > +		struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
> > +
> > +		spin_lock_init(&krcp->lock);
> > +		INIT_DELAYED_WORK(&krcp->monitor_work, kfree_rcu_monitor);
> > +	}
> > +}
> > +
> >  void __init rcu_init(void)
> >  {
> >  	int cpu;
> >  
> > +	kfree_rcu_batch_init();
> > +
> >  	rcu_early_boot_tests();
> >  
> >  	rcu_bootup_announce();
> > -- 
> > 2.23.0.rc1.153.gdeed80330f-goog
> > 

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-16 17:44   ` Joel Fernandes
@ 2019-08-16 19:16     ` Paul E. McKenney
  2019-08-17  1:32       ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-16 19:16 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Fri, Aug 16, 2019 at 01:44:29PM -0400, Joel Fernandes wrote:
> On Fri, Aug 16, 2019 at 09:43:30AM -0700, Paul E. McKenney wrote:
> > Hello, Joel,
> > 
> > I reworked the commit log as follows, but was then unsuccessful in
> > working out which -rcu commit to apply it to.  Could you please
> > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > is usually pretty good about handling minor conflicts.)
> 
> It was originally based on v5.3-rc2
> 
> I was able to apply it just now to the rcu -dev branch and I pushed it here:
> https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> 
> Let me know if any other issues, thanks for the change log rework!

Pulled and cherry-picked, thank you!

Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
results of the pull.  If you pull that branch, then run something like
"gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
two might illustrate some of the reasons for the current restrictions
on pull requests and trees subject to rebase.

							Thanx, Paul

> thanks,
> 
>  - Joel
> 
> > 							Thanx, Paul
> > 
> > On Wed, Aug 14, 2019 at 12:04:10PM -0400, Joel Fernandes (Google) wrote:
> > > A recent discussion about stability and performance systems with
> > > kfree_rcu() flooding [1] led to another discussion how to better handle
> > > this situation.
> > > 
> > > This commit starts simple, adding only basic batching support for
> > > kfree_rcu(). It is "basic" because it does none of the slab management,
> > > dynamic allocation, or code movement carried out by a previous attempt
> > > [2].  These additional improvements can be implemented later as agreement
> > > is reached on these other issues.  For example, future work might
> > > increase cache locality by applying vector object lists, kfree_bulk(),
> > > or per-slab batching to further improve handling of kfree_rcu() floods.
> > > 
> > > Performance tests are provided in a latter commmit.  These tests show a
> > > 5x reduction in number of grace periods on a 16 CPU system, with minimal
> > > increase in kfree() latency.
> > > 
> > > Note that this commit prevents rcu_barrier() from waiting for the
> > > execution of the kfree() calls associated with prior kfree_rcu()
> > > invocations.  This should not be a problem, given that the resulting
> > > pending kfree() calls do not prevent module unloading or filesystem
> > > unmounting.  The reason rcu_barrier() no longer waits for the kfree()
> > > calls is that the kfree_rcu() requests are now batched, so that at
> > > any given time there might be kfree_rcu() requests that are not yet
> > > known to the core RCU machinery.  Furthermore, once a kfree_rcu()
> > > grace period has elapsed, the actual kfree() invocations happen in
> > > workqueue context.  So rcu_barrier() no longer waits for all of the
> > > prior requests, nor it does not wait for the workqueue handlers to
> > > start, let alone complete.  If there is ever a good reason to wait for
> > > the kfree() invocation corresponding to all prior kfree_rcu() calls,
> > > an approapriate kfree_rcu_barrier() can be constructed.  However, at
> > > the moment no reasonable use case is apparent.
> > > 
> > > This commit can result in increased memory footprint because the
> > > batching can increase the kfree_rcu()-to-kfree() latency.  Later
> > > commits will reduce this memory footprint.
> > > 
> > > Later commits will also remove the special handling of kfree_rcu() by
> > > __rcu_reclaim() within the RCU core.  This will require changes to
> > > rcuperf testing and to early boot handling of kfree_rcu().
> > > 
> > > [1] http://lore.kernel.org/lkml/20190723035725-mutt-send-email-mst@kernel.org
> > > [2] https://lkml.org/lkml/2017/12/19/824
> > > 
> > > Cc: kernel-team@android.com
> > > Cc: kernel-team@lge.com
> > > Co-developed-by: Byungchul Park <byungchul.park@lge.com>
> > > Signed-off-by: Byungchul Park <byungchul.park@lge.com>
> > > Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>
> > > 
> > > ---
> > > v3->v4: Some corrections by Paul.
> > > 	Used xchg in places to simplify code.
> > > 
> > > v2->v3: Just some code comment changes thanks to Byungchul.
> > > 
> > > RFCv1->PATCH v2: Removed limits on the ->head list, just let it grow.
> > >                    Dropped KFREE_MAX_JIFFIES to HZ/50 from HZ/20 to reduce OOM occurrence.
> > >                    Removed sleeps in rcuperf test, just using cond_resched()in loop.
> > >                    Better code comments ;)
> > > 
> > >  include/linux/rcutiny.h |   5 ++
> > >  include/linux/rcutree.h |   1 +
> > >  kernel/rcu/tree.c       | 194 ++++++++++++++++++++++++++++++++++++++--
> > >  3 files changed, 194 insertions(+), 6 deletions(-)
> > > 
> > > diff --git a/include/linux/rcutiny.h b/include/linux/rcutiny.h
> > > index 8e727f57d814..383f2481750f 100644
> > > --- a/include/linux/rcutiny.h
> > > +++ b/include/linux/rcutiny.h
> > > @@ -39,6 +39,11 @@ static inline void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> > >  	call_rcu(head, func);
> > >  }
> > >  
> > > +static inline void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
> > > +{
> > > +	call_rcu(head, func);
> > > +}
> > > +
> > >  void rcu_qs(void);
> > >  
> > >  static inline void rcu_softirq_qs(void)
> > > diff --git a/include/linux/rcutree.h b/include/linux/rcutree.h
> > > index 735601ac27d3..7e38b39ec634 100644
> > > --- a/include/linux/rcutree.h
> > > +++ b/include/linux/rcutree.h
> > > @@ -34,6 +34,7 @@ static inline void rcu_virt_note_context_switch(int cpu)
> > >  
> > >  void synchronize_rcu_expedited(void);
> > >  void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func);
> > > +void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func);
> > >  
> > >  void rcu_barrier(void);
> > >  bool rcu_eqs_special_set(int cpu);
> > > diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> > > index a14e5fbbea46..1d1847cadea2 100644
> > > --- a/kernel/rcu/tree.c
> > > +++ b/kernel/rcu/tree.c
> > > @@ -2593,17 +2593,185 @@ void call_rcu(struct rcu_head *head, rcu_callback_t func)
> > >  }
> > >  EXPORT_SYMBOL_GPL(call_rcu);
> > >  
> > > +
> > > +/* Maximum number of jiffies to wait before draining a batch. */
> > > +#define KFREE_DRAIN_JIFFIES (HZ / 50)
> > > +
> > >  /*
> > > - * Queue an RCU callback for lazy invocation after a grace period.
> > > - * This will likely be later named something like "call_rcu_lazy()",
> > > - * but this change will require some way of tagging the lazy RCU
> > > - * callbacks in the list of pending callbacks. Until then, this
> > > - * function may only be called from __kfree_rcu().
> > > + * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> > > + * kfree(s) is queued for freeing after a grace period, right away.
> > >   */
> > > -void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> > > +struct kfree_rcu_cpu {
> > > +	/* The rcu_work node for queuing work with queue_rcu_work(). The work
> > > +	 * is done after a grace period.
> > > +	 */
> > > +	struct rcu_work rcu_work;
> > > +
> > > +	/* The list of objects being queued in a batch but are not yet
> > > +	 * scheduled to be freed.
> > > +	 */
> > > +	struct rcu_head *head;
> > > +
> > > +	/* The list of objects that have now left ->head and are queued for
> > > +	 * freeing after a grace period.
> > > +	 */
> > > +	struct rcu_head *head_free;
> > > +
> > > +	/* Protect concurrent access to this structure. */
> > > +	spinlock_t lock;
> > > +
> > > +	/* The delayed work that flushes ->head to ->head_free incase ->head
> > > +	 * within KFREE_DRAIN_JIFFIES. In case flushing cannot be done if RCU
> > > +	 * is busy, ->head just continues to grow and we retry flushing later.
> > > +	 */
> > > +	struct delayed_work monitor_work;
> > > +	bool monitor_todo;	/* Is a delayed work pending execution? */
> > > +};
> > > +
> > > +static DEFINE_PER_CPU(struct kfree_rcu_cpu, krc);
> > > +
> > > +/*
> > > + * This function is invoked in workqueue context after a grace period.
> > > + * It frees all the objects queued on ->head_free.
> > > + */
> > > +static void kfree_rcu_work(struct work_struct *work)
> > > +{
> > > +	unsigned long flags;
> > > +	struct rcu_head *head, *next;
> > > +	struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
> > > +					struct kfree_rcu_cpu, rcu_work);
> > > +
> > > +	spin_lock_irqsave(&krcp->lock, flags);
> > > +	head = krcp->head_free;
> > > +	krcp->head_free = NULL;
> > > +	spin_unlock_irqrestore(&krcp->lock, flags);
> > > +
> > > +	/*
> > > +	 * The head is detached and not referenced from anywhere, so lockless
> > > +	 * access is Ok.
> > > +	 */
> > > +	for (; head; head = next) {
> > > +		next = head->next;
> > > +		/* Could be possible to optimize with kfree_bulk in future */
> > > +		__rcu_reclaim(rcu_state.name, head);
> > > +		cond_resched_tasks_rcu_qs();
> > > +	}
> > > +}
> > > +
> > > +/*
> > > + * Schedule the kfree batch RCU work to run in workqueue context after a GP.
> > > + *
> > > + * This function is invoked by kfree_rcu_monitor() when the KFREE_DRAIN_JIFFIES
> > > + * timeout has been reached.
> > > + */
> > > +static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
> > > +{
> > > +	lockdep_assert_held(&krcp->lock);
> > > +
> > > +	/* If a previous RCU batch work is already in progress, we cannot queue
> > > +	 * another one, just refuse the optimization and it will be retried
> > > +	 * again in KFREE_DRAIN_JIFFIES time.
> > > +	 */
> > > +	if (krcp->head_free)
> > > +		return false;
> > > +
> > > +	krcp->head_free = krcp->head;
> > > +	krcp->head = NULL;
> > > +	INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
> > > +	queue_rcu_work(system_wq, &krcp->rcu_work);
> > > +
> > > +	return true;
> > > +}
> > > +
> > > +static inline void kfree_rcu_drain_unlock(struct kfree_rcu_cpu *krcp,
> > > +				   unsigned long flags)
> > > +{
> > > +	/* Flush ->head to ->head_free, all objects on ->head_free will be
> > > +	 * kfree'd after a grace period.
> > > +	 */
> > > +	if (queue_kfree_rcu_work(krcp)) {
> > > +		/* Success! Our job is done here. */
> > > +		spin_unlock_irqrestore(&krcp->lock, flags);
> > > +		return;
> > > +	}
> > > +
> > > +	/* Previous batch that was queued to RCU did not get free yet, let us
> > > +	 * try again soon.
> > > +	 */
> > > +	if (!xchg(&krcp->monitor_todo, true))
> > > +		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> > > +	spin_unlock_irqrestore(&krcp->lock, flags);
> > > +}
> > > +
> > > +/*
> > > + * This function is invoked after the KFREE_DRAIN_JIFFIES timeout has elapsed,
> > > + * and it drains the specified kfree_rcu_cpu structure's ->head list.
> > > + */
> > > +static void kfree_rcu_monitor(struct work_struct *work)
> > > +{
> > > +	unsigned long flags;
> > > +	struct kfree_rcu_cpu *krcp = container_of(work, struct kfree_rcu_cpu,
> > > +						 monitor_work.work);
> > > +
> > > +	spin_lock_irqsave(&krcp->lock, flags);
> > > +	if (xchg(&krcp->monitor_todo, false))
> > > +		kfree_rcu_drain_unlock(krcp, flags);
> > > +	else
> > > +		spin_unlock_irqrestore(&krcp->lock, flags);
> > > +}
> > > +
> > > +/*
> > > + * This version of kfree_call_rcu does not do batching of kfree_rcu() requests.
> > > + * Used only by rcuperf torture test for comparison with kfree_rcu_batch().
> > > + */
> > > +void kfree_call_rcu_nobatch(struct rcu_head *head, rcu_callback_t func)
> > >  {
> > >  	__call_rcu(head, func, -1, 1);
> > >  }
> > > +EXPORT_SYMBOL_GPL(kfree_call_rcu_nobatch);
> > > +
> > > +/*
> > > + * Queue a request for lazy invocation of kfree() after a grace period.
> > > + *
> > > + * Each kfree_call_rcu() request is added to a batch. The batch will be drained
> > > + * every KFREE_DRAIN_JIFFIES number of jiffies. All the objects in the batch
> > > + * will be kfree'd in workqueue context. This allows us to:
> > > + *
> > > + * 1. Batch requests together to reduce the number of grace periods during
> > > + * heavy kfree_rcu() load.
> > > + *
> > > + * 2. In the future, makes it possible to use kfree_bulk() on a large number of
> > > + * kfree_rcu() requests thus reducing the per-object overhead of kfree() and
> > > + * also reducing cache misses.
> > > + */
> > > +void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> > > +{
> > > +	unsigned long flags;
> > > +	struct kfree_rcu_cpu *krcp;
> > > +
> > > +	/* kfree_call_rcu() batching requires timers to be up. If the scheduler
> > > +	 * is not yet up, just skip batching and do the non-batched version.
> > > +	 */
> > > +	if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
> > > +		return kfree_call_rcu_nobatch(head, func);
> > > +
> > > +	head->func = func;
> > > +
> > > +	local_irq_save(flags);	/* For safely calling this_cpu_ptr(). */
> > > +	krcp = this_cpu_ptr(&krc);
> > > +	spin_lock(&krcp->lock);
> > > +
> > > +	/* Queue the kfree but don't yet schedule the batch. */
> > > +	head->next = krcp->head;
> > > +	krcp->head = head;
> > > +
> > > +	/* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
> > > +	if (!xchg(&krcp->monitor_todo, true))
> > > +		schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> > > +
> > > +	spin_unlock(&krcp->lock);
> > > +	local_irq_restore(flags);
> > > +}
> > >  EXPORT_SYMBOL_GPL(kfree_call_rcu);
> > >  
> > >  /*
> > > @@ -3455,10 +3623,24 @@ static void __init rcu_dump_rcu_node_tree(void)
> > >  struct workqueue_struct *rcu_gp_wq;
> > >  struct workqueue_struct *rcu_par_gp_wq;
> > >  
> > > +static void __init kfree_rcu_batch_init(void)
> > > +{
> > > +	int cpu;
> > > +
> > > +	for_each_possible_cpu(cpu) {
> > > +		struct kfree_rcu_cpu *krcp = per_cpu_ptr(&krc, cpu);
> > > +
> > > +		spin_lock_init(&krcp->lock);
> > > +		INIT_DELAYED_WORK(&krcp->monitor_work, kfree_rcu_monitor);
> > > +	}
> > > +}
> > > +
> > >  void __init rcu_init(void)
> > >  {
> > >  	int cpu;
> > >  
> > > +	kfree_rcu_batch_init();
> > > +
> > >  	rcu_early_boot_tests();
> > >  
> > >  	rcu_bootup_announce();
> > > -- 
> > > 2.23.0.rc1.153.gdeed80330f-goog
> > > 
> 


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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-16 19:16     ` Paul E. McKenney
@ 2019-08-17  1:32       ` Joel Fernandes
  2019-08-17  3:56         ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-17  1:32 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: LKML, kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers, Byungchul Park,
	Rao Shoaib, rcu, Steven Rostedt

Hi Paul,

On Fri, Aug 16, 2019 at 3:16 PM Paul E. McKenney <paulmck@linux.ibm.com> wrote:
> > > Hello, Joel,
> > >
> > > I reworked the commit log as follows, but was then unsuccessful in
> > > working out which -rcu commit to apply it to.  Could you please
> > > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > > is usually pretty good about handling minor conflicts.)
> >
> > It was originally based on v5.3-rc2
> >
> > I was able to apply it just now to the rcu -dev branch and I pushed it here:
> > https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> >
> > Let me know if any other issues, thanks for the change log rework!
>
> Pulled and cherry-picked, thank you!
>
> Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
> results of the pull.  If you pull that branch, then run something like
> "gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
> two might illustrate some of the reasons for the current restrictions
> on pull requests and trees subject to rebase.

Right, I did the compare and see what you mean. I guess sending any
future pull requests against Linux -next would be the best option?

thanks,

 - Joel

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-17  1:32       ` Joel Fernandes
@ 2019-08-17  3:56         ` Paul E. McKenney
  2019-08-17  4:30           ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-17  3:56 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: LKML, kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers, Byungchul Park,
	Rao Shoaib, rcu, Steven Rostedt

On Fri, Aug 16, 2019 at 09:32:23PM -0400, Joel Fernandes wrote:
> Hi Paul,
> 
> On Fri, Aug 16, 2019 at 3:16 PM Paul E. McKenney <paulmck@linux.ibm.com> wrote:
> > > > Hello, Joel,
> > > >
> > > > I reworked the commit log as follows, but was then unsuccessful in
> > > > working out which -rcu commit to apply it to.  Could you please
> > > > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > > > is usually pretty good about handling minor conflicts.)
> > >
> > > It was originally based on v5.3-rc2
> > >
> > > I was able to apply it just now to the rcu -dev branch and I pushed it here:
> > > https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> > >
> > > Let me know if any other issues, thanks for the change log rework!
> >
> > Pulled and cherry-picked, thank you!
> >
> > Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
> > results of the pull.  If you pull that branch, then run something like
> > "gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
> > two might illustrate some of the reasons for the current restrictions
> > on pull requests and trees subject to rebase.
> 
> Right, I did the compare and see what you mean. I guess sending any
> future pull requests against Linux -next would be the best option?

Hmmm...  You really want to send some pull requests, don't you?  ;-)

Suppose you had sent that pull request against Linux -next or v5.2
or wherever.  What would happen next, given the high probability of a
conflict with someone else's patch?  What would the result look like?

							Thanx, Paul


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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-17  3:56         ` Paul E. McKenney
@ 2019-08-17  4:30           ` Joel Fernandes
  2019-08-17  5:20             ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-17  4:30 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: LKML, kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers, Byungchul Park,
	Rao Shoaib, rcu, Steven Rostedt

On Fri, Aug 16, 2019 at 08:56:37PM -0700, Paul E. McKenney wrote:
> On Fri, Aug 16, 2019 at 09:32:23PM -0400, Joel Fernandes wrote:
> > Hi Paul,
> > 
> > On Fri, Aug 16, 2019 at 3:16 PM Paul E. McKenney <paulmck@linux.ibm.com> wrote:
> > > > > Hello, Joel,
> > > > >
> > > > > I reworked the commit log as follows, but was then unsuccessful in
> > > > > working out which -rcu commit to apply it to.  Could you please
> > > > > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > > > > is usually pretty good about handling minor conflicts.)
> > > >
> > > > It was originally based on v5.3-rc2
> > > >
> > > > I was able to apply it just now to the rcu -dev branch and I pushed it here:
> > > > https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> > > >
> > > > Let me know if any other issues, thanks for the change log rework!
> > >
> > > Pulled and cherry-picked, thank you!
> > >
> > > Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
> > > results of the pull.  If you pull that branch, then run something like
> > > "gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
> > > two might illustrate some of the reasons for the current restrictions
> > > on pull requests and trees subject to rebase.
> > 
> > Right, I did the compare and see what you mean. I guess sending any
> > future pull requests against Linux -next would be the best option?
> 
> Hmmm...  You really want to send some pull requests, don't you?  ;-)

I would be lying if I said I don't have the itch to ;-)

> Suppose you had sent that pull request against Linux -next or v5.2
> or wherever.  What would happen next, given the high probability of a
> conflict with someone else's patch?  What would the result look like?

One hopes that the tools are able to automatically resolve the resolution,
however adequate re-inspection of the resulting code and testing it would be
needed in either case, to ensure the conflict resolution (whether manual or
automatic) happened correctly.

IIUC, this usually depends on the maintainer's preference on which branch to
send patches against.

Are you saying -rcu's dev branch is still the best option to send patches
against, even though it is rebased often?

thanks,

 - Joel


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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-17  4:30           ` Joel Fernandes
@ 2019-08-17  5:20             ` Paul E. McKenney
  2019-08-17  5:53               ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-17  5:20 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: LKML, kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers, Byungchul Park,
	Rao Shoaib, rcu, Steven Rostedt

On Sat, Aug 17, 2019 at 12:30:24AM -0400, Joel Fernandes wrote:
> On Fri, Aug 16, 2019 at 08:56:37PM -0700, Paul E. McKenney wrote:
> > On Fri, Aug 16, 2019 at 09:32:23PM -0400, Joel Fernandes wrote:
> > > Hi Paul,
> > > 
> > > On Fri, Aug 16, 2019 at 3:16 PM Paul E. McKenney <paulmck@linux.ibm.com> wrote:
> > > > > > Hello, Joel,
> > > > > >
> > > > > > I reworked the commit log as follows, but was then unsuccessful in
> > > > > > working out which -rcu commit to apply it to.  Could you please
> > > > > > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > > > > > is usually pretty good about handling minor conflicts.)
> > > > >
> > > > > It was originally based on v5.3-rc2
> > > > >
> > > > > I was able to apply it just now to the rcu -dev branch and I pushed it here:
> > > > > https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> > > > >
> > > > > Let me know if any other issues, thanks for the change log rework!
> > > >
> > > > Pulled and cherry-picked, thank you!
> > > >
> > > > Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
> > > > results of the pull.  If you pull that branch, then run something like
> > > > "gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
> > > > two might illustrate some of the reasons for the current restrictions
> > > > on pull requests and trees subject to rebase.
> > > 
> > > Right, I did the compare and see what you mean. I guess sending any
> > > future pull requests against Linux -next would be the best option?
> > 
> > Hmmm...  You really want to send some pull requests, don't you?  ;-)
> 
> I would be lying if I said I don't have the itch to ;-)
> 
> > Suppose you had sent that pull request against Linux -next or v5.2
> > or wherever.  What would happen next, given the high probability of a
> > conflict with someone else's patch?  What would the result look like?
> 
> One hopes that the tools are able to automatically resolve the resolution,
> however adequate re-inspection of the resulting code and testing it would be
> needed in either case, to ensure the conflict resolution (whether manual or
> automatic) happened correctly.

I didn't ask you to hope.  I instead asked you what tell me what would
actually happen.  ;-)

You could actually try this by randomly grouping the patches in -rcu
(say, placing every third patch into one of three groups), generating
separate pull requests, and then merging the pull requests together.
Then you wouldn't have to hope.  You could instead look at it in (say)
gitk after the pieces were put together.

And there are more questions.  For example, how would this affect testing
given issues involving both RCU and other pieces of the kernel?

> IIUC, this usually depends on the maintainer's preference on which branch to
> send patches against.
> 
> Are you saying -rcu's dev branch is still the best option to send patches
> against, even though it is rebased often?

Sounds like we might need to discuss this face to face.

							Thanx, Paul

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-17  5:20             ` Paul E. McKenney
@ 2019-08-17  5:53               ` Joel Fernandes
  2019-08-17 21:45                 ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-17  5:53 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: LKML, kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers, Byungchul Park,
	Rao Shoaib, rcu, Steven Rostedt

On Fri, Aug 16, 2019 at 10:20:23PM -0700, Paul E. McKenney wrote:
> On Sat, Aug 17, 2019 at 12:30:24AM -0400, Joel Fernandes wrote:
> > On Fri, Aug 16, 2019 at 08:56:37PM -0700, Paul E. McKenney wrote:
> > > On Fri, Aug 16, 2019 at 09:32:23PM -0400, Joel Fernandes wrote:
> > > > Hi Paul,
> > > > 
> > > > On Fri, Aug 16, 2019 at 3:16 PM Paul E. McKenney <paulmck@linux.ibm.com> wrote:
> > > > > > > Hello, Joel,
> > > > > > >
> > > > > > > I reworked the commit log as follows, but was then unsuccessful in
> > > > > > > working out which -rcu commit to apply it to.  Could you please
> > > > > > > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > > > > > > is usually pretty good about handling minor conflicts.)
> > > > > >
> > > > > > It was originally based on v5.3-rc2
> > > > > >
> > > > > > I was able to apply it just now to the rcu -dev branch and I pushed it here:
> > > > > > https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> > > > > >
> > > > > > Let me know if any other issues, thanks for the change log rework!
> > > > >
> > > > > Pulled and cherry-picked, thank you!
> > > > >
> > > > > Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
> > > > > results of the pull.  If you pull that branch, then run something like
> > > > > "gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
> > > > > two might illustrate some of the reasons for the current restrictions
> > > > > on pull requests and trees subject to rebase.
> > > > 
> > > > Right, I did the compare and see what you mean. I guess sending any
> > > > future pull requests against Linux -next would be the best option?
> > > 
> > > Hmmm...  You really want to send some pull requests, don't you?  ;-)
> > 
> > I would be lying if I said I don't have the itch to ;-)
> > 
> > > Suppose you had sent that pull request against Linux -next or v5.2
> > > or wherever.  What would happen next, given the high probability of a
> > > conflict with someone else's patch?  What would the result look like?
> > 
> > One hopes that the tools are able to automatically resolve the resolution,
> > however adequate re-inspection of the resulting code and testing it would be
> > needed in either case, to ensure the conflict resolution (whether manual or
> > automatic) happened correctly.
> 
> I didn't ask you to hope.  I instead asked you what tell me what would
> actually happen.  ;-)
> 
> You could actually try this by randomly grouping the patches in -rcu
> (say, placing every third patch into one of three groups), generating
> separate pull requests, and then merging the pull requests together.
> Then you wouldn't have to hope.  You could instead look at it in (say)
> gitk after the pieces were put together.

So you take whatever is worked on in 'dev' and create separate branches out
of them, then merge them together later?

I have seen you doing these tricks and would love to get ideas from your
experiences on these.

> > IIUC, this usually depends on the maintainer's preference on which branch to
> > send patches against.
> > 
> > Are you saying -rcu's dev branch is still the best option to send patches
> > against, even though it is rebased often?
> 
> Sounds like we might need to discuss this face to face.

Yes, let us talk for sure at plumbers, thank you so much!

(Also I sent a patch just now to fix that xchg() issue).

 - Joel


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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-17  5:53               ` Joel Fernandes
@ 2019-08-17 21:45                 ` Paul E. McKenney
  0 siblings, 0 replies; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-17 21:45 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: LKML, kernel-team, kernel-team, Byungchul Park, Davidlohr Bueso,
	Josh Triplett, Lai Jiangshan, Mathieu Desnoyers, Byungchul Park,
	Rao Shoaib, rcu, Steven Rostedt

On Sat, Aug 17, 2019 at 01:53:29AM -0400, Joel Fernandes wrote:
> On Fri, Aug 16, 2019 at 10:20:23PM -0700, Paul E. McKenney wrote:
> > On Sat, Aug 17, 2019 at 12:30:24AM -0400, Joel Fernandes wrote:
> > > On Fri, Aug 16, 2019 at 08:56:37PM -0700, Paul E. McKenney wrote:
> > > > On Fri, Aug 16, 2019 at 09:32:23PM -0400, Joel Fernandes wrote:
> > > > > Hi Paul,
> > > > > 
> > > > > On Fri, Aug 16, 2019 at 3:16 PM Paul E. McKenney <paulmck@linux.ibm.com> wrote:
> > > > > > > > Hello, Joel,
> > > > > > > >
> > > > > > > > I reworked the commit log as follows, but was then unsuccessful in
> > > > > > > > working out which -rcu commit to apply it to.  Could you please
> > > > > > > > tell me what commit to apply this to?  (Once applied, git cherry-pick
> > > > > > > > is usually pretty good about handling minor conflicts.)
> > > > > > >
> > > > > > > It was originally based on v5.3-rc2
> > > > > > >
> > > > > > > I was able to apply it just now to the rcu -dev branch and I pushed it here:
> > > > > > > https://github.com/joelagnel/linux-kernel.git (branch paul-dev)
> > > > > > >
> > > > > > > Let me know if any other issues, thanks for the change log rework!
> > > > > >
> > > > > > Pulled and cherry-picked, thank you!
> > > > > >
> > > > > > Just for grins, I also  pushed out a from-joel.2019.08.16a showing the
> > > > > > results of the pull.  If you pull that branch, then run something like
> > > > > > "gitk v5.3-rc2..", and then do the same with branch "dev", comparing the
> > > > > > two might illustrate some of the reasons for the current restrictions
> > > > > > on pull requests and trees subject to rebase.
> > > > > 
> > > > > Right, I did the compare and see what you mean. I guess sending any
> > > > > future pull requests against Linux -next would be the best option?
> > > > 
> > > > Hmmm...  You really want to send some pull requests, don't you?  ;-)
> > > 
> > > I would be lying if I said I don't have the itch to ;-)
> > > 
> > > > Suppose you had sent that pull request against Linux -next or v5.2
> > > > or wherever.  What would happen next, given the high probability of a
> > > > conflict with someone else's patch?  What would the result look like?
> > > 
> > > One hopes that the tools are able to automatically resolve the resolution,
> > > however adequate re-inspection of the resulting code and testing it would be
> > > needed in either case, to ensure the conflict resolution (whether manual or
> > > automatic) happened correctly.
> > 
> > I didn't ask you to hope.  I instead asked you what tell me what would
> > actually happen.  ;-)
> > 
> > You could actually try this by randomly grouping the patches in -rcu
> > (say, placing every third patch into one of three groups), generating
> > separate pull requests, and then merging the pull requests together.
> > Then you wouldn't have to hope.  You could instead look at it in (say)
> > gitk after the pieces were put together.
> 
> So you take whatever is worked on in 'dev' and create separate branches out
> of them, then merge them together later?
> 
> I have seen you doing these tricks and would love to get ideas from your
> experiences on these.

If the release dates line up, perhaps I can demo it for v5.4 at LPC.

> > > IIUC, this usually depends on the maintainer's preference on which branch to
> > > send patches against.
> > > 
> > > Are you saying -rcu's dev branch is still the best option to send patches
> > > against, even though it is rebased often?
> > 
> > Sounds like we might need to discuss this face to face.
> 
> Yes, let us talk for sure at plumbers, thank you so much!
> 
> (Also I sent a patch just now to fix that xchg() issue).

Yes, I just now squashed it in, thank you!

								Thanx, Paul

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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-14 22:58   ` Paul E. McKenney
@ 2019-08-19 19:33     ` Joel Fernandes
  2019-08-19 22:23       ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-19 19:33 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Wed, Aug 14, 2019 at 03:58:50PM -0700, Paul E. McKenney wrote:
> On Wed, Aug 14, 2019 at 12:04:11PM -0400, Joel Fernandes (Google) wrote:
> > This test runs kfree_rcu in a loop to measure performance of the new
> > kfree_rcu batching functionality.
> 
> kfree_rcu().

Fixed.

> > The following table shows results when booting with arguments:
> > rcuperf.kfree_loops=200000 rcuperf.kfree_alloc_num=1000 rcuperf.kfree_rcu_test=1
> > 
> > In addition, rcuperf.kfree_no_batch is used to toggle the batching of
> > kfree_rcu()s for a test run.
> > 
> > rcuperf.kfree_no_batch	GPs	time (seconds)
> >  0 (default)		1732	15.9
> >  1			9133 	14.5
> > 
> > Note that the results are the same for the case:
> > 1. Patch is not applied and rcuperf.kfree_no_batch=0
> > 2. Patch is applied     and rcuperf.kfree_no_batch=1
> > 
> > On a 16 CPU system with the above boot parameters, we see that the total
> > number of grace periods that elapse during the test drops from 9133 when
> > not batching to 1732 when batching (a 5X improvement). The kfree_rcu()
> > flood itself slows down a bit when batching, though, as shown. This is
> > likely due to rcuperf threads contending with the additional worker
> > threads that are now running both before (the monitor) and after (the
> > work done to kfree()) the grace period.
> 
> Another possibility is that the batching approach is resulting in a
> greater number of objects waiting to be freed (noted below), and it
> takes the extra 1.4 seconds to catch up.  How would you decide which
> effect is the most important?  (Your path of least resistance is to
> remove the speculation.)

I will remove the speculation since the slightly extra time is understandable
and not concerning. I hope we agree on that.

> > Note that the active memory consumption during the kfree_rcu() flood
> > does increase to around 300-400MB due to the batching (from around 50MB
> > without batching). However, this memory consumption is relatively
> > constant and is just an effect of the buffering. In other words, the
> > system is able to keep up with the kfree_rcu() load. The memory
> > consumption comes down to 200-300MB if KFREE_DRAIN_JIFFIES is
> > increased from HZ/50 to HZ/80.
> > 
> > Also, when running the test, please disable CONFIG_DEBUG_PREEMPT and
> > CONFIG_PROVE_RCU for realistic comparisons with/without batching.
> > 
> > Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>
> 
> Looks pretty close, just a very few issues needing fixing below.

Thanks!

> > ---
> >  .../admin-guide/kernel-parameters.txt         |  17 ++
> >  kernel/rcu/rcuperf.c                          | 189 +++++++++++++++++-
> >  2 files changed, 198 insertions(+), 8 deletions(-)
> > 
> > diff --git a/Documentation/admin-guide/kernel-parameters.txt b/Documentation/admin-guide/kernel-parameters.txt
> > index 7ccd158b3894..a9156ca5de24 100644
> > --- a/Documentation/admin-guide/kernel-parameters.txt
> > +++ b/Documentation/admin-guide/kernel-parameters.txt
> > @@ -3895,6 +3895,23 @@
> >  			test until boot completes in order to avoid
> >  			interference.
> >  
> > +	rcuperf.kfree_rcu_test= [KNL]
> > +			Set to measure performance of kfree_rcu() flooding.
> > +
> > +	rcuperf.kfree_nthreads= [KNL]
> > +			The number of threads running loops of kfree_rcu().
> > +
> > +	rcuperf.kfree_alloc_num= [KNL]
> > +			Number of allocations and frees done in an iteration.
> > +
> > +	rcuperf.kfree_loops= [KNL]
> > +			Number of loops doing rcuperf.kfree_alloc_num number
> > +			of allocations and frees.
> > +
> > +	rcuperf.kfree_no_batch= [KNL]
> > +			Use the non-batching (slower) version of kfree_rcu.
> > +			This is useful for comparing with the batched version.
> 
> I suggest s/slower/more efficient/ given that the batching takes more
> wall-clock time than does the no-batching.

I think you mean, slower -> less efficient (due to taking up more grace
period cycles per second in the no batching case). I will update it
accordingly.

[snip]
> > @@ -592,6 +593,175 @@ rcu_perf_shutdown(void *arg)
> >  	return -EINVAL;
> >  }
> >  
> > +/*
> > + * kfree_rcu performance tests: Start a kfree_rcu loop on all CPUs for number
> > + * of iterations and measure total time and number of GP for all iterations to complete.
> > + */
> > +
> > +torture_param(int, kfree_nthreads, -1, "Number of threads running loops of kfree_rcu().");
> > +torture_param(int, kfree_alloc_num, 8000, "Number of allocations and frees done in an iteration.");
> > +torture_param(int, kfree_loops, 10, "Number of loops doing kfree_alloc_num allocations and frees.");
> > +torture_param(int, kfree_no_batch, 0, "Use the non-batching (slower) version of kfree_rcu.");
> > +
> > +static struct task_struct **kfree_reader_tasks;
> > +static int kfree_nrealthreads;
> > +static atomic_t n_kfree_perf_thread_started;
> > +static atomic_t n_kfree_perf_thread_ended;
> > +
> > +struct kfree_obj {
> > +	char kfree_obj[8];
> > +	struct rcu_head rh;
> > +};
> 
> (Aside from above, no need to change this part of the patch, at least not
> that I know of at the moment.)
> 
> 24 bytes on a 64-bit system, 16 on a 32-bit system.  So there might
> have been 10 million extra objects awaiting free in the batching case
> given the 400M-50M=350M excess for the batching approach.  If freeing
> each object took about 100ns, that could account for the additional
> wall-clock time for the batching approach.

Makes sense, and this comes down to 200-220MB range with the additional list.

> > +	set_user_nice(current, MAX_NICE);
> > +
> > +	alloc_ptrs = (struct kfree_obj **)kmalloc(sizeof(struct kfree_obj *) * kfree_alloc_num,
> > +						  GFP_KERNEL);
> > +	if (!alloc_ptrs)
> > +		return -ENOMEM;
> > +
> > +	start_time = ktime_get_mono_fast_ns();
> > +
> > +	if (atomic_inc_return(&n_kfree_perf_thread_started) >= kfree_nrealthreads) {
> > +		if (gp_exp)
> > +			b_rcu_gp_test_started = cur_ops->exp_completed() / 2;
> > +		else
> > +			b_rcu_gp_test_started = cur_ops->get_gp_seq();
> > +	}
> > +
> > +	do {
> > +		for (i = 0; i < kfree_alloc_num; i++) {
> > +			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> > +			if (!alloc_ptrs[i])
> > +				return -ENOMEM;
> > +		}
> > +
> > +		for (i = 0; i < kfree_alloc_num; i++) {
> > +			if (!kfree_no_batch) {
> > +				kfree_rcu(alloc_ptrs[i], rh);
> > +			} else {
> > +				rcu_callback_t cb;
> > +
> > +				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> > +				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> > +			}
> > +		}
> 
> The point of allocating a large batch and then kfree_rcu()ing them in a
> loop is to defeat the per-CPU pool optimization?  Either way, a comment
> would be very good!

It was a reasoning like this, added it as a comment:

	/* While measuring kfree_rcu() time, we also end up measuring kmalloc()
	 * time. So the strategy here is to do a few (kfree_alloc_num) number
	 * of kmalloc() and kfree_rcu() every loop so that the current loop's
	 * deferred kfree()ing overlaps with the next loop's kmalloc().
	 */

Will post it soon with other patches on top of -rcu dev.

thanks,

 - Joel


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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-19 19:33     ` Joel Fernandes
@ 2019-08-19 22:23       ` Paul E. McKenney
  2019-08-19 23:51         ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-19 22:23 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Mon, Aug 19, 2019 at 03:33:27PM -0400, Joel Fernandes wrote:
> On Wed, Aug 14, 2019 at 03:58:50PM -0700, Paul E. McKenney wrote:
> > On Wed, Aug 14, 2019 at 12:04:11PM -0400, Joel Fernandes (Google) wrote:
> > > This test runs kfree_rcu in a loop to measure performance of the new
> > > kfree_rcu batching functionality.
> > 
> > kfree_rcu().
> 
> Fixed.
> 
> > > The following table shows results when booting with arguments:
> > > rcuperf.kfree_loops=200000 rcuperf.kfree_alloc_num=1000 rcuperf.kfree_rcu_test=1
> > > 
> > > In addition, rcuperf.kfree_no_batch is used to toggle the batching of
> > > kfree_rcu()s for a test run.
> > > 
> > > rcuperf.kfree_no_batch	GPs	time (seconds)
> > >  0 (default)		1732	15.9
> > >  1			9133 	14.5
> > > 
> > > Note that the results are the same for the case:
> > > 1. Patch is not applied and rcuperf.kfree_no_batch=0
> > > 2. Patch is applied     and rcuperf.kfree_no_batch=1
> > > 
> > > On a 16 CPU system with the above boot parameters, we see that the total
> > > number of grace periods that elapse during the test drops from 9133 when
> > > not batching to 1732 when batching (a 5X improvement). The kfree_rcu()
> > > flood itself slows down a bit when batching, though, as shown. This is
> > > likely due to rcuperf threads contending with the additional worker
> > > threads that are now running both before (the monitor) and after (the
> > > work done to kfree()) the grace period.
> > 
> > Another possibility is that the batching approach is resulting in a
> > greater number of objects waiting to be freed (noted below), and it
> > takes the extra 1.4 seconds to catch up.  How would you decide which
> > effect is the most important?  (Your path of least resistance is to
> > remove the speculation.)
> 
> I will remove the speculation since the slightly extra time is understandable
> and not concerning. I hope we agree on that.

Works for me!

> > > Note that the active memory consumption during the kfree_rcu() flood
> > > does increase to around 300-400MB due to the batching (from around 50MB
> > > without batching). However, this memory consumption is relatively
> > > constant and is just an effect of the buffering. In other words, the
> > > system is able to keep up with the kfree_rcu() load. The memory
> > > consumption comes down to 200-300MB if KFREE_DRAIN_JIFFIES is
> > > increased from HZ/50 to HZ/80.
> > > 
> > > Also, when running the test, please disable CONFIG_DEBUG_PREEMPT and
> > > CONFIG_PROVE_RCU for realistic comparisons with/without batching.
> > > 
> > > Signed-off-by: Joel Fernandes (Google) <joel@joelfernandes.org>
> > 
> > Looks pretty close, just a very few issues needing fixing below.
> 
> Thanks!
> 
> > > ---
> > >  .../admin-guide/kernel-parameters.txt         |  17 ++
> > >  kernel/rcu/rcuperf.c                          | 189 +++++++++++++++++-
> > >  2 files changed, 198 insertions(+), 8 deletions(-)
> > > 
> > > diff --git a/Documentation/admin-guide/kernel-parameters.txt b/Documentation/admin-guide/kernel-parameters.txt
> > > index 7ccd158b3894..a9156ca5de24 100644
> > > --- a/Documentation/admin-guide/kernel-parameters.txt
> > > +++ b/Documentation/admin-guide/kernel-parameters.txt
> > > @@ -3895,6 +3895,23 @@
> > >  			test until boot completes in order to avoid
> > >  			interference.
> > >  
> > > +	rcuperf.kfree_rcu_test= [KNL]
> > > +			Set to measure performance of kfree_rcu() flooding.
> > > +
> > > +	rcuperf.kfree_nthreads= [KNL]
> > > +			The number of threads running loops of kfree_rcu().
> > > +
> > > +	rcuperf.kfree_alloc_num= [KNL]
> > > +			Number of allocations and frees done in an iteration.
> > > +
> > > +	rcuperf.kfree_loops= [KNL]
> > > +			Number of loops doing rcuperf.kfree_alloc_num number
> > > +			of allocations and frees.
> > > +
> > > +	rcuperf.kfree_no_batch= [KNL]
> > > +			Use the non-batching (slower) version of kfree_rcu.
> > > +			This is useful for comparing with the batched version.
> > 
> > I suggest s/slower/more efficient/ given that the batching takes more
> > wall-clock time than does the no-batching.
> 
> I think you mean, slower -> less efficient (due to taking up more grace
> period cycles per second in the no batching case). I will update it
> accordingly.

Yes, less efficient.  ;-)

> [snip]
> > > @@ -592,6 +593,175 @@ rcu_perf_shutdown(void *arg)
> > >  	return -EINVAL;
> > >  }
> > >  
> > > +/*
> > > + * kfree_rcu performance tests: Start a kfree_rcu loop on all CPUs for number
> > > + * of iterations and measure total time and number of GP for all iterations to complete.
> > > + */
> > > +
> > > +torture_param(int, kfree_nthreads, -1, "Number of threads running loops of kfree_rcu().");
> > > +torture_param(int, kfree_alloc_num, 8000, "Number of allocations and frees done in an iteration.");
> > > +torture_param(int, kfree_loops, 10, "Number of loops doing kfree_alloc_num allocations and frees.");
> > > +torture_param(int, kfree_no_batch, 0, "Use the non-batching (slower) version of kfree_rcu.");
> > > +
> > > +static struct task_struct **kfree_reader_tasks;
> > > +static int kfree_nrealthreads;
> > > +static atomic_t n_kfree_perf_thread_started;
> > > +static atomic_t n_kfree_perf_thread_ended;
> > > +
> > > +struct kfree_obj {
> > > +	char kfree_obj[8];
> > > +	struct rcu_head rh;
> > > +};
> > 
> > (Aside from above, no need to change this part of the patch, at least not
> > that I know of at the moment.)
> > 
> > 24 bytes on a 64-bit system, 16 on a 32-bit system.  So there might
> > have been 10 million extra objects awaiting free in the batching case
> > given the 400M-50M=350M excess for the batching approach.  If freeing
> > each object took about 100ns, that could account for the additional
> > wall-clock time for the batching approach.
> 
> Makes sense, and this comes down to 200-220MB range with the additional list.

Which might even match the observed numbers?

> > > +	set_user_nice(current, MAX_NICE);
> > > +
> > > +	alloc_ptrs = (struct kfree_obj **)kmalloc(sizeof(struct kfree_obj *) * kfree_alloc_num,
> > > +						  GFP_KERNEL);
> > > +	if (!alloc_ptrs)
> > > +		return -ENOMEM;
> > > +
> > > +	start_time = ktime_get_mono_fast_ns();
> > > +
> > > +	if (atomic_inc_return(&n_kfree_perf_thread_started) >= kfree_nrealthreads) {
> > > +		if (gp_exp)
> > > +			b_rcu_gp_test_started = cur_ops->exp_completed() / 2;
> > > +		else
> > > +			b_rcu_gp_test_started = cur_ops->get_gp_seq();
> > > +	}
> > > +
> > > +	do {
> > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > +			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> > > +			if (!alloc_ptrs[i])
> > > +				return -ENOMEM;
> > > +		}
> > > +
> > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > +			if (!kfree_no_batch) {
> > > +				kfree_rcu(alloc_ptrs[i], rh);
> > > +			} else {
> > > +				rcu_callback_t cb;
> > > +
> > > +				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> > > +				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> > > +			}
> > > +		}
> > 
> > The point of allocating a large batch and then kfree_rcu()ing them in a
> > loop is to defeat the per-CPU pool optimization?  Either way, a comment
> > would be very good!
> 
> It was a reasoning like this, added it as a comment:
> 
> 	/* While measuring kfree_rcu() time, we also end up measuring kmalloc()
> 	 * time. So the strategy here is to do a few (kfree_alloc_num) number
> 	 * of kmalloc() and kfree_rcu() every loop so that the current loop's
> 	 * deferred kfree()ing overlaps with the next loop's kmalloc().
> 	 */

The thought being that the CPU will be executing the two loops
concurrently?  Up to a point, agreed, but how much of an effect is
that, really?

Or is the idea to time the kfree_rcu() loop separately?  (I don't see
any such separate timing, though.)

							Thanx, Paul

> Will post it soon with other patches on top of -rcu dev.
> 
> thanks,
> 
>  - Joel
> 

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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-19 22:23       ` Paul E. McKenney
@ 2019-08-19 23:51         ` Joel Fernandes
  2019-08-20  2:50           ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-19 23:51 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Mon, Aug 19, 2019 at 03:23:30PM -0700, Paul E. McKenney wrote:
[snip]
> > [snip]
> > > > @@ -592,6 +593,175 @@ rcu_perf_shutdown(void *arg)
> > > >  	return -EINVAL;
> > > >  }
> > > >  
> > > > +/*
> > > > + * kfree_rcu performance tests: Start a kfree_rcu loop on all CPUs for number
> > > > + * of iterations and measure total time and number of GP for all iterations to complete.
> > > > + */
> > > > +
> > > > +torture_param(int, kfree_nthreads, -1, "Number of threads running loops of kfree_rcu().");
> > > > +torture_param(int, kfree_alloc_num, 8000, "Number of allocations and frees done in an iteration.");
> > > > +torture_param(int, kfree_loops, 10, "Number of loops doing kfree_alloc_num allocations and frees.");
> > > > +torture_param(int, kfree_no_batch, 0, "Use the non-batching (slower) version of kfree_rcu.");
> > > > +
> > > > +static struct task_struct **kfree_reader_tasks;
> > > > +static int kfree_nrealthreads;
> > > > +static atomic_t n_kfree_perf_thread_started;
> > > > +static atomic_t n_kfree_perf_thread_ended;
> > > > +
> > > > +struct kfree_obj {
> > > > +	char kfree_obj[8];
> > > > +	struct rcu_head rh;
> > > > +};
> > > 
> > > (Aside from above, no need to change this part of the patch, at least not
> > > that I know of at the moment.)
> > > 
> > > 24 bytes on a 64-bit system, 16 on a 32-bit system.  So there might
> > > have been 10 million extra objects awaiting free in the batching case
> > > given the 400M-50M=350M excess for the batching approach.  If freeing
> > > each object took about 100ns, that could account for the additional
> > > wall-clock time for the batching approach.
> > 
> > Makes sense, and this comes down to 200-220MB range with the additional list.
> 
> Which might even match the observed numbers?

Yes, they would. Since those *are* the observed numbers :-D ;-) ;-)

> > > > +	do {
> > > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > > +			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> > > > +			if (!alloc_ptrs[i])
> > > > +				return -ENOMEM;
> > > > +		}
> > > > +
> > > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > > +			if (!kfree_no_batch) {
> > > > +				kfree_rcu(alloc_ptrs[i], rh);
> > > > +			} else {
> > > > +				rcu_callback_t cb;
> > > > +
> > > > +				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> > > > +				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> > > > +			}
> > > > +		}
> > > 
> > > The point of allocating a large batch and then kfree_rcu()ing them in a
> > > loop is to defeat the per-CPU pool optimization?  Either way, a comment
> > > would be very good!
> > 
> > It was a reasoning like this, added it as a comment:
> > 
> > 	/* While measuring kfree_rcu() time, we also end up measuring kmalloc()
> > 	 * time. So the strategy here is to do a few (kfree_alloc_num) number
> > 	 * of kmalloc() and kfree_rcu() every loop so that the current loop's
> > 	 * deferred kfree()ing overlaps with the next loop's kmalloc().
> > 	 */
> 
> The thought being that the CPU will be executing the two loops
> concurrently?  Up to a point, agreed, but how much of an effect is
> that, really?

Yes it may not matter much. It was just a small thought when I added the
loop, I had to start somewhere, so I did it this way.

> Or is the idea to time the kfree_rcu() loop separately?  (I don't see
> any such separate timing, though.)

The kmalloc() times are included within the kfree loop. The timing of
kfree_rcu() is not separate in my patch.

thanks,

 - Joel


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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-19 23:51         ` Joel Fernandes
@ 2019-08-20  2:50           ` Paul E. McKenney
  2019-08-21  0:27             ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-20  2:50 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Mon, Aug 19, 2019 at 07:51:23PM -0400, Joel Fernandes wrote:
> On Mon, Aug 19, 2019 at 03:23:30PM -0700, Paul E. McKenney wrote:
> [snip]
> > > [snip]
> > > > > @@ -592,6 +593,175 @@ rcu_perf_shutdown(void *arg)
> > > > >  	return -EINVAL;
> > > > >  }
> > > > >  
> > > > > +/*
> > > > > + * kfree_rcu performance tests: Start a kfree_rcu loop on all CPUs for number
> > > > > + * of iterations and measure total time and number of GP for all iterations to complete.
> > > > > + */
> > > > > +
> > > > > +torture_param(int, kfree_nthreads, -1, "Number of threads running loops of kfree_rcu().");
> > > > > +torture_param(int, kfree_alloc_num, 8000, "Number of allocations and frees done in an iteration.");
> > > > > +torture_param(int, kfree_loops, 10, "Number of loops doing kfree_alloc_num allocations and frees.");
> > > > > +torture_param(int, kfree_no_batch, 0, "Use the non-batching (slower) version of kfree_rcu.");
> > > > > +
> > > > > +static struct task_struct **kfree_reader_tasks;
> > > > > +static int kfree_nrealthreads;
> > > > > +static atomic_t n_kfree_perf_thread_started;
> > > > > +static atomic_t n_kfree_perf_thread_ended;
> > > > > +
> > > > > +struct kfree_obj {
> > > > > +	char kfree_obj[8];
> > > > > +	struct rcu_head rh;
> > > > > +};
> > > > 
> > > > (Aside from above, no need to change this part of the patch, at least not
> > > > that I know of at the moment.)
> > > > 
> > > > 24 bytes on a 64-bit system, 16 on a 32-bit system.  So there might
> > > > have been 10 million extra objects awaiting free in the batching case
> > > > given the 400M-50M=350M excess for the batching approach.  If freeing
> > > > each object took about 100ns, that could account for the additional
> > > > wall-clock time for the batching approach.
> > > 
> > > Makes sense, and this comes down to 200-220MB range with the additional list.
> > 
> > Which might even match the observed numbers?
> 
> Yes, they would. Since those *are* the observed numbers :-D ;-) ;-)

;-)

> > > > > +	do {
> > > > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > > > +			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> > > > > +			if (!alloc_ptrs[i])
> > > > > +				return -ENOMEM;
> > > > > +		}
> > > > > +
> > > > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > > > +			if (!kfree_no_batch) {
> > > > > +				kfree_rcu(alloc_ptrs[i], rh);
> > > > > +			} else {
> > > > > +				rcu_callback_t cb;
> > > > > +
> > > > > +				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> > > > > +				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> > > > > +			}
> > > > > +		}
> > > > 
> > > > The point of allocating a large batch and then kfree_rcu()ing them in a
> > > > loop is to defeat the per-CPU pool optimization?  Either way, a comment
> > > > would be very good!
> > > 
> > > It was a reasoning like this, added it as a comment:
> > > 
> > > 	/* While measuring kfree_rcu() time, we also end up measuring kmalloc()
> > > 	 * time. So the strategy here is to do a few (kfree_alloc_num) number
> > > 	 * of kmalloc() and kfree_rcu() every loop so that the current loop's
> > > 	 * deferred kfree()ing overlaps with the next loop's kmalloc().
> > > 	 */
> > 
> > The thought being that the CPU will be executing the two loops
> > concurrently?  Up to a point, agreed, but how much of an effect is
> > that, really?
> 
> Yes it may not matter much. It was just a small thought when I added the
> loop, I had to start somewhere, so I did it this way.
> 
> > Or is the idea to time the kfree_rcu() loop separately?  (I don't see
> > any such separate timing, though.)
> 
> The kmalloc() times are included within the kfree loop. The timing of
> kfree_rcu() is not separate in my patch.

You lost me on this one.  What happens when you just interleave the
kmalloc() and kfree_rcu(), without looping, compared to the looping
above?  Does this get more expensive?  Cheaper?  More vulnerable to OOM?
Something else?

							Thanx, Paul

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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-20  2:50           ` Paul E. McKenney
@ 2019-08-21  0:27             ` Joel Fernandes
  2019-08-21  0:31               ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-21  0:27 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Mon, Aug 19, 2019 at 07:50:56PM -0700, Paul E. McKenney wrote:
 
> > > > > > +	do {
> > > > > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > > > > +			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> > > > > > +			if (!alloc_ptrs[i])
> > > > > > +				return -ENOMEM;
> > > > > > +		}
> > > > > > +
> > > > > > +		for (i = 0; i < kfree_alloc_num; i++) {
> > > > > > +			if (!kfree_no_batch) {
> > > > > > +				kfree_rcu(alloc_ptrs[i], rh);
> > > > > > +			} else {
> > > > > > +				rcu_callback_t cb;
> > > > > > +
> > > > > > +				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> > > > > > +				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> > > > > > +			}
> > > > > > +		}
> > > > > 
> > > > > The point of allocating a large batch and then kfree_rcu()ing them in a
> > > > > loop is to defeat the per-CPU pool optimization?  Either way, a comment
> > > > > would be very good!
> > > > 
> > > > It was a reasoning like this, added it as a comment:
> > > > 
> > > > 	/* While measuring kfree_rcu() time, we also end up measuring kmalloc()
> > > > 	 * time. So the strategy here is to do a few (kfree_alloc_num) number
> > > > 	 * of kmalloc() and kfree_rcu() every loop so that the current loop's
> > > > 	 * deferred kfree()ing overlaps with the next loop's kmalloc().
> > > > 	 */
> > > 
> > > The thought being that the CPU will be executing the two loops
> > > concurrently?  Up to a point, agreed, but how much of an effect is
> > > that, really?
> > 
> > Yes it may not matter much. It was just a small thought when I added the
> > loop, I had to start somewhere, so I did it this way.
> > 
> > > Or is the idea to time the kfree_rcu() loop separately?  (I don't see
> > > any such separate timing, though.)
> > 
> > The kmalloc() times are included within the kfree loop. The timing of
> > kfree_rcu() is not separate in my patch.
> 
> You lost me on this one.  What happens when you just interleave the
> kmalloc() and kfree_rcu(), without looping, compared to the looping
> above?  Does this get more expensive?  Cheaper?  More vulnerable to OOM?
> Something else?

You mean pairing a single kmalloc() with a single kfree_rcu() and doing this
several times? The results are very similar to doing kfree_alloc_num
kmalloc()s, then do kfree_alloc_num kfree_rcu()s; and repeat the whole thing
kfree_loops times (as done by this rcuperf patch we are reviewing).

Following are some numbers. One change is the case where we are not at all
batching does seem to complete even faster when we fully interleave kmalloc()
with kfree() while the case of batching in the same scenario completes at the
same time as did the "not fully interleaved" scenario. However, the grace
period reduction improvements and the chances of OOM'ing are pretty much the
same in either case.

Fully interleaved, single kmalloc followed by kfree_rcu, do this kfree_alloc_num * kfree_loops times.
=======================
(1) Batching
rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=0 rcuperf.kfree_rcu_test=1

root@(none):/# free -m
              total        used        free      shared  buff/cache   available
Mem:            977         261         675           0          39         674

[   15.635620] Total time taken by all kfree'ers: 14255673998 ns, loops: 20000, batches: 1596

(2) No Batching
rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=1 rcuperf.kfree_rcu_test=1

root@(none):/# free -m
             total        used        free      shared  buff/cache   available
Mem:            977          67         870           0          39         869
Swap:             0           0           0

[   12.365872] Total time taken by all kfree'ers: 10902137101 ns, loops: 20000, batches: 6893


Not fully interleaved: do kfree_alloc_num kmallocs, then do kfree_alloc_num kfree_rcu()s. And repeat this kfree_loops times.
=======================
(1) Batching
rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=0 rcuperf.kfree_rcu_test=1

root@(none):/# free -m
              total        used        free      shared  buff/cache   available
Mem:            977         251         686           0          39         684
Swap:             0           0           0

[   15.574402] Total time taken by all kfree'ers: 14185970787 ns, loops: 20000, batches: 1548

(2) No Batching
rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=1 rcuperf.kfree_rcu_test=1

root@(none):/# free -m
              total        used        free      shared  buff/cache   available
Mem:            977          82         855           0          39         853
Swap:             0           0           0

[   13.724554] Total time taken by all kfree'ers: 12246217291 ns, loops: 20000, batches: 7262


thanks,

 - Joel



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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-21  0:27             ` Joel Fernandes
@ 2019-08-21  0:31               ` Joel Fernandes
  2019-08-21  0:44                 ` Paul E. McKenney
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-08-21  0:31 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Tue, Aug 20, 2019 at 08:27:05PM -0400, Joel Fernandes wrote:
[snip]
> > > > Or is the idea to time the kfree_rcu() loop separately?  (I don't see
> > > > any such separate timing, though.)
> > > 
> > > The kmalloc() times are included within the kfree loop. The timing of
> > > kfree_rcu() is not separate in my patch.
> > 
> > You lost me on this one.  What happens when you just interleave the
> > kmalloc() and kfree_rcu(), without looping, compared to the looping
> > above?  Does this get more expensive?  Cheaper?  More vulnerable to OOM?
> > Something else?
> 
> You mean pairing a single kmalloc() with a single kfree_rcu() and doing this
> several times? The results are very similar to doing kfree_alloc_num
> kmalloc()s, then do kfree_alloc_num kfree_rcu()s; and repeat the whole thing
> kfree_loops times (as done by this rcuperf patch we are reviewing).
> 
> Following are some numbers. One change is the case where we are not at all
> batching does seem to complete even faster when we fully interleave kmalloc()
> with kfree() while the case of batching in the same scenario completes at the
> same time as did the "not fully interleaved" scenario. However, the grace
> period reduction improvements and the chances of OOM'ing are pretty much the
> same in either case.
[snip]
> Not fully interleaved: do kfree_alloc_num kmallocs, then do kfree_alloc_num kfree_rcu()s. And repeat this kfree_loops times.
> =======================
> (1) Batching
> rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=0 rcuperf.kfree_rcu_test=1
> 
> root@(none):/# free -m
>               total        used        free      shared  buff/cache   available
> Mem:            977         251         686           0          39         684
> Swap:             0           0           0
> 
> [   15.574402] Total time taken by all kfree'ers: 14185970787 ns, loops: 20000, batches: 1548
> 
> (2) No Batching
> rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=1 rcuperf.kfree_rcu_test=1
> 
> root@(none):/# free -m
>               total        used        free      shared  buff/cache   available
> Mem:            977          82         855           0          39         853
> Swap:             0           0           0
> 
> [   13.724554] Total time taken by all kfree'ers: 12246217291 ns, loops: 20000, batches: 7262

And the diff for changing the test to do this case is as follows (I don't
plan to fold this diff in, since I feel the existing test suffices and
results are similar):

diff --git a/kernel/rcu/rcuperf.c b/kernel/rcu/rcuperf.c
index 46f9c4449348..e4e4be4aaf51 100644
--- a/kernel/rcu/rcuperf.c
+++ b/kernel/rcu/rcuperf.c
@@ -618,18 +618,13 @@ kfree_perf_thread(void *arg)
 {
 	int i, loop = 0;
 	long me = (long)arg;
-	struct kfree_obj **alloc_ptrs;
+	struct kfree_obj *alloc_ptr;
 	u64 start_time, end_time;
 
 	VERBOSE_PERFOUT_STRING("kfree_perf_thread task started");
 	set_cpus_allowed_ptr(current, cpumask_of(me % nr_cpu_ids));
 	set_user_nice(current, MAX_NICE);
 
-	alloc_ptrs = (struct kfree_obj **)kmalloc(sizeof(struct kfree_obj *) * kfree_alloc_num,
-						  GFP_KERNEL);
-	if (!alloc_ptrs)
-		return -ENOMEM;
-
 	start_time = ktime_get_mono_fast_ns();
 
 	if (atomic_inc_return(&n_kfree_perf_thread_started) >= kfree_nrealthreads) {
@@ -646,19 +641,17 @@ kfree_perf_thread(void *arg)
 	 */
 	do {
 		for (i = 0; i < kfree_alloc_num; i++) {
-			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
-			if (!alloc_ptrs[i])
+			alloc_ptr = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
+			if (!alloc_ptr)
 				return -ENOMEM;
-		}
 
-		for (i = 0; i < kfree_alloc_num; i++) {
 			if (!kfree_no_batch) {
-				kfree_rcu(alloc_ptrs[i], rh);
+				kfree_rcu(alloc_ptr, rh);
 			} else {
 				rcu_callback_t cb;
 
 				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
-				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
+				kfree_call_rcu_nobatch(&(alloc_ptr->rh), cb);
 			}
 		}
 
@@ -682,7 +675,6 @@ kfree_perf_thread(void *arg)
 		}
 	}
 
-	kfree(alloc_ptrs);
 	torture_kthread_stopping("kfree_perf_thread");
 	return 0;
 }

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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-21  0:31               ` Joel Fernandes
@ 2019-08-21  0:44                 ` Paul E. McKenney
  2019-08-21  0:51                   ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-08-21  0:44 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Tue, Aug 20, 2019 at 08:31:32PM -0400, Joel Fernandes wrote:
> On Tue, Aug 20, 2019 at 08:27:05PM -0400, Joel Fernandes wrote:
> [snip]
> > > > > Or is the idea to time the kfree_rcu() loop separately?  (I don't see
> > > > > any such separate timing, though.)
> > > > 
> > > > The kmalloc() times are included within the kfree loop. The timing of
> > > > kfree_rcu() is not separate in my patch.
> > > 
> > > You lost me on this one.  What happens when you just interleave the
> > > kmalloc() and kfree_rcu(), without looping, compared to the looping
> > > above?  Does this get more expensive?  Cheaper?  More vulnerable to OOM?
> > > Something else?
> > 
> > You mean pairing a single kmalloc() with a single kfree_rcu() and doing this
> > several times? The results are very similar to doing kfree_alloc_num
> > kmalloc()s, then do kfree_alloc_num kfree_rcu()s; and repeat the whole thing
> > kfree_loops times (as done by this rcuperf patch we are reviewing).
> > 
> > Following are some numbers. One change is the case where we are not at all
> > batching does seem to complete even faster when we fully interleave kmalloc()
> > with kfree() while the case of batching in the same scenario completes at the
> > same time as did the "not fully interleaved" scenario. However, the grace
> > period reduction improvements and the chances of OOM'ing are pretty much the
> > same in either case.
> [snip]
> > Not fully interleaved: do kfree_alloc_num kmallocs, then do kfree_alloc_num kfree_rcu()s. And repeat this kfree_loops times.
> > =======================
> > (1) Batching
> > rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=0 rcuperf.kfree_rcu_test=1
> > 
> > root@(none):/# free -m
> >               total        used        free      shared  buff/cache   available
> > Mem:            977         251         686           0          39         684
> > Swap:             0           0           0
> > 
> > [   15.574402] Total time taken by all kfree'ers: 14185970787 ns, loops: 20000, batches: 1548
> > 
> > (2) No Batching
> > rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=1 rcuperf.kfree_rcu_test=1
> > 
> > root@(none):/# free -m
> >               total        used        free      shared  buff/cache   available
> > Mem:            977          82         855           0          39         853
> > Swap:             0           0           0
> > 
> > [   13.724554] Total time taken by all kfree'ers: 12246217291 ns, loops: 20000, batches: 7262
> 
> And the diff for changing the test to do this case is as follows (I don't
> plan to fold this diff in, since I feel the existing test suffices and
> results are similar):

But why not?  It does look to be a nice simplification, after all.

							Thanx, Paul

> diff --git a/kernel/rcu/rcuperf.c b/kernel/rcu/rcuperf.c
> index 46f9c4449348..e4e4be4aaf51 100644
> --- a/kernel/rcu/rcuperf.c
> +++ b/kernel/rcu/rcuperf.c
> @@ -618,18 +618,13 @@ kfree_perf_thread(void *arg)
>  {
>  	int i, loop = 0;
>  	long me = (long)arg;
> -	struct kfree_obj **alloc_ptrs;
> +	struct kfree_obj *alloc_ptr;
>  	u64 start_time, end_time;
>  
>  	VERBOSE_PERFOUT_STRING("kfree_perf_thread task started");
>  	set_cpus_allowed_ptr(current, cpumask_of(me % nr_cpu_ids));
>  	set_user_nice(current, MAX_NICE);
>  
> -	alloc_ptrs = (struct kfree_obj **)kmalloc(sizeof(struct kfree_obj *) * kfree_alloc_num,
> -						  GFP_KERNEL);
> -	if (!alloc_ptrs)
> -		return -ENOMEM;
> -
>  	start_time = ktime_get_mono_fast_ns();
>  
>  	if (atomic_inc_return(&n_kfree_perf_thread_started) >= kfree_nrealthreads) {
> @@ -646,19 +641,17 @@ kfree_perf_thread(void *arg)
>  	 */
>  	do {
>  		for (i = 0; i < kfree_alloc_num; i++) {
> -			alloc_ptrs[i] = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> -			if (!alloc_ptrs[i])
> +			alloc_ptr = kmalloc(sizeof(struct kfree_obj), GFP_KERNEL);
> +			if (!alloc_ptr)
>  				return -ENOMEM;
> -		}
>  
> -		for (i = 0; i < kfree_alloc_num; i++) {
>  			if (!kfree_no_batch) {
> -				kfree_rcu(alloc_ptrs[i], rh);
> +				kfree_rcu(alloc_ptr, rh);
>  			} else {
>  				rcu_callback_t cb;
>  
>  				cb = (rcu_callback_t)(unsigned long)offsetof(struct kfree_obj, rh);
> -				kfree_call_rcu_nobatch(&(alloc_ptrs[i]->rh), cb);
> +				kfree_call_rcu_nobatch(&(alloc_ptr->rh), cb);
>  			}
>  		}
>  
> @@ -682,7 +675,6 @@ kfree_perf_thread(void *arg)
>  		}
>  	}
>  
> -	kfree(alloc_ptrs);
>  	torture_kthread_stopping("kfree_perf_thread");
>  	return 0;
>  }

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

* Re: [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests
  2019-08-21  0:44                 ` Paul E. McKenney
@ 2019-08-21  0:51                   ` Joel Fernandes
  0 siblings, 0 replies; 30+ messages in thread
From: Joel Fernandes @ 2019-08-21  0:51 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: linux-kernel, byungchul.park, Davidlohr Bueso, Josh Triplett,
	kernel-team, kernel-team, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Tue, Aug 20, 2019 at 05:44:36PM -0700, Paul E. McKenney wrote:
> On Tue, Aug 20, 2019 at 08:31:32PM -0400, Joel Fernandes wrote:
> > On Tue, Aug 20, 2019 at 08:27:05PM -0400, Joel Fernandes wrote:
> > [snip]
> > > > > > Or is the idea to time the kfree_rcu() loop separately?  (I don't see
> > > > > > any such separate timing, though.)
> > > > > 
> > > > > The kmalloc() times are included within the kfree loop. The timing of
> > > > > kfree_rcu() is not separate in my patch.
> > > > 
> > > > You lost me on this one.  What happens when you just interleave the
> > > > kmalloc() and kfree_rcu(), without looping, compared to the looping
> > > > above?  Does this get more expensive?  Cheaper?  More vulnerable to OOM?
> > > > Something else?
> > > 
> > > You mean pairing a single kmalloc() with a single kfree_rcu() and doing this
> > > several times? The results are very similar to doing kfree_alloc_num
> > > kmalloc()s, then do kfree_alloc_num kfree_rcu()s; and repeat the whole thing
> > > kfree_loops times (as done by this rcuperf patch we are reviewing).
> > > 
> > > Following are some numbers. One change is the case where we are not at all
> > > batching does seem to complete even faster when we fully interleave kmalloc()
> > > with kfree() while the case of batching in the same scenario completes at the
> > > same time as did the "not fully interleaved" scenario. However, the grace
> > > period reduction improvements and the chances of OOM'ing are pretty much the
> > > same in either case.
> > [snip]
> > > Not fully interleaved: do kfree_alloc_num kmallocs, then do kfree_alloc_num kfree_rcu()s. And repeat this kfree_loops times.
> > > =======================
> > > (1) Batching
> > > rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=0 rcuperf.kfree_rcu_test=1
> > > 
> > > root@(none):/# free -m
> > >               total        used        free      shared  buff/cache   available
> > > Mem:            977         251         686           0          39         684
> > > Swap:             0           0           0
> > > 
> > > [   15.574402] Total time taken by all kfree'ers: 14185970787 ns, loops: 20000, batches: 1548
> > > 
> > > (2) No Batching
> > > rcuperf.kfree_loops=20000 rcuperf.kfree_alloc_num=8000 rcuperf.kfree_no_batch=1 rcuperf.kfree_rcu_test=1
> > > 
> > > root@(none):/# free -m
> > >               total        used        free      shared  buff/cache   available
> > > Mem:            977          82         855           0          39         853
> > > Swap:             0           0           0
> > > 
> > > [   13.724554] Total time taken by all kfree'ers: 12246217291 ns, loops: 20000, batches: 7262
> > 
> > And the diff for changing the test to do this case is as follows (I don't
> > plan to fold this diff in, since I feel the existing test suffices and
> > results are similar):
> 
> But why not?  It does look to be a nice simplification, after all.

That's true. Ok, I'll squash it in. thanks!

thanks,

 - Joel
[snip]

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-08-14 16:04 [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Joel Fernandes (Google)
  2019-08-14 16:04 ` [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests Joel Fernandes (Google)
  2019-08-16 16:43 ` [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Paul E. McKenney
@ 2019-09-18  9:58 ` Uladzislau Rezki
  2019-09-30 20:16   ` Joel Fernandes
  2019-12-10  9:53   ` Uladzislau Rezki
  2 siblings, 2 replies; 30+ messages in thread
From: Uladzislau Rezki @ 2019-09-18  9:58 UTC (permalink / raw)
  To: Joel Fernandes (Google)
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Paul E. McKenney, Rao Shoaib, rcu,
	Steven Rostedt

> Recently a discussion about stability and performance of a system
> involving a high rate of kfree_rcu() calls surfaced on the list [1]
> which led to another discussion how to prepare for this situation.
> 
> This patch adds basic batching support for kfree_rcu(). It is "basic"
> because we do none of the slab management, dynamic allocation, code
> moving or any of the other things, some of which previous attempts did
> [2]. These fancier improvements can be follow-up patches and there are
> different ideas being discussed in those regards. This is an effort to
> start simple, and build up from there. In the future, an extension to
> use kfree_bulk and possibly per-slab batching could be done to further
> improve performance due to cache-locality and slab-specific bulk free
> optimizations. By using an array of pointers, the worker thread
> processing the work would need to read lesser data since it does not
> need to deal with large rcu_head(s) any longer.
> 
> Torture tests follow in the next patch and show improvements of around
> 5x reduction in number of  grace periods on a 16 CPU system. More
> details and test data are in that patch.
> 
> There is an implication with rcu_barrier() with this patch. Since the
> kfree_rcu() calls can be batched, and may not be handed yet to the RCU
> machinery in fact, the monitor may not have even run yet to do the
> queue_rcu_work(), there seems no easy way of implementing rcu_barrier()
> to wait for those kfree_rcu()s that are already made. So this means a
> kfree_rcu() followed by an rcu_barrier() does not imply that memory will
> be freed once rcu_barrier() returns.
> 
> Another implication is higher active memory usage (although not
> run-away..) until the kfree_rcu() flooding ends, in comparison to
> without batching. More details about this are in the second patch which
> adds an rcuperf test.
> 
> Finally, in the near future we will get rid of kfree_rcu() special casing
> within RCU such as in rcu_do_batch and switch everything to just
> batching. Currently we don't do that since timer subsystem is not yet up
> and we cannot schedule the kfree_rcu() monitor as the timer subsystem's
> lock are not initialized. That would also mean getting rid of
> kfree_call_rcu_nobatch() entirely.
> 
Hello, Joel.

First of all thank you for improving it. I also noticed a high pressure
on RCU-machinery during performing some vmalloc tests when kfree_rcu()
flood occurred. Therefore i got rid of using kfree_rcu() there.

I have just a small question related to workloads and performance evaluation.
Are you aware of any specific workloads which benefit from it for example
mobile area, etc? I am asking because i think about backporting of it and
reuse it on our kernel. 

Thank you!

--
Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-09-18  9:58 ` Uladzislau Rezki
@ 2019-09-30 20:16   ` Joel Fernandes
  2019-10-01 11:27     ` Uladzislau Rezki
  2019-12-10  9:53   ` Uladzislau Rezki
  1 sibling, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-09-30 20:16 UTC (permalink / raw)
  To: Uladzislau Rezki
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Paul E. McKenney, Rao Shoaib, rcu,
	Steven Rostedt

On Wed, Sep 18, 2019 at 11:58:11AM +0200, Uladzislau Rezki wrote:
> > Recently a discussion about stability and performance of a system
> > involving a high rate of kfree_rcu() calls surfaced on the list [1]
> > which led to another discussion how to prepare for this situation.
> > 
> > This patch adds basic batching support for kfree_rcu(). It is "basic"
> > because we do none of the slab management, dynamic allocation, code
> > moving or any of the other things, some of which previous attempts did
> > [2]. These fancier improvements can be follow-up patches and there are
> > different ideas being discussed in those regards. This is an effort to
> > start simple, and build up from there. In the future, an extension to
> > use kfree_bulk and possibly per-slab batching could be done to further
> > improve performance due to cache-locality and slab-specific bulk free
> > optimizations. By using an array of pointers, the worker thread
> > processing the work would need to read lesser data since it does not
> > need to deal with large rcu_head(s) any longer.
> > 
> > Torture tests follow in the next patch and show improvements of around
> > 5x reduction in number of  grace periods on a 16 CPU system. More
> > details and test data are in that patch.
> > 
> > There is an implication with rcu_barrier() with this patch. Since the
> > kfree_rcu() calls can be batched, and may not be handed yet to the RCU
> > machinery in fact, the monitor may not have even run yet to do the
> > queue_rcu_work(), there seems no easy way of implementing rcu_barrier()
> > to wait for those kfree_rcu()s that are already made. So this means a
> > kfree_rcu() followed by an rcu_barrier() does not imply that memory will
> > be freed once rcu_barrier() returns.
> > 
> > Another implication is higher active memory usage (although not
> > run-away..) until the kfree_rcu() flooding ends, in comparison to
> > without batching. More details about this are in the second patch which
> > adds an rcuperf test.
> > 
> > Finally, in the near future we will get rid of kfree_rcu() special casing
> > within RCU such as in rcu_do_batch and switch everything to just
> > batching. Currently we don't do that since timer subsystem is not yet up
> > and we cannot schedule the kfree_rcu() monitor as the timer subsystem's
> > lock are not initialized. That would also mean getting rid of
> > kfree_call_rcu_nobatch() entirely.
> > 
> Hello, Joel.
> 
> First of all thank you for improving it. I also noticed a high pressure
> on RCU-machinery during performing some vmalloc tests when kfree_rcu()
> flood occurred. Therefore i got rid of using kfree_rcu() there.

Replying a bit late due to overseas conference travel and vacation.

When you say 'high pressure', do you mean memory pressure or just system
load?

Memory pressure slightly increases with the kfree_rcu() rework with the
benefit of much fewer grace periods.

> I have just a small question related to workloads and performance evaluation.
> Are you aware of any specific workloads which benefit from it for example
> mobile area, etc? I am asking because i think about backporting of it and
> reuse it on our kernel. 

I am not aware of a mobile usecase that benefits but there are server
workloads that make system more stable in the face of a kfree_rcu() flood.

For the KVA allocator work, I see it is quite similar to the way binder
allocates blocks. See function: binder_alloc_new_buf_locked(). Is there are
any chance to reuse any code? For one thing, binder also has an rbtree for
allocated blocks for fast lookup of allocated blocks. Does the KVA allocator
not have the need for that?

And, nice LPC presentation! I was there ;)

thanks,

 - Joel


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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-09-30 20:16   ` Joel Fernandes
@ 2019-10-01 11:27     ` Uladzislau Rezki
  2019-10-04 17:20       ` Joel Fernandes
  0 siblings, 1 reply; 30+ messages in thread
From: Uladzislau Rezki @ 2019-10-01 11:27 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: Uladzislau Rezki, linux-kernel, kernel-team, kernel-team,
	Byungchul Park, Davidlohr Bueso, Josh Triplett, Lai Jiangshan,
	Mathieu Desnoyers, max.byungchul.park, Paul E. McKenney,
	Rao Shoaib, rcu, Steven Rostedt

> > Hello, Joel.
> > 
> > First of all thank you for improving it. I also noticed a high pressure
> > on RCU-machinery during performing some vmalloc tests when kfree_rcu()
> > flood occurred. Therefore i got rid of using kfree_rcu() there.
> 
> Replying a bit late due to overseas conference travel and vacation.
> 
> When you say 'high pressure', do you mean memory pressure or just system
> load?
> 
>
> Memory pressure slightly increases with the kfree_rcu() rework with the
> benefit of much fewer grace periods.
> 
I meant a system load, because of high number of cycles in the kfree_rcu()
symbol under stressing. But i do not have numbers next to me, because it
was quite a long time ago. As for memory usage, i understand that.

> > I have just a small question related to workloads and performance evaluation.
> > Are you aware of any specific workloads which benefit from it for example
> > mobile area, etc? I am asking because i think about backporting of it and
> > reuse it on our kernel. 
> 
> I am not aware of a mobile usecase that benefits but there are server
> workloads that make system more stable in the face of a kfree_rcu() flood.
> 
OK, i got it. I wanted to test it finding out how it could effect mobile
workloads.

>
> For the KVA allocator work, I see it is quite similar to the way binder
> allocates blocks. See function: binder_alloc_new_buf_locked(). Is there are
> any chance to reuse any code? For one thing, binder also has an rbtree for
> allocated blocks for fast lookup of allocated blocks. Does the KVA allocator
> not have the need for that?
>
Well, there is a difference. Actually the free blocks are not sorted by
the its size like in binder layer, if understand the code correctly.

Instead, i keep them(free blocks) sorted(by start address) in ascending
order + maintain the augment value(biggest free size in left or right sub-tree)
for each node, that allows to navigate toward the lowest address and the block
that definitely suits. So as a result our allocations become sequential
what is important.

>
> And, nice LPC presentation! I was there ;)
> 
Thanks :)

--
Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-10-01 11:27     ` Uladzislau Rezki
@ 2019-10-04 17:20       ` Joel Fernandes
  2019-10-08 16:23         ` Uladzislau Rezki
  0 siblings, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-10-04 17:20 UTC (permalink / raw)
  To: Uladzislau Rezki
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Paul E. McKenney, Rao Shoaib, rcu,
	Steven Rostedt

On Tue, Oct 01, 2019 at 01:27:02PM +0200, Uladzislau Rezki wrote:
[snip] 
> > > I have just a small question related to workloads and performance evaluation.
> > > Are you aware of any specific workloads which benefit from it for example
> > > mobile area, etc? I am asking because i think about backporting of it and
> > > reuse it on our kernel. 
> > 
> > I am not aware of a mobile usecase that benefits but there are server
> > workloads that make system more stable in the face of a kfree_rcu() flood.
> > 
> OK, i got it. I wanted to test it finding out how it could effect mobile
> workloads.
> 
> >
> > For the KVA allocator work, I see it is quite similar to the way binder
> > allocates blocks. See function: binder_alloc_new_buf_locked(). Is there are
> > any chance to reuse any code? For one thing, binder also has an rbtree for
> > allocated blocks for fast lookup of allocated blocks. Does the KVA allocator
> > not have the need for that?
> >
> Well, there is a difference. Actually the free blocks are not sorted by
> the its size like in binder layer, if understand the code correctly.
> 
> Instead, i keep them(free blocks) sorted(by start address) in ascending
> order + maintain the augment value(biggest free size in left or right sub-tree)
> for each node, that allows to navigate toward the lowest address and the block
> that definitely suits. So as a result our allocations become sequential
> what is important.

Right, I realized this after sending the email that binder and kva sort
differently though they both try to use free sizes during the allocation.

Would you have any papers, which survey various rb-tree based allocator
algorithms and their tradeoffs? I am interested in studying these more
especially in relation to the binder driver. Would also be nice to make
contributions to papers surveying both these allocators to describe the state
of the art.

thanks,

 - Joel


> >
> > And, nice LPC presentation! I was there ;)
> > 
> Thanks :)
> 
> --
> Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-10-04 17:20       ` Joel Fernandes
@ 2019-10-08 16:23         ` Uladzislau Rezki
  0 siblings, 0 replies; 30+ messages in thread
From: Uladzislau Rezki @ 2019-10-08 16:23 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: Uladzislau Rezki, linux-kernel, kernel-team, kernel-team,
	Byungchul Park, Davidlohr Bueso, Josh Triplett, Lai Jiangshan,
	Mathieu Desnoyers, max.byungchul.park, Paul E. McKenney,
	Rao Shoaib, rcu, Steven Rostedt

On Fri, Oct 04, 2019 at 01:20:38PM -0400, Joel Fernandes wrote:
> On Tue, Oct 01, 2019 at 01:27:02PM +0200, Uladzislau Rezki wrote:
> [snip] 
> > > > I have just a small question related to workloads and performance evaluation.
> > > > Are you aware of any specific workloads which benefit from it for example
> > > > mobile area, etc? I am asking because i think about backporting of it and
> > > > reuse it on our kernel. 
> > > 
> > > I am not aware of a mobile usecase that benefits but there are server
> > > workloads that make system more stable in the face of a kfree_rcu() flood.
> > > 
> > OK, i got it. I wanted to test it finding out how it could effect mobile
> > workloads.
> > 
> > >
> > > For the KVA allocator work, I see it is quite similar to the way binder
> > > allocates blocks. See function: binder_alloc_new_buf_locked(). Is there are
> > > any chance to reuse any code? For one thing, binder also has an rbtree for
> > > allocated blocks for fast lookup of allocated blocks. Does the KVA allocator
> > > not have the need for that?
> > >
> > Well, there is a difference. Actually the free blocks are not sorted by
> > the its size like in binder layer, if understand the code correctly.
> > 
> > Instead, i keep them(free blocks) sorted(by start address) in ascending
> > order + maintain the augment value(biggest free size in left or right sub-tree)
> > for each node, that allows to navigate toward the lowest address and the block
> > that definitely suits. So as a result our allocations become sequential
> > what is important.
> 
> Right, I realized this after sending the email that binder and kva sort
> differently though they both try to use free sizes during the allocation.
> 
> Would you have any papers, which survey various rb-tree based allocator
> algorithms and their tradeoffs? I am interested in studying these more
> especially in relation to the binder driver. Would also be nice to make
> contributions to papers surveying both these allocators to describe the state
> of the art.
> 
So far i have not had any paper with different kind of comparison. But
that is interested for sure, especially to analyze the model for example
based on B-Tree, so when we can fully utilize a cache performance.
Because regular binary trees are just pointer chasing.

As for binder driver and its allocator, is it O(lognN) complexity? Is
there any bottleneck in its implementation?

Thanks!

--
Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-09-18  9:58 ` Uladzislau Rezki
  2019-09-30 20:16   ` Joel Fernandes
@ 2019-12-10  9:53   ` Uladzislau Rezki
  2019-12-11 23:46     ` Paul E. McKenney
  2019-12-12  5:27     ` Joel Fernandes
  1 sibling, 2 replies; 30+ messages in thread
From: Uladzislau Rezki @ 2019-12-10  9:53 UTC (permalink / raw)
  To: Joel Fernandes (Google), Paul E. McKenney
  Cc: linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Wed, Sep 18, 2019 at 11:58:11AM +0200, Uladzislau Rezki wrote:
> > Recently a discussion about stability and performance of a system
> > involving a high rate of kfree_rcu() calls surfaced on the list [1]
> > which led to another discussion how to prepare for this situation.
> > 
> > This patch adds basic batching support for kfree_rcu(). It is "basic"
> > because we do none of the slab management, dynamic allocation, code
> > moving or any of the other things, some of which previous attempts did
> > [2]. These fancier improvements can be follow-up patches and there are
> > different ideas being discussed in those regards. This is an effort to
> > start simple, and build up from there. In the future, an extension to
> > use kfree_bulk and possibly per-slab batching could be done to further
> > improve performance due to cache-locality and slab-specific bulk free
> > optimizations. By using an array of pointers, the worker thread
> > processing the work would need to read lesser data since it does not
> > need to deal with large rcu_head(s) any longer.
> > 
According to https://lkml.org/lkml/2017/12/19/706 there was an attempt
to make use of kfree_bulk() interface. I have done some tests based on
your patch and enhanced kfree_bulk() logic. Basically storing pointers 
in an array with a specific size makes sense to me and seems to others
as well. I mean in comparison with "pointer chasing" way, when there is
probably a cache misses each time the access is done to next element:

diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
index 1fe0418a5901..4f68662c1568 100644
--- a/kernel/rcu/tree.c
+++ b/kernel/rcu/tree.c
@@ -2595,6 +2595,13 @@ EXPORT_SYMBOL_GPL(call_rcu);

 /* Maximum number of jiffies to wait before draining a batch. */
 #define KFREE_DRAIN_JIFFIES (HZ / 50)
+#define KFREE_BULK_MAX_SIZE 64
+
+struct kfree_rcu_bulk_data {
+       int nr_records;
+       void *records[KFREE_BULK_MAX_SIZE];
+       struct kfree_rcu_bulk_data *next;
+};

 /*
  * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
@@ -2607,15 +2614,24 @@ struct kfree_rcu_cpu {
        struct rcu_work rcu_work;

        /* The list of objects being queued in a batch but are not yet
-        * scheduled to be freed.
+        * scheduled to be freed. For emergency path only.
         */
        struct rcu_head *head;

        /* The list of objects that have now left ->head and are queued for
-        * freeing after a grace period.
+        * freeing after a grace period. For emergency path only.
         */
        struct rcu_head *head_free;

+       /*
+        * It is a block list that keeps pointers in the array of specific
+        * size which are freed by the kfree_bulk() logic. Intends to improve
+        * drain throughput.
+        */
+       struct kfree_rcu_bulk_data *bhead;
+       struct kfree_rcu_bulk_data *bhead_free;
+       struct kfree_rcu_bulk_data *bcached;
+
        /* Protect concurrent access to this structure. */
        spinlock_t lock;
@@ -2637,23 +2653,39 @@ static void kfree_rcu_work(struct work_struct *work)
 {
        unsigned long flags;
        struct rcu_head *head, *next;
+       struct kfree_rcu_bulk_data *bhead, *bnext;
        struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
                                        struct kfree_rcu_cpu, rcu_work);
 
        spin_lock_irqsave(&krcp->lock, flags);
        head = krcp->head_free;
        krcp->head_free = NULL;
+       bhead = krcp->bhead_free;
+       krcp->bhead_free = NULL;
        spin_unlock_irqrestore(&krcp->lock, flags);
 
        /*
         * The head is detached and not referenced from anywhere, so lockless
         * access is Ok.
         */
+       for (; bhead; bhead = bnext) {
+               bnext = bhead->next;
+               kfree_bulk(bhead->nr_records, bhead->records);
+
+               if (cmpxchg(&krcp->bcached, NULL, bhead))
+                       kfree(bhead);
+
+               cond_resched_tasks_rcu_qs();
+       }
+
+       /*
+        * Emergency case only. It can happen under low
+        * memory condition when kmalloc gets failed, so
+        * the "bulk" path can not be temporary maintained.
+        */
        for (; head; head = next) {
                next = head->next;
-               /* Could be possible to optimize with kfree_bulk in future */
                __rcu_reclaim(rcu_state.name, head);
-               cond_resched_tasks_rcu_qs();
        }
 }

@@ -2671,11 +2703,15 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
         * another one, just refuse the optimization and it will be retried
         * again in KFREE_DRAIN_JIFFIES time.
         */
-       if (krcp->head_free)
+       if (krcp->bhead_free || krcp->head_free)
                return false;

        krcp->head_free = krcp->head;
        krcp->head = NULL;
+
+       krcp->bhead_free = krcp->bhead;
+       krcp->bhead = NULL;
+
        INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
        queue_rcu_work(system_wq, &krcp->rcu_work);

@@ -2747,6 +2783,7 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
 {
        unsigned long flags;
        struct kfree_rcu_cpu *krcp;
+       struct kfree_rcu_bulk_data *bnode;

        /* kfree_call_rcu() batching requires timers to be up. If the scheduler
         * is not yet up, just skip batching and do the non-batched version.
@@ -2754,16 +2791,35 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
        if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
                return kfree_call_rcu_nobatch(head, func);

-       head->func = func;
-
        local_irq_save(flags);  /* For safely calling this_cpu_ptr(). */
        krcp = this_cpu_ptr(&krc);
        spin_lock(&krcp->lock);

+       if (!krcp->bhead ||
+                       krcp->bhead->nr_records == KFREE_BULK_MAX_SIZE) {
+               /* Need a new block. */
+               if (!(bnode = xchg(&krcp->bcached, NULL)))
+                       bnode = kmalloc(sizeof(struct kfree_rcu_bulk_data),
+                               GFP_ATOMIC | __GFP_NOWARN);
+
+               /* If gets failed, maintain the list instead. */
+               if (unlikely(!bnode)) {
+                       head->func = func;
+                       head->next = krcp->head;
+                       krcp->head = head;
+                       goto check_and_schedule;
+               }
+
+               bnode->nr_records = 0;
+               bnode->next = krcp->bhead;
+               krcp->bhead = bnode;
+       }
+
        /* Queue the kfree but don't yet schedule the batch. */
-       head->next = krcp->head;
-       krcp->head = head;
+       krcp->bhead->records[krcp->bhead->nr_records++] =
+               (void *) head - (unsigned long) func;
 
+check_and_schedule:
        /* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
        if (!xchg(&krcp->monitor_todo, true))
                schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);

See below some test results with/without this patch:

# HiKey 960 8xCPUs
rcuperf.ko kfree_loops=200000 kfree_alloc_num=1000 kfree_rcu_test=1
[  159.017771] Total time taken by all kfree'ers: 92783584881 ns, loops: 200000, batches: 5117
[  126.862573] Total time taken by all kfree'ers: 70935580718 ns, loops: 200000, batches: 3953

Running the "rcuperf" shows approximately ~23% better throughput in case of using
"bulk" interface, so we have 92783584881 vs 70935580718 as total time. The "drain logic"
or its RCU callback does the work faster that leads to better throughput.

I can upload the RFC/PATCH of that change providing the test details and so on. 

Any thoughts about it?

Thank you in advance!

--
Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-12-10  9:53   ` Uladzislau Rezki
@ 2019-12-11 23:46     ` Paul E. McKenney
  2019-12-16 12:06       ` Uladzislau Rezki
  2019-12-12  5:27     ` Joel Fernandes
  1 sibling, 1 reply; 30+ messages in thread
From: Paul E. McKenney @ 2019-12-11 23:46 UTC (permalink / raw)
  To: Uladzislau Rezki
  Cc: Joel Fernandes (Google),
	linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Tue, Dec 10, 2019 at 10:53:48AM +0100, Uladzislau Rezki wrote:
> On Wed, Sep 18, 2019 at 11:58:11AM +0200, Uladzislau Rezki wrote:
> > > Recently a discussion about stability and performance of a system
> > > involving a high rate of kfree_rcu() calls surfaced on the list [1]
> > > which led to another discussion how to prepare for this situation.
> > > 
> > > This patch adds basic batching support for kfree_rcu(). It is "basic"
> > > because we do none of the slab management, dynamic allocation, code
> > > moving or any of the other things, some of which previous attempts did
> > > [2]. These fancier improvements can be follow-up patches and there are
> > > different ideas being discussed in those regards. This is an effort to
> > > start simple, and build up from there. In the future, an extension to
> > > use kfree_bulk and possibly per-slab batching could be done to further
> > > improve performance due to cache-locality and slab-specific bulk free
> > > optimizations. By using an array of pointers, the worker thread
> > > processing the work would need to read lesser data since it does not
> > > need to deal with large rcu_head(s) any longer.
> > > 
> According to https://lkml.org/lkml/2017/12/19/706 there was an attempt
> to make use of kfree_bulk() interface. I have done some tests based on
> your patch and enhanced kfree_bulk() logic. Basically storing pointers 
> in an array with a specific size makes sense to me and seems to others
> as well. I mean in comparison with "pointer chasing" way, when there is
> probably a cache misses each time the access is done to next element:

Something like this would be good!

The other thing to track besides CPU time savings (which does look good!)
is memory footprint.

And there will also need to be something visible to RCU counting the
number of outstanding kfree()s.  But on a per-CPU basis, for example,
as an atomic_long_t field in the rcu_data structure or similar.  This
is needed to help RCU work out when it needs to work harder to bring
grace periods to an end.  But that can be a separate issue.

> diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> index 1fe0418a5901..4f68662c1568 100644
> --- a/kernel/rcu/tree.c
> +++ b/kernel/rcu/tree.c
> @@ -2595,6 +2595,13 @@ EXPORT_SYMBOL_GPL(call_rcu);
> 
>  /* Maximum number of jiffies to wait before draining a batch. */
>  #define KFREE_DRAIN_JIFFIES (HZ / 50)
> +#define KFREE_BULK_MAX_SIZE 64

My guess is that performance does not depend all that much on the
exact number.  Does that match your testing?

> +
> +struct kfree_rcu_bulk_data {
> +       int nr_records;
> +       void *records[KFREE_BULK_MAX_SIZE];
> +       struct kfree_rcu_bulk_data *next;
> +};
> 
>  /*
>   * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> @@ -2607,15 +2614,24 @@ struct kfree_rcu_cpu {
>         struct rcu_work rcu_work;
> 
>         /* The list of objects being queued in a batch but are not yet
> -        * scheduled to be freed.
> +        * scheduled to be freed. For emergency path only.
>          */
>         struct rcu_head *head;
> 
>         /* The list of objects that have now left ->head and are queued for
> -        * freeing after a grace period.
> +        * freeing after a grace period. For emergency path only.
>          */
>         struct rcu_head *head_free;
> 
> +       /*
> +        * It is a block list that keeps pointers in the array of specific
> +        * size which are freed by the kfree_bulk() logic. Intends to improve
> +        * drain throughput.
> +        */
> +       struct kfree_rcu_bulk_data *bhead;
> +       struct kfree_rcu_bulk_data *bhead_free;
> +       struct kfree_rcu_bulk_data *bcached;

So ->bcached keeps at most one kfree_rcu_bulk_data around for later use,
correct?  And ->bhead is where new memory is placed, while ->bhead_free
contains those waiting for a grace period, right?  (It would be good
to make the comment explicit about this.)

> +
>         /* Protect concurrent access to this structure. */
>         spinlock_t lock;
> @@ -2637,23 +2653,39 @@ static void kfree_rcu_work(struct work_struct *work)
>  {
>         unsigned long flags;
>         struct rcu_head *head, *next;
> +       struct kfree_rcu_bulk_data *bhead, *bnext;
>         struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
>                                         struct kfree_rcu_cpu, rcu_work);
>  
>         spin_lock_irqsave(&krcp->lock, flags);
>         head = krcp->head_free;
>         krcp->head_free = NULL;
> +       bhead = krcp->bhead_free;
> +       krcp->bhead_free = NULL;
>         spin_unlock_irqrestore(&krcp->lock, flags);
>  
>         /*
>          * The head is detached and not referenced from anywhere, so lockless
>          * access is Ok.
>          */
> +       for (; bhead; bhead = bnext) {
> +               bnext = bhead->next;
> +               kfree_bulk(bhead->nr_records, bhead->records);
> +
> +               if (cmpxchg(&krcp->bcached, NULL, bhead))
> +                       kfree(bhead);
> +
> +               cond_resched_tasks_rcu_qs();
> +       }
> +
> +       /*
> +        * Emergency case only. It can happen under low
> +        * memory condition when kmalloc gets failed, so
> +        * the "bulk" path can not be temporary maintained.
> +        */
>         for (; head; head = next) {
>                 next = head->next;
> -               /* Could be possible to optimize with kfree_bulk in future */
>                 __rcu_reclaim(rcu_state.name, head);
> -               cond_resched_tasks_rcu_qs();
>         }
>  }
> 
> @@ -2671,11 +2703,15 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
>          * another one, just refuse the optimization and it will be retried
>          * again in KFREE_DRAIN_JIFFIES time.
>          */
> -       if (krcp->head_free)
> +       if (krcp->bhead_free || krcp->head_free)
>                 return false;
> 
>         krcp->head_free = krcp->head;
>         krcp->head = NULL;
> +
> +       krcp->bhead_free = krcp->bhead;
> +       krcp->bhead = NULL;
> +
>         INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
>         queue_rcu_work(system_wq, &krcp->rcu_work);
> 
> @@ -2747,6 +2783,7 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
>  {
>         unsigned long flags;
>         struct kfree_rcu_cpu *krcp;
> +       struct kfree_rcu_bulk_data *bnode;
> 
>         /* kfree_call_rcu() batching requires timers to be up. If the scheduler
>          * is not yet up, just skip batching and do the non-batched version.
> @@ -2754,16 +2791,35 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
>         if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
>                 return kfree_call_rcu_nobatch(head, func);
> 
> -       head->func = func;
> -
>         local_irq_save(flags);  /* For safely calling this_cpu_ptr(). */
>         krcp = this_cpu_ptr(&krc);
>         spin_lock(&krcp->lock);
> 
> +       if (!krcp->bhead ||
> +                       krcp->bhead->nr_records == KFREE_BULK_MAX_SIZE) {
> +               /* Need a new block. */
> +               if (!(bnode = xchg(&krcp->bcached, NULL)))
> +                       bnode = kmalloc(sizeof(struct kfree_rcu_bulk_data),
> +                               GFP_ATOMIC | __GFP_NOWARN);
> +
> +               /* If gets failed, maintain the list instead. */
> +               if (unlikely(!bnode)) {
> +                       head->func = func;
> +                       head->next = krcp->head;
> +                       krcp->head = head;
> +                       goto check_and_schedule;

It should be possible to move this code out to follow the "Queue the
next" comment, thus avoiding the goto.  Setting krcp->bhead to NULL
here should set up for the check below, right?

> +               }
> +
> +               bnode->nr_records = 0;
> +               bnode->next = krcp->bhead;
> +               krcp->bhead = bnode;
> +       }
> +
>         /* Queue the kfree but don't yet schedule the batch. */
> -       head->next = krcp->head;
> -       krcp->head = head;
> +       krcp->bhead->records[krcp->bhead->nr_records++] =
> +               (void *) head - (unsigned long) func;
>  
> +check_and_schedule:
>         /* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
>         if (!xchg(&krcp->monitor_todo, true))
>                 schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> 
> See below some test results with/without this patch:
> 
> # HiKey 960 8xCPUs
> rcuperf.ko kfree_loops=200000 kfree_alloc_num=1000 kfree_rcu_test=1
> [  159.017771] Total time taken by all kfree'ers: 92783584881 ns, loops: 200000, batches: 5117
> [  126.862573] Total time taken by all kfree'ers: 70935580718 ns, loops: 200000, batches: 3953
> 
> Running the "rcuperf" shows approximately ~23% better throughput in case of using
> "bulk" interface, so we have 92783584881 vs 70935580718 as total time. The "drain logic"
> or its RCU callback does the work faster that leads to better throughput.
> 
> I can upload the RFC/PATCH of that change providing the test details and so on. 
> 
> Any thoughts about it?

Again nice improvement!  Please also check memory footprint.  I would
not expect much difference, but...

							Thanx, Paul

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-12-10  9:53   ` Uladzislau Rezki
  2019-12-11 23:46     ` Paul E. McKenney
@ 2019-12-12  5:27     ` Joel Fernandes
  2019-12-16 12:46       ` Uladzislau Rezki
  1 sibling, 1 reply; 30+ messages in thread
From: Joel Fernandes @ 2019-12-12  5:27 UTC (permalink / raw)
  To: Uladzislau Rezki
  Cc: Paul E. McKenney, linux-kernel, kernel-team, kernel-team,
	Byungchul Park, Davidlohr Bueso, Josh Triplett, Lai Jiangshan,
	Mathieu Desnoyers, max.byungchul.park, Rao Shoaib, rcu,
	Steven Rostedt

On Tue, Dec 10, 2019 at 10:53:48AM +0100, Uladzislau Rezki wrote:
> On Wed, Sep 18, 2019 at 11:58:11AM +0200, Uladzislau Rezki wrote:
> > > Recently a discussion about stability and performance of a system
> > > involving a high rate of kfree_rcu() calls surfaced on the list [1]
> > > which led to another discussion how to prepare for this situation.
> > > 
> > > This patch adds basic batching support for kfree_rcu(). It is "basic"
> > > because we do none of the slab management, dynamic allocation, code
> > > moving or any of the other things, some of which previous attempts did
> > > [2]. These fancier improvements can be follow-up patches and there are
> > > different ideas being discussed in those regards. This is an effort to
> > > start simple, and build up from there. In the future, an extension to
> > > use kfree_bulk and possibly per-slab batching could be done to further
> > > improve performance due to cache-locality and slab-specific bulk free
> > > optimizations. By using an array of pointers, the worker thread
> > > processing the work would need to read lesser data since it does not
> > > need to deal with large rcu_head(s) any longer.
> > > 
> According to https://lkml.org/lkml/2017/12/19/706 there was an attempt
> to make use of kfree_bulk() interface. I have done some tests based on
> your patch and enhanced kfree_bulk() logic. Basically storing pointers 
> in an array with a specific size makes sense to me and seems to others
> as well. I mean in comparison with "pointer chasing" way, when there is
> probably a cache misses each time the access is done to next element:

This looks like a great idea to me, that is chaining blocks together.

> 
> diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> index 1fe0418a5901..4f68662c1568 100644
> --- a/kernel/rcu/tree.c
> +++ b/kernel/rcu/tree.c
> @@ -2595,6 +2595,13 @@ EXPORT_SYMBOL_GPL(call_rcu);
> 
>  /* Maximum number of jiffies to wait before draining a batch. */
>  #define KFREE_DRAIN_JIFFIES (HZ / 50)
> +#define KFREE_BULK_MAX_SIZE 64
> +
> +struct kfree_rcu_bulk_data {
> +       int nr_records;
> +       void *records[KFREE_BULK_MAX_SIZE];
> +       struct kfree_rcu_bulk_data *next;
> +};
> 
>  /*
>   * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> @@ -2607,15 +2614,24 @@ struct kfree_rcu_cpu {
>         struct rcu_work rcu_work;
> 
>         /* The list of objects being queued in a batch but are not yet
> -        * scheduled to be freed.
> +        * scheduled to be freed. For emergency path only.
>          */
>         struct rcu_head *head;
> 
>         /* The list of objects that have now left ->head and are queued for
> -        * freeing after a grace period.
> +        * freeing after a grace period. For emergency path only.
>          */
>         struct rcu_head *head_free;
> 
> +       /*
> +        * It is a block list that keeps pointers in the array of specific
> +        * size which are freed by the kfree_bulk() logic. Intends to improve
> +        * drain throughput.
> +        */
> +       struct kfree_rcu_bulk_data *bhead;
> +       struct kfree_rcu_bulk_data *bhead_free;
> +       struct kfree_rcu_bulk_data *bcached;
> +
>         /* Protect concurrent access to this structure. */
>         spinlock_t lock;
> @@ -2637,23 +2653,39 @@ static void kfree_rcu_work(struct work_struct *work)
>  {
>         unsigned long flags;
>         struct rcu_head *head, *next;
> +       struct kfree_rcu_bulk_data *bhead, *bnext;
>         struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
>                                         struct kfree_rcu_cpu, rcu_work);
>  
>         spin_lock_irqsave(&krcp->lock, flags);
>         head = krcp->head_free;
>         krcp->head_free = NULL;
> +       bhead = krcp->bhead_free;
> +       krcp->bhead_free = NULL;
>         spin_unlock_irqrestore(&krcp->lock, flags);
>  
>         /*
>          * The head is detached and not referenced from anywhere, so lockless
>          * access is Ok.
>          */
> +       for (; bhead; bhead = bnext) {
> +               bnext = bhead->next;
> +               kfree_bulk(bhead->nr_records, bhead->records);
> +
> +               if (cmpxchg(&krcp->bcached, NULL, bhead))
> +                       kfree(bhead);

After the first iteration of loop, say cmpxchg succeeded, then is there a
point it doing repeated cmpxchg in future loops? AIUI, cmpxchg has a
serializing cost and better be avoided where possible.

But... there can be a case where bcached was used up while the loop is
running. Then there could be a point in reassigning it in future loop
iterations.

> +
> +               cond_resched_tasks_rcu_qs();
> +       }
> +
> +       /*
> +        * Emergency case only. It can happen under low
> +        * memory condition when kmalloc gets failed, so
> +        * the "bulk" path can not be temporary maintained.
> +        */
>         for (; head; head = next) {
>                 next = head->next;
> -               /* Could be possible to optimize with kfree_bulk in future */

I'm glad I had left this comment ;-) ;-)

>                 __rcu_reclaim(rcu_state.name, head);
> -               cond_resched_tasks_rcu_qs();

Why take off this cond_resched..() ?

>         }
>  }
> 
> @@ -2671,11 +2703,15 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
>          * another one, just refuse the optimization and it will be retried
>          * again in KFREE_DRAIN_JIFFIES time.
>          */
> -       if (krcp->head_free)
> +       if (krcp->bhead_free || krcp->head_free)
>                 return false;
> 
>         krcp->head_free = krcp->head;
>         krcp->head = NULL;
> +
> +       krcp->bhead_free = krcp->bhead;
> +       krcp->bhead = NULL;
> +
>         INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
>         queue_rcu_work(system_wq, &krcp->rcu_work);
> 
> @@ -2747,6 +2783,7 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
>  {
>         unsigned long flags;
>         struct kfree_rcu_cpu *krcp;
> +       struct kfree_rcu_bulk_data *bnode;
> 
>         /* kfree_call_rcu() batching requires timers to be up. If the scheduler
>          * is not yet up, just skip batching and do the non-batched version.
> @@ -2754,16 +2791,35 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
>         if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
>                 return kfree_call_rcu_nobatch(head, func);
> 
> -       head->func = func;
> -
>         local_irq_save(flags);  /* For safely calling this_cpu_ptr(). */
>         krcp = this_cpu_ptr(&krc);
>         spin_lock(&krcp->lock);
> 
> +       if (!krcp->bhead ||
> +                       krcp->bhead->nr_records == KFREE_BULK_MAX_SIZE) {
> +               /* Need a new block. */
> +               if (!(bnode = xchg(&krcp->bcached, NULL)))

Is it better to cache more than 1 block? But this is also Ok I think.

> +                       bnode = kmalloc(sizeof(struct kfree_rcu_bulk_data),
> +                               GFP_ATOMIC | __GFP_NOWARN);
> +
> +               /* If gets failed, maintain the list instead. */
> +               if (unlikely(!bnode)) {
> +                       head->func = func;
> +                       head->next = krcp->head;
> +                       krcp->head = head;
> +                       goto check_and_schedule;
> +               }
> +
> +               bnode->nr_records = 0;
> +               bnode->next = krcp->bhead;
> +               krcp->bhead = bnode;
> +       }
> +
>         /* Queue the kfree but don't yet schedule the batch. */
> -       head->next = krcp->head;
> -       krcp->head = head;
> +       krcp->bhead->records[krcp->bhead->nr_records++] =
> +               (void *) head - (unsigned long) func;
>  
> +check_and_schedule:

>         /* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
>         if (!xchg(&krcp->monitor_todo, true))
>                 schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> 
> See below some test results with/without this patch:
> 
> # HiKey 960 8xCPUs
> rcuperf.ko kfree_loops=200000 kfree_alloc_num=1000 kfree_rcu_test=1
> [  159.017771] Total time taken by all kfree'ers: 92783584881 ns, loops: 200000, batches: 5117
> [  126.862573] Total time taken by all kfree'ers: 70935580718 ns, loops: 200000, batches: 3953
> 
> Running the "rcuperf" shows approximately ~23% better throughput in case of using

Awesome. Wow. Is this +23% with a slab allocator configuration?

You mentioned you will post a new version. Once you do it, I can take
another look and run some tests. Then I'll give your patch the Reviewed-by tag.

Thanks Uladzislau and Paul!

 - Joel

> or its RCU callback does the work faster that leads to better throughput.
> 
> I can upload the RFC/PATCH of that change providing the test details and so on. 
> 
> Any thoughts about it?
> 
> Thank you in advance!
> 
> --
> Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-12-11 23:46     ` Paul E. McKenney
@ 2019-12-16 12:06       ` Uladzislau Rezki
  0 siblings, 0 replies; 30+ messages in thread
From: Uladzislau Rezki @ 2019-12-16 12:06 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: Uladzislau Rezki, Joel Fernandes (Google),
	linux-kernel, kernel-team, kernel-team, Byungchul Park,
	Davidlohr Bueso, Josh Triplett, Lai Jiangshan, Mathieu Desnoyers,
	max.byungchul.park, Rao Shoaib, rcu, Steven Rostedt

On Wed, Dec 11, 2019 at 03:46:48PM -0800, Paul E. McKenney wrote:
> On Tue, Dec 10, 2019 at 10:53:48AM +0100, Uladzislau Rezki wrote:
> > On Wed, Sep 18, 2019 at 11:58:11AM +0200, Uladzislau Rezki wrote:
> > > > Recently a discussion about stability and performance of a system
> > > > involving a high rate of kfree_rcu() calls surfaced on the list [1]
> > > > which led to another discussion how to prepare for this situation.
> > > > 
> > > > This patch adds basic batching support for kfree_rcu(). It is "basic"
> > > > because we do none of the slab management, dynamic allocation, code
> > > > moving or any of the other things, some of which previous attempts did
> > > > [2]. These fancier improvements can be follow-up patches and there are
> > > > different ideas being discussed in those regards. This is an effort to
> > > > start simple, and build up from there. In the future, an extension to
> > > > use kfree_bulk and possibly per-slab batching could be done to further
> > > > improve performance due to cache-locality and slab-specific bulk free
> > > > optimizations. By using an array of pointers, the worker thread
> > > > processing the work would need to read lesser data since it does not
> > > > need to deal with large rcu_head(s) any longer.
> > > > 
> > According to https://lkml.org/lkml/2017/12/19/706 there was an attempt
> > to make use of kfree_bulk() interface. I have done some tests based on
> > your patch and enhanced kfree_bulk() logic. Basically storing pointers 
> > in an array with a specific size makes sense to me and seems to others
> > as well. I mean in comparison with "pointer chasing" way, when there is
> > probably a cache misses each time the access is done to next element:
> 
> Something like this would be good!
> 
> The other thing to track besides CPU time savings (which does look good!)
> is memory footprint.
>
I will double check how much extra memory it requires, but it depends on
how many elements we have in "bulk list" and the size of the
kfree_rcu_bulk_data structure. So, i will run "rcuperf" to see what we
have.

> 
> And there will also need to be something visible to RCU counting the
> number of outstanding kfree()s.  But on a per-CPU basis, for example,
> as an atomic_long_t field in the rcu_data structure or similar.  This
> is needed to help RCU work out when it needs to work harder to bring
> grace periods to an end.  But that can be a separate issue.
> 
OK, i see that. As far as i see we need to have per-cpu implementation
first, i mean kfree_rcu() should be per-cpu.

> > diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> > index 1fe0418a5901..4f68662c1568 100644
> > --- a/kernel/rcu/tree.c
> > +++ b/kernel/rcu/tree.c
> > @@ -2595,6 +2595,13 @@ EXPORT_SYMBOL_GPL(call_rcu);
> > 
> >  /* Maximum number of jiffies to wait before draining a batch. */
> >  #define KFREE_DRAIN_JIFFIES (HZ / 50)
> > +#define KFREE_BULK_MAX_SIZE 64
> 
> My guess is that performance does not depend all that much on the
> exact number.  Does that match your testing?
> 
Not really. It does not depend on exact number, whereas it is clear
that setting it to 1 does not make sense :) Also the size of the
kfree_rcu_bulk_data struct should not be more then PAGE_SIZE due to
memory fragmentation problems.

> > +
> > +struct kfree_rcu_bulk_data {
> > +       int nr_records;
> > +       void *records[KFREE_BULK_MAX_SIZE];
> > +       struct kfree_rcu_bulk_data *next;
> > +};
> > 
> >  /*
> >   * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> > @@ -2607,15 +2614,24 @@ struct kfree_rcu_cpu {
> >         struct rcu_work rcu_work;
> > 
> >         /* The list of objects being queued in a batch but are not yet
> > -        * scheduled to be freed.
> > +        * scheduled to be freed. For emergency path only.
> >          */
> >         struct rcu_head *head;
> > 
> >         /* The list of objects that have now left ->head and are queued for
> > -        * freeing after a grace period.
> > +        * freeing after a grace period. For emergency path only.
> >          */
> >         struct rcu_head *head_free;
> > 
> > +       /*
> > +        * It is a block list that keeps pointers in the array of specific
> > +        * size which are freed by the kfree_bulk() logic. Intends to improve
> > +        * drain throughput.
> > +        */
> > +       struct kfree_rcu_bulk_data *bhead;
> > +       struct kfree_rcu_bulk_data *bhead_free;
> > +       struct kfree_rcu_bulk_data *bcached;
> 
> So ->bcached keeps at most one kfree_rcu_bulk_data around for later use,
> correct?  And ->bhead is where new memory is placed, while ->bhead_free
> contains those waiting for a grace period, right?  (It would be good
> to make the comment explicit about this.)
> 
Correct. I will add some extra comments.

> > +
> >         /* Protect concurrent access to this structure. */
> >         spinlock_t lock;
> > @@ -2637,23 +2653,39 @@ static void kfree_rcu_work(struct work_struct *work)
> >  {
> >         unsigned long flags;
> >         struct rcu_head *head, *next;
> > +       struct kfree_rcu_bulk_data *bhead, *bnext;
> >         struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
> >                                         struct kfree_rcu_cpu, rcu_work);
> >  
> >         spin_lock_irqsave(&krcp->lock, flags);
> >         head = krcp->head_free;
> >         krcp->head_free = NULL;
> > +       bhead = krcp->bhead_free;
> > +       krcp->bhead_free = NULL;
> >         spin_unlock_irqrestore(&krcp->lock, flags);
> >  
> >         /*
> >          * The head is detached and not referenced from anywhere, so lockless
> >          * access is Ok.
> >          */
> > +       for (; bhead; bhead = bnext) {
> > +               bnext = bhead->next;
> > +               kfree_bulk(bhead->nr_records, bhead->records);
> > +
> > +               if (cmpxchg(&krcp->bcached, NULL, bhead))
> > +                       kfree(bhead);
> > +
> > +               cond_resched_tasks_rcu_qs();
> > +       }
> > +
> > +       /*
> > +        * Emergency case only. It can happen under low
> > +        * memory condition when kmalloc gets failed, so
> > +        * the "bulk" path can not be temporary maintained.
> > +        */
> >         for (; head; head = next) {
> >                 next = head->next;
> > -               /* Could be possible to optimize with kfree_bulk in future */
> >                 __rcu_reclaim(rcu_state.name, head);
> > -               cond_resched_tasks_rcu_qs();
> >         }
> >  }
> > 
> > @@ -2671,11 +2703,15 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
> >          * another one, just refuse the optimization and it will be retried
> >          * again in KFREE_DRAIN_JIFFIES time.
> >          */
> > -       if (krcp->head_free)
> > +       if (krcp->bhead_free || krcp->head_free)
> >                 return false;
> > 
> >         krcp->head_free = krcp->head;
> >         krcp->head = NULL;
> > +
> > +       krcp->bhead_free = krcp->bhead;
> > +       krcp->bhead = NULL;
> > +
> >         INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
> >         queue_rcu_work(system_wq, &krcp->rcu_work);
> > 
> > @@ -2747,6 +2783,7 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> >  {
> >         unsigned long flags;
> >         struct kfree_rcu_cpu *krcp;
> > +       struct kfree_rcu_bulk_data *bnode;
> > 
> >         /* kfree_call_rcu() batching requires timers to be up. If the scheduler
> >          * is not yet up, just skip batching and do the non-batched version.
> > @@ -2754,16 +2791,35 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> >         if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
> >                 return kfree_call_rcu_nobatch(head, func);
> > 
> > -       head->func = func;
> > -
> >         local_irq_save(flags);  /* For safely calling this_cpu_ptr(). */
> >         krcp = this_cpu_ptr(&krc);
> >         spin_lock(&krcp->lock);
> > 
> > +       if (!krcp->bhead ||
> > +                       krcp->bhead->nr_records == KFREE_BULK_MAX_SIZE) {
> > +               /* Need a new block. */
> > +               if (!(bnode = xchg(&krcp->bcached, NULL)))
> > +                       bnode = kmalloc(sizeof(struct kfree_rcu_bulk_data),
> > +                               GFP_ATOMIC | __GFP_NOWARN);
> > +
> > +               /* If gets failed, maintain the list instead. */
> > +               if (unlikely(!bnode)) {
> > +                       head->func = func;
> > +                       head->next = krcp->head;
> > +                       krcp->head = head;
> > +                       goto check_and_schedule;
> 
> It should be possible to move this code out to follow the "Queue the
> next" comment, thus avoiding the goto.  Setting krcp->bhead to NULL
> here should set up for the check below, right?
> 
Yes it should be possible. If we set krcp->bhead to NULL in case of
failure we can loose previous queued "bulk elements". But i see your
point and will rework it, trying to get rid of "goto" jump.

> > +               }
> > +
> > +               bnode->nr_records = 0;
> > +               bnode->next = krcp->bhead;
> > +               krcp->bhead = bnode;
> > +       }
> > +
> >         /* Queue the kfree but don't yet schedule the batch. */
> > -       head->next = krcp->head;
> > -       krcp->head = head;
> > +       krcp->bhead->records[krcp->bhead->nr_records++] =
> > +               (void *) head - (unsigned long) func;
> >  
> > +check_and_schedule:
> >         /* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
> >         if (!xchg(&krcp->monitor_todo, true))
> >                 schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> > 
> > See below some test results with/without this patch:
> > 
> > # HiKey 960 8xCPUs
> > rcuperf.ko kfree_loops=200000 kfree_alloc_num=1000 kfree_rcu_test=1
> > [  159.017771] Total time taken by all kfree'ers: 92783584881 ns, loops: 200000, batches: 5117
> > [  126.862573] Total time taken by all kfree'ers: 70935580718 ns, loops: 200000, batches: 3953
> > 
> > Running the "rcuperf" shows approximately ~23% better throughput in case of using
> > "bulk" interface, so we have 92783584881 vs 70935580718 as total time. The "drain logic"
> > or its RCU callback does the work faster that leads to better throughput.
> > 
> > I can upload the RFC/PATCH of that change providing the test details and so on. 
> > 
> > Any thoughts about it?
> 
> Again nice improvement!  Please also check memory footprint.  I would
> not expect much difference, but...
> 
Thank you Paul for your comments! Will check it.

--
Vlad Rezki

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

* Re: [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching
  2019-12-12  5:27     ` Joel Fernandes
@ 2019-12-16 12:46       ` Uladzislau Rezki
  0 siblings, 0 replies; 30+ messages in thread
From: Uladzislau Rezki @ 2019-12-16 12:46 UTC (permalink / raw)
  To: Joel Fernandes
  Cc: Uladzislau Rezki, Paul E. McKenney, linux-kernel, kernel-team,
	kernel-team, Byungchul Park, Davidlohr Bueso, Josh Triplett,
	Lai Jiangshan, Mathieu Desnoyers, max.byungchul.park, Rao Shoaib,
	rcu, Steven Rostedt

On Thu, Dec 12, 2019 at 12:27:27AM -0500, Joel Fernandes wrote:
> On Tue, Dec 10, 2019 at 10:53:48AM +0100, Uladzislau Rezki wrote:
> > On Wed, Sep 18, 2019 at 11:58:11AM +0200, Uladzislau Rezki wrote:
> > > > Recently a discussion about stability and performance of a system
> > > > involving a high rate of kfree_rcu() calls surfaced on the list [1]
> > > > which led to another discussion how to prepare for this situation.
> > > > 
> > > > This patch adds basic batching support for kfree_rcu(). It is "basic"
> > > > because we do none of the slab management, dynamic allocation, code
> > > > moving or any of the other things, some of which previous attempts did
> > > > [2]. These fancier improvements can be follow-up patches and there are
> > > > different ideas being discussed in those regards. This is an effort to
> > > > start simple, and build up from there. In the future, an extension to
> > > > use kfree_bulk and possibly per-slab batching could be done to further
> > > > improve performance due to cache-locality and slab-specific bulk free
> > > > optimizations. By using an array of pointers, the worker thread
> > > > processing the work would need to read lesser data since it does not
> > > > need to deal with large rcu_head(s) any longer.
> > > > 
> > According to https://lkml.org/lkml/2017/12/19/706 there was an attempt
> > to make use of kfree_bulk() interface. I have done some tests based on
> > your patch and enhanced kfree_bulk() logic. Basically storing pointers 
> > in an array with a specific size makes sense to me and seems to others
> > as well. I mean in comparison with "pointer chasing" way, when there is
> > probably a cache misses each time the access is done to next element:
> 
> This looks like a great idea to me, that is chaining blocks together.
> 
> > 
> > diff --git a/kernel/rcu/tree.c b/kernel/rcu/tree.c
> > index 1fe0418a5901..4f68662c1568 100644
> > --- a/kernel/rcu/tree.c
> > +++ b/kernel/rcu/tree.c
> > @@ -2595,6 +2595,13 @@ EXPORT_SYMBOL_GPL(call_rcu);
> > 
> >  /* Maximum number of jiffies to wait before draining a batch. */
> >  #define KFREE_DRAIN_JIFFIES (HZ / 50)
> > +#define KFREE_BULK_MAX_SIZE 64
> > +
> > +struct kfree_rcu_bulk_data {
> > +       int nr_records;
> > +       void *records[KFREE_BULK_MAX_SIZE];
> > +       struct kfree_rcu_bulk_data *next;
> > +};
> > 
> >  /*
> >   * Maximum number of kfree(s) to batch, if this limit is hit then the batch of
> > @@ -2607,15 +2614,24 @@ struct kfree_rcu_cpu {
> >         struct rcu_work rcu_work;
> > 
> >         /* The list of objects being queued in a batch but are not yet
> > -        * scheduled to be freed.
> > +        * scheduled to be freed. For emergency path only.
> >          */
> >         struct rcu_head *head;
> > 
> >         /* The list of objects that have now left ->head and are queued for
> > -        * freeing after a grace period.
> > +        * freeing after a grace period. For emergency path only.
> >          */
> >         struct rcu_head *head_free;
> > 
> > +       /*
> > +        * It is a block list that keeps pointers in the array of specific
> > +        * size which are freed by the kfree_bulk() logic. Intends to improve
> > +        * drain throughput.
> > +        */
> > +       struct kfree_rcu_bulk_data *bhead;
> > +       struct kfree_rcu_bulk_data *bhead_free;
> > +       struct kfree_rcu_bulk_data *bcached;
> > +
> >         /* Protect concurrent access to this structure. */
> >         spinlock_t lock;
> > @@ -2637,23 +2653,39 @@ static void kfree_rcu_work(struct work_struct *work)
> >  {
> >         unsigned long flags;
> >         struct rcu_head *head, *next;
> > +       struct kfree_rcu_bulk_data *bhead, *bnext;
> >         struct kfree_rcu_cpu *krcp = container_of(to_rcu_work(work),
> >                                         struct kfree_rcu_cpu, rcu_work);
> >  
> >         spin_lock_irqsave(&krcp->lock, flags);
> >         head = krcp->head_free;
> >         krcp->head_free = NULL;
> > +       bhead = krcp->bhead_free;
> > +       krcp->bhead_free = NULL;
> >         spin_unlock_irqrestore(&krcp->lock, flags);
> >  
> >         /*
> >          * The head is detached and not referenced from anywhere, so lockless
> >          * access is Ok.
> >          */
> > +       for (; bhead; bhead = bnext) {
> > +               bnext = bhead->next;
> > +               kfree_bulk(bhead->nr_records, bhead->records);
> > +
> > +               if (cmpxchg(&krcp->bcached, NULL, bhead))
> > +                       kfree(bhead);
> 
> After the first iteration of loop, say cmpxchg succeeded, then is there a
> point it doing repeated cmpxchg in future loops? AIUI, cmpxchg has a
> serializing cost and better be avoided where possible.
> 
I see your point. I also do not like calling cmpxchg() in the loop. From
the other hand the number of loops depends on how many "chain blocks" we
have in our list. To reduce that number we can increase KFREE_BULK_MAX_SIZE,
i.e. the list will become shorter.

>
> But... there can be a case where bcached was used up while the loop is
> running. Then there could be a point in reassigning it in future loop
> iterations.
> 
Sound like that, therefore i keep it in the loop.

> > +
> > +               cond_resched_tasks_rcu_qs();
> > +       }
> > +
> > +       /*
> > +        * Emergency case only. It can happen under low
> > +        * memory condition when kmalloc gets failed, so
> > +        * the "bulk" path can not be temporary maintained.
> > +        */
> >         for (; head; head = next) {
> >                 next = head->next;
> > -               /* Could be possible to optimize with kfree_bulk in future */
> 
> I'm glad I had left this comment ;-) ;-)
> 
It is always good to leave valuable comments :)


> >                 __rcu_reclaim(rcu_state.name, head);
> > -               cond_resched_tasks_rcu_qs();
> 
> Why take off this cond_resched..() ?
> 
I can keep it. I removed it because it becomes as "emergency path"
only that is most likely never triggered.

> >         }
> >  }
> > 
> > @@ -2671,11 +2703,15 @@ static inline bool queue_kfree_rcu_work(struct kfree_rcu_cpu *krcp)
> >          * another one, just refuse the optimization and it will be retried
> >          * again in KFREE_DRAIN_JIFFIES time.
> >          */
> > -       if (krcp->head_free)
> > +       if (krcp->bhead_free || krcp->head_free)
> >                 return false;
> > 
> >         krcp->head_free = krcp->head;
> >         krcp->head = NULL;
> > +
> > +       krcp->bhead_free = krcp->bhead;
> > +       krcp->bhead = NULL;
> > +
> >         INIT_RCU_WORK(&krcp->rcu_work, kfree_rcu_work);
> >         queue_rcu_work(system_wq, &krcp->rcu_work);
> > 
> > @@ -2747,6 +2783,7 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> >  {
> >         unsigned long flags;
> >         struct kfree_rcu_cpu *krcp;
> > +       struct kfree_rcu_bulk_data *bnode;
> > 
> >         /* kfree_call_rcu() batching requires timers to be up. If the scheduler
> >          * is not yet up, just skip batching and do the non-batched version.
> > @@ -2754,16 +2791,35 @@ void kfree_call_rcu(struct rcu_head *head, rcu_callback_t func)
> >         if (rcu_scheduler_active != RCU_SCHEDULER_RUNNING)
> >                 return kfree_call_rcu_nobatch(head, func);
> > 
> > -       head->func = func;
> > -
> >         local_irq_save(flags);  /* For safely calling this_cpu_ptr(). */
> >         krcp = this_cpu_ptr(&krc);
> >         spin_lock(&krcp->lock);
> > 
> > +       if (!krcp->bhead ||
> > +                       krcp->bhead->nr_records == KFREE_BULK_MAX_SIZE) {
> > +               /* Need a new block. */
> > +               if (!(bnode = xchg(&krcp->bcached, NULL)))
> 
> Is it better to cache more than 1 block? But this is also Ok I think.
> 
I was thinking about it. In that case we need to answer the question
how many blocks are worth to cache. As of now we can go with one if
nobody minds.

> > +                       bnode = kmalloc(sizeof(struct kfree_rcu_bulk_data),
> > +                               GFP_ATOMIC | __GFP_NOWARN);
> > +
> > +               /* If gets failed, maintain the list instead. */
> > +               if (unlikely(!bnode)) {
> > +                       head->func = func;
> > +                       head->next = krcp->head;
> > +                       krcp->head = head;
> > +                       goto check_and_schedule;
> > +               }
> > +
> > +               bnode->nr_records = 0;
> > +               bnode->next = krcp->bhead;
> > +               krcp->bhead = bnode;
> > +       }
> > +
> >         /* Queue the kfree but don't yet schedule the batch. */
> > -       head->next = krcp->head;
> > -       krcp->head = head;
> > +       krcp->bhead->records[krcp->bhead->nr_records++] =
> > +               (void *) head - (unsigned long) func;
> >  
> > +check_and_schedule:
> 
> >         /* Schedule monitor for timely drain after KFREE_DRAIN_JIFFIES. */
> >         if (!xchg(&krcp->monitor_todo, true))
> >                 schedule_delayed_work(&krcp->monitor_work, KFREE_DRAIN_JIFFIES);
> > 
> > See below some test results with/without this patch:
> > 
> > # HiKey 960 8xCPUs
> > rcuperf.ko kfree_loops=200000 kfree_alloc_num=1000 kfree_rcu_test=1
> > [  159.017771] Total time taken by all kfree'ers: 92783584881 ns, loops: 200000, batches: 5117
> > [  126.862573] Total time taken by all kfree'ers: 70935580718 ns, loops: 200000, batches: 3953
> > 
> > Running the "rcuperf" shows approximately ~23% better throughput in case of using
> 
> Awesome. Wow. Is this +23% with a slab allocator configuration?
> 
I have checked CONFIG_SLAB but on the virtual machine running under KVM
the difference was ~15%. As for HiKey board and provided results above,
i have just checked, it uses CONFIG_SLUB allocator. I can also check
CONFIG_SLAB on HiKey 960 8xCPUs board.

>
> You mentioned you will post a new version. Once you do it, I can take
> another look and run some tests. Then I'll give your patch the Reviewed-by tag.
> 
OK.

Thank you Joel for your comments!

--
Vlad Rezki

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

end of thread, other threads:[~2019-12-16 12:46 UTC | newest]

Thread overview: 30+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-08-14 16:04 [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Joel Fernandes (Google)
2019-08-14 16:04 ` [PATCH v4 2/2] rcuperf: Add kfree_rcu() performance Tests Joel Fernandes (Google)
2019-08-14 22:58   ` Paul E. McKenney
2019-08-19 19:33     ` Joel Fernandes
2019-08-19 22:23       ` Paul E. McKenney
2019-08-19 23:51         ` Joel Fernandes
2019-08-20  2:50           ` Paul E. McKenney
2019-08-21  0:27             ` Joel Fernandes
2019-08-21  0:31               ` Joel Fernandes
2019-08-21  0:44                 ` Paul E. McKenney
2019-08-21  0:51                   ` Joel Fernandes
2019-08-16 16:43 ` [PATCH v4 1/2] rcu/tree: Add basic support for kfree_rcu() batching Paul E. McKenney
2019-08-16 17:44   ` Joel Fernandes
2019-08-16 19:16     ` Paul E. McKenney
2019-08-17  1:32       ` Joel Fernandes
2019-08-17  3:56         ` Paul E. McKenney
2019-08-17  4:30           ` Joel Fernandes
2019-08-17  5:20             ` Paul E. McKenney
2019-08-17  5:53               ` Joel Fernandes
2019-08-17 21:45                 ` Paul E. McKenney
2019-09-18  9:58 ` Uladzislau Rezki
2019-09-30 20:16   ` Joel Fernandes
2019-10-01 11:27     ` Uladzislau Rezki
2019-10-04 17:20       ` Joel Fernandes
2019-10-08 16:23         ` Uladzislau Rezki
2019-12-10  9:53   ` Uladzislau Rezki
2019-12-11 23:46     ` Paul E. McKenney
2019-12-16 12:06       ` Uladzislau Rezki
2019-12-12  5:27     ` Joel Fernandes
2019-12-16 12:46       ` Uladzislau Rezki

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).