linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations
@ 2024-08-13  4:29 Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 01/13] uprobes: revamp uprobe refcounting and lifetime management Andrii Nakryiko
                   ` (13 more replies)
  0 siblings, 14 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

( In addition to previously posted first 8 patches, I'm sending 5 more as an
RFC for people to get the general gist of where this work heading and what
uprobe performance is now achievable. I think first 8 patches are ready to be
applied and I'd appreciate early feedback on the remaining 5 ones.

Patches #9 and #10 implement SRCU-based refcounting avoidance for
return_instance and single-stepped uprobes, and lockless SRCU expiration by
timeout. Please look at hprobe_*() helpers for most of the details.

Last three patches are implementing lockless VMA -> uprobe resolution logic by
relying on a) SLAB_TYPESAFE_BY_RCU protection for files_cachep and b) on
Suren's mm_lock_seq changes. Note benchmarking numbers in the last patch.

The remainder of cover letter is written as applied to only first 8 patches. )

This patch set is heavily inspired by Peter Zijlstra's uprobe optimization
patches ([0]) and continue that work, albeit trying to keep complexity to the
minimum, and attepting to reuse existing primitives as much as possible. The
goal here is to optimize obvious uprobe triggering hot path, while keeping the
rest of locking mostly intact.

I've added uprobe_unregister_sync() into the error handling code path inside
uprobe_unregister(). This is due to recent refactorings from Oleg Nesterov
([1]), which necessitates this addition.

Except for refcounting change patch (which I stongly believe is a good
improvement we should do and forget about quasi-refcounting schema of
uprobe->consumers list), the rest of the changes are similar to Peter's
initial changes in [0].

Main differences would be:
  - no special RCU protection for mmap and fork handling, we just stick to
    refcounts there, as those are infrequent and not performance-sensitive
    code, while being complex and thus benefiting from proper locking;
  - the above means we don't need to do any custom SRCU additions to handle
    forking code path;
  - I handled UPROBE_HANDLER_REMOVE problem in handler_chain() differently,
    again, leveraging existing locking scheam;
  - I kept refcount usage for uretprobe and single-stepping uprobes, I plan to
    address that in a separate follow up patches. The plan is to avoid
    task_work, but I need to sit down and write and test the code.
  - finally, I dutifully was using SRCU throughout all the changes, and only
    last patch switches SRCU to RCU Tasks Trace and demonstrates significant
    performance and scalability gains from this.

The changes in this patch set were tested using BPF selftests and using
uprobe-stress ([2]) tool. One recent BPF selftest (uprobe_multi/consumers),
only recently added by Jiri Olsa will need a single-line adjustment to the
counting logic, but the patch itself is in bpf-next/master, so we'll have to
address that once linux-trace or tip and bpf-next trees merge. I'll take care
of that when this happens.

Now, for the benchmarking results. I've used the following script (which
utilizes BPF selftests-based bench tool). The CPU used was 80-core Intel Xeon
Gold 6138 CPU @ 2.00GHz running kernel with production-like config. I minimized
background noise by stopping any service I could identify and stop, so results
are pretty stable and variability is pretty small, overall.

Benchmark script:

#!/bin/bash

set -eufo pipefail

for i in uprobe-nop uretprobe-nop; do
    for p in 1 2 4 8 16 32 64; do
        summary=$(sudo ./bench -w3 -d5 -p$p -a trig-$i | tail -n1)
        total=$(echo "$summary" | cut -d'(' -f1 | cut -d' ' -f3-)
        percpu=$(echo "$summary" | cut -d'(' -f2 | cut -d')' -f1 | cut -d'/' -f1)
        printf "%-15s (%2d cpus): %s (%s/s/cpu)\n" $i $p "$total" "$percpu"
    done
    echo
done

With all the lock-avoiding changes done in this patch set, we get a pretty
decent improvement in performance and scalability of uprobes with number of
CPUs, even though we are still nowhere near linear scalability. This is due to
the remaining mmap_lock, which is currently taken to resolve interrupt address
to inode+offset and then uprobe instance. And, of course, uretprobes still need
similar RCU to avoid refcount in the hot path, which will be addressed in the
follow up patches. (Again, note, I left the benchmark numbers with lockless
RB-tree patches in.)

BASELINE (on top of Oleg's clean up patches)
============================================
uprobe-nop      ( 1 cpus):    3.032 ± 0.023M/s  (  3.032M/s/cpu)
uprobe-nop      ( 2 cpus):    3.452 ± 0.005M/s  (  1.726M/s/cpu)
uprobe-nop      ( 4 cpus):    3.663 ± 0.005M/s  (  0.916M/s/cpu)
uprobe-nop      ( 8 cpus):    3.718 ± 0.038M/s  (  0.465M/s/cpu)
uprobe-nop      (16 cpus):    3.344 ± 0.008M/s  (  0.209M/s/cpu)
uprobe-nop      (32 cpus):    2.288 ± 0.021M/s  (  0.071M/s/cpu)
uprobe-nop      (64 cpus):    3.205 ± 0.004M/s  (  0.050M/s/cpu)

uretprobe-nop   ( 1 cpus):    1.979 ± 0.005M/s  (  1.979M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.361 ± 0.005M/s  (  1.180M/s/cpu)
uretprobe-nop   ( 4 cpus):    2.309 ± 0.002M/s  (  0.577M/s/cpu)
uretprobe-nop   ( 8 cpus):    2.253 ± 0.001M/s  (  0.282M/s/cpu)
uretprobe-nop   (16 cpus):    2.007 ± 0.000M/s  (  0.125M/s/cpu)
uretprobe-nop   (32 cpus):    1.624 ± 0.003M/s  (  0.051M/s/cpu)
uretprobe-nop   (64 cpus):    2.149 ± 0.001M/s  (  0.034M/s/cpu)

Up to second-to-last patch (i.e., SRCU-based optimizations)
===========================================================
uprobe-nop      ( 1 cpus):    3.276 ± 0.005M/s  (  3.276M/s/cpu)
uprobe-nop      ( 2 cpus):    4.125 ± 0.002M/s  (  2.063M/s/cpu)
uprobe-nop      ( 4 cpus):    7.713 ± 0.002M/s  (  1.928M/s/cpu)
uprobe-nop      ( 8 cpus):    8.097 ± 0.006M/s  (  1.012M/s/cpu)
uprobe-nop      (16 cpus):    6.501 ± 0.056M/s  (  0.406M/s/cpu)
uprobe-nop      (32 cpus):    4.398 ± 0.084M/s  (  0.137M/s/cpu)
uprobe-nop      (64 cpus):    6.452 ± 0.000M/s  (  0.101M/s/cpu)

uretprobe-nop   ( 1 cpus):    2.055 ± 0.001M/s  (  2.055M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.677 ± 0.000M/s  (  1.339M/s/cpu)
uretprobe-nop   ( 4 cpus):    4.561 ± 0.003M/s  (  1.140M/s/cpu)
uretprobe-nop   ( 8 cpus):    5.291 ± 0.002M/s  (  0.661M/s/cpu)
uretprobe-nop   (16 cpus):    5.065 ± 0.019M/s  (  0.317M/s/cpu)
uretprobe-nop   (32 cpus):    3.622 ± 0.003M/s  (  0.113M/s/cpu)
uretprobe-nop   (64 cpus):    3.723 ± 0.002M/s  (  0.058M/s/cpu)

RCU Tasks Trace
===============
uprobe-nop      ( 1 cpus):    3.396 ± 0.002M/s  (  3.396M/s/cpu)
uprobe-nop      ( 2 cpus):    4.271 ± 0.006M/s  (  2.135M/s/cpu)
uprobe-nop      ( 4 cpus):    8.499 ± 0.015M/s  (  2.125M/s/cpu)
uprobe-nop      ( 8 cpus):   10.355 ± 0.028M/s  (  1.294M/s/cpu)
uprobe-nop      (16 cpus):    7.615 ± 0.099M/s  (  0.476M/s/cpu)
uprobe-nop      (32 cpus):    4.430 ± 0.007M/s  (  0.138M/s/cpu)
uprobe-nop      (64 cpus):    6.887 ± 0.020M/s  (  0.108M/s/cpu)

uretprobe-nop   ( 1 cpus):    2.174 ± 0.001M/s  (  2.174M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.853 ± 0.001M/s  (  1.426M/s/cpu)
uretprobe-nop   ( 4 cpus):    4.913 ± 0.002M/s  (  1.228M/s/cpu)
uretprobe-nop   ( 8 cpus):    5.883 ± 0.002M/s  (  0.735M/s/cpu)
uretprobe-nop   (16 cpus):    5.147 ± 0.001M/s  (  0.322M/s/cpu)
uretprobe-nop   (32 cpus):    3.738 ± 0.008M/s  (  0.117M/s/cpu)
uretprobe-nop   (64 cpus):    4.397 ± 0.002M/s  (  0.069M/s/cpu)

For baseline vs SRCU, peak througput increased from 3.7 M/s (million uprobe
triggerings per second) up to about 8 M/s. For uretprobes it's a bit more
modest with bump from 2.4 M/s to 5 M/s.

For SRCU vs RCU Tasks Trace, peak throughput for uprobes increases further from
8 M/s to 10.3 M/s (+28%!), and for uretprobes from 5.3 M/s to 5.8 M/s (+11%),
as we have more work to do on uretprobes side.

Even single-thread (no contention) performance is slightly better: 3.276 M/s to
3.396 M/s (+3.5%) for uprobes, and 2.055 M/s to 2.174 M/s (+5.8%)
for uretprobes.

  [0] https://lore.kernel.org/linux-trace-kernel/20240711110235.098009979@infradead.org/
  [1] https://lore.kernel.org/linux-trace-kernel/20240729134444.GA12293@redhat.com/
  [2] https://github.com/libbpf/libbpf-bootstrap/tree/uprobe-stress

v2->v3:
  - undid rcu and rb_node fields colocation which were causing crashes (Oleg);
  - ensure synchronize_srcu() on registration failure in patch #4 (Oleg);
v1->v2:
  - added back missed kfree() in patch #1 (Oleg);
  - forgot the rest, but there were a few small things here and there.

Andrii Nakryiko (10):
  uprobes: revamp uprobe refcounting and lifetime management
  uprobes: protected uprobe lifetime with SRCU
  uprobes: get rid of enum uprobe_filter_ctx in uprobe filter callbacks
  uprobes: travers uprobe's consumer list locklessly under SRCU
    protection
  uprobes: perform lockless SRCU-protected uprobes_tree lookup
  uprobes: switch to RCU Tasks Trace flavor for better performance
  uprobes: SRCU-protect uretprobe lifetime (with timeout)
  uprobes: implement SRCU-protected lifetime for single-stepped uprobe
  mm: add SLAB_TYPESAFE_BY_RCU to files_cache
  uprobes: add speculative lockless VMA to inode resolution

Peter Zijlstra (2):
  perf/uprobe: split uprobe_unregister()
  rbtree: provide rb_find_rcu() / rb_find_add_rcu()

Suren Baghdasaryan (1):
  mm: introduce mmap_lock_speculation_{start|end}

 include/linux/mm_types.h                      |   3 +
 include/linux/mmap_lock.h                     |  53 +-
 include/linux/rbtree.h                        |  67 ++
 include/linux/uprobes.h                       |  73 +-
 kernel/events/uprobes.c                       | 733 +++++++++++++-----
 kernel/fork.c                                 |   7 +-
 kernel/trace/bpf_trace.c                      |   8 +-
 kernel/trace/trace_uprobe.c                   |  15 +-
 .../selftests/bpf/bpf_testmod/bpf_testmod.c   |   3 +-
 9 files changed, 722 insertions(+), 240 deletions(-)

-- 
2.43.5



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

* [PATCH v3 01/13] uprobes: revamp uprobe refcounting and lifetime management
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 02/13] uprobes: protected uprobe lifetime with SRCU Andrii Nakryiko
                   ` (12 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

Revamp how struct uprobe is refcounted, and thus how its lifetime is
managed.

Right now, there are a few possible "owners" of uprobe refcount:
  - uprobes_tree RB tree assumes one refcount when uprobe is registered
    and added to the lookup tree;
  - while uprobe is triggered and kernel is handling it in the breakpoint
    handler code, temporary refcount bump is done to keep uprobe from
    being freed;
  - if we have uretprobe requested on a given struct uprobe instance, we
    take another refcount to keep uprobe alive until user space code
    returns from the function and triggers return handler.

The uprobe_tree's extra refcount of 1 is confusing and problematic. No
matter how many actual consumers are attached, they all share the same
refcount, and we have an extra logic to drop the "last" (which might not
really be last) refcount once uprobe's consumer list becomes empty.

This is unconventional and has to be kept in mind as a special case all
the time. Further, because of this design we have the situations where
find_uprobe() will find uprobe, bump refcount, return it to the caller,
but that uprobe will still need uprobe_is_active() check, after which
the caller is required to drop refcount and try again. This is just too
many details leaking to the higher level logic.

This patch changes refcounting scheme in such a way as to not have
uprobes_tree keeping extra refcount for struct uprobe. Instead, each
uprobe_consumer is assuming its own refcount, which will be dropped
when consumer is unregistered. Other than that, all the active users of
uprobe (entry and return uprobe handling code) keeps exactly the same
refcounting approach.

With the above setup, once uprobe's refcount drops to zero, we need to
make sure that uprobe's "destructor" removes uprobe from uprobes_tree,
of course. This, though, races with uprobe entry handling code in
handle_swbp(), which, through find_active_uprobe()->find_uprobe() lookup,
can race with uprobe being destroyed after refcount drops to zero (e.g.,
due to uprobe_consumer unregistering). So we add try_get_uprobe(), which
will attempt to bump refcount, unless it already is zero. Caller needs
to guarantee that uprobe instance won't be freed in parallel, which is
the case while we keep uprobes_treelock (for read or write, doesn't
matter).

Note also, we now don't leak the race between registration and
unregistration, so we remove the retry logic completely. If
find_uprobe() returns valid uprobe, it's guaranteed to remain in
uprobes_tree with properly incremented refcount. The race is handled
inside __insert_uprobe() and put_uprobe() working together:
__insert_uprobe() will remove uprobe from RB-tree, if it can't bump
refcount and will retry to insert the new uprobe instance. put_uprobe()
won't attempt to remove uprobe from RB-tree, if it's already not there.
All that is protected by uprobes_treelock, which keeps things simple.

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 kernel/events/uprobes.c | 179 +++++++++++++++++++++++-----------------
 1 file changed, 101 insertions(+), 78 deletions(-)

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 33349cc8de0c..147561c19d57 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -109,6 +109,11 @@ struct xol_area {
 	unsigned long 			vaddr;		/* Page(s) of instruction slots */
 };
 
+static void uprobe_warn(struct task_struct *t, const char *msg)
+{
+	pr_warn("uprobe: %s:%d failed to %s\n", current->comm, current->pid, msg);
+}
+
 /*
  * valid_vma: Verify if the specified vma is an executable vma
  * Relax restrictions while unregistering: vm_flags might have
@@ -587,25 +592,53 @@ set_orig_insn(struct arch_uprobe *auprobe, struct mm_struct *mm, unsigned long v
 			*(uprobe_opcode_t *)&auprobe->insn);
 }
 
+/* uprobe should have guaranteed positive refcount */
 static struct uprobe *get_uprobe(struct uprobe *uprobe)
 {
 	refcount_inc(&uprobe->ref);
 	return uprobe;
 }
 
+/*
+ * uprobe should have guaranteed lifetime, which can be either of:
+ *   - caller already has refcount taken (and wants an extra one);
+ *   - uprobe is RCU protected and won't be freed until after grace period;
+ *   - we are holding uprobes_treelock (for read or write, doesn't matter).
+ */
+static struct uprobe *try_get_uprobe(struct uprobe *uprobe)
+{
+	if (refcount_inc_not_zero(&uprobe->ref))
+		return uprobe;
+	return NULL;
+}
+
+static inline bool uprobe_is_active(struct uprobe *uprobe)
+{
+	return !RB_EMPTY_NODE(&uprobe->rb_node);
+}
+
 static void put_uprobe(struct uprobe *uprobe)
 {
-	if (refcount_dec_and_test(&uprobe->ref)) {
-		/*
-		 * If application munmap(exec_vma) before uprobe_unregister()
-		 * gets called, we don't get a chance to remove uprobe from
-		 * delayed_uprobe_list from remove_breakpoint(). Do it here.
-		 */
-		mutex_lock(&delayed_uprobe_lock);
-		delayed_uprobe_remove(uprobe, NULL);
-		mutex_unlock(&delayed_uprobe_lock);
-		kfree(uprobe);
-	}
+	if (!refcount_dec_and_test(&uprobe->ref))
+		return;
+
+	write_lock(&uprobes_treelock);
+
+	if (uprobe_is_active(uprobe))
+		rb_erase(&uprobe->rb_node, &uprobes_tree);
+
+	write_unlock(&uprobes_treelock);
+
+	/*
+	 * If application munmap(exec_vma) before uprobe_unregister()
+	 * gets called, we don't get a chance to remove uprobe from
+	 * delayed_uprobe_list from remove_breakpoint(). Do it here.
+	 */
+	mutex_lock(&delayed_uprobe_lock);
+	delayed_uprobe_remove(uprobe, NULL);
+	mutex_unlock(&delayed_uprobe_lock);
+
+	kfree(uprobe);
 }
 
 static __always_inline
@@ -656,7 +689,7 @@ static struct uprobe *__find_uprobe(struct inode *inode, loff_t offset)
 	struct rb_node *node = rb_find(&key, &uprobes_tree, __uprobe_cmp_key);
 
 	if (node)
-		return get_uprobe(__node_2_uprobe(node));
+		return try_get_uprobe(__node_2_uprobe(node));
 
 	return NULL;
 }
@@ -676,26 +709,44 @@ static struct uprobe *find_uprobe(struct inode *inode, loff_t offset)
 	return uprobe;
 }
 
+/*
+ * Attempt to insert a new uprobe into uprobes_tree.
+ *
+ * If uprobe already exists (for given inode+offset), we just increment
+ * refcount of previously existing uprobe.
+ *
+ * If not, a provided new instance of uprobe is inserted into the tree (with
+ * assumed initial refcount == 1).
+ *
+ * In any case, we return a uprobe instance that ends up being in uprobes_tree.
+ * Caller has to clean up new uprobe instance, if it ended up not being
+ * inserted into the tree.
+ *
+ * We assume that uprobes_treelock is held for writing.
+ */
 static struct uprobe *__insert_uprobe(struct uprobe *uprobe)
 {
 	struct rb_node *node;
-
+again:
 	node = rb_find_add(&uprobe->rb_node, &uprobes_tree, __uprobe_cmp);
-	if (node)
-		return get_uprobe(__node_2_uprobe(node));
+	if (node) {
+		struct uprobe *u = __node_2_uprobe(node);
 
-	/* get access + creation ref */
-	refcount_set(&uprobe->ref, 2);
-	return NULL;
+		if (!try_get_uprobe(u)) {
+			rb_erase(node, &uprobes_tree);
+			RB_CLEAR_NODE(&u->rb_node);
+			goto again;
+		}
+
+		return u;
+	}
+
+	return uprobe;
 }
 
 /*
- * Acquire uprobes_treelock.
- * Matching uprobe already exists in rbtree;
- *	increment (access refcount) and return the matching uprobe.
- *
- * No matching uprobe; insert the uprobe in rb_tree;
- *	get a double refcount (access + creation) and return NULL.
+ * Acquire uprobes_treelock and insert uprobe into uprobes_tree
+ * (or reuse existing one, see __insert_uprobe() comments above).
  */
 static struct uprobe *insert_uprobe(struct uprobe *uprobe)
 {
@@ -732,11 +783,13 @@ static struct uprobe *alloc_uprobe(struct inode *inode, loff_t offset,
 	uprobe->ref_ctr_offset = ref_ctr_offset;
 	init_rwsem(&uprobe->register_rwsem);
 	init_rwsem(&uprobe->consumer_rwsem);
+	RB_CLEAR_NODE(&uprobe->rb_node);
+	refcount_set(&uprobe->ref, 1);
 
 	/* add to uprobes_tree, sorted on inode:offset */
 	cur_uprobe = insert_uprobe(uprobe);
 	/* a uprobe exists for this inode:offset combination */
-	if (cur_uprobe) {
+	if (cur_uprobe != uprobe) {
 		if (cur_uprobe->ref_ctr_offset != uprobe->ref_ctr_offset) {
 			ref_ctr_mismatch_warn(cur_uprobe, uprobe);
 			put_uprobe(cur_uprobe);
@@ -921,26 +974,6 @@ remove_breakpoint(struct uprobe *uprobe, struct mm_struct *mm, unsigned long vad
 	return set_orig_insn(&uprobe->arch, mm, vaddr);
 }
 
-static inline bool uprobe_is_active(struct uprobe *uprobe)
-{
-	return !RB_EMPTY_NODE(&uprobe->rb_node);
-}
-/*
- * There could be threads that have already hit the breakpoint. They
- * will recheck the current insn and restart if find_uprobe() fails.
- * See find_active_uprobe().
- */
-static void delete_uprobe(struct uprobe *uprobe)
-{
-	if (WARN_ON(!uprobe_is_active(uprobe)))
-		return;
-
-	write_lock(&uprobes_treelock);
-	rb_erase(&uprobe->rb_node, &uprobes_tree);
-	write_unlock(&uprobes_treelock);
-	RB_CLEAR_NODE(&uprobe->rb_node); /* for uprobe_is_active() */
-}
-
 struct map_info {
 	struct map_info *next;
 	struct mm_struct *mm;
@@ -1094,17 +1127,13 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
 	int err;
 
 	down_write(&uprobe->register_rwsem);
-	if (WARN_ON(!consumer_del(uprobe, uc)))
+	if (WARN_ON(!consumer_del(uprobe, uc))) {
 		err = -ENOENT;
-	else
+	} else {
 		err = register_for_each_vma(uprobe, NULL);
-
-	/* TODO : cant unregister? schedule a worker thread */
-	if (!err) {
-		if (!uprobe->consumers)
-			delete_uprobe(uprobe);
-		else
-			err = -EBUSY;
+		/* TODO : cant unregister? schedule a worker thread */
+		if (unlikely(err))
+			uprobe_warn(current, "unregister, leaking uprobe");
 	}
 	up_write(&uprobe->register_rwsem);
 
@@ -1159,27 +1188,16 @@ struct uprobe *uprobe_register(struct inode *inode,
 	if (!IS_ALIGNED(ref_ctr_offset, sizeof(short)))
 		return ERR_PTR(-EINVAL);
 
- retry:
 	uprobe = alloc_uprobe(inode, offset, ref_ctr_offset);
 	if (IS_ERR(uprobe))
 		return uprobe;
 
-	/*
-	 * We can race with uprobe_unregister()->delete_uprobe().
-	 * Check uprobe_is_active() and retry if it is false.
-	 */
 	down_write(&uprobe->register_rwsem);
-	ret = -EAGAIN;
-	if (likely(uprobe_is_active(uprobe))) {
-		consumer_add(uprobe, uc);
-		ret = register_for_each_vma(uprobe, uc);
-	}
+	consumer_add(uprobe, uc);
+	ret = register_for_each_vma(uprobe, uc);
 	up_write(&uprobe->register_rwsem);
-	put_uprobe(uprobe);
 
 	if (ret) {
-		if (unlikely(ret == -EAGAIN))
-			goto retry;
 		uprobe_unregister(uprobe, uc);
 		return ERR_PTR(ret);
 	}
@@ -1286,15 +1304,17 @@ static void build_probe_list(struct inode *inode,
 			u = rb_entry(t, struct uprobe, rb_node);
 			if (u->inode != inode || u->offset < min)
 				break;
-			list_add(&u->pending_list, head);
-			get_uprobe(u);
+			/* if uprobe went away, it's safe to ignore it */
+			if (try_get_uprobe(u))
+				list_add(&u->pending_list, head);
 		}
 		for (t = n; (t = rb_next(t)); ) {
 			u = rb_entry(t, struct uprobe, rb_node);
 			if (u->inode != inode || u->offset > max)
 				break;
-			list_add(&u->pending_list, head);
-			get_uprobe(u);
+			/* if uprobe went away, it's safe to ignore it */
+			if (try_get_uprobe(u))
+				list_add(&u->pending_list, head);
 		}
 	}
 	read_unlock(&uprobes_treelock);
@@ -1752,6 +1772,12 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
 			return -ENOMEM;
 
 		*n = *o;
+		/*
+		 * uprobe's refcnt has to be positive at this point, kept by
+		 * utask->return_instances items; return_instances can't be
+		 * removed right now, as task is blocked due to duping; so
+		 * get_uprobe() is safe to use here.
+		 */
 		get_uprobe(n->uprobe);
 		n->next = NULL;
 
@@ -1763,12 +1789,6 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
 	return 0;
 }
 
-static void uprobe_warn(struct task_struct *t, const char *msg)
-{
-	pr_warn("uprobe: %s:%d failed to %s\n",
-			current->comm, current->pid, msg);
-}
-
 static void dup_xol_work(struct callback_head *work)
 {
 	if (current->flags & PF_EXITING)
@@ -1894,7 +1914,10 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 		}
 		orig_ret_vaddr = utask->return_instances->orig_ret_vaddr;
 	}
-
+	/*
+	 * uprobe's refcnt is positive, held by caller, so it's safe to
+	 * unconditionally bump it one more time here
+	 */
 	ri->uprobe = get_uprobe(uprobe);
 	ri->func = instruction_pointer(regs);
 	ri->stack = user_stack_pointer(regs);
-- 
2.43.5



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

* [PATCH v3 02/13] uprobes: protected uprobe lifetime with SRCU
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 01/13] uprobes: revamp uprobe refcounting and lifetime management Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 03/13] uprobes: get rid of enum uprobe_filter_ctx in uprobe filter callbacks Andrii Nakryiko
                   ` (11 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

To avoid unnecessarily taking a (brief) refcount on uprobe during
breakpoint handling in handle_swbp for entry uprobes, make find_uprobe()
not take refcount, but protect the lifetime of a uprobe instance with
RCU. This improves scalability, as refcount gets quite expensive due to
cache line bouncing between multiple CPUs.

Specifically, we utilize our own uprobe-specific SRCU instance for this
RCU protection. put_uprobe() will delay actual kfree() using call_srcu().

For now, uretprobe and single-stepping handling will still acquire
refcount as necessary. We'll address these issues in follow up patches
by making them use SRCU with timeout.

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 kernel/events/uprobes.c | 94 +++++++++++++++++++++++------------------
 1 file changed, 54 insertions(+), 40 deletions(-)

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 147561c19d57..3e3595753e2c 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -41,6 +41,8 @@ static struct rb_root uprobes_tree = RB_ROOT;
 
 static DEFINE_RWLOCK(uprobes_treelock);	/* serialize rbtree access */
 
+DEFINE_STATIC_SRCU(uprobes_srcu);
+
 #define UPROBES_HASH_SZ	13
 /* serialize uprobe->pending_list */
 static struct mutex uprobes_mmap_mutex[UPROBES_HASH_SZ];
@@ -59,6 +61,7 @@ struct uprobe {
 	struct list_head	pending_list;
 	struct uprobe_consumer	*consumers;
 	struct inode		*inode;		/* Also hold a ref to inode */
+	struct rcu_head		rcu;
 	loff_t			offset;
 	loff_t			ref_ctr_offset;
 	unsigned long		flags;
@@ -617,6 +620,13 @@ static inline bool uprobe_is_active(struct uprobe *uprobe)
 	return !RB_EMPTY_NODE(&uprobe->rb_node);
 }
 
+static void uprobe_free_rcu(struct rcu_head *rcu)
+{
+	struct uprobe *uprobe = container_of(rcu, struct uprobe, rcu);
+
+	kfree(uprobe);
+}
+
 static void put_uprobe(struct uprobe *uprobe)
 {
 	if (!refcount_dec_and_test(&uprobe->ref))
@@ -638,7 +648,7 @@ static void put_uprobe(struct uprobe *uprobe)
 	delayed_uprobe_remove(uprobe, NULL);
 	mutex_unlock(&delayed_uprobe_lock);
 
-	kfree(uprobe);
+	call_srcu(&uprobes_srcu, &uprobe->rcu, uprobe_free_rcu);
 }
 
 static __always_inline
@@ -680,33 +690,25 @@ static inline int __uprobe_cmp(struct rb_node *a, const struct rb_node *b)
 	return uprobe_cmp(u->inode, u->offset, __node_2_uprobe(b));
 }
 
-static struct uprobe *__find_uprobe(struct inode *inode, loff_t offset)
+/*
+ * Assumes being inside RCU protected region.
+ * No refcount is taken on returned uprobe.
+ */
+static struct uprobe *find_uprobe_rcu(struct inode *inode, loff_t offset)
 {
 	struct __uprobe_key key = {
 		.inode = inode,
 		.offset = offset,
 	};
-	struct rb_node *node = rb_find(&key, &uprobes_tree, __uprobe_cmp_key);
-
-	if (node)
-		return try_get_uprobe(__node_2_uprobe(node));
-
-	return NULL;
-}
+	struct rb_node *node;
 
-/*
- * Find a uprobe corresponding to a given inode:offset
- * Acquires uprobes_treelock
- */
-static struct uprobe *find_uprobe(struct inode *inode, loff_t offset)
-{
-	struct uprobe *uprobe;
+	lockdep_assert(srcu_read_lock_held(&uprobes_srcu));
 
 	read_lock(&uprobes_treelock);
-	uprobe = __find_uprobe(inode, offset);
+	node = rb_find(&key, &uprobes_tree, __uprobe_cmp_key);
 	read_unlock(&uprobes_treelock);
 
-	return uprobe;
+	return node ? __node_2_uprobe(node) : NULL;
 }
 
 /*
@@ -1080,10 +1082,10 @@ register_for_each_vma(struct uprobe *uprobe, struct uprobe_consumer *new)
 			goto free;
 		/*
 		 * We take mmap_lock for writing to avoid the race with
-		 * find_active_uprobe() which takes mmap_lock for reading.
+		 * find_active_uprobe_rcu() which takes mmap_lock for reading.
 		 * Thus this install_breakpoint() can not make
-		 * is_trap_at_addr() true right after find_uprobe()
-		 * returns NULL in find_active_uprobe().
+		 * is_trap_at_addr() true right after find_uprobe_rcu()
+		 * returns NULL in find_active_uprobe_rcu().
 		 */
 		mmap_write_lock(mm);
 		vma = find_vma(mm, info->vaddr);
@@ -1885,9 +1887,13 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 		return;
 	}
 
+	/* we need to bump refcount to store uprobe in utask */
+	if (!try_get_uprobe(uprobe))
+		return;
+
 	ri = kmalloc(sizeof(struct return_instance), GFP_KERNEL);
 	if (!ri)
-		return;
+		goto fail;
 
 	trampoline_vaddr = uprobe_get_trampoline_vaddr();
 	orig_ret_vaddr = arch_uretprobe_hijack_return_addr(trampoline_vaddr, regs);
@@ -1914,11 +1920,7 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 		}
 		orig_ret_vaddr = utask->return_instances->orig_ret_vaddr;
 	}
-	/*
-	 * uprobe's refcnt is positive, held by caller, so it's safe to
-	 * unconditionally bump it one more time here
-	 */
-	ri->uprobe = get_uprobe(uprobe);
+	ri->uprobe = uprobe;
 	ri->func = instruction_pointer(regs);
 	ri->stack = user_stack_pointer(regs);
 	ri->orig_ret_vaddr = orig_ret_vaddr;
@@ -1929,8 +1931,9 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 	utask->return_instances = ri;
 
 	return;
- fail:
+fail:
 	kfree(ri);
+	put_uprobe(uprobe);
 }
 
 /* Prepare to single-step probed instruction out of line. */
@@ -1945,9 +1948,14 @@ pre_ssout(struct uprobe *uprobe, struct pt_regs *regs, unsigned long bp_vaddr)
 	if (!utask)
 		return -ENOMEM;
 
+	if (!try_get_uprobe(uprobe))
+		return -EINVAL;
+
 	xol_vaddr = xol_get_insn_slot(uprobe);
-	if (!xol_vaddr)
-		return -ENOMEM;
+	if (!xol_vaddr) {
+		err = -ENOMEM;
+		goto err_out;
+	}
 
 	utask->xol_vaddr = xol_vaddr;
 	utask->vaddr = bp_vaddr;
@@ -1955,12 +1963,15 @@ pre_ssout(struct uprobe *uprobe, struct pt_regs *regs, unsigned long bp_vaddr)
 	err = arch_uprobe_pre_xol(&uprobe->arch, regs);
 	if (unlikely(err)) {
 		xol_free_insn_slot(current);
-		return err;
+		goto err_out;
 	}
 
 	utask->active_uprobe = uprobe;
 	utask->state = UTASK_SSTEP;
 	return 0;
+err_out:
+	put_uprobe(uprobe);
+	return err;
 }
 
 /*
@@ -2043,7 +2054,8 @@ static int is_trap_at_addr(struct mm_struct *mm, unsigned long vaddr)
 	return is_trap_insn(&opcode);
 }
 
-static struct uprobe *find_active_uprobe(unsigned long bp_vaddr, int *is_swbp)
+/* assumes being inside RCU protected region */
+static struct uprobe *find_active_uprobe_rcu(unsigned long bp_vaddr, int *is_swbp)
 {
 	struct mm_struct *mm = current->mm;
 	struct uprobe *uprobe = NULL;
@@ -2056,7 +2068,7 @@ static struct uprobe *find_active_uprobe(unsigned long bp_vaddr, int *is_swbp)
 			struct inode *inode = file_inode(vma->vm_file);
 			loff_t offset = vaddr_to_offset(vma, bp_vaddr);
 
-			uprobe = find_uprobe(inode, offset);
+			uprobe = find_uprobe_rcu(inode, offset);
 		}
 
 		if (!uprobe)
@@ -2202,13 +2214,15 @@ static void handle_swbp(struct pt_regs *regs)
 {
 	struct uprobe *uprobe;
 	unsigned long bp_vaddr;
-	int is_swbp;
+	int is_swbp, srcu_idx;
 
 	bp_vaddr = uprobe_get_swbp_addr(regs);
 	if (bp_vaddr == uprobe_get_trampoline_vaddr())
 		return uprobe_handle_trampoline(regs);
 
-	uprobe = find_active_uprobe(bp_vaddr, &is_swbp);
+	srcu_idx = srcu_read_lock(&uprobes_srcu);
+
+	uprobe = find_active_uprobe_rcu(bp_vaddr, &is_swbp);
 	if (!uprobe) {
 		if (is_swbp > 0) {
 			/* No matching uprobe; signal SIGTRAP. */
@@ -2224,7 +2238,7 @@ static void handle_swbp(struct pt_regs *regs)
 			 */
 			instruction_pointer_set(regs, bp_vaddr);
 		}
-		return;
+		goto out;
 	}
 
 	/* change it in advance for ->handler() and restart */
@@ -2259,12 +2273,12 @@ static void handle_swbp(struct pt_regs *regs)
 	if (arch_uprobe_skip_sstep(&uprobe->arch, regs))
 		goto out;
 
-	if (!pre_ssout(uprobe, regs, bp_vaddr))
-		return;
+	if (pre_ssout(uprobe, regs, bp_vaddr))
+		goto out;
 
-	/* arch_uprobe_skip_sstep() succeeded, or restart if can't singlestep */
 out:
-	put_uprobe(uprobe);
+	/* arch_uprobe_skip_sstep() succeeded, or restart if can't singlestep */
+	srcu_read_unlock(&uprobes_srcu, srcu_idx);
 }
 
 /*
-- 
2.43.5



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

* [PATCH v3 03/13] uprobes: get rid of enum uprobe_filter_ctx in uprobe filter callbacks
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 01/13] uprobes: revamp uprobe refcounting and lifetime management Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 02/13] uprobes: protected uprobe lifetime with SRCU Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection Andrii Nakryiko
                   ` (10 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

It serves no purpose beyond adding unnecessray argument passed to the
filter callback. Just get rid of it, no one is actually using it.

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 include/linux/uprobes.h     | 10 +---------
 kernel/events/uprobes.c     | 18 +++++++-----------
 kernel/trace/bpf_trace.c    |  3 +--
 kernel/trace/trace_uprobe.c |  9 +++------
 4 files changed, 12 insertions(+), 28 deletions(-)

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index 6332c111036e..9cf0dce62e4c 100644
--- a/include/linux/uprobes.h
+++ b/include/linux/uprobes.h
@@ -28,20 +28,12 @@ struct page;
 
 #define MAX_URETPROBE_DEPTH		64
 
-enum uprobe_filter_ctx {
-	UPROBE_FILTER_REGISTER,
-	UPROBE_FILTER_UNREGISTER,
-	UPROBE_FILTER_MMAP,
-};
-
 struct uprobe_consumer {
 	int (*handler)(struct uprobe_consumer *self, struct pt_regs *regs);
 	int (*ret_handler)(struct uprobe_consumer *self,
 				unsigned long func,
 				struct pt_regs *regs);
-	bool (*filter)(struct uprobe_consumer *self,
-				enum uprobe_filter_ctx ctx,
-				struct mm_struct *mm);
+	bool (*filter)(struct uprobe_consumer *self, struct mm_struct *mm);
 
 	struct uprobe_consumer *next;
 };
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 3e3595753e2c..8bdcdc6901b2 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -918,21 +918,19 @@ static int prepare_uprobe(struct uprobe *uprobe, struct file *file,
 	return ret;
 }
 
-static inline bool consumer_filter(struct uprobe_consumer *uc,
-				   enum uprobe_filter_ctx ctx, struct mm_struct *mm)
+static inline bool consumer_filter(struct uprobe_consumer *uc, struct mm_struct *mm)
 {
-	return !uc->filter || uc->filter(uc, ctx, mm);
+	return !uc->filter || uc->filter(uc, mm);
 }
 
-static bool filter_chain(struct uprobe *uprobe,
-			 enum uprobe_filter_ctx ctx, struct mm_struct *mm)
+static bool filter_chain(struct uprobe *uprobe, struct mm_struct *mm)
 {
 	struct uprobe_consumer *uc;
 	bool ret = false;
 
 	down_read(&uprobe->consumer_rwsem);
 	for (uc = uprobe->consumers; uc; uc = uc->next) {
-		ret = consumer_filter(uc, ctx, mm);
+		ret = consumer_filter(uc, mm);
 		if (ret)
 			break;
 	}
@@ -1099,12 +1097,10 @@ register_for_each_vma(struct uprobe *uprobe, struct uprobe_consumer *new)
 
 		if (is_register) {
 			/* consult only the "caller", new consumer. */
-			if (consumer_filter(new,
-					UPROBE_FILTER_REGISTER, mm))
+			if (consumer_filter(new, mm))
 				err = install_breakpoint(uprobe, mm, vma, info->vaddr);
 		} else if (test_bit(MMF_HAS_UPROBES, &mm->flags)) {
-			if (!filter_chain(uprobe,
-					UPROBE_FILTER_UNREGISTER, mm))
+			if (!filter_chain(uprobe, mm))
 				err |= remove_breakpoint(uprobe, mm, info->vaddr);
 		}
 
@@ -1387,7 +1383,7 @@ int uprobe_mmap(struct vm_area_struct *vma)
 	 */
 	list_for_each_entry_safe(uprobe, u, &tmp_list, pending_list) {
 		if (!fatal_signal_pending(current) &&
-		    filter_chain(uprobe, UPROBE_FILTER_MMAP, vma->vm_mm)) {
+		    filter_chain(uprobe, vma->vm_mm)) {
 			unsigned long vaddr = offset_to_vaddr(vma, uprobe->offset);
 			install_breakpoint(uprobe, vma->vm_mm, vma, vaddr);
 		}
diff --git a/kernel/trace/bpf_trace.c b/kernel/trace/bpf_trace.c
index 4e391daafa64..73c570b5988b 100644
--- a/kernel/trace/bpf_trace.c
+++ b/kernel/trace/bpf_trace.c
@@ -3320,8 +3320,7 @@ static int uprobe_prog_run(struct bpf_uprobe *uprobe,
 }
 
 static bool
-uprobe_multi_link_filter(struct uprobe_consumer *con, enum uprobe_filter_ctx ctx,
-			 struct mm_struct *mm)
+uprobe_multi_link_filter(struct uprobe_consumer *con, struct mm_struct *mm)
 {
 	struct bpf_uprobe *uprobe;
 
diff --git a/kernel/trace/trace_uprobe.c b/kernel/trace/trace_uprobe.c
index 52e76a73fa7c..7eb79e0a5352 100644
--- a/kernel/trace/trace_uprobe.c
+++ b/kernel/trace/trace_uprobe.c
@@ -1078,9 +1078,7 @@ print_uprobe_event(struct trace_iterator *iter, int flags, struct trace_event *e
 	return trace_handle_return(s);
 }
 
-typedef bool (*filter_func_t)(struct uprobe_consumer *self,
-				enum uprobe_filter_ctx ctx,
-				struct mm_struct *mm);
+typedef bool (*filter_func_t)(struct uprobe_consumer *self, struct mm_struct *mm);
 
 static int trace_uprobe_enable(struct trace_uprobe *tu, filter_func_t filter)
 {
@@ -1339,8 +1337,7 @@ static int uprobe_perf_open(struct trace_event_call *call,
 	return err;
 }
 
-static bool uprobe_perf_filter(struct uprobe_consumer *uc,
-				enum uprobe_filter_ctx ctx, struct mm_struct *mm)
+static bool uprobe_perf_filter(struct uprobe_consumer *uc, struct mm_struct *mm)
 {
 	struct trace_uprobe_filter *filter;
 	struct trace_uprobe *tu;
@@ -1426,7 +1423,7 @@ static void __uprobe_perf_func(struct trace_uprobe *tu,
 static int uprobe_perf_func(struct trace_uprobe *tu, struct pt_regs *regs,
 			    struct uprobe_cpu_buffer **ucbp)
 {
-	if (!uprobe_perf_filter(&tu->consumer, 0, current->mm))
+	if (!uprobe_perf_filter(&tu->consumer, current->mm))
 		return UPROBE_HANDLER_REMOVE;
 
 	if (!is_ret_probe(tu))
-- 
2.43.5



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

* [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (2 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH v3 03/13] uprobes: get rid of enum uprobe_filter_ctx in uprobe filter callbacks Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-22 14:22   ` Jiri Olsa
  2024-08-13  4:29 ` [PATCH v3 05/13] perf/uprobe: split uprobe_unregister() Andrii Nakryiko
                   ` (9 subsequent siblings)
  13 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

uprobe->register_rwsem is one of a few big bottlenecks to scalability of
uprobes, so we need to get rid of it to improve uprobe performance and
multi-CPU scalability.

First, we turn uprobe's consumer list to a typical doubly-linked list
and utilize existing RCU-aware helpers for traversing such lists, as
well as adding and removing elements from it.

For entry uprobes we already have SRCU protection active since before
uprobe lookup. For uretprobe we keep refcount, guaranteeing that uprobe
won't go away from under us, but we add SRCU protection around consumer
list traversal.

Lastly, to keep handler_chain()'s UPROBE_HANDLER_REMOVE handling simple,
we remember whether any removal was requested during handler calls, but
then we double-check the decision under a proper register_rwsem using
consumers' filter callbacks. Handler removal is very rare, so this extra
lock won't hurt performance, overall, but we also avoid the need for any
extra protection (e.g., seqcount locks).

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 include/linux/uprobes.h |   2 +-
 kernel/events/uprobes.c | 111 ++++++++++++++++++++++------------------
 2 files changed, 61 insertions(+), 52 deletions(-)

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index 9cf0dce62e4c..29c935b0d504 100644
--- a/include/linux/uprobes.h
+++ b/include/linux/uprobes.h
@@ -35,7 +35,7 @@ struct uprobe_consumer {
 				struct pt_regs *regs);
 	bool (*filter)(struct uprobe_consumer *self, struct mm_struct *mm);
 
-	struct uprobe_consumer *next;
+	struct list_head cons_node;
 };
 
 #ifdef CONFIG_UPROBES
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 8bdcdc6901b2..7de1aaf50394 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -59,7 +59,7 @@ struct uprobe {
 	struct rw_semaphore	register_rwsem;
 	struct rw_semaphore	consumer_rwsem;
 	struct list_head	pending_list;
-	struct uprobe_consumer	*consumers;
+	struct list_head	consumers;
 	struct inode		*inode;		/* Also hold a ref to inode */
 	struct rcu_head		rcu;
 	loff_t			offset;
@@ -783,6 +783,7 @@ static struct uprobe *alloc_uprobe(struct inode *inode, loff_t offset,
 	uprobe->inode = inode;
 	uprobe->offset = offset;
 	uprobe->ref_ctr_offset = ref_ctr_offset;
+	INIT_LIST_HEAD(&uprobe->consumers);
 	init_rwsem(&uprobe->register_rwsem);
 	init_rwsem(&uprobe->consumer_rwsem);
 	RB_CLEAR_NODE(&uprobe->rb_node);
@@ -808,34 +809,10 @@ static struct uprobe *alloc_uprobe(struct inode *inode, loff_t offset,
 static void consumer_add(struct uprobe *uprobe, struct uprobe_consumer *uc)
 {
 	down_write(&uprobe->consumer_rwsem);
-	uc->next = uprobe->consumers;
-	uprobe->consumers = uc;
+	list_add_rcu(&uc->cons_node, &uprobe->consumers);
 	up_write(&uprobe->consumer_rwsem);
 }
 
-/*
- * For uprobe @uprobe, delete the consumer @uc.
- * Return true if the @uc is deleted successfully
- * or return false.
- */
-static bool consumer_del(struct uprobe *uprobe, struct uprobe_consumer *uc)
-{
-	struct uprobe_consumer **con;
-	bool ret = false;
-
-	down_write(&uprobe->consumer_rwsem);
-	for (con = &uprobe->consumers; *con; con = &(*con)->next) {
-		if (*con == uc) {
-			*con = uc->next;
-			ret = true;
-			break;
-		}
-	}
-	up_write(&uprobe->consumer_rwsem);
-
-	return ret;
-}
-
 static int __copy_insn(struct address_space *mapping, struct file *filp,
 			void *insn, int nbytes, loff_t offset)
 {
@@ -929,7 +906,8 @@ static bool filter_chain(struct uprobe *uprobe, struct mm_struct *mm)
 	bool ret = false;
 
 	down_read(&uprobe->consumer_rwsem);
-	for (uc = uprobe->consumers; uc; uc = uc->next) {
+	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
+				 srcu_read_lock_held(&uprobes_srcu)) {
 		ret = consumer_filter(uc, mm);
 		if (ret)
 			break;
@@ -1125,18 +1103,31 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
 	int err;
 
 	down_write(&uprobe->register_rwsem);
-	if (WARN_ON(!consumer_del(uprobe, uc))) {
-		err = -ENOENT;
-	} else {
-		err = register_for_each_vma(uprobe, NULL);
-		/* TODO : cant unregister? schedule a worker thread */
-		if (unlikely(err))
-			uprobe_warn(current, "unregister, leaking uprobe");
-	}
+
+	list_del_rcu(&uc->cons_node);
+	err = register_for_each_vma(uprobe, NULL);
+
 	up_write(&uprobe->register_rwsem);
 
-	if (!err)
-		put_uprobe(uprobe);
+	/* TODO : cant unregister? schedule a worker thread */
+	if (unlikely(err)) {
+		uprobe_warn(current, "unregister, leaking uprobe");
+		goto out_sync;
+	}
+
+	put_uprobe(uprobe);
+
+out_sync:
+	/*
+	 * Now that handler_chain() and handle_uretprobe_chain() iterate over
+	 * uprobe->consumers list under RCU protection without holding
+	 * uprobe->register_rwsem, we need to wait for RCU grace period to
+	 * make sure that we can't call into just unregistered
+	 * uprobe_consumer's callbacks anymore. If we don't do that, fast and
+	 * unlucky enough caller can free consumer's memory and cause
+	 * handler_chain() or handle_uretprobe_chain() to do an use-after-free.
+	 */
+	synchronize_srcu(&uprobes_srcu);
 }
 EXPORT_SYMBOL_GPL(uprobe_unregister);
 
@@ -1214,13 +1205,20 @@ EXPORT_SYMBOL_GPL(uprobe_register);
 int uprobe_apply(struct uprobe *uprobe, struct uprobe_consumer *uc, bool add)
 {
 	struct uprobe_consumer *con;
-	int ret = -ENOENT;
+	int ret = -ENOENT, srcu_idx;
 
 	down_write(&uprobe->register_rwsem);
-	for (con = uprobe->consumers; con && con != uc ; con = con->next)
-		;
-	if (con)
-		ret = register_for_each_vma(uprobe, add ? uc : NULL);
+
+	srcu_idx = srcu_read_lock(&uprobes_srcu);
+	list_for_each_entry_srcu(con, &uprobe->consumers, cons_node,
+				 srcu_read_lock_held(&uprobes_srcu)) {
+		if (con == uc) {
+			ret = register_for_each_vma(uprobe, add ? uc : NULL);
+			break;
+		}
+	}
+	srcu_read_unlock(&uprobes_srcu, srcu_idx);
+
 	up_write(&uprobe->register_rwsem);
 
 	return ret;
@@ -2085,10 +2083,12 @@ static void handler_chain(struct uprobe *uprobe, struct pt_regs *regs)
 	struct uprobe_consumer *uc;
 	int remove = UPROBE_HANDLER_REMOVE;
 	bool need_prep = false; /* prepare return uprobe, when needed */
+	bool has_consumers = false;
 
-	down_read(&uprobe->register_rwsem);
 	current->utask->auprobe = &uprobe->arch;
-	for (uc = uprobe->consumers; uc; uc = uc->next) {
+
+	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
+				 srcu_read_lock_held(&uprobes_srcu)) {
 		int rc = 0;
 
 		if (uc->handler) {
@@ -2101,17 +2101,24 @@ static void handler_chain(struct uprobe *uprobe, struct pt_regs *regs)
 			need_prep = true;
 
 		remove &= rc;
+		has_consumers = true;
 	}
 	current->utask->auprobe = NULL;
 
 	if (need_prep && !remove)
 		prepare_uretprobe(uprobe, regs); /* put bp at return */
 
-	if (remove && uprobe->consumers) {
-		WARN_ON(!uprobe_is_active(uprobe));
-		unapply_uprobe(uprobe, current->mm);
+	if (remove && has_consumers) {
+		down_read(&uprobe->register_rwsem);
+
+		/* re-check that removal is still required, this time under lock */
+		if (!filter_chain(uprobe, current->mm)) {
+			WARN_ON(!uprobe_is_active(uprobe));
+			unapply_uprobe(uprobe, current->mm);
+		}
+
+		up_read(&uprobe->register_rwsem);
 	}
-	up_read(&uprobe->register_rwsem);
 }
 
 static void
@@ -2119,13 +2126,15 @@ handle_uretprobe_chain(struct return_instance *ri, struct pt_regs *regs)
 {
 	struct uprobe *uprobe = ri->uprobe;
 	struct uprobe_consumer *uc;
+	int srcu_idx;
 
-	down_read(&uprobe->register_rwsem);
-	for (uc = uprobe->consumers; uc; uc = uc->next) {
+	srcu_idx = srcu_read_lock(&uprobes_srcu);
+	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
+				 srcu_read_lock_held(&uprobes_srcu)) {
 		if (uc->ret_handler)
 			uc->ret_handler(uc, ri->func, regs);
 	}
-	up_read(&uprobe->register_rwsem);
+	srcu_read_unlock(&uprobes_srcu, srcu_idx);
 }
 
 static struct return_instance *find_next_ret_chain(struct return_instance *ri)
-- 
2.43.5



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

* [PATCH v3 05/13] perf/uprobe: split uprobe_unregister()
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (3 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 06/13] rbtree: provide rb_find_rcu() / rb_find_add_rcu() Andrii Nakryiko
                   ` (8 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

From: Peter Zijlstra <peterz@infradead.org>

With uprobe_unregister() having grown a synchronize_srcu(), it becomes
fairly slow to call. Esp. since both users of this API call it in a
loop.

Peel off the sync_srcu() and do it once, after the loop.

We also need to add uprobe_unregister_sync() into uprobe_register()'s
error handling path, as we need to be careful about returning to the
caller before we have a guarantee that partially attached consumer won't
be called anymore. This is an unlikely slow path and this should be
totally fine to be slow in the case of a failed attach.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Co-developed-by: Andrii Nakryiko <andrii@kernel.org>
Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 include/linux/uprobes.h                       |  8 +++++--
 kernel/events/uprobes.c                       | 21 +++++++++++++------
 kernel/trace/bpf_trace.c                      |  5 ++++-
 kernel/trace/trace_uprobe.c                   |  6 +++++-
 .../selftests/bpf/bpf_testmod/bpf_testmod.c   |  3 ++-
 5 files changed, 32 insertions(+), 11 deletions(-)

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index 29c935b0d504..e41cdae5597b 100644
--- a/include/linux/uprobes.h
+++ b/include/linux/uprobes.h
@@ -108,7 +108,8 @@ extern unsigned long uprobe_get_trap_addr(struct pt_regs *regs);
 extern int uprobe_write_opcode(struct arch_uprobe *auprobe, struct mm_struct *mm, unsigned long vaddr, uprobe_opcode_t);
 extern struct uprobe *uprobe_register(struct inode *inode, loff_t offset, loff_t ref_ctr_offset, struct uprobe_consumer *uc);
 extern int uprobe_apply(struct uprobe *uprobe, struct uprobe_consumer *uc, bool);
-extern void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc);
+extern void uprobe_unregister_nosync(struct uprobe *uprobe, struct uprobe_consumer *uc);
+extern void uprobe_unregister_sync(void);
 extern int uprobe_mmap(struct vm_area_struct *vma);
 extern void uprobe_munmap(struct vm_area_struct *vma, unsigned long start, unsigned long end);
 extern void uprobe_start_dup_mmap(void);
@@ -157,7 +158,10 @@ uprobe_apply(struct uprobe* uprobe, struct uprobe_consumer *uc, bool add)
 	return -ENOSYS;
 }
 static inline void
-uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
+uprobe_unregister_nosync(struct uprobe *uprobe, struct uprobe_consumer *uc)
+{
+}
+static inline void uprobe_unregister_sync(void)
 {
 }
 static inline int uprobe_mmap(struct vm_area_struct *vma)
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 7de1aaf50394..0b6d4c0a0088 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -1094,11 +1094,11 @@ register_for_each_vma(struct uprobe *uprobe, struct uprobe_consumer *new)
 }
 
 /**
- * uprobe_unregister - unregister an already registered probe.
+ * uprobe_unregister_nosync - unregister an already registered probe.
  * @uprobe: uprobe to remove
  * @uc: identify which probe if multiple probes are colocated.
  */
-void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
+void uprobe_unregister_nosync(struct uprobe *uprobe, struct uprobe_consumer *uc)
 {
 	int err;
 
@@ -1112,12 +1112,15 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
 	/* TODO : cant unregister? schedule a worker thread */
 	if (unlikely(err)) {
 		uprobe_warn(current, "unregister, leaking uprobe");
-		goto out_sync;
+		return;
 	}
 
 	put_uprobe(uprobe);
+}
+EXPORT_SYMBOL_GPL(uprobe_unregister_nosync);
 
-out_sync:
+void uprobe_unregister_sync(void)
+{
 	/*
 	 * Now that handler_chain() and handle_uretprobe_chain() iterate over
 	 * uprobe->consumers list under RCU protection without holding
@@ -1129,7 +1132,7 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
 	 */
 	synchronize_srcu(&uprobes_srcu);
 }
-EXPORT_SYMBOL_GPL(uprobe_unregister);
+EXPORT_SYMBOL_GPL(uprobe_unregister_sync);
 
 /**
  * uprobe_register - register a probe
@@ -1187,7 +1190,13 @@ struct uprobe *uprobe_register(struct inode *inode,
 	up_write(&uprobe->register_rwsem);
 
 	if (ret) {
-		uprobe_unregister(uprobe, uc);
+		uprobe_unregister_nosync(uprobe, uc);
+		/*
+		 * Registration might have partially succeeded, so we can have
+		 * this consumer being called right at this time. We need to
+		 * sync here. It's ok, it's unlikely slow path.
+		 */
+		uprobe_unregister_sync();
 		return ERR_PTR(ret);
 	}
 
diff --git a/kernel/trace/bpf_trace.c b/kernel/trace/bpf_trace.c
index 73c570b5988b..6b632710c98e 100644
--- a/kernel/trace/bpf_trace.c
+++ b/kernel/trace/bpf_trace.c
@@ -3184,7 +3184,10 @@ static void bpf_uprobe_unregister(struct bpf_uprobe *uprobes, u32 cnt)
 	u32 i;
 
 	for (i = 0; i < cnt; i++)
-		uprobe_unregister(uprobes[i].uprobe, &uprobes[i].consumer);
+		uprobe_unregister_nosync(uprobes[i].uprobe, &uprobes[i].consumer);
+
+	if (cnt)
+		uprobe_unregister_sync();
 }
 
 static void bpf_uprobe_multi_link_release(struct bpf_link *link)
diff --git a/kernel/trace/trace_uprobe.c b/kernel/trace/trace_uprobe.c
index 7eb79e0a5352..f7443e996b1b 100644
--- a/kernel/trace/trace_uprobe.c
+++ b/kernel/trace/trace_uprobe.c
@@ -1097,6 +1097,7 @@ static int trace_uprobe_enable(struct trace_uprobe *tu, filter_func_t filter)
 static void __probe_event_disable(struct trace_probe *tp)
 {
 	struct trace_uprobe *tu;
+	bool sync = false;
 
 	tu = container_of(tp, struct trace_uprobe, tp);
 	WARN_ON(!uprobe_filter_is_empty(tu->tp.event->filter));
@@ -1105,9 +1106,12 @@ static void __probe_event_disable(struct trace_probe *tp)
 		if (!tu->uprobe)
 			continue;
 
-		uprobe_unregister(tu->uprobe, &tu->consumer);
+		uprobe_unregister_nosync(tu->uprobe, &tu->consumer);
+		sync = true;
 		tu->uprobe = NULL;
 	}
+	if (sync)
+		uprobe_unregister_sync();
 }
 
 static int probe_event_enable(struct trace_event_call *call,
diff --git a/tools/testing/selftests/bpf/bpf_testmod/bpf_testmod.c b/tools/testing/selftests/bpf/bpf_testmod/bpf_testmod.c
index 3c0515a27842..1fc16657cf42 100644
--- a/tools/testing/selftests/bpf/bpf_testmod/bpf_testmod.c
+++ b/tools/testing/selftests/bpf/bpf_testmod/bpf_testmod.c
@@ -475,7 +475,8 @@ static void testmod_unregister_uprobe(void)
 	mutex_lock(&testmod_uprobe_mutex);
 
 	if (uprobe.uprobe) {
-		uprobe_unregister(uprobe.uprobe, &uprobe.consumer);
+		uprobe_unregister_nosync(uprobe.uprobe, &uprobe.consumer);
+		uprobe_unregister_sync();
 		path_put(&uprobe.path);
 		uprobe.uprobe = NULL;
 	}
-- 
2.43.5



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

* [PATCH v3 06/13] rbtree: provide rb_find_rcu() / rb_find_add_rcu()
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (4 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH v3 05/13] perf/uprobe: split uprobe_unregister() Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 07/13] uprobes: perform lockless SRCU-protected uprobes_tree lookup Andrii Nakryiko
                   ` (7 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

From: Peter Zijlstra <peterz@infradead.org>

Much like latch_tree, add two RCU methods for the regular RB-tree,
which can be used in conjunction with a seqcount to provide lockless
lookups.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Reviewed-by: Masami Hiramatsu (Google) <mhiramat@kernel.org>
---
 include/linux/rbtree.h | 67 ++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 67 insertions(+)

diff --git a/include/linux/rbtree.h b/include/linux/rbtree.h
index f7edca369eda..7c173aa64e1e 100644
--- a/include/linux/rbtree.h
+++ b/include/linux/rbtree.h
@@ -244,6 +244,42 @@ rb_find_add(struct rb_node *node, struct rb_root *tree,
 	return NULL;
 }
 
+/**
+ * rb_find_add_rcu() - find equivalent @node in @tree, or add @node
+ * @node: node to look-for / insert
+ * @tree: tree to search / modify
+ * @cmp: operator defining the node order
+ *
+ * Adds a Store-Release for link_node.
+ *
+ * Returns the rb_node matching @node, or NULL when no match is found and @node
+ * is inserted.
+ */
+static __always_inline struct rb_node *
+rb_find_add_rcu(struct rb_node *node, struct rb_root *tree,
+		int (*cmp)(struct rb_node *, const struct rb_node *))
+{
+	struct rb_node **link = &tree->rb_node;
+	struct rb_node *parent = NULL;
+	int c;
+
+	while (*link) {
+		parent = *link;
+		c = cmp(node, parent);
+
+		if (c < 0)
+			link = &parent->rb_left;
+		else if (c > 0)
+			link = &parent->rb_right;
+		else
+			return parent;
+	}
+
+	rb_link_node_rcu(node, parent, link);
+	rb_insert_color(node, tree);
+	return NULL;
+}
+
 /**
  * rb_find() - find @key in tree @tree
  * @key: key to match
@@ -272,6 +308,37 @@ rb_find(const void *key, const struct rb_root *tree,
 	return NULL;
 }
 
+/**
+ * rb_find_rcu() - find @key in tree @tree
+ * @key: key to match
+ * @tree: tree to search
+ * @cmp: operator defining the node order
+ *
+ * Notably, tree descent vs concurrent tree rotations is unsound and can result
+ * in false-negatives.
+ *
+ * Returns the rb_node matching @key or NULL.
+ */
+static __always_inline struct rb_node *
+rb_find_rcu(const void *key, const struct rb_root *tree,
+	    int (*cmp)(const void *key, const struct rb_node *))
+{
+	struct rb_node *node = tree->rb_node;
+
+	while (node) {
+		int c = cmp(key, node);
+
+		if (c < 0)
+			node = rcu_dereference_raw(node->rb_left);
+		else if (c > 0)
+			node = rcu_dereference_raw(node->rb_right);
+		else
+			return node;
+	}
+
+	return NULL;
+}
+
 /**
  * rb_find_first() - find the first @key in @tree
  * @key: key to match
-- 
2.43.5



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

* [PATCH v3 07/13] uprobes: perform lockless SRCU-protected uprobes_tree lookup
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (5 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH v3 06/13] rbtree: provide rb_find_rcu() / rb_find_add_rcu() Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH v3 08/13] uprobes: switch to RCU Tasks Trace flavor for better performance Andrii Nakryiko
                   ` (6 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

Another big bottleneck to scalablity is uprobe_treelock that's taken in
a very hot path in handle_swbp(). Now that uprobes are SRCU-protected,
take advantage of that and make uprobes_tree RB-tree look up lockless.

To make RB-tree RCU-protected lockless lookup correct, we need to take
into account that such RB-tree lookup can return false negatives if there
are parallel RB-tree modifications (rotations) going on. We use seqcount
lock to detect whether RB-tree changed, and if we find nothing while
RB-tree got modified inbetween, we just retry. If uprobe was found, then
it's guaranteed to be a correct lookup.

With all the lock-avoiding changes done, we get a pretty decent
improvement in performance and scalability of uprobes with number of
CPUs, even though we are still nowhere near linear scalability. This is
due to SRCU not really scaling very well with number of CPUs on
a particular hardware that was used for testing (80-core Intel Xeon Gold
6138 CPU @ 2.00GHz), but also due to the remaning mmap_lock, which is
currently taken to resolve interrupt address to inode+offset and then
uprobe instance. And, of course, uretprobes still need similar RCU to
avoid refcount in the hot path, which will be addressed in the follow up
patches.

Nevertheless, the improvement is good. We used BPF selftest-based
uprobe-nop and uretprobe-nop benchmarks to get the below numbers,
varying number of CPUs on which uprobes and uretprobes are triggered.

BASELINE
========
uprobe-nop      ( 1 cpus):    3.032 ± 0.023M/s  (  3.032M/s/cpu)
uprobe-nop      ( 2 cpus):    3.452 ± 0.005M/s  (  1.726M/s/cpu)
uprobe-nop      ( 4 cpus):    3.663 ± 0.005M/s  (  0.916M/s/cpu)
uprobe-nop      ( 8 cpus):    3.718 ± 0.038M/s  (  0.465M/s/cpu)
uprobe-nop      (16 cpus):    3.344 ± 0.008M/s  (  0.209M/s/cpu)
uprobe-nop      (32 cpus):    2.288 ± 0.021M/s  (  0.071M/s/cpu)
uprobe-nop      (64 cpus):    3.205 ± 0.004M/s  (  0.050M/s/cpu)

uretprobe-nop   ( 1 cpus):    1.979 ± 0.005M/s  (  1.979M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.361 ± 0.005M/s  (  1.180M/s/cpu)
uretprobe-nop   ( 4 cpus):    2.309 ± 0.002M/s  (  0.577M/s/cpu)
uretprobe-nop   ( 8 cpus):    2.253 ± 0.001M/s  (  0.282M/s/cpu)
uretprobe-nop   (16 cpus):    2.007 ± 0.000M/s  (  0.125M/s/cpu)
uretprobe-nop   (32 cpus):    1.624 ± 0.003M/s  (  0.051M/s/cpu)
uretprobe-nop   (64 cpus):    2.149 ± 0.001M/s  (  0.034M/s/cpu)

SRCU CHANGES
============
uprobe-nop      ( 1 cpus):    3.276 ± 0.005M/s  (  3.276M/s/cpu)
uprobe-nop      ( 2 cpus):    4.125 ± 0.002M/s  (  2.063M/s/cpu)
uprobe-nop      ( 4 cpus):    7.713 ± 0.002M/s  (  1.928M/s/cpu)
uprobe-nop      ( 8 cpus):    8.097 ± 0.006M/s  (  1.012M/s/cpu)
uprobe-nop      (16 cpus):    6.501 ± 0.056M/s  (  0.406M/s/cpu)
uprobe-nop      (32 cpus):    4.398 ± 0.084M/s  (  0.137M/s/cpu)
uprobe-nop      (64 cpus):    6.452 ± 0.000M/s  (  0.101M/s/cpu)

uretprobe-nop   ( 1 cpus):    2.055 ± 0.001M/s  (  2.055M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.677 ± 0.000M/s  (  1.339M/s/cpu)
uretprobe-nop   ( 4 cpus):    4.561 ± 0.003M/s  (  1.140M/s/cpu)
uretprobe-nop   ( 8 cpus):    5.291 ± 0.002M/s  (  0.661M/s/cpu)
uretprobe-nop   (16 cpus):    5.065 ± 0.019M/s  (  0.317M/s/cpu)
uretprobe-nop   (32 cpus):    3.622 ± 0.003M/s  (  0.113M/s/cpu)
uretprobe-nop   (64 cpus):    3.723 ± 0.002M/s  (  0.058M/s/cpu)

Peak througput increased from 3.7 mln/s (uprobe triggerings) up to about
8 mln/s. For uretprobes it's a bit more modest with bump from 2.4 mln/s
to 5mln/s.

Suggested-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 kernel/events/uprobes.c | 30 ++++++++++++++++++++++++------
 1 file changed, 24 insertions(+), 6 deletions(-)

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 0b6d4c0a0088..8559ca365679 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -40,6 +40,7 @@ static struct rb_root uprobes_tree = RB_ROOT;
 #define no_uprobe_events()	RB_EMPTY_ROOT(&uprobes_tree)
 
 static DEFINE_RWLOCK(uprobes_treelock);	/* serialize rbtree access */
+static seqcount_rwlock_t uprobes_seqcount = SEQCNT_RWLOCK_ZERO(uprobes_seqcount, &uprobes_treelock);
 
 DEFINE_STATIC_SRCU(uprobes_srcu);
 
@@ -634,8 +635,11 @@ static void put_uprobe(struct uprobe *uprobe)
 
 	write_lock(&uprobes_treelock);
 
-	if (uprobe_is_active(uprobe))
+	if (uprobe_is_active(uprobe)) {
+		write_seqcount_begin(&uprobes_seqcount);
 		rb_erase(&uprobe->rb_node, &uprobes_tree);
+		write_seqcount_end(&uprobes_seqcount);
+	}
 
 	write_unlock(&uprobes_treelock);
 
@@ -701,14 +705,26 @@ static struct uprobe *find_uprobe_rcu(struct inode *inode, loff_t offset)
 		.offset = offset,
 	};
 	struct rb_node *node;
+	unsigned int seq;
 
 	lockdep_assert(srcu_read_lock_held(&uprobes_srcu));
 
-	read_lock(&uprobes_treelock);
-	node = rb_find(&key, &uprobes_tree, __uprobe_cmp_key);
-	read_unlock(&uprobes_treelock);
+	do {
+		seq = read_seqcount_begin(&uprobes_seqcount);
+		node = rb_find_rcu(&key, &uprobes_tree, __uprobe_cmp_key);
+		/*
+		 * Lockless RB-tree lookups can result only in false negatives.
+		 * If the element is found, it is correct and can be returned
+		 * under RCU protection. If we find nothing, we need to
+		 * validate that seqcount didn't change. If it did, we have to
+		 * try again as we might have missed the element (false
+		 * negative). If seqcount is unchanged, search truly failed.
+		 */
+		if (node)
+			return __node_2_uprobe(node);
+	} while (read_seqcount_retry(&uprobes_seqcount, seq));
 
-	return node ? __node_2_uprobe(node) : NULL;
+	return NULL;
 }
 
 /*
@@ -730,7 +746,7 @@ static struct uprobe *__insert_uprobe(struct uprobe *uprobe)
 {
 	struct rb_node *node;
 again:
-	node = rb_find_add(&uprobe->rb_node, &uprobes_tree, __uprobe_cmp);
+	node = rb_find_add_rcu(&uprobe->rb_node, &uprobes_tree, __uprobe_cmp);
 	if (node) {
 		struct uprobe *u = __node_2_uprobe(node);
 
@@ -755,7 +771,9 @@ static struct uprobe *insert_uprobe(struct uprobe *uprobe)
 	struct uprobe *u;
 
 	write_lock(&uprobes_treelock);
+	write_seqcount_begin(&uprobes_seqcount);
 	u = __insert_uprobe(uprobe);
+	write_seqcount_end(&uprobes_seqcount);
 	write_unlock(&uprobes_treelock);
 
 	return u;
-- 
2.43.5



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

* [PATCH v3 08/13] uprobes: switch to RCU Tasks Trace flavor for better performance
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (6 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH v3 07/13] uprobes: perform lockless SRCU-protected uprobes_tree lookup Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
                   ` (5 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

This patch switches uprobes SRCU usage to RCU Tasks Trace flavor, which
is optimized for more lightweight and quick readers (at the expense of
slower writers, which for uprobes is a fine tradeof) and has better
performance and scalability with number of CPUs.

Similarly to baseline vs SRCU, we've benchmarked SRCU-based
implementation vs RCU Tasks Trace implementation.

SRCU
====
uprobe-nop      ( 1 cpus):    3.276 ± 0.005M/s  (  3.276M/s/cpu)
uprobe-nop      ( 2 cpus):    4.125 ± 0.002M/s  (  2.063M/s/cpu)
uprobe-nop      ( 4 cpus):    7.713 ± 0.002M/s  (  1.928M/s/cpu)
uprobe-nop      ( 8 cpus):    8.097 ± 0.006M/s  (  1.012M/s/cpu)
uprobe-nop      (16 cpus):    6.501 ± 0.056M/s  (  0.406M/s/cpu)
uprobe-nop      (32 cpus):    4.398 ± 0.084M/s  (  0.137M/s/cpu)
uprobe-nop      (64 cpus):    6.452 ± 0.000M/s  (  0.101M/s/cpu)

uretprobe-nop   ( 1 cpus):    2.055 ± 0.001M/s  (  2.055M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.677 ± 0.000M/s  (  1.339M/s/cpu)
uretprobe-nop   ( 4 cpus):    4.561 ± 0.003M/s  (  1.140M/s/cpu)
uretprobe-nop   ( 8 cpus):    5.291 ± 0.002M/s  (  0.661M/s/cpu)
uretprobe-nop   (16 cpus):    5.065 ± 0.019M/s  (  0.317M/s/cpu)
uretprobe-nop   (32 cpus):    3.622 ± 0.003M/s  (  0.113M/s/cpu)
uretprobe-nop   (64 cpus):    3.723 ± 0.002M/s  (  0.058M/s/cpu)

RCU Tasks Trace
===============
uprobe-nop      ( 1 cpus):    3.396 ± 0.002M/s  (  3.396M/s/cpu)
uprobe-nop      ( 2 cpus):    4.271 ± 0.006M/s  (  2.135M/s/cpu)
uprobe-nop      ( 4 cpus):    8.499 ± 0.015M/s  (  2.125M/s/cpu)
uprobe-nop      ( 8 cpus):   10.355 ± 0.028M/s  (  1.294M/s/cpu)
uprobe-nop      (16 cpus):    7.615 ± 0.099M/s  (  0.476M/s/cpu)
uprobe-nop      (32 cpus):    4.430 ± 0.007M/s  (  0.138M/s/cpu)
uprobe-nop      (64 cpus):    6.887 ± 0.020M/s  (  0.108M/s/cpu)

uretprobe-nop   ( 1 cpus):    2.174 ± 0.001M/s  (  2.174M/s/cpu)
uretprobe-nop   ( 2 cpus):    2.853 ± 0.001M/s  (  1.426M/s/cpu)
uretprobe-nop   ( 4 cpus):    4.913 ± 0.002M/s  (  1.228M/s/cpu)
uretprobe-nop   ( 8 cpus):    5.883 ± 0.002M/s  (  0.735M/s/cpu)
uretprobe-nop   (16 cpus):    5.147 ± 0.001M/s  (  0.322M/s/cpu)
uretprobe-nop   (32 cpus):    3.738 ± 0.008M/s  (  0.117M/s/cpu)
uretprobe-nop   (64 cpus):    4.397 ± 0.002M/s  (  0.069M/s/cpu)

Peak throughput for uprobes increases from 8 mln/s to 10.3 mln/s
(+28%!), and for uretprobes from 5.3 mln/s to 5.8 mln/s (+11%), as we
have more work to do on uretprobes side.

Even single-thread (no contention) performance is slightly better: 3.276
mln/s to 3.396 mln/s (+3.5%) for uprobes, and 2.055 mln/s to 2.174 mln/s
(+5.8%) for uretprobes.

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 kernel/events/uprobes.c | 37 +++++++++++++++----------------------
 1 file changed, 15 insertions(+), 22 deletions(-)

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 8559ca365679..0480ad841942 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -42,8 +42,6 @@ static struct rb_root uprobes_tree = RB_ROOT;
 static DEFINE_RWLOCK(uprobes_treelock);	/* serialize rbtree access */
 static seqcount_rwlock_t uprobes_seqcount = SEQCNT_RWLOCK_ZERO(uprobes_seqcount, &uprobes_treelock);
 
-DEFINE_STATIC_SRCU(uprobes_srcu);
-
 #define UPROBES_HASH_SZ	13
 /* serialize uprobe->pending_list */
 static struct mutex uprobes_mmap_mutex[UPROBES_HASH_SZ];
@@ -652,7 +650,7 @@ static void put_uprobe(struct uprobe *uprobe)
 	delayed_uprobe_remove(uprobe, NULL);
 	mutex_unlock(&delayed_uprobe_lock);
 
-	call_srcu(&uprobes_srcu, &uprobe->rcu, uprobe_free_rcu);
+	call_rcu_tasks_trace(&uprobe->rcu, uprobe_free_rcu);
 }
 
 static __always_inline
@@ -707,7 +705,7 @@ static struct uprobe *find_uprobe_rcu(struct inode *inode, loff_t offset)
 	struct rb_node *node;
 	unsigned int seq;
 
-	lockdep_assert(srcu_read_lock_held(&uprobes_srcu));
+	lockdep_assert(rcu_read_lock_trace_held());
 
 	do {
 		seq = read_seqcount_begin(&uprobes_seqcount);
@@ -924,8 +922,7 @@ static bool filter_chain(struct uprobe *uprobe, struct mm_struct *mm)
 	bool ret = false;
 
 	down_read(&uprobe->consumer_rwsem);
-	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
-				 srcu_read_lock_held(&uprobes_srcu)) {
+	list_for_each_entry_rcu(uc, &uprobe->consumers, cons_node, rcu_read_lock_trace_held()) {
 		ret = consumer_filter(uc, mm);
 		if (ret)
 			break;
@@ -1148,7 +1145,7 @@ void uprobe_unregister_sync(void)
 	 * unlucky enough caller can free consumer's memory and cause
 	 * handler_chain() or handle_uretprobe_chain() to do an use-after-free.
 	 */
-	synchronize_srcu(&uprobes_srcu);
+	synchronize_rcu_tasks_trace();
 }
 EXPORT_SYMBOL_GPL(uprobe_unregister_sync);
 
@@ -1232,19 +1229,18 @@ EXPORT_SYMBOL_GPL(uprobe_register);
 int uprobe_apply(struct uprobe *uprobe, struct uprobe_consumer *uc, bool add)
 {
 	struct uprobe_consumer *con;
-	int ret = -ENOENT, srcu_idx;
+	int ret = -ENOENT;
 
 	down_write(&uprobe->register_rwsem);
 
-	srcu_idx = srcu_read_lock(&uprobes_srcu);
-	list_for_each_entry_srcu(con, &uprobe->consumers, cons_node,
-				 srcu_read_lock_held(&uprobes_srcu)) {
+	rcu_read_lock_trace();
+	list_for_each_entry_rcu(con, &uprobe->consumers, cons_node, rcu_read_lock_trace_held()) {
 		if (con == uc) {
 			ret = register_for_each_vma(uprobe, add ? uc : NULL);
 			break;
 		}
 	}
-	srcu_read_unlock(&uprobes_srcu, srcu_idx);
+	rcu_read_unlock_trace();
 
 	up_write(&uprobe->register_rwsem);
 
@@ -2114,8 +2110,7 @@ static void handler_chain(struct uprobe *uprobe, struct pt_regs *regs)
 
 	current->utask->auprobe = &uprobe->arch;
 
-	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
-				 srcu_read_lock_held(&uprobes_srcu)) {
+	list_for_each_entry_rcu(uc, &uprobe->consumers, cons_node, rcu_read_lock_trace_held()) {
 		int rc = 0;
 
 		if (uc->handler) {
@@ -2153,15 +2148,13 @@ handle_uretprobe_chain(struct return_instance *ri, struct pt_regs *regs)
 {
 	struct uprobe *uprobe = ri->uprobe;
 	struct uprobe_consumer *uc;
-	int srcu_idx;
 
-	srcu_idx = srcu_read_lock(&uprobes_srcu);
-	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
-				 srcu_read_lock_held(&uprobes_srcu)) {
+	rcu_read_lock_trace();
+	list_for_each_entry_rcu(uc, &uprobe->consumers, cons_node, rcu_read_lock_trace_held()) {
 		if (uc->ret_handler)
 			uc->ret_handler(uc, ri->func, regs);
 	}
-	srcu_read_unlock(&uprobes_srcu, srcu_idx);
+	rcu_read_unlock_trace();
 }
 
 static struct return_instance *find_next_ret_chain(struct return_instance *ri)
@@ -2246,13 +2239,13 @@ static void handle_swbp(struct pt_regs *regs)
 {
 	struct uprobe *uprobe;
 	unsigned long bp_vaddr;
-	int is_swbp, srcu_idx;
+	int is_swbp;
 
 	bp_vaddr = uprobe_get_swbp_addr(regs);
 	if (bp_vaddr == uprobe_get_trampoline_vaddr())
 		return uprobe_handle_trampoline(regs);
 
-	srcu_idx = srcu_read_lock(&uprobes_srcu);
+	rcu_read_lock_trace();
 
 	uprobe = find_active_uprobe_rcu(bp_vaddr, &is_swbp);
 	if (!uprobe) {
@@ -2310,7 +2303,7 @@ static void handle_swbp(struct pt_regs *regs)
 
 out:
 	/* arch_uprobe_skip_sstep() succeeded, or restart if can't singlestep */
-	srcu_read_unlock(&uprobes_srcu, srcu_idx);
+	rcu_read_unlock_trace();
 }
 
 /*
-- 
2.43.5



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

* [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (7 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH v3 08/13] uprobes: switch to RCU Tasks Trace flavor for better performance Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-19 13:41   ` Oleg Nesterov
  2024-08-13  4:29 ` [PATCH RFC v3 10/13] uprobes: implement SRCU-protected lifetime for single-stepped uprobe Andrii Nakryiko
                   ` (4 subsequent siblings)
  13 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

Avoid taking refcount on uprobe in prepare_uretprobe(), instead take
uretprobe-specific SRCU lock and keep it active as kernel transfers
control back to user space.

Given we can't rely on user space returning from traced function within
reasonable time period, we need to make sure not to keep SRCU lock
active for too long, though. To that effect, we employ a timer callback
which is meant to terminate SRCU lock region after predefined timeout
(currently set to 100ms), and instead transfer underlying struct
uprobe's lifetime protection to refcounting.

This fallback to less scalable refcounting after 100ms is a fine
tradeoff from uretprobe's scalability and performance perspective,
because uretprobing long running user functions inherently doesn't run
into scalability issues (there is just not enough frequency to cause
noticeable issues with either performance or scalability).

The overall trick is in ensuring synchronization between current thread
and timer's callback fired on some other thread. To cope with that with
minimal logic complications, we add hprobe wrapper which is used to
contain all the racy and synchronization related issues behind a small
number of basic helpers: hprobe_expire() and a matching pair of
hprobe_consume() and hprobe_finalize(). Other than that whatever current
thread's logic is there stays the same, as timer thread cannot modify
return_instance state (or add new/remove old return_instances). It only
takes care of SRCU unlock and uprobe refcounting, which is hidden from
the higher-level uretprobe handling logic.

We use atomic xchg() in hprobe_consume(), which is called from
performance critical handle_uretprobe_chain() function run in the
current context. When uncontended, this xchg() doesn't seem to hurt
performance as there are no other competing CPUs fighting for the same
cache line.  We also mark struct return_instance as
____cacheline_aligned to ensure no false sharing can happen.

Another technical moment, we need to make sure that the list of return
instances can be safely traversed under RCU from timer callback, so we
delay return_instance freeing with kfree_rcu() and make sure that list
modifications use RCU-aware operations.

Also, given SRCU lock survives transition from kernel to user space and
back we need to use lower-level __srcu_read_lock() and
__srcu_read_unlock() to avoid lockdep complaining.

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 include/linux/uprobes.h |  49 ++++++-
 kernel/events/uprobes.c | 294 ++++++++++++++++++++++++++++++++++------
 2 files changed, 301 insertions(+), 42 deletions(-)

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index e41cdae5597b..9a0aa0b2a5fe 100644
--- a/include/linux/uprobes.h
+++ b/include/linux/uprobes.h
@@ -15,6 +15,7 @@
 #include <linux/rbtree.h>
 #include <linux/types.h>
 #include <linux/wait.h>
+#include <linux/timer.h>
 
 struct uprobe;
 struct vm_area_struct;
@@ -48,6 +49,45 @@ enum uprobe_task_state {
 	UTASK_SSTEP_TRAPPED,
 };
 
+/*
+ * Hybrid lifetime uprobe. Represents a uprobe instance that could be either
+ * SRCU protected (with SRCU protection eventually potentially timing out),
+ * refcounted using uprobe->ref, or there could be no valid uprobe (NULL).
+ *
+ * hprobe's internal state is setup such that background timer thread can
+ * atomically "downgrade" temporarily RCU-protected uprobe into refcounted one
+ * (or no uprobe, if refcounting failed).
+ *
+ * *stable* pointer always point to the uprobe (or could be NULL if there is
+ * was no valid underlying uprobe to begin with).
+ *
+ * *leased* pointer is the key to achieving race-free atomic lifetime state
+ * transition and can have three possible states:
+ *   - either the same non-NULL value as *stable*, in which case uprobe is
+ *     SRCU-protected;
+ *   - NULL, in which case uprobe (if there is any) is refcounted;
+ *   - special __UPROBE_DEAD value, which represents an uprobe that was SRCU
+ *     protected initially, but SRCU period timed out and we attempted to
+ *     convert it to refcounted, but refcount_inc_not_zero() failed, because
+ *     uprobe effectively went away (the last consumer unsubscribed). In this
+ *     case it's important to know that *stable* pointer (which still has
+ *     non-NULL uprobe pointer) shouldn't be used, because lifetime of
+ *     underlying uprobe is not guaranteed anymore. __UPROBE_DEAD is just an
+ *     internal marker and is handled transparently by hprobe_fetch() helper.
+ *
+ * When uprobe is SRCU-protected, we also record srcu_idx value, necessary for
+ * SRCU unlocking.
+ *
+ * See hprobe_expire() and hprobe_fetch() for details of race-free uprobe
+ * state transitioning details. It all hinges on atomic xchg() over *leaded*
+ * pointer. *stable* pointer, once initially set, is not modified concurrently.
+ */
+struct hprobe {
+	struct uprobe *stable;
+	struct uprobe *leased;
+	int srcu_idx;
+};
+
 /*
  * uprobe_task: Metadata of a task while it singlesteps.
  */
@@ -68,6 +108,7 @@ struct uprobe_task {
 	};
 
 	struct uprobe			*active_uprobe;
+	struct timer_list		ri_timer;
 	unsigned long			xol_vaddr;
 
 	struct arch_uprobe              *auprobe;
@@ -77,14 +118,18 @@ struct uprobe_task {
 };
 
 struct return_instance {
-	struct uprobe		*uprobe;
 	unsigned long		func;
 	unsigned long		stack;		/* stack pointer */
 	unsigned long		orig_ret_vaddr; /* original return address */
 	bool			chained;	/* true, if instance is nested */
 
 	struct return_instance	*next;		/* keep as stack */
-};
+
+	union {
+		struct hprobe		hprobe;
+		struct rcu_head		rcu;
+	};
+} ____cacheline_aligned;
 
 enum rp_check {
 	RP_CHECK_CALL,
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 0480ad841942..26acd06871e6 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -26,6 +26,7 @@
 #include <linux/task_work.h>
 #include <linux/shmem_fs.h>
 #include <linux/khugepaged.h>
+#include <linux/srcu.h>
 
 #include <linux/uprobes.h>
 
@@ -49,6 +50,9 @@ static struct mutex uprobes_mmap_mutex[UPROBES_HASH_SZ];
 
 DEFINE_STATIC_PERCPU_RWSEM(dup_mmap_sem);
 
+/* Covers return_instance's uprobe lifetime. */
+DEFINE_STATIC_SRCU(uretprobes_srcu);
+
 /* Have a copy of original instruction */
 #define UPROBE_COPY_INSN	0
 
@@ -594,13 +598,6 @@ set_orig_insn(struct arch_uprobe *auprobe, struct mm_struct *mm, unsigned long v
 			*(uprobe_opcode_t *)&auprobe->insn);
 }
 
-/* uprobe should have guaranteed positive refcount */
-static struct uprobe *get_uprobe(struct uprobe *uprobe)
-{
-	refcount_inc(&uprobe->ref);
-	return uprobe;
-}
-
 /*
  * uprobe should have guaranteed lifetime, which can be either of:
  *   - caller already has refcount taken (and wants an extra one);
@@ -619,13 +616,20 @@ static inline bool uprobe_is_active(struct uprobe *uprobe)
 	return !RB_EMPTY_NODE(&uprobe->rb_node);
 }
 
-static void uprobe_free_rcu(struct rcu_head *rcu)
+static void uprobe_free_rcu_tasks_trace(struct rcu_head *rcu)
 {
 	struct uprobe *uprobe = container_of(rcu, struct uprobe, rcu);
 
 	kfree(uprobe);
 }
 
+static void uprobe_free_srcu(struct rcu_head *rcu)
+{
+	struct uprobe *uprobe = container_of(rcu, struct uprobe, rcu);
+
+	call_rcu_tasks_trace(&uprobe->rcu, uprobe_free_rcu_tasks_trace);
+}
+
 static void put_uprobe(struct uprobe *uprobe)
 {
 	if (!refcount_dec_and_test(&uprobe->ref))
@@ -650,7 +654,146 @@ static void put_uprobe(struct uprobe *uprobe)
 	delayed_uprobe_remove(uprobe, NULL);
 	mutex_unlock(&delayed_uprobe_lock);
 
-	call_rcu_tasks_trace(&uprobe->rcu, uprobe_free_rcu);
+	/* start srcu -> rcu_tasks_trace -> kfree chain */
+	call_srcu(&uretprobes_srcu, &uprobe->rcu, uprobe_free_srcu);
+}
+
+/*
+ * Special marker pointer for when ri_timer() expired, unlocking RCU, but
+ * failed to acquire refcount on uprobe (because it doesn't have any
+ * associated consumer anymore, for example). In such case it's important for
+ * hprobe_consume() to return NULL uprobe, instead of "stable" uprobe pointer,
+ * as that one isn't protected by either refcount nor RCU region now.
+ */
+#define __UPROBE_DEAD ((struct uprobe *)(-0xdead))
+
+#define RI_TIMER_PERIOD (HZ/10) /* 100 ms */
+
+/* Initialize hprobe as SRCU-protected "leased" uprobe */
+static void hprobe_init_leased(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
+{
+	hprobe->srcu_idx = srcu_idx;
+	hprobe->stable = uprobe;
+	hprobe->leased = uprobe;
+}
+
+/* Initialize hprobe as refcounted ("stable") uprobe (uprobe can be NULL). */
+static void hprobe_init_stable(struct hprobe *hprobe, struct uprobe *uprobe)
+{
+	hprobe->srcu_idx = -1;
+	hprobe->stable = uprobe;
+	hprobe->leased = NULL;
+}
+
+/*
+ * hprobe_consume() fetches hprobe's underlying uprobe and detects whether
+ * uprobe is still SRCU protected, or is refcounted. hprobe_consume() can be
+ * used only once for a given hprobe.
+ *
+ * Caller has to perform SRCU unlock if under_rcu is set to true;
+ * otherwise, either properly refcounted uprobe is returned or NULL.
+ */
+static inline struct uprobe *hprobe_consume(struct hprobe *hprobe, bool *under_rcu)
+{
+	struct uprobe *uprobe;
+
+	uprobe = xchg(&hprobe->leased, NULL);
+	if (uprobe) {
+		if (unlikely(uprobe == __UPROBE_DEAD)) {
+			*under_rcu = false;
+			return NULL;
+		}
+
+		*under_rcu = true;
+		return uprobe;
+	}
+
+	*under_rcu = false;
+	return hprobe->stable;
+}
+
+/*
+ * Reset hprobe state and, if under_rcu is true, release SRCU lock.
+ * hprobe_finalize() can only be used from current context after
+ * hprobe_consume() call (which determines uprobe and under_rcu value).
+ */
+static void hprobe_finalize(struct hprobe *hprobe, struct uprobe *uprobe, bool under_rcu)
+{
+	if (under_rcu)
+		__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
+	else if (uprobe)
+		put_uprobe(uprobe);
+	/* prevent free_ret_instance() from double-putting uprobe */
+	hprobe->stable = NULL;
+}
+
+/*
+ * Attempt to switch (atomically) uprobe from being RCU protected ("leased")
+ * to refcounted ("stable") state. Competes with hprobe_consume(), only one of
+ * them can win the race to perform SRCU unlocking. Whoever wins must perform
+ * SRCU unlock.
+ *
+ * Returns underlying valid uprobe or NULL, if there was no underlying uprobe
+ * to begin with or we failed to bump its refcount and it's going away.
+ *
+ * Returned non-NULL uprobe can be still safely used within an ongoing SRCU
+ * locked region. It's not guaranteed that returned uprobe has a positive
+ * refcount, so caller has to attempt try_get_uprobe(), if it needs to use
+ * returned uprobe instance beyond ongoing SRCU lock region. See dup_utask().
+ */
+static struct uprobe* hprobe_expire(struct hprobe *hprobe)
+{
+	struct uprobe *uprobe;
+
+	/*
+	 * return_instance's hprobe is protected by RCU.
+	 * Underlying uprobe is itself protected from reuse by SRCU.
+	 */
+	lockdep_assert(rcu_read_lock_held() && srcu_read_lock_held(&uretprobes_srcu));
+
+	/*
+	 * Leased pointer can only be NULL, __UPROBE_DEAD, or some valid uprobe
+	 * pointer. This pointer can only be updated to NULL or __UPROBE_DEAD,
+	 * not any other valid uprobe pointer. So it's safe to fetch it with
+	 * READ_ONCE() and try to refcount it, if it's not NULL or __UPROBE_DEAD.
+	 */
+	uprobe = data_race(READ_ONCE(hprobe->leased));
+	if (!uprobe || uprobe == __UPROBE_DEAD)
+		return NULL;
+
+	if (!try_get_uprobe(uprobe)) {
+		/*
+		 * hprobe_consume() might have xchg()'ed to NULL already,
+		 * in which case we shouldn't set __UPROBE_DEAD.
+		 */
+		cmpxchg(&hprobe->leased, uprobe, __UPROBE_DEAD);
+		return NULL;
+	}
+
+	/*
+	 * Even if hprobe_consume() won and unlocked SRCU, we still have
+	 * a guarantee that uprobe won't be freed (and thus won't be reused)
+	 * because out caller maintains its own SRCU locked region.
+	 * So cmpxchg() below is well-formed.
+	 */
+	if (cmpxchg(&hprobe->leased, uprobe, NULL)) {
+		/*
+		 * At this point uprobe is properly refcounted, so it's safe
+		 * to end its original SRCU locked region.
+		 */
+		__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
+		return uprobe;
+	}
+
+	/* We lost the race, undo our refcount bump. It can drop to zero. */
+	put_uprobe(uprobe);
+
+	/*
+	 * We return underlying uprobe nevertheless because it's still valid
+	 * until the end of current SRCU locked region, and can be used to
+	 * try_get_uprobe(). This is used in dup_utask().
+	 */
+	return uprobe;
 }
 
 static __always_inline
@@ -1727,11 +1870,18 @@ unsigned long uprobe_get_trap_addr(struct pt_regs *regs)
 	return instruction_pointer(regs);
 }
 
-static struct return_instance *free_ret_instance(struct return_instance *ri)
+static struct return_instance *free_ret_instance(struct return_instance *ri, bool cleanup_hprobe)
 {
 	struct return_instance *next = ri->next;
-	put_uprobe(ri->uprobe);
-	kfree(ri);
+	struct uprobe *uprobe;
+	bool under_rcu;
+
+	if (cleanup_hprobe) {
+		uprobe = hprobe_consume(&ri->hprobe, &under_rcu);
+		hprobe_finalize(&ri->hprobe, uprobe, under_rcu);
+	}
+
+	kfree_rcu(ri, rcu);
 	return next;
 }
 
@@ -1747,18 +1897,51 @@ void uprobe_free_utask(struct task_struct *t)
 	if (!utask)
 		return;
 
+	timer_delete_sync(&utask->ri_timer);
+
 	if (utask->active_uprobe)
 		put_uprobe(utask->active_uprobe);
 
 	ri = utask->return_instances;
 	while (ri)
-		ri = free_ret_instance(ri);
+		ri = free_ret_instance(ri, true /* cleanup_hprobe */);
 
 	xol_free_insn_slot(t);
 	kfree(utask);
 	t->utask = NULL;
 }
 
+#define for_each_ret_instance_rcu(pos, head) \
+	for (pos = rcu_dereference_raw(head); pos; pos = rcu_dereference_raw(pos->next))
+
+static void ri_timer(struct timer_list *timer)
+{
+	struct uprobe_task *utask = container_of(timer, struct uprobe_task, ri_timer);
+	struct return_instance *ri;
+
+	/* SRCU protects uprobe from reuse for the cmpxchg() inside hprobe_expire(). */
+	guard(srcu)(&uretprobes_srcu);
+	/* RCU protects return_instance from freeing. */
+	guard(rcu)();
+
+	for_each_ret_instance_rcu(ri, utask->return_instances) {
+		hprobe_expire(&ri->hprobe);
+	}
+}
+
+static struct uprobe_task *alloc_utask(void)
+{
+	struct uprobe_task *utask;
+
+	utask = kzalloc(sizeof(*utask), GFP_KERNEL);
+	if (!utask)
+		return NULL;
+
+	timer_setup(&utask->ri_timer, ri_timer, 0);
+
+	return utask;
+}
+
 /*
  * Allocate a uprobe_task object for the task if necessary.
  * Called when the thread hits a breakpoint.
@@ -1770,7 +1953,7 @@ void uprobe_free_utask(struct task_struct *t)
 static struct uprobe_task *get_utask(void)
 {
 	if (!current->utask)
-		current->utask = kzalloc(sizeof(struct uprobe_task), GFP_KERNEL);
+		current->utask = alloc_utask();
 	return current->utask;
 }
 
@@ -1778,12 +1961,16 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
 {
 	struct uprobe_task *n_utask;
 	struct return_instance **p, *o, *n;
+	struct uprobe *uprobe;
 
-	n_utask = kzalloc(sizeof(struct uprobe_task), GFP_KERNEL);
+	n_utask = alloc_utask();
 	if (!n_utask)
 		return -ENOMEM;
 	t->utask = n_utask;
 
+	/* protect uprobes from freeing, we'll need try_get_uprobe() them */
+	guard(srcu)(&uretprobes_srcu);
+
 	p = &n_utask->return_instances;
 	for (o = o_utask->return_instances; o; o = o->next) {
 		n = kmalloc(sizeof(struct return_instance), GFP_KERNEL);
@@ -1791,17 +1978,24 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
 			return -ENOMEM;
 
 		*n = *o;
+
+		/* see hprobe_expire() comments */
+		uprobe = hprobe_expire(&o->hprobe);
+		if (uprobe) /* refcount bump for new utask */
+			uprobe = try_get_uprobe(uprobe);
+
 		/*
-		 * uprobe's refcnt has to be positive at this point, kept by
-		 * utask->return_instances items; return_instances can't be
-		 * removed right now, as task is blocked due to duping; so
-		 * get_uprobe() is safe to use here.
+		 * New utask will have stable properly refcounted uprobe or NULL.
+		 * Even if we failed to get refcounted uprobe, we still need
+		 * to preserve full set of return_instances for proper
+		 * uretprobe handling and nesting in forked task.
 		 */
-		get_uprobe(n->uprobe);
-		n->next = NULL;
+		hprobe_init_stable(&n->hprobe, uprobe);
 
-		*p = n;
+		n->next = NULL;
+		rcu_assign_pointer(*p, n);
 		p = &n->next;
+
 		n_utask->depth++;
 	}
 
@@ -1877,10 +2071,10 @@ static void cleanup_return_instances(struct uprobe_task *utask, bool chained,
 	enum rp_check ctx = chained ? RP_CHECK_CHAIN_CALL : RP_CHECK_CALL;
 
 	while (ri && !arch_uretprobe_is_alive(ri, ctx, regs)) {
-		ri = free_ret_instance(ri);
+		ri = free_ret_instance(ri, true /* cleanup_hprobe */);
 		utask->depth--;
 	}
-	utask->return_instances = ri;
+	rcu_assign_pointer(utask->return_instances, ri);
 }
 
 static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
@@ -1889,6 +2083,7 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 	struct uprobe_task *utask;
 	unsigned long orig_ret_vaddr, trampoline_vaddr;
 	bool chained;
+	int srcu_idx;
 
 	if (!get_xol_area())
 		return;
@@ -1904,10 +2099,6 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 		return;
 	}
 
-	/* we need to bump refcount to store uprobe in utask */
-	if (!try_get_uprobe(uprobe))
-		return;
-
 	ri = kmalloc(sizeof(struct return_instance), GFP_KERNEL);
 	if (!ri)
 		goto fail;
@@ -1937,20 +2128,36 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 		}
 		orig_ret_vaddr = utask->return_instances->orig_ret_vaddr;
 	}
-	ri->uprobe = uprobe;
+
+	/* __srcu_read_lock() because SRCU lock survives switch to user space */
+	srcu_idx = __srcu_read_lock(&uretprobes_srcu);
+
 	ri->func = instruction_pointer(regs);
 	ri->stack = user_stack_pointer(regs);
 	ri->orig_ret_vaddr = orig_ret_vaddr;
 	ri->chained = chained;
 
 	utask->depth++;
+
+	hprobe_init_leased(&ri->hprobe, uprobe, srcu_idx);
 	ri->next = utask->return_instances;
-	utask->return_instances = ri;
+	rcu_assign_pointer(utask->return_instances, ri);
+
+	/*
+	 * Don't reschedule if timer is already active. This way we have
+	 * a guaranteed cap on maximum timer period (SRCU expiration duration)
+	 * regardless of how long and well-timed uretprobe chain user space
+	 * might cause. At worst we'll just have a few extra inconsequential
+	 * refcount bumps even if we could, technically, get away with just an
+	 * SRCU lock. On the other hand, we get timer expiration logic
+	 * triggered and tested regularly even for very short-running uretprobes.
+	 */
+	if (!timer_pending(&utask->ri_timer))
+		mod_timer(&utask->ri_timer, jiffies + RI_TIMER_PERIOD);
 
 	return;
 fail:
 	kfree(ri);
-	put_uprobe(uprobe);
 }
 
 /* Prepare to single-step probed instruction out of line. */
@@ -2144,11 +2351,14 @@ static void handler_chain(struct uprobe *uprobe, struct pt_regs *regs)
 }
 
 static void
-handle_uretprobe_chain(struct return_instance *ri, struct pt_regs *regs)
+handle_uretprobe_chain(struct return_instance *ri, struct uprobe *uprobe, struct pt_regs *regs)
 {
-	struct uprobe *uprobe = ri->uprobe;
 	struct uprobe_consumer *uc;
 
+	/* all consumers unsubscribed meanwhile */
+	if (unlikely(!uprobe))
+		return;
+
 	rcu_read_lock_trace();
 	list_for_each_entry_rcu(uc, &uprobe->consumers, cons_node, rcu_read_lock_trace_held()) {
 		if (uc->ret_handler)
@@ -2173,7 +2383,8 @@ void uprobe_handle_trampoline(struct pt_regs *regs)
 {
 	struct uprobe_task *utask;
 	struct return_instance *ri, *next;
-	bool valid;
+	struct uprobe *uprobe;
+	bool valid, under_rcu;
 
 	utask = current->utask;
 	if (!utask)
@@ -2203,21 +2414,24 @@ void uprobe_handle_trampoline(struct pt_regs *regs)
 			 * trampoline addresses on the stack are replaced with correct
 			 * original return addresses
 			 */
-			utask->return_instances = ri->next;
+			rcu_assign_pointer(utask->return_instances, ri->next);
+
+			uprobe = hprobe_consume(&ri->hprobe, &under_rcu);
 			if (valid)
-				handle_uretprobe_chain(ri, regs);
-			ri = free_ret_instance(ri);
+				handle_uretprobe_chain(ri, uprobe, regs);
+			hprobe_finalize(&ri->hprobe, uprobe, under_rcu);
+
+			/* We already took care of hprobe, no need to waste more time on that. */
+			ri = free_ret_instance(ri, false /* !cleanup_hprobe */);
 			utask->depth--;
 		} while (ri != next);
 	} while (!valid);
 
-	utask->return_instances = ri;
 	return;
 
- sigill:
+sigill:
 	uprobe_warn(current, "handle uretprobe, sending SIGILL.");
 	force_sig(SIGILL);
-
 }
 
 bool __weak arch_uprobe_ignore(struct arch_uprobe *aup, struct pt_regs *regs)
-- 
2.43.5



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

* [PATCH RFC v3 10/13] uprobes: implement SRCU-protected lifetime for single-stepped uprobe
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (8 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH RFC v3 11/13] mm: introduce mmap_lock_speculation_{start|end} Andrii Nakryiko
                   ` (3 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

Similarly to how we SRCU-protect uprobe instance (and avoid refcounting
it unnecessarily) when waiting for return probe hit, use hprobe approach
to do the same with single-stepped uprobe. Same hprobe_* primitives are
used. We also reuse ri_timer() callback to expire both pending
single-step uprobe and return instances.

Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 include/linux/uprobes.h |  4 +--
 kernel/events/uprobes.c | 56 ++++++++++++++++++++++++-----------------
 2 files changed, 35 insertions(+), 25 deletions(-)

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index 9a0aa0b2a5fe..cc9d3cb055b5 100644
--- a/include/linux/uprobes.h
+++ b/include/linux/uprobes.h
@@ -107,7 +107,7 @@ struct uprobe_task {
 		};
 	};
 
-	struct uprobe			*active_uprobe;
+	struct hprobe			active_hprobe;
 	struct timer_list		ri_timer;
 	unsigned long			xol_vaddr;
 
@@ -115,7 +115,7 @@ struct uprobe_task {
 
 	struct return_instance		*return_instances;
 	unsigned int			depth;
-};
+} ____cacheline_aligned;
 
 struct return_instance {
 	unsigned long		func;
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 26acd06871e6..713824c8ca77 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -1860,11 +1860,16 @@ unsigned long __weak uprobe_get_swbp_addr(struct pt_regs *regs)
 	return instruction_pointer(regs) - UPROBE_SWBP_INSN_SIZE;
 }
 
+static bool utask_has_pending_sstep_uprobe(struct uprobe_task *utask)
+{
+	return utask->active_hprobe.stable != NULL;
+}
+
 unsigned long uprobe_get_trap_addr(struct pt_regs *regs)
 {
 	struct uprobe_task *utask = current->utask;
 
-	if (unlikely(utask && utask->active_uprobe))
+	if (unlikely(utask && utask_has_pending_sstep_uprobe(utask)))
 		return utask->vaddr;
 
 	return instruction_pointer(regs);
@@ -1893,14 +1898,17 @@ void uprobe_free_utask(struct task_struct *t)
 {
 	struct uprobe_task *utask = t->utask;
 	struct return_instance *ri;
+	struct uprobe *uprobe;
+	bool under_rcu;
 
 	if (!utask)
 		return;
 
 	timer_delete_sync(&utask->ri_timer);
 
-	if (utask->active_uprobe)
-		put_uprobe(utask->active_uprobe);
+	/* clean up pending single-stepped uprobe */
+	uprobe = hprobe_consume(&utask->active_hprobe, &under_rcu);
+	hprobe_finalize(&utask->active_hprobe, uprobe, under_rcu);
 
 	ri = utask->return_instances;
 	while (ri)
@@ -1924,6 +1932,8 @@ static void ri_timer(struct timer_list *timer)
 	/* RCU protects return_instance from freeing. */
 	guard(rcu)();
 
+	hprobe_expire(&utask->active_hprobe);
+
 	for_each_ret_instance_rcu(ri, utask->return_instances) {
 		hprobe_expire(&ri->hprobe);
 	}
@@ -2166,20 +2176,15 @@ pre_ssout(struct uprobe *uprobe, struct pt_regs *regs, unsigned long bp_vaddr)
 {
 	struct uprobe_task *utask;
 	unsigned long xol_vaddr;
-	int err;
+	int err, srcu_idx;
 
 	utask = get_utask();
 	if (!utask)
 		return -ENOMEM;
 
-	if (!try_get_uprobe(uprobe))
-		return -EINVAL;
-
 	xol_vaddr = xol_get_insn_slot(uprobe);
-	if (!xol_vaddr) {
-		err = -ENOMEM;
-		goto err_out;
-	}
+	if (!xol_vaddr)
+		return -ENOMEM;
 
 	utask->xol_vaddr = xol_vaddr;
 	utask->vaddr = bp_vaddr;
@@ -2187,15 +2192,18 @@ pre_ssout(struct uprobe *uprobe, struct pt_regs *regs, unsigned long bp_vaddr)
 	err = arch_uprobe_pre_xol(&uprobe->arch, regs);
 	if (unlikely(err)) {
 		xol_free_insn_slot(current);
-		goto err_out;
+		return err;
 	}
 
-	utask->active_uprobe = uprobe;
+	srcu_idx = __srcu_read_lock(&uretprobes_srcu);
+
+	hprobe_init_leased(&utask->active_hprobe, uprobe, srcu_idx);
 	utask->state = UTASK_SSTEP;
+
+	if (!timer_pending(&utask->ri_timer))
+		mod_timer(&utask->ri_timer, jiffies + RI_TIMER_PERIOD);
+
 	return 0;
-err_out:
-	put_uprobe(uprobe);
-	return err;
 }
 
 /*
@@ -2212,7 +2220,7 @@ bool uprobe_deny_signal(void)
 	struct task_struct *t = current;
 	struct uprobe_task *utask = t->utask;
 
-	if (likely(!utask || !utask->active_uprobe))
+	if (likely(!utask || !utask_has_pending_sstep_uprobe(utask)))
 		return false;
 
 	WARN_ON_ONCE(utask->state != UTASK_SSTEP);
@@ -2528,8 +2536,10 @@ static void handle_singlestep(struct uprobe_task *utask, struct pt_regs *regs)
 {
 	struct uprobe *uprobe;
 	int err = 0;
+	bool under_rcu;
+
+	uprobe = hprobe_consume(&utask->active_hprobe, &under_rcu);
 
-	uprobe = utask->active_uprobe;
 	if (utask->state == UTASK_SSTEP_ACK)
 		err = arch_uprobe_post_xol(&uprobe->arch, regs);
 	else if (utask->state == UTASK_SSTEP_TRAPPED)
@@ -2537,8 +2547,8 @@ static void handle_singlestep(struct uprobe_task *utask, struct pt_regs *regs)
 	else
 		WARN_ON_ONCE(1);
 
-	put_uprobe(uprobe);
-	utask->active_uprobe = NULL;
+	hprobe_finalize(&utask->active_hprobe, uprobe, under_rcu);
+
 	utask->state = UTASK_RUNNING;
 	xol_free_insn_slot(current);
 
@@ -2556,7 +2566,7 @@ static void handle_singlestep(struct uprobe_task *utask, struct pt_regs *regs)
 /*
  * On breakpoint hit, breakpoint notifier sets the TIF_UPROBE flag and
  * allows the thread to return from interrupt. After that handle_swbp()
- * sets utask->active_uprobe.
+ * sets utask->active_hprobe.
  *
  * On singlestep exception, singlestep notifier sets the TIF_UPROBE flag
  * and allows the thread to return from interrupt.
@@ -2571,7 +2581,7 @@ void uprobe_notify_resume(struct pt_regs *regs)
 	clear_thread_flag(TIF_UPROBE);
 
 	utask = current->utask;
-	if (utask && utask->active_uprobe)
+	if (utask && utask_has_pending_sstep_uprobe(utask))
 		handle_singlestep(utask, regs);
 	else
 		handle_swbp(regs);
@@ -2602,7 +2612,7 @@ int uprobe_post_sstep_notifier(struct pt_regs *regs)
 {
 	struct uprobe_task *utask = current->utask;
 
-	if (!current->mm || !utask || !utask->active_uprobe)
+	if (!current->mm || !utask || !utask_has_pending_sstep_uprobe(utask))
 		/* task is currently not uprobed */
 		return 0;
 
-- 
2.43.5



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

* [PATCH RFC v3 11/13] mm: introduce mmap_lock_speculation_{start|end}
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (9 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH RFC v3 10/13] uprobes: implement SRCU-protected lifetime for single-stepped uprobe Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  4:29 ` [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache Andrii Nakryiko
                   ` (2 subsequent siblings)
  13 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko, Andrii Nakryiko

From: Suren Baghdasaryan <surenb@google.com>

Add helper functions to speculatively perform operations without
read-locking mmap_lock, expecting that mmap_lock will not be
write-locked and mm is not modified from under us.

Signed-off-by: Suren Baghdasaryan <surenb@google.com>
Suggested-by: Peter Zijlstra <peterz@infradead.org>
Cc: Andrii Nakryiko <andrii.nakryiko@gmail.com>
---
 include/linux/mm_types.h  |  3 +++
 include/linux/mmap_lock.h | 53 +++++++++++++++++++++++++++++++--------
 kernel/fork.c             |  3 ---
 3 files changed, 46 insertions(+), 13 deletions(-)

diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index 485424979254..d5e3f907eea4 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -876,6 +876,9 @@ struct mm_struct {
 		 * Roughly speaking, incrementing the sequence number is
 		 * equivalent to releasing locks on VMAs; reading the sequence
 		 * number can be part of taking a read lock on a VMA.
+		 * Incremented every time mmap_lock is write-locked/unlocked.
+		 * Initialized to 0, therefore odd values indicate mmap_lock
+		 * is write-locked and even values that it's released.
 		 *
 		 * Can be modified under write mmap_lock using RELEASE
 		 * semantics.
diff --git a/include/linux/mmap_lock.h b/include/linux/mmap_lock.h
index de9dc20b01ba..5410ce741d75 100644
--- a/include/linux/mmap_lock.h
+++ b/include/linux/mmap_lock.h
@@ -71,15 +71,12 @@ static inline void mmap_assert_write_locked(const struct mm_struct *mm)
 }
 
 #ifdef CONFIG_PER_VMA_LOCK
-/*
- * Drop all currently-held per-VMA locks.
- * This is called from the mmap_lock implementation directly before releasing
- * a write-locked mmap_lock (or downgrading it to read-locked).
- * This should normally NOT be called manually from other places.
- * If you want to call this manually anyway, keep in mind that this will release
- * *all* VMA write locks, including ones from further up the stack.
- */
-static inline void vma_end_write_all(struct mm_struct *mm)
+static inline void init_mm_lock_seq(struct mm_struct *mm)
+{
+	mm->mm_lock_seq = 0;
+}
+
+static inline void inc_mm_lock_seq(struct mm_struct *mm)
 {
 	mmap_assert_write_locked(mm);
 	/*
@@ -91,19 +88,52 @@ static inline void vma_end_write_all(struct mm_struct *mm)
 	 */
 	smp_store_release(&mm->mm_lock_seq, mm->mm_lock_seq + 1);
 }
+
+static inline bool mmap_lock_speculation_start(struct mm_struct *mm, int *seq)
+{
+	/* Pairs with RELEASE semantics in inc_mm_lock_seq(). */
+	*seq = smp_load_acquire(&mm->mm_lock_seq);
+	/* Allow speculation if mmap_lock is not write-locked */
+	return (*seq & 1) == 0;
+}
+
+static inline bool mmap_lock_speculation_end(struct mm_struct *mm, int seq)
+{
+	/* Pairs with RELEASE semantics in inc_mm_lock_seq(). */
+	return seq == smp_load_acquire(&mm->mm_lock_seq);
+}
+
 #else
-static inline void vma_end_write_all(struct mm_struct *mm) {}
+static inline void init_mm_lock_seq(struct mm_struct *mm) {}
+static inline void inc_mm_lock_seq(struct mm_struct *mm) {}
+static inline bool mmap_lock_speculation_start(struct mm_struct *mm, int *seq) { return false; }
+static inline bool mmap_lock_speculation_end(struct mm_struct *mm, int seq) { return false; }
 #endif
 
+/*
+ * Drop all currently-held per-VMA locks.
+ * This is called from the mmap_lock implementation directly before releasing
+ * a write-locked mmap_lock (or downgrading it to read-locked).
+ * This should normally NOT be called manually from other places.
+ * If you want to call this manually anyway, keep in mind that this will release
+ * *all* VMA write locks, including ones from further up the stack.
+ */
+static inline void vma_end_write_all(struct mm_struct *mm)
+{
+	inc_mm_lock_seq(mm);
+}
+
 static inline void mmap_init_lock(struct mm_struct *mm)
 {
 	init_rwsem(&mm->mmap_lock);
+	init_mm_lock_seq(mm);
 }
 
 static inline void mmap_write_lock(struct mm_struct *mm)
 {
 	__mmap_lock_trace_start_locking(mm, true);
 	down_write(&mm->mmap_lock);
+	inc_mm_lock_seq(mm);
 	__mmap_lock_trace_acquire_returned(mm, true, true);
 }
 
@@ -111,6 +141,7 @@ static inline void mmap_write_lock_nested(struct mm_struct *mm, int subclass)
 {
 	__mmap_lock_trace_start_locking(mm, true);
 	down_write_nested(&mm->mmap_lock, subclass);
+	inc_mm_lock_seq(mm);
 	__mmap_lock_trace_acquire_returned(mm, true, true);
 }
 
@@ -120,6 +151,8 @@ static inline int mmap_write_lock_killable(struct mm_struct *mm)
 
 	__mmap_lock_trace_start_locking(mm, true);
 	ret = down_write_killable(&mm->mmap_lock);
+	if (!ret)
+		inc_mm_lock_seq(mm);
 	__mmap_lock_trace_acquire_returned(mm, true, ret == 0);
 	return ret;
 }
diff --git a/kernel/fork.c b/kernel/fork.c
index cc760491f201..76ebafb956a6 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -1259,9 +1259,6 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
 	seqcount_init(&mm->write_protect_seq);
 	mmap_init_lock(mm);
 	INIT_LIST_HEAD(&mm->mmlist);
-#ifdef CONFIG_PER_VMA_LOCK
-	mm->mm_lock_seq = 0;
-#endif
 	mm_pgtables_bytes_init(mm);
 	mm->map_count = 0;
 	mm->locked_vm = 0;
-- 
2.43.5



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

* [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (10 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH RFC v3 11/13] mm: introduce mmap_lock_speculation_{start|end} Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  6:07   ` Mateusz Guzik
  2024-08-13  4:29 ` [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution Andrii Nakryiko
  2024-08-15 13:24 ` [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Oleg Nesterov
  13 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

Add RCU protection for file struct's backing memory by adding
SLAB_TYPESAFE_BY_RCU flag to files_cachep. This will allow to locklessly
access struct file's fields under RCU lock protection without having to
take much more expensive and contended locks.

This is going to be used for lockless uprobe look up in the next patch.

Suggested-by: Matthew Wilcox <willy@infradead.org>
Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 kernel/fork.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/kernel/fork.c b/kernel/fork.c
index 76ebafb956a6..91ecc32a491c 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -3157,8 +3157,8 @@ void __init proc_caches_init(void)
 			NULL);
 	files_cachep = kmem_cache_create("files_cache",
 			sizeof(struct files_struct), 0,
-			SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
-			NULL);
+			SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_TYPESAFE_BY_RCU|
+			SLAB_ACCOUNT, NULL);
 	fs_cachep = kmem_cache_create("fs_cache",
 			sizeof(struct fs_struct), 0,
 			SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
-- 
2.43.5



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

* [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (11 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache Andrii Nakryiko
@ 2024-08-13  4:29 ` Andrii Nakryiko
  2024-08-13  6:17   ` Mateusz Guzik
  2024-08-15 13:24 ` [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Oleg Nesterov
  13 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13  4:29 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	surenb, akpm, linux-mm, Andrii Nakryiko

Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
attempting uprobe look up speculatively.

We rely on newly added mmap_lock_speculation_{start,end}() helpers to
validate that mm_struct stays intact for entire duration of this
speculation. If not, we fall back to mmap_lock-protected lookup.

This allows to avoid contention on mmap_lock in absolutely majority of
cases, nicely improving uprobe/uretprobe scalability.

BEFORE
======
uprobe-nop            ( 1 cpus):    3.417 ± 0.013M/s  (  3.417M/s/cpu)
uprobe-nop            ( 2 cpus):    5.724 ± 0.006M/s  (  2.862M/s/cpu)
uprobe-nop            ( 3 cpus):    8.543 ± 0.012M/s  (  2.848M/s/cpu)
uprobe-nop            ( 4 cpus):   11.094 ± 0.004M/s  (  2.774M/s/cpu)
uprobe-nop            ( 5 cpus):   13.703 ± 0.006M/s  (  2.741M/s/cpu)
uprobe-nop            ( 6 cpus):   16.350 ± 0.010M/s  (  2.725M/s/cpu)
uprobe-nop            ( 7 cpus):   19.100 ± 0.031M/s  (  2.729M/s/cpu)
uprobe-nop            ( 8 cpus):   20.138 ± 0.029M/s  (  2.517M/s/cpu)
uprobe-nop            (10 cpus):   20.161 ± 0.020M/s  (  2.016M/s/cpu)
uprobe-nop            (12 cpus):   15.129 ± 0.011M/s  (  1.261M/s/cpu)
uprobe-nop            (14 cpus):   15.013 ± 0.013M/s  (  1.072M/s/cpu)
uprobe-nop            (16 cpus):   13.352 ± 0.007M/s  (  0.834M/s/cpu)
uprobe-nop            (24 cpus):   12.470 ± 0.005M/s  (  0.520M/s/cpu)
uprobe-nop            (32 cpus):   11.252 ± 0.042M/s  (  0.352M/s/cpu)
uprobe-nop            (40 cpus):   10.308 ± 0.001M/s  (  0.258M/s/cpu)
uprobe-nop            (48 cpus):   11.037 ± 0.007M/s  (  0.230M/s/cpu)
uprobe-nop            (56 cpus):   12.055 ± 0.002M/s  (  0.215M/s/cpu)
uprobe-nop            (64 cpus):   12.895 ± 0.004M/s  (  0.201M/s/cpu)
uprobe-nop            (72 cpus):   13.995 ± 0.005M/s  (  0.194M/s/cpu)
uprobe-nop            (80 cpus):   15.224 ± 0.030M/s  (  0.190M/s/cpu)

AFTER
=====
uprobe-nop            ( 1 cpus):    3.562 ± 0.006M/s  (  3.562M/s/cpu)
uprobe-nop            ( 2 cpus):    6.751 ± 0.007M/s  (  3.376M/s/cpu)
uprobe-nop            ( 3 cpus):   10.121 ± 0.007M/s  (  3.374M/s/cpu)
uprobe-nop            ( 4 cpus):   13.100 ± 0.007M/s  (  3.275M/s/cpu)
uprobe-nop            ( 5 cpus):   16.321 ± 0.008M/s  (  3.264M/s/cpu)
uprobe-nop            ( 6 cpus):   19.612 ± 0.004M/s  (  3.269M/s/cpu)
uprobe-nop            ( 7 cpus):   22.910 ± 0.037M/s  (  3.273M/s/cpu)
uprobe-nop            ( 8 cpus):   24.705 ± 0.011M/s  (  3.088M/s/cpu)
uprobe-nop            (10 cpus):   30.772 ± 0.020M/s  (  3.077M/s/cpu)
uprobe-nop            (12 cpus):   33.614 ± 0.009M/s  (  2.801M/s/cpu)
uprobe-nop            (14 cpus):   39.166 ± 0.004M/s  (  2.798M/s/cpu)
uprobe-nop            (16 cpus):   41.692 ± 0.014M/s  (  2.606M/s/cpu)
uprobe-nop            (24 cpus):   64.802 ± 0.048M/s  (  2.700M/s/cpu)
uprobe-nop            (32 cpus):   84.226 ± 0.223M/s  (  2.632M/s/cpu)
uprobe-nop            (40 cpus):  102.071 ± 0.067M/s  (  2.552M/s/cpu)
uprobe-nop            (48 cpus):  106.603 ± 1.198M/s  (  2.221M/s/cpu)
uprobe-nop            (56 cpus):  117.695 ± 0.059M/s  (  2.102M/s/cpu)
uprobe-nop            (64 cpus):  124.291 ± 0.485M/s  (  1.942M/s/cpu)
uprobe-nop            (72 cpus):  135.527 ± 0.134M/s  (  1.882M/s/cpu)
uprobe-nop            (80 cpus):  146.195 ± 0.230M/s  (  1.827M/s/cpu)

Previously total throughput was maxing out at 20mln/s with 8-10 cores,
declining afterwards. With this change, it now keeps growing with each
added CPU, reaching 146mln/s at 80 CPUs (this was measured on a 80-core
Intel(R) Xeon(R) Gold 6138 CPU @ 2.00GHz).

Suggested-by: Matthew Wilcox <willy@infradead.org>
Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
---
 kernel/events/uprobes.c | 51 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 51 insertions(+)

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 713824c8ca77..12f3edf2ffb1 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -2286,6 +2286,53 @@ static int is_trap_at_addr(struct mm_struct *mm, unsigned long vaddr)
 	return is_trap_insn(&opcode);
 }
 
+static struct uprobe *find_active_uprobe_speculative(unsigned long bp_vaddr)
+{
+	const vm_flags_t flags = VM_HUGETLB | VM_MAYEXEC | VM_MAYSHARE;
+	struct mm_struct *mm = current->mm;
+	struct uprobe *uprobe;
+	struct vm_area_struct *vma;
+	struct file *vm_file;
+	struct inode *vm_inode;
+	unsigned long vm_pgoff, vm_start;
+	int seq;
+	loff_t offset;
+
+	if (!mmap_lock_speculation_start(mm, &seq))
+		return NULL;
+
+	rcu_read_lock();
+
+	vma = vma_lookup(mm, bp_vaddr);
+	if (!vma)
+		goto bail;
+
+	vm_file = data_race(vma->vm_file);
+	if (!vm_file || (vma->vm_flags & flags) != VM_MAYEXEC)
+		goto bail;
+
+	vm_inode = data_race(vm_file->f_inode);
+	vm_pgoff = data_race(vma->vm_pgoff);
+	vm_start = data_race(vma->vm_start);
+
+	offset = (loff_t)(vm_pgoff << PAGE_SHIFT) + (bp_vaddr - vm_start);
+	uprobe = find_uprobe_rcu(vm_inode, offset);
+	if (!uprobe)
+		goto bail;
+
+	/* now double check that nothing about MM changed */
+	if (!mmap_lock_speculation_end(mm, seq))
+		goto bail;
+
+	rcu_read_unlock();
+
+	/* happy case, we speculated successfully */
+	return uprobe;
+bail:
+	rcu_read_unlock();
+	return NULL;
+}
+
 /* assumes being inside RCU protected region */
 static struct uprobe *find_active_uprobe_rcu(unsigned long bp_vaddr, int *is_swbp)
 {
@@ -2293,6 +2340,10 @@ static struct uprobe *find_active_uprobe_rcu(unsigned long bp_vaddr, int *is_swb
 	struct uprobe *uprobe = NULL;
 	struct vm_area_struct *vma;
 
+	uprobe = find_active_uprobe_speculative(bp_vaddr);
+	if (uprobe)
+		return uprobe;
+
 	mmap_read_lock(mm);
 	vma = vma_lookup(mm, bp_vaddr);
 	if (vma) {
-- 
2.43.5



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

* Re: [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache
  2024-08-13  4:29 ` [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache Andrii Nakryiko
@ 2024-08-13  6:07   ` Mateusz Guzik
  2024-08-13 14:49     ` Suren Baghdasaryan
  0 siblings, 1 reply; 40+ messages in thread
From: Mateusz Guzik @ 2024-08-13  6:07 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, peterz, oleg, rostedt, mhiramat, bpf,
	linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm

On Mon, Aug 12, 2024 at 09:29:16PM -0700, Andrii Nakryiko wrote:
> Add RCU protection for file struct's backing memory by adding
> SLAB_TYPESAFE_BY_RCU flag to files_cachep. This will allow to locklessly
> access struct file's fields under RCU lock protection without having to
> take much more expensive and contended locks.
> 
> This is going to be used for lockless uprobe look up in the next patch.
> 
> Suggested-by: Matthew Wilcox <willy@infradead.org>
> Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
> ---
>  kernel/fork.c | 4 ++--
>  1 file changed, 2 insertions(+), 2 deletions(-)
> 
> diff --git a/kernel/fork.c b/kernel/fork.c
> index 76ebafb956a6..91ecc32a491c 100644
> --- a/kernel/fork.c
> +++ b/kernel/fork.c
> @@ -3157,8 +3157,8 @@ void __init proc_caches_init(void)
>  			NULL);
>  	files_cachep = kmem_cache_create("files_cache",
>  			sizeof(struct files_struct), 0,
> -			SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
> -			NULL);
> +			SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_TYPESAFE_BY_RCU|
> +			SLAB_ACCOUNT, NULL);
>  	fs_cachep = kmem_cache_create("fs_cache",
>  			sizeof(struct fs_struct), 0,
>  			SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,

Did you mean to add it to the cache backing 'struct file' allocations?

That cache is created in fs/file_table.c and already has the flag:
        filp_cachep = kmem_cache_create("filp", sizeof(struct file), 0,
                                SLAB_TYPESAFE_BY_RCU | SLAB_HWCACHE_ALIGN |
                                SLAB_PANIC | SLAB_ACCOUNT, NULL);

The cache you are modifying in this patch contains the fd array et al
and is of no consequence to "uprobes: add speculative lockless VMA to
inode resolution".

iow this patch needs to be dropped


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-13  4:29 ` [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution Andrii Nakryiko
@ 2024-08-13  6:17   ` Mateusz Guzik
  2024-08-13 15:36     ` Suren Baghdasaryan
  0 siblings, 1 reply; 40+ messages in thread
From: Mateusz Guzik @ 2024-08-13  6:17 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, peterz, oleg, rostedt, mhiramat, bpf,
	linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm

On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> attempting uprobe look up speculatively.
> 
> We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> validate that mm_struct stays intact for entire duration of this
> speculation. If not, we fall back to mmap_lock-protected lookup.
> 
> This allows to avoid contention on mmap_lock in absolutely majority of
> cases, nicely improving uprobe/uretprobe scalability.
> 

Here I have to admit to being mostly ignorant about the mm, so bear with
me. :>

I note the result of find_active_uprobe_speculative is immediately stale
in face of modifications.

The thing I'm after is that the mmap_lock_speculation business adds
overhead on archs where a release fence is not a de facto nop and I
don't believe the commit message justifies it. Definitely a bummer to
add merely it for uprobes. If there are bigger plans concerning it
that's a different story of course.

With this in mind I have to ask if instead you could perhaps get away
with the already present per-vma sequence counter?


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

* Re: [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache
  2024-08-13  6:07   ` Mateusz Guzik
@ 2024-08-13 14:49     ` Suren Baghdasaryan
  2024-08-13 18:15       ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Suren Baghdasaryan @ 2024-08-13 14:49 UTC (permalink / raw)
  To: Mateusz Guzik
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Mon, Aug 12, 2024 at 11:07 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
>
> On Mon, Aug 12, 2024 at 09:29:16PM -0700, Andrii Nakryiko wrote:
> > Add RCU protection for file struct's backing memory by adding
> > SLAB_TYPESAFE_BY_RCU flag to files_cachep. This will allow to locklessly
> > access struct file's fields under RCU lock protection without having to
> > take much more expensive and contended locks.
> >
> > This is going to be used for lockless uprobe look up in the next patch.
> >
> > Suggested-by: Matthew Wilcox <willy@infradead.org>
> > Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
> > ---
> >  kernel/fork.c | 4 ++--
> >  1 file changed, 2 insertions(+), 2 deletions(-)
> >
> > diff --git a/kernel/fork.c b/kernel/fork.c
> > index 76ebafb956a6..91ecc32a491c 100644
> > --- a/kernel/fork.c
> > +++ b/kernel/fork.c
> > @@ -3157,8 +3157,8 @@ void __init proc_caches_init(void)
> >                       NULL);
> >       files_cachep = kmem_cache_create("files_cache",
> >                       sizeof(struct files_struct), 0,
> > -                     SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
> > -                     NULL);
> > +                     SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_TYPESAFE_BY_RCU|
> > +                     SLAB_ACCOUNT, NULL);
> >       fs_cachep = kmem_cache_create("fs_cache",
> >                       sizeof(struct fs_struct), 0,
> >                       SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
>
> Did you mean to add it to the cache backing 'struct file' allocations?
>
> That cache is created in fs/file_table.c and already has the flag:
>         filp_cachep = kmem_cache_create("filp", sizeof(struct file), 0,
>                                 SLAB_TYPESAFE_BY_RCU | SLAB_HWCACHE_ALIGN |
>                                 SLAB_PANIC | SLAB_ACCOUNT, NULL);

Oh, I completely missed the SLAB_TYPESAFE_BY_RCU for this cache, and
here I was telling Andrii that it's RCU unsafe to access
vma->vm_file... Mea culpa.

>
> The cache you are modifying in this patch contains the fd array et al
> and is of no consequence to "uprobes: add speculative lockless VMA to
> inode resolution".
>
> iow this patch needs to be dropped

I believe you are correct.


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-13  6:17   ` Mateusz Guzik
@ 2024-08-13 15:36     ` Suren Baghdasaryan
  2024-08-15 13:44       ` Mateusz Guzik
  0 siblings, 1 reply; 40+ messages in thread
From: Suren Baghdasaryan @ 2024-08-13 15:36 UTC (permalink / raw)
  To: Mateusz Guzik
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
>
> On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > attempting uprobe look up speculatively.
> >
> > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > validate that mm_struct stays intact for entire duration of this
> > speculation. If not, we fall back to mmap_lock-protected lookup.
> >
> > This allows to avoid contention on mmap_lock in absolutely majority of
> > cases, nicely improving uprobe/uretprobe scalability.
> >
>
> Here I have to admit to being mostly ignorant about the mm, so bear with
> me. :>
>
> I note the result of find_active_uprobe_speculative is immediately stale
> in face of modifications.
>
> The thing I'm after is that the mmap_lock_speculation business adds
> overhead on archs where a release fence is not a de facto nop and I
> don't believe the commit message justifies it. Definitely a bummer to
> add merely it for uprobes. If there are bigger plans concerning it
> that's a different story of course.
>
> With this in mind I have to ask if instead you could perhaps get away
> with the already present per-vma sequence counter?

per-vma sequence counter does not implement acquire/release logic, it
relies on vma->vm_lock for synchronization. So if we want to use it,
we would have to add additional memory barriers here. This is likely
possible but as I mentioned before we would need to ensure the
pagefault path does not regress. OTOH mm->mm_lock_seq already halfway
there (it implements acquire/release logic), we just had to ensure
mmap_write_lock() increments mm->mm_lock_seq.

So, from the release fence overhead POV I think whether we use
mm->mm_lock_seq or vma->vm_lock, we would still need a proper fence
here.


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

* Re: [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache
  2024-08-13 14:49     ` Suren Baghdasaryan
@ 2024-08-13 18:15       ` Andrii Nakryiko
  0 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-13 18:15 UTC (permalink / raw)
  To: Suren Baghdasaryan
  Cc: Mateusz Guzik, Andrii Nakryiko, linux-trace-kernel, peterz, oleg,
	rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Tue, Aug 13, 2024 at 7:49 AM Suren Baghdasaryan <surenb@google.com> wrote:
>
> On Mon, Aug 12, 2024 at 11:07 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> >
> > On Mon, Aug 12, 2024 at 09:29:16PM -0700, Andrii Nakryiko wrote:
> > > Add RCU protection for file struct's backing memory by adding
> > > SLAB_TYPESAFE_BY_RCU flag to files_cachep. This will allow to locklessly
> > > access struct file's fields under RCU lock protection without having to
> > > take much more expensive and contended locks.
> > >
> > > This is going to be used for lockless uprobe look up in the next patch.
> > >
> > > Suggested-by: Matthew Wilcox <willy@infradead.org>
> > > Signed-off-by: Andrii Nakryiko <andrii@kernel.org>
> > > ---
> > >  kernel/fork.c | 4 ++--
> > >  1 file changed, 2 insertions(+), 2 deletions(-)
> > >
> > > diff --git a/kernel/fork.c b/kernel/fork.c
> > > index 76ebafb956a6..91ecc32a491c 100644
> > > --- a/kernel/fork.c
> > > +++ b/kernel/fork.c
> > > @@ -3157,8 +3157,8 @@ void __init proc_caches_init(void)
> > >                       NULL);
> > >       files_cachep = kmem_cache_create("files_cache",
> > >                       sizeof(struct files_struct), 0,
> > > -                     SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
> > > -                     NULL);
> > > +                     SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_TYPESAFE_BY_RCU|
> > > +                     SLAB_ACCOUNT, NULL);
> > >       fs_cachep = kmem_cache_create("fs_cache",
> > >                       sizeof(struct fs_struct), 0,
> > >                       SLAB_HWCACHE_ALIGN|SLAB_PANIC|SLAB_ACCOUNT,
> >
> > Did you mean to add it to the cache backing 'struct file' allocations?

Yep, thanks for catching this!

> >
> > That cache is created in fs/file_table.c and already has the flag:
> >         filp_cachep = kmem_cache_create("filp", sizeof(struct file), 0,
> >                                 SLAB_TYPESAFE_BY_RCU | SLAB_HWCACHE_ALIGN |
> >                                 SLAB_PANIC | SLAB_ACCOUNT, NULL);
>
> Oh, I completely missed the SLAB_TYPESAFE_BY_RCU for this cache, and
> here I was telling Andrii that it's RCU unsafe to access
> vma->vm_file... Mea culpa.
>

Well, my bad for not double-checking and going just by the name.
filp_cachep vs files_cachep is easy to mix up.

> >
> > The cache you are modifying in this patch contains the fd array et al
> > and is of no consequence to "uprobes: add speculative lockless VMA to
> > inode resolution".
> >
> > iow this patch needs to be dropped
>
> I believe you are correct.
>

I'm happy that we already have SLAB_TYPESAFE_BY_RCU on filp_cachep,
I'll just drop this patch.


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

* Re: [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations
  2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
                   ` (12 preceding siblings ...)
  2024-08-13  4:29 ` [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution Andrii Nakryiko
@ 2024-08-15 13:24 ` Oleg Nesterov
  2024-08-15 16:49   ` Andrii Nakryiko
  13 siblings, 1 reply; 40+ messages in thread
From: Oleg Nesterov @ 2024-08-15 13:24 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, peterz, rostedt, mhiramat, bpf, linux-kernel,
	jolsa, paulmck, willy, surenb, akpm, linux-mm

On 08/12, Andrii Nakryiko wrote:
>
> ( In addition to previously posted first 8 patches, I'm sending 5 more as an
> RFC for people to get the general gist of where this work heading and what
> uprobe performance is now achievable. I think first 8 patches are ready to be
> applied and I'd appreciate early feedback on the remaining 5 ones.

I didn't read the "RFC" patches yet, will try to do on weekend.

As for 1-8, I failed to find any problem:
Reviewed-by: Oleg Nesterov <oleg@redhat.com>



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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-13 15:36     ` Suren Baghdasaryan
@ 2024-08-15 13:44       ` Mateusz Guzik
  2024-08-15 16:47         ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Mateusz Guzik @ 2024-08-15 13:44 UTC (permalink / raw)
  To: Suren Baghdasaryan
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Tue, Aug 13, 2024 at 08:36:03AM -0700, Suren Baghdasaryan wrote:
> On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> >
> > On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > attempting uprobe look up speculatively.
> > >
> > > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > > validate that mm_struct stays intact for entire duration of this
> > > speculation. If not, we fall back to mmap_lock-protected lookup.
> > >
> > > This allows to avoid contention on mmap_lock in absolutely majority of
> > > cases, nicely improving uprobe/uretprobe scalability.
> > >
> >
> > Here I have to admit to being mostly ignorant about the mm, so bear with
> > me. :>
> >
> > I note the result of find_active_uprobe_speculative is immediately stale
> > in face of modifications.
> >
> > The thing I'm after is that the mmap_lock_speculation business adds
> > overhead on archs where a release fence is not a de facto nop and I
> > don't believe the commit message justifies it. Definitely a bummer to
> > add merely it for uprobes. If there are bigger plans concerning it
> > that's a different story of course.
> >
> > With this in mind I have to ask if instead you could perhaps get away
> > with the already present per-vma sequence counter?
> 
> per-vma sequence counter does not implement acquire/release logic, it
> relies on vma->vm_lock for synchronization. So if we want to use it,
> we would have to add additional memory barriers here. This is likely
> possible but as I mentioned before we would need to ensure the
> pagefault path does not regress. OTOH mm->mm_lock_seq already halfway
> there (it implements acquire/release logic), we just had to ensure
> mmap_write_lock() increments mm->mm_lock_seq.
> 
> So, from the release fence overhead POV I think whether we use
> mm->mm_lock_seq or vma->vm_lock, we would still need a proper fence
> here.
> 

Per my previous e-mail I'm not particularly familiar with mm internals,
so I'm going to handwave a little bit with my $0,03 concerning multicore
in general and if you disagree with it that's your business. For the
time being I have no interest in digging into any of this.

Before I do, to prevent this thread from being a total waste, here are
some remarks concerning the patch with the assumption that the core idea
lands.

From the commit message:
> Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> attempting uprobe look up speculatively.

Just in case I'll note a nit that this paragraph will need to be removed
since the patch adding the flag is getting dropped.

A non-nit which may or may not end up mattering is that the flag (which
*is* set on the filep slab cache) makes things more difficult to
validate. Normal RCU usage guarantees that the object itself wont be
freed as long you follow the rules. However, the SLAB_TYPESAFE_BY_RCU
flag weakens it significantly -- the thing at hand will always be a
'struct file', but it may get reallocated to *another* file from under
you. Whether this aspect plays a role here I don't know.

> +static struct uprobe *find_active_uprobe_speculative(unsigned long bp_vaddr)
> +{
> +	const vm_flags_t flags = VM_HUGETLB | VM_MAYEXEC | VM_MAYSHARE;
> +	struct mm_struct *mm = current->mm;
> +	struct uprobe *uprobe;
> +	struct vm_area_struct *vma;
> +	struct file *vm_file;
> +	struct inode *vm_inode;
> +	unsigned long vm_pgoff, vm_start;
> +	int seq;
> +	loff_t offset;
> +
> +	if (!mmap_lock_speculation_start(mm, &seq))
> +		return NULL;
> +
> +	rcu_read_lock();
> +

I don't think there is a correctness problem here, but entering rcu
*after* deciding to speculatively do the lookup feels backwards.

> +	vma = vma_lookup(mm, bp_vaddr);
> +	if (!vma)
> +		goto bail;
> +
> +	vm_file = data_race(vma->vm_file);
> +	if (!vm_file || (vma->vm_flags & flags) != VM_MAYEXEC)
> +		goto bail;
> +

If vma teardown is allowed to progress and the file got fput'ed...

> +	vm_inode = data_race(vm_file->f_inode);

... the inode can be NULL, I don't know if that's handled.

More importantly though, per my previous description of
SLAB_TYPESAFE_BY_RCU, by now the file could have been reallocated and
the inode you did find is completely unrelated.

I understand the intent is to backpedal from everything should the mm
seqc change, but the above may happen to matter.

> +	vm_pgoff = data_race(vma->vm_pgoff);
> +	vm_start = data_race(vma->vm_start);
> +
> +	offset = (loff_t)(vm_pgoff << PAGE_SHIFT) + (bp_vaddr - vm_start);
> +	uprobe = find_uprobe_rcu(vm_inode, offset);
> +	if (!uprobe)
> +		goto bail;
> +
> +	/* now double check that nothing about MM changed */
> +	if (!mmap_lock_speculation_end(mm, seq))
> +		goto bail;

This leaks the reference obtained by find_uprobe_rcu().

> +
> +	rcu_read_unlock();
> +
> +	/* happy case, we speculated successfully */
> +	return uprobe;
> +bail:
> +	rcu_read_unlock();
> +	return NULL;
> +}

Now to some handwaving, here it is:

The core of my concern is that adding more work to down_write on the
mmap semaphore comes with certain side-effects and plausibly more than a
sufficient speed up can be achieved without doing it.

An mm-wide mechanism is just incredibly coarse-grained and it may happen
to perform poorly when faced with a program which likes to mess with its
address space -- the fast path is going to keep failing and only
inducing *more* overhead as the code decides to down_read the mmap
semaphore.

Furthermore there may be work currently synchronized with down_write
which perhaps can transition to "merely" down_read, but by the time it
happens this and possibly other consumers expect a change in the
sequence counter, messing with it.

To my understanding the kernel supports parallel faults with per-vma
locking. I would find it surprising if the same machinery could not be
used to sort out uprobe handling above.

I presume a down_read on vma around all the work would also sort out any
issues concerning stability of the file or inode objects.

Of course single-threaded performance would take a hit due to atomic
stemming from down/up_read and parallel uprobe lookups on the same vma
would also get slower, but I don't know if that's a problem for a real
workload.

I would not have any comments if all speed ups were achieved without
modifying non-uprobe code.


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 13:44       ` Mateusz Guzik
@ 2024-08-15 16:47         ` Andrii Nakryiko
  2024-08-15 17:45           ` Suren Baghdasaryan
  0 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-15 16:47 UTC (permalink / raw)
  To: Mateusz Guzik
  Cc: Suren Baghdasaryan, Andrii Nakryiko, linux-trace-kernel, peterz,
	oleg, rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	akpm, linux-mm

On Thu, Aug 15, 2024 at 6:44 AM Mateusz Guzik <mjguzik@gmail.com> wrote:
>
> On Tue, Aug 13, 2024 at 08:36:03AM -0700, Suren Baghdasaryan wrote:
> > On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > >
> > > On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > attempting uprobe look up speculatively.
> > > >
> > > > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > > > validate that mm_struct stays intact for entire duration of this
> > > > speculation. If not, we fall back to mmap_lock-protected lookup.
> > > >
> > > > This allows to avoid contention on mmap_lock in absolutely majority of
> > > > cases, nicely improving uprobe/uretprobe scalability.
> > > >
> > >
> > > Here I have to admit to being mostly ignorant about the mm, so bear with
> > > me. :>
> > >
> > > I note the result of find_active_uprobe_speculative is immediately stale
> > > in face of modifications.
> > >
> > > The thing I'm after is that the mmap_lock_speculation business adds
> > > overhead on archs where a release fence is not a de facto nop and I
> > > don't believe the commit message justifies it. Definitely a bummer to
> > > add merely it for uprobes. If there are bigger plans concerning it
> > > that's a different story of course.
> > >
> > > With this in mind I have to ask if instead you could perhaps get away
> > > with the already present per-vma sequence counter?
> >
> > per-vma sequence counter does not implement acquire/release logic, it
> > relies on vma->vm_lock for synchronization. So if we want to use it,
> > we would have to add additional memory barriers here. This is likely
> > possible but as I mentioned before we would need to ensure the
> > pagefault path does not regress. OTOH mm->mm_lock_seq already halfway
> > there (it implements acquire/release logic), we just had to ensure
> > mmap_write_lock() increments mm->mm_lock_seq.
> >
> > So, from the release fence overhead POV I think whether we use
> > mm->mm_lock_seq or vma->vm_lock, we would still need a proper fence
> > here.
> >
>
> Per my previous e-mail I'm not particularly familiar with mm internals,
> so I'm going to handwave a little bit with my $0,03 concerning multicore
> in general and if you disagree with it that's your business. For the
> time being I have no interest in digging into any of this.
>
> Before I do, to prevent this thread from being a total waste, here are
> some remarks concerning the patch with the assumption that the core idea
> lands.
>
> From the commit message:
> > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > attempting uprobe look up speculatively.
>
> Just in case I'll note a nit that this paragraph will need to be removed
> since the patch adding the flag is getting dropped.

Yep, of course, I'll update all that for the next revision (I'll wait
for non-RFC patches to land first before reposting).

>
> A non-nit which may or may not end up mattering is that the flag (which
> *is* set on the filep slab cache) makes things more difficult to
> validate. Normal RCU usage guarantees that the object itself wont be
> freed as long you follow the rules. However, the SLAB_TYPESAFE_BY_RCU
> flag weakens it significantly -- the thing at hand will always be a
> 'struct file', but it may get reallocated to *another* file from under
> you. Whether this aspect plays a role here I don't know.

Yes, that's ok and is accounted for. We care about that memory not
going even from under us (I'm not even sure if it matters that it is
still a struct file, tbh; I think that shouldn't matter as we are
prepared to deal with completely garbage values read from struct
file).

>
> > +static struct uprobe *find_active_uprobe_speculative(unsigned long bp_vaddr)
> > +{
> > +     const vm_flags_t flags = VM_HUGETLB | VM_MAYEXEC | VM_MAYSHARE;
> > +     struct mm_struct *mm = current->mm;
> > +     struct uprobe *uprobe;
> > +     struct vm_area_struct *vma;
> > +     struct file *vm_file;
> > +     struct inode *vm_inode;
> > +     unsigned long vm_pgoff, vm_start;
> > +     int seq;
> > +     loff_t offset;
> > +
> > +     if (!mmap_lock_speculation_start(mm, &seq))
> > +             return NULL;
> > +
> > +     rcu_read_lock();
> > +
>
> I don't think there is a correctness problem here, but entering rcu
> *after* deciding to speculatively do the lookup feels backwards.

RCU should protect VMA and file, mm itself won't go anywhere, so this seems ok.

>
> > +     vma = vma_lookup(mm, bp_vaddr);
> > +     if (!vma)
> > +             goto bail;
> > +
> > +     vm_file = data_race(vma->vm_file);
> > +     if (!vm_file || (vma->vm_flags & flags) != VM_MAYEXEC)
> > +             goto bail;
> > +
>
> If vma teardown is allowed to progress and the file got fput'ed...
>
> > +     vm_inode = data_race(vm_file->f_inode);
>
> ... the inode can be NULL, I don't know if that's handled.
>

Yep, inode pointer value is part of RB-tree key, so if it's NULL, we
just won't find a matching uprobe. Same for any other "garbage"
f_inode value. Importantly, we never should dereference such inode
pointers, at least until we find a valid uprobe (in which case we keep
inode reference to it).

> More importantly though, per my previous description of
> SLAB_TYPESAFE_BY_RCU, by now the file could have been reallocated and
> the inode you did find is completely unrelated.
>
> I understand the intent is to backpedal from everything should the mm
> seqc change, but the above may happen to matter.

Yes, I think we took that into account. All that we care about is
memory "type safety", i.e., even if struct file's memory is reused,
it's ok, we'll eventually detect the change and will discard wrong
uprobe that we might by accident lookup (though probably in most cases
we just won't find a uprobe at all).

>
> > +     vm_pgoff = data_race(vma->vm_pgoff);
> > +     vm_start = data_race(vma->vm_start);
> > +
> > +     offset = (loff_t)(vm_pgoff << PAGE_SHIFT) + (bp_vaddr - vm_start);
> > +     uprobe = find_uprobe_rcu(vm_inode, offset);
> > +     if (!uprobe)
> > +             goto bail;
> > +
> > +     /* now double check that nothing about MM changed */
> > +     if (!mmap_lock_speculation_end(mm, seq))
> > +             goto bail;
>
> This leaks the reference obtained by find_uprobe_rcu().

find_uprobe_rcu() doesn't obtain a reference, uprobe is RCU-protected,
and if caller need a refcount bump it will have to use
try_get_uprobe() (which might fail).

>
> > +
> > +     rcu_read_unlock();
> > +
> > +     /* happy case, we speculated successfully */
> > +     return uprobe;
> > +bail:
> > +     rcu_read_unlock();
> > +     return NULL;
> > +}
>
> Now to some handwaving, here it is:
>
> The core of my concern is that adding more work to down_write on the
> mmap semaphore comes with certain side-effects and plausibly more than a
> sufficient speed up can be achieved without doing it.
>
> An mm-wide mechanism is just incredibly coarse-grained and it may happen
> to perform poorly when faced with a program which likes to mess with its
> address space -- the fast path is going to keep failing and only
> inducing *more* overhead as the code decides to down_read the mmap
> semaphore.
>
> Furthermore there may be work currently synchronized with down_write
> which perhaps can transition to "merely" down_read, but by the time it
> happens this and possibly other consumers expect a change in the
> sequence counter, messing with it.
>
> To my understanding the kernel supports parallel faults with per-vma
> locking. I would find it surprising if the same machinery could not be
> used to sort out uprobe handling above.

per-vma locking is still *locking*. Which means memory sharing between
multiple CPUs, which means limited scalability. Lots of work in this
series went to avoid even refcounting (as I pointed out for
find_uprobe_rcu()) due to the same reason, and so relying on per-VMA
locking is just shifting the bottleneck from mmap_lock to
vma->vm_lock. Worst (and not uncommon) case is the same uprobe in the
same process (and thus vma) being hit on multiple CPUs at the same
time. Whether that's protected by mmap_lock or vma->vm_lock is
immaterial at that point (from scalability standpoint).

>
> I presume a down_read on vma around all the work would also sort out any
> issues concerning stability of the file or inode objects.
>
> Of course single-threaded performance would take a hit due to atomic
> stemming from down/up_read and parallel uprobe lookups on the same vma
> would also get slower, but I don't know if that's a problem for a real
> workload.
>
> I would not have any comments if all speed ups were achieved without
> modifying non-uprobe code.

I'm also not a mm-focused person, so I'll let Suren and others address
mm-specific concerns, but I (hopefully) addressed all the
uprobe-related questions and concerns you had.


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

* Re: [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations
  2024-08-15 13:24 ` [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Oleg Nesterov
@ 2024-08-15 16:49   ` Andrii Nakryiko
  2024-08-21 16:41     ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-15 16:49 UTC (permalink / raw)
  To: Oleg Nesterov, peterz
  Cc: Andrii Nakryiko, linux-trace-kernel, rostedt, mhiramat, bpf,
	linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm

On Thu, Aug 15, 2024 at 6:25 AM Oleg Nesterov <oleg@redhat.com> wrote:
>
> On 08/12, Andrii Nakryiko wrote:
> >
> > ( In addition to previously posted first 8 patches, I'm sending 5 more as an
> > RFC for people to get the general gist of where this work heading and what
> > uprobe performance is now achievable. I think first 8 patches are ready to be
> > applied and I'd appreciate early feedback on the remaining 5 ones.
>
> I didn't read the "RFC" patches yet, will try to do on weekend.
>
> As for 1-8, I failed to find any problem:
> Reviewed-by: Oleg Nesterov <oleg@redhat.com>
>

Great, thanks a lot for all the thorough reviews you've provided (and
hopefully will keep providing ;).

Peter, if you don't see any problems with first 8 patches, could you
please apply them to tip/perf/core some time soon, so that subsequent
work (SRCU+timeout and, separately, lockless VMA->inode->uprobe
lookup) can be split into independent pieces and reviewed/landed
separately? Thanks!


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 16:47         ` Andrii Nakryiko
@ 2024-08-15 17:45           ` Suren Baghdasaryan
  2024-08-15 18:24             ` Mateusz Guzik
  2024-08-15 18:58             ` Jann Horn
  0 siblings, 2 replies; 40+ messages in thread
From: Suren Baghdasaryan @ 2024-08-15 17:45 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: Mateusz Guzik, Andrii Nakryiko, linux-trace-kernel, peterz, oleg,
	rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm, Jann Horn

On Thu, Aug 15, 2024 at 9:47 AM Andrii Nakryiko
<andrii.nakryiko@gmail.com> wrote:
>
> On Thu, Aug 15, 2024 at 6:44 AM Mateusz Guzik <mjguzik@gmail.com> wrote:
> >
> > On Tue, Aug 13, 2024 at 08:36:03AM -0700, Suren Baghdasaryan wrote:
> > > On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > > >
> > > > On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > > attempting uprobe look up speculatively.
> > > > >
> > > > > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > > > > validate that mm_struct stays intact for entire duration of this
> > > > > speculation. If not, we fall back to mmap_lock-protected lookup.
> > > > >
> > > > > This allows to avoid contention on mmap_lock in absolutely majority of
> > > > > cases, nicely improving uprobe/uretprobe scalability.
> > > > >
> > > >
> > > > Here I have to admit to being mostly ignorant about the mm, so bear with
> > > > me. :>
> > > >
> > > > I note the result of find_active_uprobe_speculative is immediately stale
> > > > in face of modifications.
> > > >
> > > > The thing I'm after is that the mmap_lock_speculation business adds
> > > > overhead on archs where a release fence is not a de facto nop and I
> > > > don't believe the commit message justifies it. Definitely a bummer to
> > > > add merely it for uprobes. If there are bigger plans concerning it
> > > > that's a different story of course.
> > > >
> > > > With this in mind I have to ask if instead you could perhaps get away
> > > > with the already present per-vma sequence counter?
> > >
> > > per-vma sequence counter does not implement acquire/release logic, it
> > > relies on vma->vm_lock for synchronization. So if we want to use it,
> > > we would have to add additional memory barriers here. This is likely
> > > possible but as I mentioned before we would need to ensure the
> > > pagefault path does not regress. OTOH mm->mm_lock_seq already halfway
> > > there (it implements acquire/release logic), we just had to ensure
> > > mmap_write_lock() increments mm->mm_lock_seq.
> > >
> > > So, from the release fence overhead POV I think whether we use
> > > mm->mm_lock_seq or vma->vm_lock, we would still need a proper fence
> > > here.
> > >
> >
> > Per my previous e-mail I'm not particularly familiar with mm internals,
> > so I'm going to handwave a little bit with my $0,03 concerning multicore
> > in general and if you disagree with it that's your business. For the
> > time being I have no interest in digging into any of this.
> >
> > Before I do, to prevent this thread from being a total waste, here are
> > some remarks concerning the patch with the assumption that the core idea
> > lands.
> >
> > From the commit message:
> > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > attempting uprobe look up speculatively.
> >
> > Just in case I'll note a nit that this paragraph will need to be removed
> > since the patch adding the flag is getting dropped.
>
> Yep, of course, I'll update all that for the next revision (I'll wait
> for non-RFC patches to land first before reposting).
>
> >
> > A non-nit which may or may not end up mattering is that the flag (which
> > *is* set on the filep slab cache) makes things more difficult to
> > validate. Normal RCU usage guarantees that the object itself wont be
> > freed as long you follow the rules. However, the SLAB_TYPESAFE_BY_RCU
> > flag weakens it significantly -- the thing at hand will always be a
> > 'struct file', but it may get reallocated to *another* file from under
> > you. Whether this aspect plays a role here I don't know.
>
> Yes, that's ok and is accounted for. We care about that memory not
> going even from under us (I'm not even sure if it matters that it is
> still a struct file, tbh; I think that shouldn't matter as we are
> prepared to deal with completely garbage values read from struct
> file).

Correct, with SLAB_TYPESAFE_BY_RCU we do need an additional check that
vma->vm_file has not been freed and reused. That's where
mmap_lock_speculation_{start|end} helps us. For vma->vm_file to change
from under us one would have to take mmap_lock for write. If that
happens mmap_lock_speculation_{start|end} should detect that and
terminate our speculation.

>
> >
> > > +static struct uprobe *find_active_uprobe_speculative(unsigned long bp_vaddr)
> > > +{
> > > +     const vm_flags_t flags = VM_HUGETLB | VM_MAYEXEC | VM_MAYSHARE;
> > > +     struct mm_struct *mm = current->mm;
> > > +     struct uprobe *uprobe;
> > > +     struct vm_area_struct *vma;
> > > +     struct file *vm_file;
> > > +     struct inode *vm_inode;
> > > +     unsigned long vm_pgoff, vm_start;
> > > +     int seq;
> > > +     loff_t offset;
> > > +
> > > +     if (!mmap_lock_speculation_start(mm, &seq))
> > > +             return NULL;
> > > +
> > > +     rcu_read_lock();
> > > +
> >
> > I don't think there is a correctness problem here, but entering rcu
> > *after* deciding to speculatively do the lookup feels backwards.
>
> RCU should protect VMA and file, mm itself won't go anywhere, so this seems ok.
>
> >
> > > +     vma = vma_lookup(mm, bp_vaddr);
> > > +     if (!vma)
> > > +             goto bail;
> > > +
> > > +     vm_file = data_race(vma->vm_file);
> > > +     if (!vm_file || (vma->vm_flags & flags) != VM_MAYEXEC)
> > > +             goto bail;
> > > +
> >
> > If vma teardown is allowed to progress and the file got fput'ed...
> >
> > > +     vm_inode = data_race(vm_file->f_inode);
> >
> > ... the inode can be NULL, I don't know if that's handled.
> >
>
> Yep, inode pointer value is part of RB-tree key, so if it's NULL, we
> just won't find a matching uprobe. Same for any other "garbage"
> f_inode value. Importantly, we never should dereference such inode
> pointers, at least until we find a valid uprobe (in which case we keep
> inode reference to it).
>
> > More importantly though, per my previous description of
> > SLAB_TYPESAFE_BY_RCU, by now the file could have been reallocated and
> > the inode you did find is completely unrelated.
> >
> > I understand the intent is to backpedal from everything should the mm
> > seqc change, but the above may happen to matter.
>
> Yes, I think we took that into account. All that we care about is
> memory "type safety", i.e., even if struct file's memory is reused,
> it's ok, we'll eventually detect the change and will discard wrong
> uprobe that we might by accident lookup (though probably in most cases
> we just won't find a uprobe at all).
>
> >
> > > +     vm_pgoff = data_race(vma->vm_pgoff);
> > > +     vm_start = data_race(vma->vm_start);
> > > +
> > > +     offset = (loff_t)(vm_pgoff << PAGE_SHIFT) + (bp_vaddr - vm_start);
> > > +     uprobe = find_uprobe_rcu(vm_inode, offset);
> > > +     if (!uprobe)
> > > +             goto bail;
> > > +
> > > +     /* now double check that nothing about MM changed */
> > > +     if (!mmap_lock_speculation_end(mm, seq))
> > > +             goto bail;
> >
> > This leaks the reference obtained by find_uprobe_rcu().
>
> find_uprobe_rcu() doesn't obtain a reference, uprobe is RCU-protected,
> and if caller need a refcount bump it will have to use
> try_get_uprobe() (which might fail).
>
> >
> > > +
> > > +     rcu_read_unlock();
> > > +
> > > +     /* happy case, we speculated successfully */
> > > +     return uprobe;
> > > +bail:
> > > +     rcu_read_unlock();
> > > +     return NULL;
> > > +}
> >
> > Now to some handwaving, here it is:
> >
> > The core of my concern is that adding more work to down_write on the
> > mmap semaphore comes with certain side-effects and plausibly more than a
> > sufficient speed up can be achieved without doing it.

AFAIK writers of mmap_lock are not considered a fast path. In a sense
yes, we made any writer a bit heavier but OTOH we also made
mm->mm_lock_seq a proper sequence count which allows us to locklessly
check if mmap_lock is write-locked. I think you asked whether there
will be other uses for mmap_lock_speculation_{start|end} and yes. For
example, I am planning to use them for printing /proc/{pid}/maps
without taking mmap_lock (when it's uncontended). If we have VMA seq
counter-based detection it would be better (see below).

> >
> > An mm-wide mechanism is just incredibly coarse-grained and it may happen
> > to perform poorly when faced with a program which likes to mess with its
> > address space -- the fast path is going to keep failing and only
> > inducing *more* overhead as the code decides to down_read the mmap
> > semaphore.
> >
> > Furthermore there may be work currently synchronized with down_write
> > which perhaps can transition to "merely" down_read, but by the time it
> > happens this and possibly other consumers expect a change in the
> > sequence counter, messing with it.
> >
> > To my understanding the kernel supports parallel faults with per-vma
> > locking. I would find it surprising if the same machinery could not be
> > used to sort out uprobe handling above.

From all the above, my understanding of your objection is that
checking mmap_lock during our speculation is too coarse-grained and
you would prefer to use the VMA seq counter to check that the VMA we
are working on is unchanged. I agree, that would be ideal. I had a
quick chat with Jann about this and the conclusion we came to is that
we would need to add an additional smp_wmb() barrier inside
vma_start_write() and a smp_rmb() in the speculation code:

static inline void vma_start_write(struct vm_area_struct *vma)
{
        int mm_lock_seq;

        if (__is_vma_write_locked(vma, &mm_lock_seq))
                return;

        down_write(&vma->vm_lock->lock);
        /*
         * We should use WRITE_ONCE() here because we can have concurrent reads
         * from the early lockless pessimistic check in vma_start_read().
         * We don't really care about the correctness of that early check, but
         * we should use WRITE_ONCE() for cleanliness and to keep KCSAN happy.
         */
        WRITE_ONCE(vma->vm_lock_seq, mm_lock_seq);
+        smp_wmb();
        up_write(&vma->vm_lock->lock);
}

Note: up_write(&vma->vm_lock->lock) in the vma_start_write() is not
enough because it's one-way permeable (it's a "RELEASE operation") and
later vma->vm_file store (or any other VMA modification) can move
before our vma->vm_lock_seq store.

This makes vma_start_write() heavier but again, it's write-locking, so
should not be considered a fast path.
With this change we can use the code suggested by Andrii in
https://lore.kernel.org/all/CAEf4BzZeLg0WsYw2M7KFy0+APrPaPVBY7FbawB9vjcA2+6k69Q@mail.gmail.com/
with an additional smp_rmb():

rcu_read_lock()
vma = find_vma(...)
if (!vma) /* bail */

vm_lock_seq = smp_load_acquire(&vma->vm_lock_seq);
mm_lock_seq = smp_load_acquire(&vma->mm->mm_lock_seq);
/* I think vm_lock has to be acquired first to avoid the race */
if (mm_lock_seq == vm_lock_seq)
        /* bail, vma is write-locked */
... perform uprobe lookup logic based on vma->vm_file->f_inode ...
smp_rmb();
if (vma->vm_lock_seq != vm_lock_seq)
        /* bail, VMA might have changed */

The smp_rmb() is needed so that vma->vm_lock_seq load does not get
reordered and moved up before speculation.

I'm CC'ing Jann since he understands memory barriers way better than
me and will keep me honest.


>
> per-vma locking is still *locking*. Which means memory sharing between
> multiple CPUs, which means limited scalability. Lots of work in this
> series went to avoid even refcounting (as I pointed out for
> find_uprobe_rcu()) due to the same reason, and so relying on per-VMA
> locking is just shifting the bottleneck from mmap_lock to
> vma->vm_lock. Worst (and not uncommon) case is the same uprobe in the
> same process (and thus vma) being hit on multiple CPUs at the same
> time. Whether that's protected by mmap_lock or vma->vm_lock is
> immaterial at that point (from scalability standpoint).
>
> >
> > I presume a down_read on vma around all the work would also sort out any
> > issues concerning stability of the file or inode objects.
> >
> > Of course single-threaded performance would take a hit due to atomic
> > stemming from down/up_read and parallel uprobe lookups on the same vma
> > would also get slower, but I don't know if that's a problem for a real
> > workload.
> >
> > I would not have any comments if all speed ups were achieved without
> > modifying non-uprobe code.
>
> I'm also not a mm-focused person, so I'll let Suren and others address
> mm-specific concerns, but I (hopefully) addressed all the
> uprobe-related questions and concerns you had.


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 17:45           ` Suren Baghdasaryan
@ 2024-08-15 18:24             ` Mateusz Guzik
  2024-08-15 18:58             ` Jann Horn
  1 sibling, 0 replies; 40+ messages in thread
From: Mateusz Guzik @ 2024-08-15 18:24 UTC (permalink / raw)
  To: Suren Baghdasaryan
  Cc: Andrii Nakryiko, Andrii Nakryiko, linux-trace-kernel, peterz,
	oleg, rostedt, mhiramat, bpf, linux-kernel, jolsa, paulmck, willy,
	akpm, linux-mm, Jann Horn

On Thu, Aug 15, 2024 at 10:45:45AM -0700, Suren Baghdasaryan wrote:
> >From all the above, my understanding of your objection is that
> checking mmap_lock during our speculation is too coarse-grained and
> you would prefer to use the VMA seq counter to check that the VMA we
> are working on is unchanged. I agree, that would be ideal. I had a
> quick chat with Jann about this and the conclusion we came to is that
> we would need to add an additional smp_wmb() barrier inside
> vma_start_write() and a smp_rmb() in the speculation code:
> 
> static inline void vma_start_write(struct vm_area_struct *vma)
> {
>         int mm_lock_seq;
> 
>         if (__is_vma_write_locked(vma, &mm_lock_seq))
>                 return;
> 
>         down_write(&vma->vm_lock->lock);
>         /*
>          * We should use WRITE_ONCE() here because we can have concurrent reads
>          * from the early lockless pessimistic check in vma_start_read().
>          * We don't really care about the correctness of that early check, but
>          * we should use WRITE_ONCE() for cleanliness and to keep KCSAN happy.
>          */
>         WRITE_ONCE(vma->vm_lock_seq, mm_lock_seq);
> +        smp_wmb();
>         up_write(&vma->vm_lock->lock);
> }
> 
> Note: up_write(&vma->vm_lock->lock) in the vma_start_write() is not
> enough because it's one-way permeable (it's a "RELEASE operation") and
> later vma->vm_file store (or any other VMA modification) can move
> before our vma->vm_lock_seq store.
> 
> This makes vma_start_write() heavier but again, it's write-locking, so
> should not be considered a fast path.
> With this change we can use the code suggested by Andrii in
> https://lore.kernel.org/all/CAEf4BzZeLg0WsYw2M7KFy0+APrPaPVBY7FbawB9vjcA2+6k69Q@mail.gmail.com/
> with an additional smp_rmb():
> 
> rcu_read_lock()
> vma = find_vma(...)
> if (!vma) /* bail */
> 
> vm_lock_seq = smp_load_acquire(&vma->vm_lock_seq);
> mm_lock_seq = smp_load_acquire(&vma->mm->mm_lock_seq);
> /* I think vm_lock has to be acquired first to avoid the race */
> if (mm_lock_seq == vm_lock_seq)
>         /* bail, vma is write-locked */
> ... perform uprobe lookup logic based on vma->vm_file->f_inode ...
> smp_rmb();
> if (vma->vm_lock_seq != vm_lock_seq)
>         /* bail, VMA might have changed */
> 
> The smp_rmb() is needed so that vma->vm_lock_seq load does not get
> reordered and moved up before speculation.
> 
> I'm CC'ing Jann since he understands memory barriers way better than
> me and will keep me honest.
> 

So I briefly noted that maybe down_read on the vma would do it, but per
Andrii parallel lookups on the same vma on multiple CPUs are expected,
which whacks that out.

When I initially mentioned per-vma sequence counters I blindly assumed
they worked the usual way. I don't believe any fancy rework here is
warranted especially given that the per-mm counter thing is expected to
have other uses.

However, chances are decent this can still be worked out with per-vma
granualarity all while avoiding any stores on lookup and without
invasive (or complicated) changes. The lockless uprobe code claims to
guarantee only false negatives and the miss always falls back to the
mmap semaphore lookup. There may be something here, I'm going to chew on
it.

That said, thank you both for writeup so far.


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 17:45           ` Suren Baghdasaryan
  2024-08-15 18:24             ` Mateusz Guzik
@ 2024-08-15 18:58             ` Jann Horn
  2024-08-15 19:07               ` Mateusz Guzik
                                 ` (2 more replies)
  1 sibling, 3 replies; 40+ messages in thread
From: Jann Horn @ 2024-08-15 18:58 UTC (permalink / raw)
  To: Suren Baghdasaryan, Christian Brauner
  Cc: Andrii Nakryiko, Mateusz Guzik, Andrii Nakryiko,
	linux-trace-kernel, peterz, oleg, rostedt, mhiramat, bpf,
	linux-kernel, jolsa, paulmck, willy, akpm, linux-mm

+brauner for "struct file" lifetime

On Thu, Aug 15, 2024 at 7:45 PM Suren Baghdasaryan <surenb@google.com> wrote:
> On Thu, Aug 15, 2024 at 9:47 AM Andrii Nakryiko
> <andrii.nakryiko@gmail.com> wrote:
> >
> > On Thu, Aug 15, 2024 at 6:44 AM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > >
> > > On Tue, Aug 13, 2024 at 08:36:03AM -0700, Suren Baghdasaryan wrote:
> > > > On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > > > >
> > > > > On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > > > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > > > attempting uprobe look up speculatively.

Stupid question: Is this uprobe stuff actually such a hot codepath
that it makes sense to optimize it to be faster than the page fault
path?

(Sidenote: I find it kinda interesting that this is sort of going back
in the direction of the old Speculative Page Faults design.)

> > > > > > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > > > > > validate that mm_struct stays intact for entire duration of this
> > > > > > speculation. If not, we fall back to mmap_lock-protected lookup.
> > > > > >
> > > > > > This allows to avoid contention on mmap_lock in absolutely majority of
> > > > > > cases, nicely improving uprobe/uretprobe scalability.
> > > > > >
> > > > >
> > > > > Here I have to admit to being mostly ignorant about the mm, so bear with
> > > > > me. :>
> > > > >
> > > > > I note the result of find_active_uprobe_speculative is immediately stale
> > > > > in face of modifications.
> > > > >
> > > > > The thing I'm after is that the mmap_lock_speculation business adds
> > > > > overhead on archs where a release fence is not a de facto nop and I
> > > > > don't believe the commit message justifies it. Definitely a bummer to
> > > > > add merely it for uprobes. If there are bigger plans concerning it
> > > > > that's a different story of course.
> > > > >
> > > > > With this in mind I have to ask if instead you could perhaps get away
> > > > > with the already present per-vma sequence counter?
> > > >
> > > > per-vma sequence counter does not implement acquire/release logic, it
> > > > relies on vma->vm_lock for synchronization. So if we want to use it,
> > > > we would have to add additional memory barriers here. This is likely
> > > > possible but as I mentioned before we would need to ensure the
> > > > pagefault path does not regress. OTOH mm->mm_lock_seq already halfway
> > > > there (it implements acquire/release logic), we just had to ensure
> > > > mmap_write_lock() increments mm->mm_lock_seq.
> > > >
> > > > So, from the release fence overhead POV I think whether we use
> > > > mm->mm_lock_seq or vma->vm_lock, we would still need a proper fence
> > > > here.
> > > >
> > >
> > > Per my previous e-mail I'm not particularly familiar with mm internals,
> > > so I'm going to handwave a little bit with my $0,03 concerning multicore
> > > in general and if you disagree with it that's your business. For the
> > > time being I have no interest in digging into any of this.
> > >
> > > Before I do, to prevent this thread from being a total waste, here are
> > > some remarks concerning the patch with the assumption that the core idea
> > > lands.
> > >
> > > From the commit message:
> > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > attempting uprobe look up speculatively.
> > >
> > > Just in case I'll note a nit that this paragraph will need to be removed
> > > since the patch adding the flag is getting dropped.
> >
> > Yep, of course, I'll update all that for the next revision (I'll wait
> > for non-RFC patches to land first before reposting).
> >
> > >
> > > A non-nit which may or may not end up mattering is that the flag (which
> > > *is* set on the filep slab cache) makes things more difficult to
> > > validate. Normal RCU usage guarantees that the object itself wont be
> > > freed as long you follow the rules. However, the SLAB_TYPESAFE_BY_RCU
> > > flag weakens it significantly -- the thing at hand will always be a
> > > 'struct file', but it may get reallocated to *another* file from under
> > > you. Whether this aspect plays a role here I don't know.
> >
> > Yes, that's ok and is accounted for. We care about that memory not
> > going even from under us (I'm not even sure if it matters that it is
> > still a struct file, tbh; I think that shouldn't matter as we are
> > prepared to deal with completely garbage values read from struct
> > file).
>
> Correct, with SLAB_TYPESAFE_BY_RCU we do need an additional check that
> vma->vm_file has not been freed and reused. That's where
> mmap_lock_speculation_{start|end} helps us. For vma->vm_file to change
> from under us one would have to take mmap_lock for write. If that
> happens mmap_lock_speculation_{start|end} should detect that and
> terminate our speculation.
>
> >
> > >
> > > > +static struct uprobe *find_active_uprobe_speculative(unsigned long bp_vaddr)
> > > > +{
> > > > +     const vm_flags_t flags = VM_HUGETLB | VM_MAYEXEC | VM_MAYSHARE;
> > > > +     struct mm_struct *mm = current->mm;
> > > > +     struct uprobe *uprobe;
> > > > +     struct vm_area_struct *vma;
> > > > +     struct file *vm_file;
> > > > +     struct inode *vm_inode;
> > > > +     unsigned long vm_pgoff, vm_start;
> > > > +     int seq;
> > > > +     loff_t offset;
> > > > +
> > > > +     if (!mmap_lock_speculation_start(mm, &seq))
> > > > +             return NULL;
> > > > +
> > > > +     rcu_read_lock();
> > > > +
> > >
> > > I don't think there is a correctness problem here, but entering rcu
> > > *after* deciding to speculatively do the lookup feels backwards.
> >
> > RCU should protect VMA and file, mm itself won't go anywhere, so this seems ok.
> >
> > >
> > > > +     vma = vma_lookup(mm, bp_vaddr);
> > > > +     if (!vma)
> > > > +             goto bail;
> > > > +
> > > > +     vm_file = data_race(vma->vm_file);
> > > > +     if (!vm_file || (vma->vm_flags & flags) != VM_MAYEXEC)
> > > > +             goto bail;
> > > > +
> > >
> > > If vma teardown is allowed to progress and the file got fput'ed...
> > >
> > > > +     vm_inode = data_race(vm_file->f_inode);
> > >
> > > ... the inode can be NULL, I don't know if that's handled.
> > >
> >
> > Yep, inode pointer value is part of RB-tree key, so if it's NULL, we
> > just won't find a matching uprobe. Same for any other "garbage"
> > f_inode value. Importantly, we never should dereference such inode
> > pointers, at least until we find a valid uprobe (in which case we keep
> > inode reference to it).
> >
> > > More importantly though, per my previous description of
> > > SLAB_TYPESAFE_BY_RCU, by now the file could have been reallocated and
> > > the inode you did find is completely unrelated.
> > >
> > > I understand the intent is to backpedal from everything should the mm
> > > seqc change, but the above may happen to matter.
> >
> > Yes, I think we took that into account. All that we care about is
> > memory "type safety", i.e., even if struct file's memory is reused,
> > it's ok, we'll eventually detect the change and will discard wrong
> > uprobe that we might by accident lookup (though probably in most cases
> > we just won't find a uprobe at all).
> >
> > >
> > > > +     vm_pgoff = data_race(vma->vm_pgoff);
> > > > +     vm_start = data_race(vma->vm_start);
> > > > +
> > > > +     offset = (loff_t)(vm_pgoff << PAGE_SHIFT) + (bp_vaddr - vm_start);
> > > > +     uprobe = find_uprobe_rcu(vm_inode, offset);
> > > > +     if (!uprobe)
> > > > +             goto bail;
> > > > +
> > > > +     /* now double check that nothing about MM changed */
> > > > +     if (!mmap_lock_speculation_end(mm, seq))
> > > > +             goto bail;
> > >
> > > This leaks the reference obtained by find_uprobe_rcu().
> >
> > find_uprobe_rcu() doesn't obtain a reference, uprobe is RCU-protected,
> > and if caller need a refcount bump it will have to use
> > try_get_uprobe() (which might fail).
> >
> > >
> > > > +
> > > > +     rcu_read_unlock();
> > > > +
> > > > +     /* happy case, we speculated successfully */
> > > > +     return uprobe;
> > > > +bail:
> > > > +     rcu_read_unlock();
> > > > +     return NULL;
> > > > +}
> > >
> > > Now to some handwaving, here it is:
> > >
> > > The core of my concern is that adding more work to down_write on the
> > > mmap semaphore comes with certain side-effects and plausibly more than a
> > > sufficient speed up can be achieved without doing it.
>
> AFAIK writers of mmap_lock are not considered a fast path. In a sense
> yes, we made any writer a bit heavier but OTOH we also made
> mm->mm_lock_seq a proper sequence count which allows us to locklessly
> check if mmap_lock is write-locked. I think you asked whether there
> will be other uses for mmap_lock_speculation_{start|end} and yes. For
> example, I am planning to use them for printing /proc/{pid}/maps
> without taking mmap_lock (when it's uncontended).

What would be the goal of this - to avoid cacheline bouncing of the
mmap lock between readers? Or to allow mmap_write_lock() to preempt
/proc/{pid}/maps readers who started out uncontended?

Is the idea that you'd change show_map_vma() to first do something
like get_file_active() to increment the file refcount (because
otherwise the dentry can be freed under you and you need the dentry
for path printing), then recheck your sequence count on the mm or vma
(to avoid accessing the dentry of an unrelated file that hasn't become
userspace-visible yet and may not have a proper dentry pointer yet),
then print the file path, drop the file reference again, and in the
end recheck the sequence count again before actually returning the
printed data to userspace?

> If we have VMA seq
> counter-based detection it would be better (see below).
>
> > >
> > > An mm-wide mechanism is just incredibly coarse-grained and it may happen
> > > to perform poorly when faced with a program which likes to mess with its
> > > address space -- the fast path is going to keep failing and only
> > > inducing *more* overhead as the code decides to down_read the mmap
> > > semaphore.
> > >
> > > Furthermore there may be work currently synchronized with down_write
> > > which perhaps can transition to "merely" down_read, but by the time it
> > > happens this and possibly other consumers expect a change in the
> > > sequence counter, messing with it.
> > >
> > > To my understanding the kernel supports parallel faults with per-vma
> > > locking. I would find it surprising if the same machinery could not be
> > > used to sort out uprobe handling above.
>
> From all the above, my understanding of your objection is that
> checking mmap_lock during our speculation is too coarse-grained and
> you would prefer to use the VMA seq counter to check that the VMA we
> are working on is unchanged. I agree, that would be ideal. I had a
> quick chat with Jann about this and the conclusion we came to is that
> we would need to add an additional smp_wmb() barrier inside
> vma_start_write() and a smp_rmb() in the speculation code:
>
> static inline void vma_start_write(struct vm_area_struct *vma)
> {
>         int mm_lock_seq;
>
>         if (__is_vma_write_locked(vma, &mm_lock_seq))
>                 return;
>
>         down_write(&vma->vm_lock->lock);
>         /*
>          * We should use WRITE_ONCE() here because we can have concurrent reads
>          * from the early lockless pessimistic check in vma_start_read().
>          * We don't really care about the correctness of that early check, but
>          * we should use WRITE_ONCE() for cleanliness and to keep KCSAN happy.
>          */
>         WRITE_ONCE(vma->vm_lock_seq, mm_lock_seq);
> +        smp_wmb();
>         up_write(&vma->vm_lock->lock);
> }
>
> Note: up_write(&vma->vm_lock->lock) in the vma_start_write() is not
> enough because it's one-way permeable (it's a "RELEASE operation") and
> later vma->vm_file store (or any other VMA modification) can move
> before our vma->vm_lock_seq store.
>
> This makes vma_start_write() heavier but again, it's write-locking, so
> should not be considered a fast path.
> With this change we can use the code suggested by Andrii in
> https://lore.kernel.org/all/CAEf4BzZeLg0WsYw2M7KFy0+APrPaPVBY7FbawB9vjcA2+6k69Q@mail.gmail.com/
> with an additional smp_rmb():
>
> rcu_read_lock()
> vma = find_vma(...)
> if (!vma) /* bail */

And maybe add some comments like:

/*
 * Load the current VMA lock sequence - we will detect if anyone concurrently
 * locks the VMA after this point.
 * Pairs with smp_wmb() in vma_start_write().
 */
> vm_lock_seq = smp_load_acquire(&vma->vm_lock_seq);
/*
 * Now we just have to detect if the VMA is already locked with its current
 * sequence count.
 *
 * The following load is ordered against the vm_lock_seq load above (using
 * smp_load_acquire() for the load above), and pairs with implicit memory
 * ordering between the mm_lock_seq write in mmap_write_unlock() and the
 * vm_lock_seq write in the next vma_start_write() after that (which can only
 * occur after an mmap_write_lock()).
 */
> mm_lock_seq = smp_load_acquire(&vma->mm->mm_lock_seq);
> /* I think vm_lock has to be acquired first to avoid the race */
> if (mm_lock_seq == vm_lock_seq)
>         /* bail, vma is write-locked */
> ... perform uprobe lookup logic based on vma->vm_file->f_inode ...
/*
 * Order the speculative accesses above against the following vm_lock_seq
 * recheck.
 */
> smp_rmb();
> if (vma->vm_lock_seq != vm_lock_seq)

(As I said on the other thread: Since this now relies on
vma->vm_lock_seq not wrapping back to the same value for correctness,
I'd like to see vma->vm_lock_seq being at least an "unsigned long", or
even better, an atomic64_t... though I realize we don't currently do
that for seqlocks either.)

>         /* bail, VMA might have changed */
>
> The smp_rmb() is needed so that vma->vm_lock_seq load does not get
> reordered and moved up before speculation.
>
> I'm CC'ing Jann since he understands memory barriers way better than
> me and will keep me honest.


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 18:58             ` Jann Horn
@ 2024-08-15 19:07               ` Mateusz Guzik
  2024-08-15 19:17                 ` Arnaldo Carvalho de Melo
  2024-08-15 19:44               ` Suren Baghdasaryan
  2024-08-15 20:17               ` Andrii Nakryiko
  2 siblings, 1 reply; 40+ messages in thread
From: Mateusz Guzik @ 2024-08-15 19:07 UTC (permalink / raw)
  To: Jann Horn
  Cc: Suren Baghdasaryan, Christian Brauner, Andrii Nakryiko,
	Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Thu, Aug 15, 2024 at 8:58 PM Jann Horn <jannh@google.com> wrote:
> Stupid question: Is this uprobe stuff actually such a hot codepath
> that it makes sense to optimize it to be faster than the page fault
> path?
>

That's what I implicitly asked, hoping a down_read on vma would do it,
but Andrii claims multiple parallel lookups on the same vma are a
problem.

Even so, I suspect something *simple* is doable here which avoids any
writes to vmas and does not need the mm-wide sequence counter. It may
be requirements are lax enough that merely observing some state is the
same before and after uprobe lookup will be sufficient, or maybe some
other hackery is viable without messing with fences in
vma_start_write.
-- 
Mateusz Guzik <mjguzik gmail.com>


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 19:07               ` Mateusz Guzik
@ 2024-08-15 19:17                 ` Arnaldo Carvalho de Melo
  2024-08-15 19:18                   ` Arnaldo Carvalho de Melo
  0 siblings, 1 reply; 40+ messages in thread
From: Arnaldo Carvalho de Melo @ 2024-08-15 19:17 UTC (permalink / raw)
  To: Mateusz Guzik
  Cc: Jann Horn, Suren Baghdasaryan, Christian Brauner, Andrii Nakryiko,
	Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

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

Um exemplo de uma modificação numa área central derivada de uma necessidade
específica: melhorar a performance de uprobes

On Thu, Aug 15, 2024, 4:07 PM Mateusz Guzik <mjguzik@gmail.com> wrote:

> On Thu, Aug 15, 2024 at 8:58 PM Jann Horn <jannh@google.com> wrote:
> > Stupid question: Is this uprobe stuff actually such a hot codepath
> > that it makes sense to optimize it to be faster than the page fault
> > path?
> >
>
> That's what I implicitly asked, hoping a down_read on vma would do it,
> but Andrii claims multiple parallel lookups on the same vma are a
> problem.
>
> Even so, I suspect something *simple* is doable here which avoids any
> writes to vmas and does not need the mm-wide sequence counter. It may
> be requirements are lax enough that merely observing some state is the
> same before and after uprobe lookup will be sufficient, or maybe some
> other hackery is viable without messing with fences in
> vma_start_write.
> --
> Mateusz Guzik <mjguzik gmail.com>
>
>

[-- Attachment #2: Type: text/html, Size: 1485 bytes --]

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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 19:17                 ` Arnaldo Carvalho de Melo
@ 2024-08-15 19:18                   ` Arnaldo Carvalho de Melo
  0 siblings, 0 replies; 40+ messages in thread
From: Arnaldo Carvalho de Melo @ 2024-08-15 19:18 UTC (permalink / raw)
  To: Mateusz Guzik
  Cc: Jann Horn, Suren Baghdasaryan, Christian Brauner, Andrii Nakryiko,
	Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

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

Sorry, nevermind, wrong forward :-/

On Thu, Aug 15, 2024, 4:17 PM Arnaldo Carvalho de Melo <
arnaldo.melo@gmail.com> wrote:

> Um exemplo de uma modificação numa área central derivada de uma
> necessidade específica: melhorar a performance de uprobes
>
> On Thu, Aug 15, 2024, 4:07 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
>
>> On Thu, Aug 15, 2024 at 8:58 PM Jann Horn <jannh@google.com> wrote:
>> > Stupid question: Is this uprobe stuff actually such a hot codepath
>> > that it makes sense to optimize it to be faster than the page fault
>> > path?
>> >
>>
>> That's what I implicitly asked, hoping a down_read on vma would do it,
>> but Andrii claims multiple parallel lookups on the same vma are a
>> problem.
>>
>> Even so, I suspect something *simple* is doable here which avoids any
>> writes to vmas and does not need the mm-wide sequence counter. It may
>> be requirements are lax enough that merely observing some state is the
>> same before and after uprobe lookup will be sufficient, or maybe some
>> other hackery is viable without messing with fences in
>> vma_start_write.
>> --
>> Mateusz Guzik <mjguzik gmail.com>
>>
>>

[-- Attachment #2: Type: text/html, Size: 1928 bytes --]

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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 18:58             ` Jann Horn
  2024-08-15 19:07               ` Mateusz Guzik
@ 2024-08-15 19:44               ` Suren Baghdasaryan
  2024-08-15 20:17               ` Andrii Nakryiko
  2 siblings, 0 replies; 40+ messages in thread
From: Suren Baghdasaryan @ 2024-08-15 19:44 UTC (permalink / raw)
  To: Jann Horn
  Cc: Christian Brauner, Andrii Nakryiko, Mateusz Guzik,
	Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Thu, Aug 15, 2024 at 11:58 AM Jann Horn <jannh@google.com> wrote:
>
> +brauner for "struct file" lifetime
>
> On Thu, Aug 15, 2024 at 7:45 PM Suren Baghdasaryan <surenb@google.com> wrote:
> > On Thu, Aug 15, 2024 at 9:47 AM Andrii Nakryiko
> > <andrii.nakryiko@gmail.com> wrote:
> > >
> > > On Thu, Aug 15, 2024 at 6:44 AM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > > >
> > > > On Tue, Aug 13, 2024 at 08:36:03AM -0700, Suren Baghdasaryan wrote:
> > > > > On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > > > > >
> > > > > > On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > > > > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > > > > attempting uprobe look up speculatively.
>
> Stupid question: Is this uprobe stuff actually such a hot codepath
> that it makes sense to optimize it to be faster than the page fault
> path?
>
> (Sidenote: I find it kinda interesting that this is sort of going back
> in the direction of the old Speculative Page Faults design.)
>
> > > > > > > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > > > > > > validate that mm_struct stays intact for entire duration of this
> > > > > > > speculation. If not, we fall back to mmap_lock-protected lookup.
> > > > > > >
> > > > > > > This allows to avoid contention on mmap_lock in absolutely majority of
> > > > > > > cases, nicely improving uprobe/uretprobe scalability.
> > > > > > >
> > > > > >
> > > > > > Here I have to admit to being mostly ignorant about the mm, so bear with
> > > > > > me. :>
> > > > > >
> > > > > > I note the result of find_active_uprobe_speculative is immediately stale
> > > > > > in face of modifications.
> > > > > >
> > > > > > The thing I'm after is that the mmap_lock_speculation business adds
> > > > > > overhead on archs where a release fence is not a de facto nop and I
> > > > > > don't believe the commit message justifies it. Definitely a bummer to
> > > > > > add merely it for uprobes. If there are bigger plans concerning it
> > > > > > that's a different story of course.
> > > > > >
> > > > > > With this in mind I have to ask if instead you could perhaps get away
> > > > > > with the already present per-vma sequence counter?
> > > > >
> > > > > per-vma sequence counter does not implement acquire/release logic, it
> > > > > relies on vma->vm_lock for synchronization. So if we want to use it,
> > > > > we would have to add additional memory barriers here. This is likely
> > > > > possible but as I mentioned before we would need to ensure the
> > > > > pagefault path does not regress. OTOH mm->mm_lock_seq already halfway
> > > > > there (it implements acquire/release logic), we just had to ensure
> > > > > mmap_write_lock() increments mm->mm_lock_seq.
> > > > >
> > > > > So, from the release fence overhead POV I think whether we use
> > > > > mm->mm_lock_seq or vma->vm_lock, we would still need a proper fence
> > > > > here.
> > > > >
> > > >
> > > > Per my previous e-mail I'm not particularly familiar with mm internals,
> > > > so I'm going to handwave a little bit with my $0,03 concerning multicore
> > > > in general and if you disagree with it that's your business. For the
> > > > time being I have no interest in digging into any of this.
> > > >
> > > > Before I do, to prevent this thread from being a total waste, here are
> > > > some remarks concerning the patch with the assumption that the core idea
> > > > lands.
> > > >
> > > > From the commit message:
> > > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > > attempting uprobe look up speculatively.
> > > >
> > > > Just in case I'll note a nit that this paragraph will need to be removed
> > > > since the patch adding the flag is getting dropped.
> > >
> > > Yep, of course, I'll update all that for the next revision (I'll wait
> > > for non-RFC patches to land first before reposting).
> > >
> > > >
> > > > A non-nit which may or may not end up mattering is that the flag (which
> > > > *is* set on the filep slab cache) makes things more difficult to
> > > > validate. Normal RCU usage guarantees that the object itself wont be
> > > > freed as long you follow the rules. However, the SLAB_TYPESAFE_BY_RCU
> > > > flag weakens it significantly -- the thing at hand will always be a
> > > > 'struct file', but it may get reallocated to *another* file from under
> > > > you. Whether this aspect plays a role here I don't know.
> > >
> > > Yes, that's ok and is accounted for. We care about that memory not
> > > going even from under us (I'm not even sure if it matters that it is
> > > still a struct file, tbh; I think that shouldn't matter as we are
> > > prepared to deal with completely garbage values read from struct
> > > file).
> >
> > Correct, with SLAB_TYPESAFE_BY_RCU we do need an additional check that
> > vma->vm_file has not been freed and reused. That's where
> > mmap_lock_speculation_{start|end} helps us. For vma->vm_file to change
> > from under us one would have to take mmap_lock for write. If that
> > happens mmap_lock_speculation_{start|end} should detect that and
> > terminate our speculation.
> >
> > >
> > > >
> > > > > +static struct uprobe *find_active_uprobe_speculative(unsigned long bp_vaddr)
> > > > > +{
> > > > > +     const vm_flags_t flags = VM_HUGETLB | VM_MAYEXEC | VM_MAYSHARE;
> > > > > +     struct mm_struct *mm = current->mm;
> > > > > +     struct uprobe *uprobe;
> > > > > +     struct vm_area_struct *vma;
> > > > > +     struct file *vm_file;
> > > > > +     struct inode *vm_inode;
> > > > > +     unsigned long vm_pgoff, vm_start;
> > > > > +     int seq;
> > > > > +     loff_t offset;
> > > > > +
> > > > > +     if (!mmap_lock_speculation_start(mm, &seq))
> > > > > +             return NULL;
> > > > > +
> > > > > +     rcu_read_lock();
> > > > > +
> > > >
> > > > I don't think there is a correctness problem here, but entering rcu
> > > > *after* deciding to speculatively do the lookup feels backwards.
> > >
> > > RCU should protect VMA and file, mm itself won't go anywhere, so this seems ok.
> > >
> > > >
> > > > > +     vma = vma_lookup(mm, bp_vaddr);
> > > > > +     if (!vma)
> > > > > +             goto bail;
> > > > > +
> > > > > +     vm_file = data_race(vma->vm_file);
> > > > > +     if (!vm_file || (vma->vm_flags & flags) != VM_MAYEXEC)
> > > > > +             goto bail;
> > > > > +
> > > >
> > > > If vma teardown is allowed to progress and the file got fput'ed...
> > > >
> > > > > +     vm_inode = data_race(vm_file->f_inode);
> > > >
> > > > ... the inode can be NULL, I don't know if that's handled.
> > > >
> > >
> > > Yep, inode pointer value is part of RB-tree key, so if it's NULL, we
> > > just won't find a matching uprobe. Same for any other "garbage"
> > > f_inode value. Importantly, we never should dereference such inode
> > > pointers, at least until we find a valid uprobe (in which case we keep
> > > inode reference to it).
> > >
> > > > More importantly though, per my previous description of
> > > > SLAB_TYPESAFE_BY_RCU, by now the file could have been reallocated and
> > > > the inode you did find is completely unrelated.
> > > >
> > > > I understand the intent is to backpedal from everything should the mm
> > > > seqc change, but the above may happen to matter.
> > >
> > > Yes, I think we took that into account. All that we care about is
> > > memory "type safety", i.e., even if struct file's memory is reused,
> > > it's ok, we'll eventually detect the change and will discard wrong
> > > uprobe that we might by accident lookup (though probably in most cases
> > > we just won't find a uprobe at all).
> > >
> > > >
> > > > > +     vm_pgoff = data_race(vma->vm_pgoff);
> > > > > +     vm_start = data_race(vma->vm_start);
> > > > > +
> > > > > +     offset = (loff_t)(vm_pgoff << PAGE_SHIFT) + (bp_vaddr - vm_start);
> > > > > +     uprobe = find_uprobe_rcu(vm_inode, offset);
> > > > > +     if (!uprobe)
> > > > > +             goto bail;
> > > > > +
> > > > > +     /* now double check that nothing about MM changed */
> > > > > +     if (!mmap_lock_speculation_end(mm, seq))
> > > > > +             goto bail;
> > > >
> > > > This leaks the reference obtained by find_uprobe_rcu().
> > >
> > > find_uprobe_rcu() doesn't obtain a reference, uprobe is RCU-protected,
> > > and if caller need a refcount bump it will have to use
> > > try_get_uprobe() (which might fail).
> > >
> > > >
> > > > > +
> > > > > +     rcu_read_unlock();
> > > > > +
> > > > > +     /* happy case, we speculated successfully */
> > > > > +     return uprobe;
> > > > > +bail:
> > > > > +     rcu_read_unlock();
> > > > > +     return NULL;
> > > > > +}
> > > >
> > > > Now to some handwaving, here it is:
> > > >
> > > > The core of my concern is that adding more work to down_write on the
> > > > mmap semaphore comes with certain side-effects and plausibly more than a
> > > > sufficient speed up can be achieved without doing it.
> >
> > AFAIK writers of mmap_lock are not considered a fast path. In a sense
> > yes, we made any writer a bit heavier but OTOH we also made
> > mm->mm_lock_seq a proper sequence count which allows us to locklessly
> > check if mmap_lock is write-locked. I think you asked whether there
> > will be other uses for mmap_lock_speculation_{start|end} and yes. For
> > example, I am planning to use them for printing /proc/{pid}/maps
> > without taking mmap_lock (when it's uncontended).
>
> What would be the goal of this - to avoid cacheline bouncing of the
> mmap lock between readers? Or to allow mmap_write_lock() to preempt
> /proc/{pid}/maps readers who started out uncontended?

The latter, from my early patchset which I need to refine
(https://lore.kernel.org/all/20240123231014.3801041-3-surenb@google.com/):

This change is designed to reduce mmap_lock contention and prevent a
process reading /proc/pid/maps files (often a low priority task, such as
monitoring/data collection services) from blocking address space updates.

>
> Is the idea that you'd change show_map_vma() to first do something
> like get_file_active() to increment the file refcount (because
> otherwise the dentry can be freed under you and you need the dentry
> for path printing), then recheck your sequence count on the mm or vma
> (to avoid accessing the dentry of an unrelated file that hasn't become
> userspace-visible yet and may not have a proper dentry pointer yet),
> then print the file path, drop the file reference again, and in the
> end recheck the sequence count again before actually returning the
> printed data to userspace?

Yeah, you can see the details in that link I posted above. See
get_vma_snapshot() function.

>
> > If we have VMA seq
> > counter-based detection it would be better (see below).
> >
> > > >
> > > > An mm-wide mechanism is just incredibly coarse-grained and it may happen
> > > > to perform poorly when faced with a program which likes to mess with its
> > > > address space -- the fast path is going to keep failing and only
> > > > inducing *more* overhead as the code decides to down_read the mmap
> > > > semaphore.
> > > >
> > > > Furthermore there may be work currently synchronized with down_write
> > > > which perhaps can transition to "merely" down_read, but by the time it
> > > > happens this and possibly other consumers expect a change in the
> > > > sequence counter, messing with it.
> > > >
> > > > To my understanding the kernel supports parallel faults with per-vma
> > > > locking. I would find it surprising if the same machinery could not be
> > > > used to sort out uprobe handling above.
> >
> > From all the above, my understanding of your objection is that
> > checking mmap_lock during our speculation is too coarse-grained and
> > you would prefer to use the VMA seq counter to check that the VMA we
> > are working on is unchanged. I agree, that would be ideal. I had a
> > quick chat with Jann about this and the conclusion we came to is that
> > we would need to add an additional smp_wmb() barrier inside
> > vma_start_write() and a smp_rmb() in the speculation code:
> >
> > static inline void vma_start_write(struct vm_area_struct *vma)
> > {
> >         int mm_lock_seq;
> >
> >         if (__is_vma_write_locked(vma, &mm_lock_seq))
> >                 return;
> >
> >         down_write(&vma->vm_lock->lock);
> >         /*
> >          * We should use WRITE_ONCE() here because we can have concurrent reads
> >          * from the early lockless pessimistic check in vma_start_read().
> >          * We don't really care about the correctness of that early check, but
> >          * we should use WRITE_ONCE() for cleanliness and to keep KCSAN happy.
> >          */
> >         WRITE_ONCE(vma->vm_lock_seq, mm_lock_seq);
> > +        smp_wmb();
> >         up_write(&vma->vm_lock->lock);
> > }
> >
> > Note: up_write(&vma->vm_lock->lock) in the vma_start_write() is not
> > enough because it's one-way permeable (it's a "RELEASE operation") and
> > later vma->vm_file store (or any other VMA modification) can move
> > before our vma->vm_lock_seq store.
> >
> > This makes vma_start_write() heavier but again, it's write-locking, so
> > should not be considered a fast path.
> > With this change we can use the code suggested by Andrii in
> > https://lore.kernel.org/all/CAEf4BzZeLg0WsYw2M7KFy0+APrPaPVBY7FbawB9vjcA2+6k69Q@mail.gmail.com/
> > with an additional smp_rmb():
> >
> > rcu_read_lock()
> > vma = find_vma(...)
> > if (!vma) /* bail */
>
> And maybe add some comments like:
>
> /*
>  * Load the current VMA lock sequence - we will detect if anyone concurrently
>  * locks the VMA after this point.
>  * Pairs with smp_wmb() in vma_start_write().
>  */
> > vm_lock_seq = smp_load_acquire(&vma->vm_lock_seq);
> /*
>  * Now we just have to detect if the VMA is already locked with its current
>  * sequence count.
>  *
>  * The following load is ordered against the vm_lock_seq load above (using
>  * smp_load_acquire() for the load above), and pairs with implicit memory
>  * ordering between the mm_lock_seq write in mmap_write_unlock() and the
>  * vm_lock_seq write in the next vma_start_write() after that (which can only
>  * occur after an mmap_write_lock()).
>  */
> > mm_lock_seq = smp_load_acquire(&vma->mm->mm_lock_seq);
> > /* I think vm_lock has to be acquired first to avoid the race */
> > if (mm_lock_seq == vm_lock_seq)
> >         /* bail, vma is write-locked */
> > ... perform uprobe lookup logic based on vma->vm_file->f_inode ...
> /*
>  * Order the speculative accesses above against the following vm_lock_seq
>  * recheck.
>  */
> > smp_rmb();
> > if (vma->vm_lock_seq != vm_lock_seq)
>
> (As I said on the other thread: Since this now relies on
> vma->vm_lock_seq not wrapping back to the same value for correctness,
> I'd like to see vma->vm_lock_seq being at least an "unsigned long", or
> even better, an atomic64_t... though I realize we don't currently do
> that for seqlocks either.)
>
> >         /* bail, VMA might have changed */
> >
> > The smp_rmb() is needed so that vma->vm_lock_seq load does not get
> > reordered and moved up before speculation.
> >
> > I'm CC'ing Jann since he understands memory barriers way better than
> > me and will keep me honest.


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

* Re: [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution
  2024-08-15 18:58             ` Jann Horn
  2024-08-15 19:07               ` Mateusz Guzik
  2024-08-15 19:44               ` Suren Baghdasaryan
@ 2024-08-15 20:17               ` Andrii Nakryiko
  2 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-15 20:17 UTC (permalink / raw)
  To: Jann Horn
  Cc: Suren Baghdasaryan, Christian Brauner, Mateusz Guzik,
	Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, jolsa, paulmck, willy, akpm,
	linux-mm

On Thu, Aug 15, 2024 at 11:58 AM Jann Horn <jannh@google.com> wrote:
>
> +brauner for "struct file" lifetime
>
> On Thu, Aug 15, 2024 at 7:45 PM Suren Baghdasaryan <surenb@google.com> wrote:
> > On Thu, Aug 15, 2024 at 9:47 AM Andrii Nakryiko
> > <andrii.nakryiko@gmail.com> wrote:
> > >
> > > On Thu, Aug 15, 2024 at 6:44 AM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > > >
> > > > On Tue, Aug 13, 2024 at 08:36:03AM -0700, Suren Baghdasaryan wrote:
> > > > > On Mon, Aug 12, 2024 at 11:18 PM Mateusz Guzik <mjguzik@gmail.com> wrote:
> > > > > >
> > > > > > On Mon, Aug 12, 2024 at 09:29:17PM -0700, Andrii Nakryiko wrote:
> > > > > > > Now that files_cachep is SLAB_TYPESAFE_BY_RCU, we can safely access
> > > > > > > vma->vm_file->f_inode lockless only under rcu_read_lock() protection,
> > > > > > > attempting uprobe look up speculatively.
>
> Stupid question: Is this uprobe stuff actually such a hot codepath
> that it makes sense to optimize it to be faster than the page fault
> path?

Not a stupid question, but yes, generally speaking uprobe performance
is critical for a bunch of tracing use cases. And having independent
threads implicitly contending with each other just because of uprobe's
internal implementation detail (while conceptually there should be no
dependencies for triggering uprobe from multiple parallel threads) is
a big surprise to users and affects production use cases beyond just
uprobe-handling BPF logic overhead ("useful overhead") they assume.

>
> (Sidenote: I find it kinda interesting that this is sort of going back
> in the direction of the old Speculative Page Faults design.)
>
> > > > > > > We rely on newly added mmap_lock_speculation_{start,end}() helpers to
> > > > > > > validate that mm_struct stays intact for entire duration of this
> > > > > > > speculation. If not, we fall back to mmap_lock-protected lookup.
> > > > > > >
> > > > > > > This allows to avoid contention on mmap_lock in absolutely majority of
> > > > > > > cases, nicely improving uprobe/uretprobe scalability.
> > > > > > >
> > > > > >

[...]

> > Note: up_write(&vma->vm_lock->lock) in the vma_start_write() is not
> > enough because it's one-way permeable (it's a "RELEASE operation") and
> > later vma->vm_file store (or any other VMA modification) can move
> > before our vma->vm_lock_seq store.
> >
> > This makes vma_start_write() heavier but again, it's write-locking, so
> > should not be considered a fast path.
> > With this change we can use the code suggested by Andrii in
> > https://lore.kernel.org/all/CAEf4BzZeLg0WsYw2M7KFy0+APrPaPVBY7FbawB9vjcA2+6k69Q@mail.gmail.com/
> > with an additional smp_rmb():
> >
> > rcu_read_lock()
> > vma = find_vma(...)
> > if (!vma) /* bail */
>
> And maybe add some comments like:
>
> /*
>  * Load the current VMA lock sequence - we will detect if anyone concurrently
>  * locks the VMA after this point.
>  * Pairs with smp_wmb() in vma_start_write().
>  */
> > vm_lock_seq = smp_load_acquire(&vma->vm_lock_seq);
> /*
>  * Now we just have to detect if the VMA is already locked with its current
>  * sequence count.
>  *
>  * The following load is ordered against the vm_lock_seq load above (using
>  * smp_load_acquire() for the load above), and pairs with implicit memory
>  * ordering between the mm_lock_seq write in mmap_write_unlock() and the
>  * vm_lock_seq write in the next vma_start_write() after that (which can only
>  * occur after an mmap_write_lock()).
>  */
> > mm_lock_seq = smp_load_acquire(&vma->mm->mm_lock_seq);
> > /* I think vm_lock has to be acquired first to avoid the race */
> > if (mm_lock_seq == vm_lock_seq)
> >         /* bail, vma is write-locked */
> > ... perform uprobe lookup logic based on vma->vm_file->f_inode ...
> /*
>  * Order the speculative accesses above against the following vm_lock_seq
>  * recheck.
>  */
> > smp_rmb();
> > if (vma->vm_lock_seq != vm_lock_seq)
>

thanks, will incorporate these comments into the next revision

> (As I said on the other thread: Since this now relies on
> vma->vm_lock_seq not wrapping back to the same value for correctness,
> I'd like to see vma->vm_lock_seq being at least an "unsigned long", or
> even better, an atomic64_t... though I realize we don't currently do
> that for seqlocks either.)
>
> >         /* bail, VMA might have changed */
> >
> > The smp_rmb() is needed so that vma->vm_lock_seq load does not get
> > reordered and moved up before speculation.
> >
> > I'm CC'ing Jann since he understands memory barriers way better than
> > me and will keep me honest.


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

* Re: [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-08-13  4:29 ` [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
@ 2024-08-19 13:41   ` Oleg Nesterov
  2024-08-19 20:34     ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Oleg Nesterov @ 2024-08-19 13:41 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, peterz, rostedt, mhiramat, bpf, linux-kernel,
	jolsa, paulmck, willy, surenb, akpm, linux-mm

On 08/12, Andrii Nakryiko wrote:
>
> Avoid taking refcount on uprobe in prepare_uretprobe(), instead take
> uretprobe-specific SRCU lock and keep it active as kernel transfers
> control back to user space.
...
>  include/linux/uprobes.h |  49 ++++++-
>  kernel/events/uprobes.c | 294 ++++++++++++++++++++++++++++++++++------
>  2 files changed, 301 insertions(+), 42 deletions(-)

Oh. To be honest I don't like this patch.

I would like to know what other reviewers think, but to me it adds too many
complications that I can't even fully understand...

And how much does it help performance-wise?

I'll try to take another look, and I'll try to think about other approaches,
not that I have something better in mind...


But lets forgets this patch for the moment. The next one adds even more
complications, and I think it doesn't make sense.

As I have already mentioned in the previous discussions, we can simply kill
utask->active_uprobe. And utask->auprobe.

So can't we start with the patch below? On top of your 08/13. It doesn't kill
utask->auprobe yet, this needs a bit more trivial changes.

What do you think?

Oleg.

-------------------------------------------------------------------------------
From d7cb674eb6f7bb891408b2b6a5fb872a6c2f0f6c Mon Sep 17 00:00:00 2001
From: Oleg Nesterov <oleg@redhat.com>
Date: Mon, 19 Aug 2024 15:34:55 +0200
Subject: [RFC PATCH] uprobe: kill uprobe_task->active_uprobe

Untested, not for inclusion yet, and I need to split it into 2 changes.
It does 2 simple things:

	1. active_uprobe != NULL is possible if and only if utask->state != 0,
	   so it turns the active_uprobe checks into the utask->state checks.

	2. handle_singlestep() doesn't really need ->active_uprobe, it only
	   needs uprobe->arch which is "const" after prepare_uprobe().

	   So this patch adds the new "arch_uprobe uarch" member into utask
	   and changes pre_ssout() to do memcpy(&utask->uarch, &uprobe->arch).
---
 include/linux/uprobes.h |  2 +-
 kernel/events/uprobes.c | 37 +++++++++++--------------------------
 2 files changed, 12 insertions(+), 27 deletions(-)

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index 3a3154b74fe0..df6f3dab032c 100644
--- a/include/linux/uprobes.h
+++ b/include/linux/uprobes.h
@@ -56,6 +56,7 @@ struct uprobe_task {
 
 	union {
 		struct {
+			struct arch_uprobe	uarch;
 			struct arch_uprobe_task	autask;
 			unsigned long		vaddr;
 		};
@@ -66,7 +67,6 @@ struct uprobe_task {
 		};
 	};
 
-	struct uprobe			*active_uprobe;
 	unsigned long			xol_vaddr;
 
 	struct arch_uprobe              *auprobe;
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index acc73c1bc54c..9689b557a5cf 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -1721,7 +1721,7 @@ unsigned long uprobe_get_trap_addr(struct pt_regs *regs)
 {
 	struct uprobe_task *utask = current->utask;
 
-	if (unlikely(utask && utask->active_uprobe))
+	if (unlikely(utask && utask->state))
 		return utask->vaddr;
 
 	return instruction_pointer(regs);
@@ -1747,9 +1747,6 @@ void uprobe_free_utask(struct task_struct *t)
 	if (!utask)
 		return;
 
-	if (utask->active_uprobe)
-		put_uprobe(utask->active_uprobe);
-
 	ri = utask->return_instances;
 	while (ri)
 		ri = free_ret_instance(ri);
@@ -1965,14 +1962,9 @@ pre_ssout(struct uprobe *uprobe, struct pt_regs *regs, unsigned long bp_vaddr)
 	if (!utask)
 		return -ENOMEM;
 
-	if (!try_get_uprobe(uprobe))
-		return -EINVAL;
-
 	xol_vaddr = xol_get_insn_slot(uprobe);
-	if (!xol_vaddr) {
-		err = -ENOMEM;
-		goto err_out;
-	}
+	if (!xol_vaddr)
+		return -ENOMEM;
 
 	utask->xol_vaddr = xol_vaddr;
 	utask->vaddr = bp_vaddr;
@@ -1980,15 +1972,12 @@ pre_ssout(struct uprobe *uprobe, struct pt_regs *regs, unsigned long bp_vaddr)
 	err = arch_uprobe_pre_xol(&uprobe->arch, regs);
 	if (unlikely(err)) {
 		xol_free_insn_slot(current);
-		goto err_out;
+		return err;
 	}
 
-	utask->active_uprobe = uprobe;
+	memcpy(&utask->uarch, &uprobe->arch, sizeof(utask->uarch));
 	utask->state = UTASK_SSTEP;
 	return 0;
-err_out:
-	put_uprobe(uprobe);
-	return err;
 }
 
 /*
@@ -2005,7 +1994,7 @@ bool uprobe_deny_signal(void)
 	struct task_struct *t = current;
 	struct uprobe_task *utask = t->utask;
 
-	if (likely(!utask || !utask->active_uprobe))
+	if (likely(!utask || !utask->state))
 		return false;
 
 	WARN_ON_ONCE(utask->state != UTASK_SSTEP);
@@ -2313,19 +2302,15 @@ static void handle_swbp(struct pt_regs *regs)
  */
 static void handle_singlestep(struct uprobe_task *utask, struct pt_regs *regs)
 {
-	struct uprobe *uprobe;
 	int err = 0;
 
-	uprobe = utask->active_uprobe;
 	if (utask->state == UTASK_SSTEP_ACK)
-		err = arch_uprobe_post_xol(&uprobe->arch, regs);
+		err = arch_uprobe_post_xol(&utask->uarch, regs);
 	else if (utask->state == UTASK_SSTEP_TRAPPED)
-		arch_uprobe_abort_xol(&uprobe->arch, regs);
+		arch_uprobe_abort_xol(&utask->uarch, regs);
 	else
 		WARN_ON_ONCE(1);
 
-	put_uprobe(uprobe);
-	utask->active_uprobe = NULL;
 	utask->state = UTASK_RUNNING;
 	xol_free_insn_slot(current);
 
@@ -2342,7 +2327,7 @@ static void handle_singlestep(struct uprobe_task *utask, struct pt_regs *regs)
 /*
  * On breakpoint hit, breakpoint notifier sets the TIF_UPROBE flag and
  * allows the thread to return from interrupt. After that handle_swbp()
- * sets utask->active_uprobe.
+ * sets utask->state != 0.
  *
  * On singlestep exception, singlestep notifier sets the TIF_UPROBE flag
  * and allows the thread to return from interrupt.
@@ -2357,7 +2342,7 @@ void uprobe_notify_resume(struct pt_regs *regs)
 	clear_thread_flag(TIF_UPROBE);
 
 	utask = current->utask;
-	if (utask && utask->active_uprobe)
+	if (utask && utask->state)
 		handle_singlestep(utask, regs);
 	else
 		handle_swbp(regs);
@@ -2388,7 +2373,7 @@ int uprobe_post_sstep_notifier(struct pt_regs *regs)
 {
 	struct uprobe_task *utask = current->utask;
 
-	if (!current->mm || !utask || !utask->active_uprobe)
+	if (!current->mm || !utask || !utask->state)
 		/* task is currently not uprobed */
 		return 0;
 
-- 
2.25.1.362.g51ebf55




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

* Re: [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-08-19 13:41   ` Oleg Nesterov
@ 2024-08-19 20:34     ` Andrii Nakryiko
  2024-08-20 15:05       ` Oleg Nesterov
  0 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-19 20:34 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, rostedt, mhiramat,
	bpf, linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm

On Mon, Aug 19, 2024 at 6:41 AM Oleg Nesterov <oleg@redhat.com> wrote:
>
> On 08/12, Andrii Nakryiko wrote:
> >
> > Avoid taking refcount on uprobe in prepare_uretprobe(), instead take
> > uretprobe-specific SRCU lock and keep it active as kernel transfers
> > control back to user space.
> ...
> >  include/linux/uprobes.h |  49 ++++++-
> >  kernel/events/uprobes.c | 294 ++++++++++++++++++++++++++++++++++------
> >  2 files changed, 301 insertions(+), 42 deletions(-)
>
> Oh. To be honest I don't like this patch.
>
> I would like to know what other reviewers think, but to me it adds too many
> complications that I can't even fully understand...

Which parts? The atomic xchg() and cmpxchg() parts? What exactly do
you feel like you don't fully understand?

>
> And how much does it help performance-wise?

A lot, as we increase uprobe parallelism. Here's a subset of
benchmarks for 1-4, 8, 16, 32, 64, and 80 threads firing uretprobe.
With and without this SRCU change, but including all the other
changes, including the lockless VMA lookup. It's noticeable already
with just two competing CPUs/threads, and it just gets much worse from
there.

Of course in production you shouldn't come close to such rates of
uprobe/uretprobe firing, so this is definitely a microbenchmark
emphasizing the sharing between CPUs, but it still adds up. And we do
have production use cases that would like to fire uprobes at 100K+ per
second rates.

WITH SRCU for uretprobes
========================
uretprobe-nop         ( 1 cpus):    1.968 ± 0.001M/s  (  1.968M/s/cpu)
uretprobe-nop         ( 2 cpus):    3.739 ± 0.003M/s  (  1.869M/s/cpu)
uretprobe-nop         ( 3 cpus):    5.616 ± 0.003M/s  (  1.872M/s/cpu)
uretprobe-nop         ( 4 cpus):    7.286 ± 0.002M/s  (  1.822M/s/cpu)
uretprobe-nop         ( 8 cpus):   13.657 ± 0.007M/s  (  1.707M/s/cpu)
uretprobe-nop         (32 cpus):   45.305 ± 0.066M/s  (  1.416M/s/cpu)
uretprobe-nop         (64 cpus):   42.390 ± 0.922M/s  (  0.662M/s/cpu)
uretprobe-nop         (80 cpus):   47.554 ± 2.411M/s  (  0.594M/s/cpu)

WITHOUT SRCU for uretprobes
===========================
uretprobe-nop         ( 1 cpus):    2.197 ± 0.002M/s  (  2.197M/s/cpu)
uretprobe-nop         ( 2 cpus):    3.325 ± 0.001M/s  (  1.662M/s/cpu)
uretprobe-nop         ( 3 cpus):    4.129 ± 0.002M/s  (  1.376M/s/cpu)
uretprobe-nop         ( 4 cpus):    6.180 ± 0.003M/s  (  1.545M/s/cpu)
uretprobe-nop         ( 8 cpus):    7.323 ± 0.005M/s  (  0.915M/s/cpu)
uretprobe-nop         (16 cpus):    6.943 ± 0.005M/s  (  0.434M/s/cpu)
uretprobe-nop         (32 cpus):    5.931 ± 0.014M/s  (  0.185M/s/cpu)
uretprobe-nop         (64 cpus):    5.145 ± 0.003M/s  (  0.080M/s/cpu)
uretprobe-nop         (80 cpus):    4.925 ± 0.005M/s  (  0.062M/s/cpu)

>
> I'll try to take another look, and I'll try to think about other approaches,
> not that I have something better in mind...

Ok.

>
>
> But lets forgets this patch for the moment. The next one adds even more
> complications, and I think it doesn't make sense.
>

"Even more complications" is a bit of an overstatement. It just
applies everything we do for uretprobes in this patch to a very
straightforward single-stepped case.

> As I have already mentioned in the previous discussions, we can simply kill
> utask->active_uprobe. And utask->auprobe.

I don't have anything against that, in principle, but let's benchmark
and test that thoroughly. I'm a bit uneasy about the possibility that
some arch-specific code will do container_of() on this arch_uprobe in
order to get to uprobe, we'd need to audit all the code to make sure
that can't happen. Also it's a bit unfortunate that we have to assume
that struct arch_uprobe is small on all architectures, and there is no
code that assumes it can't be moved, etc, etc. (I also don't get why
you need memcpy

>
> So can't we start with the patch below? On top of your 08/13. It doesn't kill
> utask->auprobe yet, this needs a bit more trivial changes.
>
> What do you think?

I think that single-stepped case isn't the main use case (typically
uprobe/uretprobe will be installed on nop or push %rbp, both
emulated). uretprobes, though, are the main use case (along with
optimized entry uprobes). So what we do about single-stepped is a bit
secondary (for me, looking at production use cases).

But we do need to do something with uretprobes first and foremost.

>
> Oleg.
>
> -------------------------------------------------------------------------------
> From d7cb674eb6f7bb891408b2b6a5fb872a6c2f0f6c Mon Sep 17 00:00:00 2001
> From: Oleg Nesterov <oleg@redhat.com>
> Date: Mon, 19 Aug 2024 15:34:55 +0200
> Subject: [RFC PATCH] uprobe: kill uprobe_task->active_uprobe
>
> Untested, not for inclusion yet, and I need to split it into 2 changes.
> It does 2 simple things:
>
>         1. active_uprobe != NULL is possible if and only if utask->state != 0,
>            so it turns the active_uprobe checks into the utask->state checks.
>
>         2. handle_singlestep() doesn't really need ->active_uprobe, it only
>            needs uprobe->arch which is "const" after prepare_uprobe().
>
>            So this patch adds the new "arch_uprobe uarch" member into utask
>            and changes pre_ssout() to do memcpy(&utask->uarch, &uprobe->arch).
> ---
>  include/linux/uprobes.h |  2 +-
>  kernel/events/uprobes.c | 37 +++++++++++--------------------------
>  2 files changed, 12 insertions(+), 27 deletions(-)

[...]


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

* Re: [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-08-19 20:34     ` Andrii Nakryiko
@ 2024-08-20 15:05       ` Oleg Nesterov
  2024-08-20 18:01         ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Oleg Nesterov @ 2024-08-20 15:05 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, rostedt, mhiramat,
	bpf, linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm

On 08/19, Andrii Nakryiko wrote:
>
> On Mon, Aug 19, 2024 at 6:41 AM Oleg Nesterov <oleg@redhat.com> wrote:
> >
> > On 08/12, Andrii Nakryiko wrote:
> > >
> > > Avoid taking refcount on uprobe in prepare_uretprobe(), instead take
> > > uretprobe-specific SRCU lock and keep it active as kernel transfers
> > > control back to user space.
> > ...
> > >  include/linux/uprobes.h |  49 ++++++-
> > >  kernel/events/uprobes.c | 294 ++++++++++++++++++++++++++++++++++------
> > >  2 files changed, 301 insertions(+), 42 deletions(-)
> >
> > Oh. To be honest I don't like this patch.
> >
> > I would like to know what other reviewers think, but to me it adds too many
> > complications that I can't even fully understand...
>
> Which parts? The atomic xchg() and cmpxchg() parts? What exactly do
> you feel like you don't fully understand?

Heh, everything looks too complex for me ;)

Say, hprobe_expire(). It is also called by ri_timer() in softirq context,
right? And it does

	/* We lost the race, undo our refcount bump. It can drop to zero. */
	put_uprobe(uprobe);

How so? If the refcount goes to zero, put_uprobe() does mutex_lock(),
but it must not sleep in softirq context.


Or, prepare_uretprobe() which does

	rcu_assign_pointer(utask->return_instances, ri);

	if (!timer_pending(&utask->ri_timer))
		mod_timer(&utask->ri_timer, ...);

Suppose that the timer was pending and it was fired right before
rcu_assign_pointer(). What guarantees that prepare_uretprobe() will see
timer_pending() == false?

rcu_assign_pointer()->smp_store_release() is a one-way barrier. This
timer_pending() check may appear to happen before rcu_assign_pointer()
completes.

In this (yes, theoretical) case ri_timer() can miss the new return_instance,
while prepare_uretprobe() can miss the necessary mod_timer(). I think this
needs another mb() in between.


And I can't convince myself hprobe_expire() is correct... OK, I don't
fully understand the logic, but why data_race(READ_ONCE(hprobe->leased)) ?
READ_ONCE() should be enough in this case?


> > As I have already mentioned in the previous discussions, we can simply kill
> > utask->active_uprobe. And utask->auprobe.
>
> I don't have anything against that, in principle, but let's benchmark
> and test that thoroughly. I'm a bit uneasy about the possibility that
> some arch-specific code will do container_of() on this arch_uprobe in
> order to get to uprobe,

Well, struct uprobe is not "exported", the arch-specific code can't do this.

Oleg.



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

* Re: [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-08-20 15:05       ` Oleg Nesterov
@ 2024-08-20 18:01         ` Andrii Nakryiko
  0 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-20 18:01 UTC (permalink / raw)
  To: Oleg Nesterov
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, rostedt, mhiramat,
	bpf, linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm

On Tue, Aug 20, 2024 at 8:06 AM Oleg Nesterov <oleg@redhat.com> wrote:
>
> On 08/19, Andrii Nakryiko wrote:
> >
> > On Mon, Aug 19, 2024 at 6:41 AM Oleg Nesterov <oleg@redhat.com> wrote:
> > >
> > > On 08/12, Andrii Nakryiko wrote:
> > > >
> > > > Avoid taking refcount on uprobe in prepare_uretprobe(), instead take
> > > > uretprobe-specific SRCU lock and keep it active as kernel transfers
> > > > control back to user space.
> > > ...
> > > >  include/linux/uprobes.h |  49 ++++++-
> > > >  kernel/events/uprobes.c | 294 ++++++++++++++++++++++++++++++++++------
> > > >  2 files changed, 301 insertions(+), 42 deletions(-)
> > >
> > > Oh. To be honest I don't like this patch.
> > >
> > > I would like to know what other reviewers think, but to me it adds too many
> > > complications that I can't even fully understand...
> >
> > Which parts? The atomic xchg() and cmpxchg() parts? What exactly do
> > you feel like you don't fully understand?
>
> Heh, everything looks too complex for me ;)

Well, the best code is no code. But I'm not doing this just for fun,
so I'm happy with the simplest solution *that works*.

>
> Say, hprobe_expire(). It is also called by ri_timer() in softirq context,
> right? And it does
>
>         /* We lost the race, undo our refcount bump. It can drop to zero. */
>         put_uprobe(uprobe);
>
> How so? If the refcount goes to zero, put_uprobe() does mutex_lock(),
> but it must not sleep in softirq context.
>

Now we are talking about specific issues, thank you! It's hard to
discuss "I don't like".

Yes, I think you are right and it is certainly a problem with
put_uprobe() that it can't be called from softirq (just having to
remember that is error-prone, as is evidenced by me forgetting about
this issue).

It's easy enough to solve. We can either schedule work from timer
thread (and that will solve this particular issue only), or we can
teach put_uprobe() to schedule work item if it drops refcount to zero
from softirq and other restricted contexts.

I vote for making put_uprobe() flexible in this regard, add
work_struct to uprobe, and schedule all this to be done in the work
queue callback. WDYT?

>
> Or, prepare_uretprobe() which does
>
>         rcu_assign_pointer(utask->return_instances, ri);
>
>         if (!timer_pending(&utask->ri_timer))
>                 mod_timer(&utask->ri_timer, ...);
>
> Suppose that the timer was pending and it was fired right before
> rcu_assign_pointer(). What guarantees that prepare_uretprobe() will see
> timer_pending() == false?
>
> rcu_assign_pointer()->smp_store_release() is a one-way barrier. This
> timer_pending() check may appear to happen before rcu_assign_pointer()
> completes.
>
> In this (yes, theoretical) case ri_timer() can miss the new return_instance,
> while prepare_uretprobe() can miss the necessary mod_timer(). I think this
> needs another mb() in between.
>

Ok, that's fair. I felt like this pattern might be a bit problematic,
but I also felt like it's good to have to ensure that we do
occasionally have a race between timer callback and uretprobe, even if
uretprobe returns very quickly. (and I did confirm we get those races
and they seem to be handled fine, i.e., I saw uprobes being "expired"
into refcounted ones from ri_timer)

But the really simple way to solve this is to do unconditional
mod_timer(), so I can do just that to keep this less tricky. Would you
be ok with that?

>
> And I can't convince myself hprobe_expire() is correct... OK, I don't
> fully understand the logic, but why data_race(READ_ONCE(hprobe->leased)) ?
> READ_ONCE() should be enough in this case?

you mean why data_race() annotation? To appease KCSAN, given that we
modify hprobe->leased with xchg/cmpxchg, but read here with
READ_ONCE(). Maybe I'm overthinking it, not sure. There is a reason
why this is an RFC ;)

>
>
> > > As I have already mentioned in the previous discussions, we can simply kill
> > > utask->active_uprobe. And utask->auprobe.
> >
> > I don't have anything against that, in principle, but let's benchmark
> > and test that thoroughly. I'm a bit uneasy about the possibility that
> > some arch-specific code will do container_of() on this arch_uprobe in
> > order to get to uprobe,
>
> Well, struct uprobe is not "exported", the arch-specific code can't do this.
>

Ah, good point, that's great. But as I said, uretprobe is actually
*the common* use case, not single-stepped uprobe. Still not very happy
about that memcpy() and assumption that it's cheap, but that's minor.

But no matter what we do for single-stepped one, uretprobe needs some
solution. (and if that solution works for uretprobe, why wouldn't it
work for single-step?...)

> Oleg.
>


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

* Re: [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations
  2024-08-15 16:49   ` Andrii Nakryiko
@ 2024-08-21 16:41     ` Andrii Nakryiko
  0 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-21 16:41 UTC (permalink / raw)
  To: peterz
  Cc: Andrii Nakryiko, linux-trace-kernel, rostedt, mhiramat, bpf,
	linux-kernel, jolsa, paulmck, willy, surenb, akpm, linux-mm,
	Oleg Nesterov

On Thu, Aug 15, 2024 at 9:49 AM Andrii Nakryiko
<andrii.nakryiko@gmail.com> wrote:
>
> On Thu, Aug 15, 2024 at 6:25 AM Oleg Nesterov <oleg@redhat.com> wrote:
> >
> > On 08/12, Andrii Nakryiko wrote:
> > >
> > > ( In addition to previously posted first 8 patches, I'm sending 5 more as an
> > > RFC for people to get the general gist of where this work heading and what
> > > uprobe performance is now achievable. I think first 8 patches are ready to be
> > > applied and I'd appreciate early feedback on the remaining 5 ones.
> >
> > I didn't read the "RFC" patches yet, will try to do on weekend.
> >
> > As for 1-8, I failed to find any problem:
> > Reviewed-by: Oleg Nesterov <oleg@redhat.com>
> >
>
> Great, thanks a lot for all the thorough reviews you've provided (and
> hopefully will keep providing ;).
>
> Peter, if you don't see any problems with first 8 patches, could you
> please apply them to tip/perf/core some time soon, so that subsequent
> work (SRCU+timeout and, separately, lockless VMA->inode->uprobe
> lookup) can be split into independent pieces and reviewed/landed
> separately? Thanks!

If there are no concerns about the first 8 patches (and Oleg doesn't
have any), it would be nice to apply them. The remaining two sets of
changes are parallelizable in terms of discussion and landing them
independently, so it would help to move the uprobe optimizations works
along nicely. Thank you!


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

* Re: [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection
  2024-08-13  4:29 ` [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection Andrii Nakryiko
@ 2024-08-22 14:22   ` Jiri Olsa
  2024-08-22 16:59     ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Jiri Olsa @ 2024-08-22 14:22 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, peterz, oleg, rostedt, mhiramat, bpf,
	linux-kernel, paulmck, willy, surenb, akpm, linux-mm

On Mon, Aug 12, 2024 at 09:29:08PM -0700, Andrii Nakryiko wrote:

SNIP

> @@ -1125,18 +1103,31 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
>  	int err;
>  
>  	down_write(&uprobe->register_rwsem);
> -	if (WARN_ON(!consumer_del(uprobe, uc))) {
> -		err = -ENOENT;
> -	} else {
> -		err = register_for_each_vma(uprobe, NULL);
> -		/* TODO : cant unregister? schedule a worker thread */
> -		if (unlikely(err))
> -			uprobe_warn(current, "unregister, leaking uprobe");
> -	}
> +
> +	list_del_rcu(&uc->cons_node);

hi,
I'm using this patchset as base for my changes and stumbled on this today,
I'm probably missing something, but should we keep the 'uprobe->consumer_rwsem'
lock around the list_del_rcu?

jirka


> +	err = register_for_each_vma(uprobe, NULL);
> +
>  	up_write(&uprobe->register_rwsem);
>  
> -	if (!err)
> -		put_uprobe(uprobe);
> +	/* TODO : cant unregister? schedule a worker thread */
> +	if (unlikely(err)) {
> +		uprobe_warn(current, "unregister, leaking uprobe");
> +		goto out_sync;
> +	}
> +
> +	put_uprobe(uprobe);
> +
> +out_sync:
> +	/*
> +	 * Now that handler_chain() and handle_uretprobe_chain() iterate over
> +	 * uprobe->consumers list under RCU protection without holding
> +	 * uprobe->register_rwsem, we need to wait for RCU grace period to
> +	 * make sure that we can't call into just unregistered
> +	 * uprobe_consumer's callbacks anymore. If we don't do that, fast and
> +	 * unlucky enough caller can free consumer's memory and cause
> +	 * handler_chain() or handle_uretprobe_chain() to do an use-after-free.
> +	 */
> +	synchronize_srcu(&uprobes_srcu);
>  }
>  EXPORT_SYMBOL_GPL(uprobe_unregister);
>  
> @@ -1214,13 +1205,20 @@ EXPORT_SYMBOL_GPL(uprobe_register);
>  int uprobe_apply(struct uprobe *uprobe, struct uprobe_consumer *uc, bool add)
>  {
>  	struct uprobe_consumer *con;
> -	int ret = -ENOENT;
> +	int ret = -ENOENT, srcu_idx;
>  
>  	down_write(&uprobe->register_rwsem);
> -	for (con = uprobe->consumers; con && con != uc ; con = con->next)
> -		;
> -	if (con)
> -		ret = register_for_each_vma(uprobe, add ? uc : NULL);
> +
> +	srcu_idx = srcu_read_lock(&uprobes_srcu);
> +	list_for_each_entry_srcu(con, &uprobe->consumers, cons_node,
> +				 srcu_read_lock_held(&uprobes_srcu)) {
> +		if (con == uc) {
> +			ret = register_for_each_vma(uprobe, add ? uc : NULL);
> +			break;
> +		}
> +	}
> +	srcu_read_unlock(&uprobes_srcu, srcu_idx);
> +
>  	up_write(&uprobe->register_rwsem);
>  
>  	return ret;
> @@ -2085,10 +2083,12 @@ static void handler_chain(struct uprobe *uprobe, struct pt_regs *regs)
>  	struct uprobe_consumer *uc;
>  	int remove = UPROBE_HANDLER_REMOVE;
>  	bool need_prep = false; /* prepare return uprobe, when needed */
> +	bool has_consumers = false;
>  
> -	down_read(&uprobe->register_rwsem);
>  	current->utask->auprobe = &uprobe->arch;
> -	for (uc = uprobe->consumers; uc; uc = uc->next) {
> +
> +	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
> +				 srcu_read_lock_held(&uprobes_srcu)) {
>  		int rc = 0;
>  
>  		if (uc->handler) {
> @@ -2101,17 +2101,24 @@ static void handler_chain(struct uprobe *uprobe, struct pt_regs *regs)
>  			need_prep = true;
>  
>  		remove &= rc;
> +		has_consumers = true;
>  	}
>  	current->utask->auprobe = NULL;
>  
>  	if (need_prep && !remove)
>  		prepare_uretprobe(uprobe, regs); /* put bp at return */
>  
> -	if (remove && uprobe->consumers) {
> -		WARN_ON(!uprobe_is_active(uprobe));
> -		unapply_uprobe(uprobe, current->mm);
> +	if (remove && has_consumers) {
> +		down_read(&uprobe->register_rwsem);
> +
> +		/* re-check that removal is still required, this time under lock */
> +		if (!filter_chain(uprobe, current->mm)) {
> +			WARN_ON(!uprobe_is_active(uprobe));
> +			unapply_uprobe(uprobe, current->mm);
> +		}
> +
> +		up_read(&uprobe->register_rwsem);
>  	}
> -	up_read(&uprobe->register_rwsem);
>  }
>  
>  static void
> @@ -2119,13 +2126,15 @@ handle_uretprobe_chain(struct return_instance *ri, struct pt_regs *regs)
>  {
>  	struct uprobe *uprobe = ri->uprobe;
>  	struct uprobe_consumer *uc;
> +	int srcu_idx;
>  
> -	down_read(&uprobe->register_rwsem);
> -	for (uc = uprobe->consumers; uc; uc = uc->next) {
> +	srcu_idx = srcu_read_lock(&uprobes_srcu);
> +	list_for_each_entry_srcu(uc, &uprobe->consumers, cons_node,
> +				 srcu_read_lock_held(&uprobes_srcu)) {
>  		if (uc->ret_handler)
>  			uc->ret_handler(uc, ri->func, regs);
>  	}
> -	up_read(&uprobe->register_rwsem);
> +	srcu_read_unlock(&uprobes_srcu, srcu_idx);
>  }
>  
>  static struct return_instance *find_next_ret_chain(struct return_instance *ri)
> -- 
> 2.43.5
> 


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

* Re: [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection
  2024-08-22 14:22   ` Jiri Olsa
@ 2024-08-22 16:59     ` Andrii Nakryiko
  2024-08-22 17:35       ` Jiri Olsa
  0 siblings, 1 reply; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-22 16:59 UTC (permalink / raw)
  To: Jiri Olsa
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, paulmck, willy, surenb, akpm,
	linux-mm

On Thu, Aug 22, 2024 at 7:22 AM Jiri Olsa <olsajiri@gmail.com> wrote:
>
> On Mon, Aug 12, 2024 at 09:29:08PM -0700, Andrii Nakryiko wrote:
>
> SNIP
>
> > @@ -1125,18 +1103,31 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
> >       int err;
> >
> >       down_write(&uprobe->register_rwsem);
> > -     if (WARN_ON(!consumer_del(uprobe, uc))) {
> > -             err = -ENOENT;
> > -     } else {
> > -             err = register_for_each_vma(uprobe, NULL);
> > -             /* TODO : cant unregister? schedule a worker thread */
> > -             if (unlikely(err))
> > -                     uprobe_warn(current, "unregister, leaking uprobe");
> > -     }
> > +
> > +     list_del_rcu(&uc->cons_node);
>
> hi,
> I'm using this patchset as base for my changes and stumbled on this today,
> I'm probably missing something, but should we keep the 'uprobe->consumer_rwsem'
> lock around the list_del_rcu?
>

Note that original code also didn't take consumer_rwsem, but rather
kept register_rwsem (which we still use).

There is a bit of mix of using register_rwsem and consumer_rwsem for
working with consumer list. Code hints at this as being undesirable
and "temporary", but you know, it's not broken :)

Anyways, my point is that we didn't change the behavior, this should
be fine. That _rcu() in list_del_rcu() is not about lockless
modification of the list, but rather modification in such a way as to
keep lockless RCU-protected *readers* correct. It just does some more
memory barrier/release operations more carefully.

> jirka
>
>
> > +     err = register_for_each_vma(uprobe, NULL);
> > +
> >       up_write(&uprobe->register_rwsem);
> >
> > -     if (!err)
> > -             put_uprobe(uprobe);
> > +     /* TODO : cant unregister? schedule a worker thread */
> > +     if (unlikely(err)) {
> > +             uprobe_warn(current, "unregister, leaking uprobe");
> > +             goto out_sync;
> > +     }
> > +
> > +     put_uprobe(uprobe);
> > +


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

* Re: [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection
  2024-08-22 16:59     ` Andrii Nakryiko
@ 2024-08-22 17:35       ` Jiri Olsa
  2024-08-22 17:51         ` Andrii Nakryiko
  0 siblings, 1 reply; 40+ messages in thread
From: Jiri Olsa @ 2024-08-22 17:35 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: Jiri Olsa, Andrii Nakryiko, linux-trace-kernel, peterz, oleg,
	rostedt, mhiramat, bpf, linux-kernel, paulmck, willy, surenb,
	akpm, linux-mm

On Thu, Aug 22, 2024 at 09:59:29AM -0700, Andrii Nakryiko wrote:
> On Thu, Aug 22, 2024 at 7:22 AM Jiri Olsa <olsajiri@gmail.com> wrote:
> >
> > On Mon, Aug 12, 2024 at 09:29:08PM -0700, Andrii Nakryiko wrote:
> >
> > SNIP
> >
> > > @@ -1125,18 +1103,31 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
> > >       int err;
> > >
> > >       down_write(&uprobe->register_rwsem);
> > > -     if (WARN_ON(!consumer_del(uprobe, uc))) {
> > > -             err = -ENOENT;
> > > -     } else {
> > > -             err = register_for_each_vma(uprobe, NULL);
> > > -             /* TODO : cant unregister? schedule a worker thread */
> > > -             if (unlikely(err))
> > > -                     uprobe_warn(current, "unregister, leaking uprobe");
> > > -     }
> > > +
> > > +     list_del_rcu(&uc->cons_node);
> >
> > hi,
> > I'm using this patchset as base for my changes and stumbled on this today,
> > I'm probably missing something, but should we keep the 'uprobe->consumer_rwsem'
> > lock around the list_del_rcu?
> >
> 
> Note that original code also didn't take consumer_rwsem, but rather
> kept register_rwsem (which we still use).

humm, consumer_del took consumer_rwsem, right?

jirka

> 
> There is a bit of mix of using register_rwsem and consumer_rwsem for
> working with consumer list. Code hints at this as being undesirable
> and "temporary", but you know, it's not broken :)
> 
> Anyways, my point is that we didn't change the behavior, this should
> be fine. That _rcu() in list_del_rcu() is not about lockless
> modification of the list, but rather modification in such a way as to
> keep lockless RCU-protected *readers* correct. It just does some more
> memory barrier/release operations more carefully.
> 
> > jirka
> >
> >
> > > +     err = register_for_each_vma(uprobe, NULL);
> > > +
> > >       up_write(&uprobe->register_rwsem);
> > >
> > > -     if (!err)
> > > -             put_uprobe(uprobe);
> > > +     /* TODO : cant unregister? schedule a worker thread */
> > > +     if (unlikely(err)) {
> > > +             uprobe_warn(current, "unregister, leaking uprobe");
> > > +             goto out_sync;
> > > +     }
> > > +
> > > +     put_uprobe(uprobe);
> > > +


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

* Re: [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection
  2024-08-22 17:35       ` Jiri Olsa
@ 2024-08-22 17:51         ` Andrii Nakryiko
  0 siblings, 0 replies; 40+ messages in thread
From: Andrii Nakryiko @ 2024-08-22 17:51 UTC (permalink / raw)
  To: Jiri Olsa
  Cc: Andrii Nakryiko, linux-trace-kernel, peterz, oleg, rostedt,
	mhiramat, bpf, linux-kernel, paulmck, willy, surenb, akpm,
	linux-mm

On Thu, Aug 22, 2024 at 10:35 AM Jiri Olsa <olsajiri@gmail.com> wrote:
>
> On Thu, Aug 22, 2024 at 09:59:29AM -0700, Andrii Nakryiko wrote:
> > On Thu, Aug 22, 2024 at 7:22 AM Jiri Olsa <olsajiri@gmail.com> wrote:
> > >
> > > On Mon, Aug 12, 2024 at 09:29:08PM -0700, Andrii Nakryiko wrote:
> > >
> > > SNIP
> > >
> > > > @@ -1125,18 +1103,31 @@ void uprobe_unregister(struct uprobe *uprobe, struct uprobe_consumer *uc)
> > > >       int err;
> > > >
> > > >       down_write(&uprobe->register_rwsem);
> > > > -     if (WARN_ON(!consumer_del(uprobe, uc))) {
> > > > -             err = -ENOENT;
> > > > -     } else {
> > > > -             err = register_for_each_vma(uprobe, NULL);
> > > > -             /* TODO : cant unregister? schedule a worker thread */
> > > > -             if (unlikely(err))
> > > > -                     uprobe_warn(current, "unregister, leaking uprobe");
> > > > -     }
> > > > +
> > > > +     list_del_rcu(&uc->cons_node);
> > >
> > > hi,
> > > I'm using this patchset as base for my changes and stumbled on this today,
> > > I'm probably missing something, but should we keep the 'uprobe->consumer_rwsem'
> > > lock around the list_del_rcu?
> > >
> >
> > Note that original code also didn't take consumer_rwsem, but rather
> > kept register_rwsem (which we still use).
>
> humm, consumer_del took consumer_rwsem, right?
>

Ah, it was inside consume_del(), sorry, my bad. I can add nested
consumer_rwsem back, but what I mentioned earlier, regiser_rwsem is
sort of interchangeable and sufficient enough for working with
consumer list, it seems. There are a bunch of places where we iterated
this list without holding consumer_rwsem lock and that doesn't break
anything.

Also, consumer_add() and consumer_del() are always called with
register_rwsem, so that consumer_rwsem isn't necessary.

We also have prepare_uprobe() holding consumer_rwsem and there is a
comment about abuse of that rwsem and suggestion to move it to
registration, I never completely understood that. But prepare_uprobe()
doesn't seem to modify consumers list at all.

And the one remaining use of consumer_rwsem is filter_chain(), which
for handler_chain() will be also called under register_rwsem, if
purely lockless traversal is not enough.

There are two other calls to filter_chain() that are not protected by
register_rwsem, so just because of those two maybe we should keep
consumer_rwsem, but so far all the stress testing never caught any
problem.


> jirka
>
> >
> > There is a bit of mix of using register_rwsem and consumer_rwsem for
> > working with consumer list. Code hints at this as being undesirable
> > and "temporary", but you know, it's not broken :)
> >
> > Anyways, my point is that we didn't change the behavior, this should
> > be fine. That _rcu() in list_del_rcu() is not about lockless
> > modification of the list, but rather modification in such a way as to
> > keep lockless RCU-protected *readers* correct. It just does some more
> > memory barrier/release operations more carefully.
> >
> > > jirka
> > >
> > >
> > > > +     err = register_for_each_vma(uprobe, NULL);
> > > > +
> > > >       up_write(&uprobe->register_rwsem);
> > > >
> > > > -     if (!err)
> > > > -             put_uprobe(uprobe);
> > > > +     /* TODO : cant unregister? schedule a worker thread */
> > > > +     if (unlikely(err)) {
> > > > +             uprobe_warn(current, "unregister, leaking uprobe");
> > > > +             goto out_sync;
> > > > +     }
> > > > +
> > > > +     put_uprobe(uprobe);
> > > > +


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

end of thread, other threads:[~2024-08-22 17:52 UTC | newest]

Thread overview: 40+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2024-08-13  4:29 [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 01/13] uprobes: revamp uprobe refcounting and lifetime management Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 02/13] uprobes: protected uprobe lifetime with SRCU Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 03/13] uprobes: get rid of enum uprobe_filter_ctx in uprobe filter callbacks Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 04/13] uprobes: travers uprobe's consumer list locklessly under SRCU protection Andrii Nakryiko
2024-08-22 14:22   ` Jiri Olsa
2024-08-22 16:59     ` Andrii Nakryiko
2024-08-22 17:35       ` Jiri Olsa
2024-08-22 17:51         ` Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 05/13] perf/uprobe: split uprobe_unregister() Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 06/13] rbtree: provide rb_find_rcu() / rb_find_add_rcu() Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 07/13] uprobes: perform lockless SRCU-protected uprobes_tree lookup Andrii Nakryiko
2024-08-13  4:29 ` [PATCH v3 08/13] uprobes: switch to RCU Tasks Trace flavor for better performance Andrii Nakryiko
2024-08-13  4:29 ` [PATCH RFC v3 09/13] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
2024-08-19 13:41   ` Oleg Nesterov
2024-08-19 20:34     ` Andrii Nakryiko
2024-08-20 15:05       ` Oleg Nesterov
2024-08-20 18:01         ` Andrii Nakryiko
2024-08-13  4:29 ` [PATCH RFC v3 10/13] uprobes: implement SRCU-protected lifetime for single-stepped uprobe Andrii Nakryiko
2024-08-13  4:29 ` [PATCH RFC v3 11/13] mm: introduce mmap_lock_speculation_{start|end} Andrii Nakryiko
2024-08-13  4:29 ` [PATCH RFC v3 12/13] mm: add SLAB_TYPESAFE_BY_RCU to files_cache Andrii Nakryiko
2024-08-13  6:07   ` Mateusz Guzik
2024-08-13 14:49     ` Suren Baghdasaryan
2024-08-13 18:15       ` Andrii Nakryiko
2024-08-13  4:29 ` [PATCH RFC v3 13/13] uprobes: add speculative lockless VMA to inode resolution Andrii Nakryiko
2024-08-13  6:17   ` Mateusz Guzik
2024-08-13 15:36     ` Suren Baghdasaryan
2024-08-15 13:44       ` Mateusz Guzik
2024-08-15 16:47         ` Andrii Nakryiko
2024-08-15 17:45           ` Suren Baghdasaryan
2024-08-15 18:24             ` Mateusz Guzik
2024-08-15 18:58             ` Jann Horn
2024-08-15 19:07               ` Mateusz Guzik
2024-08-15 19:17                 ` Arnaldo Carvalho de Melo
2024-08-15 19:18                   ` Arnaldo Carvalho de Melo
2024-08-15 19:44               ` Suren Baghdasaryan
2024-08-15 20:17               ` Andrii Nakryiko
2024-08-15 13:24 ` [PATCH v3 00/13] uprobes: RCU-protected hot path optimizations Oleg Nesterov
2024-08-15 16:49   ` Andrii Nakryiko
2024-08-21 16:41     ` Andrii Nakryiko

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