linux-trace-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v2 tip/perf/core 0/2] SRCU-protected uretprobes hot path
@ 2024-10-08  0:25 Andrii Nakryiko
  2024-10-08  0:25 ` [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context Andrii Nakryiko
  2024-10-08  0:25 ` [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
  0 siblings, 2 replies; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-08  0:25 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, mingo, bpf, linux-kernel, jolsa, paulmck,
	Andrii Nakryiko

Recently landed changes make uprobe entry hot code path makes use of RCU Tasks
Trace to avoid touching uprobe refcount, which at high frequency of uprobe
triggering leads to excessive cache line bouncing and limited scalability with
increased number of CPUs that simultaneously execute uprobe handlers.

This patch set adds return uprobe (uretprobe) side of this, this time
utilizing SRCU for the same reasons. Given the time between entry uprobe
activation (at which point uretprobe code hijacks user-space stack to get
activated on user function return) and uretprobe activation can be arbitrarily
long and is completely under control of user code, we need to protect
ourselves from too long or unbounded SRCU grace periods.

To that end we keep SRCU protection only for a limited time, and if user space
code takes longer to return, pending uretprobe instances are "downgraded" to
refcounted ones. This gives us best scalability and performance for
high-frequency uretprobes, and keeps upper bound on SRCU grace period duration
for low frequency uretprobes.

There are a bunch of synchronization issues between timer callback running in
IRQ handler and current thread executing uretprobe handlers, which is
abstracted away behind "hybrid lifetime uprobe" (hprobe) wrapper around uprobe
instance itself.

There is now a speculative try_get_uprobe() and, possibly, a compensating
put_uprobe() being done from the timer thread (softirq), so we need to make
sure that put_uprobe() is working well from any context. This is what patch #1
does, employing deferred work callback, and shifting all the locking to it.

v1->v2:
  - dropped single-stepped uprobes changes to make this change a bit more
    palatable to Oleg and get some good will from him :)
  - fixed the bug with not calling __srcu_read_unlock when "expiring" leased
    uprobe, but failing to get refcount;
  - switched hprobe implementation to an explicit state machine, which seems
    to make logic more straightforward, evidenced by this allowing me to spot
    the above subtle LEASED -> GONE transition bug;
  - re-ran uprobe-stress many-many times, it was instrumental for getting
    confidence in implementation and spotting subtle bugs (including the above
    one, once I modified timer logic to ran at fixed interval to increase the
    probability of races with the normal uretprobe consumer code);
rfc->v1:
  - made put_uprobe() work in any context, not just user context (Oleg);
  - changed to unconditional mod_timer() usage to avoid races (Oleg).
  - I kept single-stepped uprobe changes, as they have a simple use of all the
    hprobe functionality developed in patch #1.


Andrii Nakryiko (2):
  uprobes: allow put_uprobe() from non-sleepable softirq context
  uprobes: SRCU-protect uretprobe lifetime (with timeout)

 include/linux/uprobes.h |  54 ++++++-
 kernel/events/uprobes.c | 312 ++++++++++++++++++++++++++++++++++------
 2 files changed, 321 insertions(+), 45 deletions(-)

-- 
2.43.5


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

* [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context
  2024-10-08  0:25 [PATCH v2 tip/perf/core 0/2] SRCU-protected uretprobes hot path Andrii Nakryiko
@ 2024-10-08  0:25 ` Andrii Nakryiko
  2024-10-18  8:26   ` Peter Zijlstra
  2024-10-08  0:25 ` [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
  1 sibling, 1 reply; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-08  0:25 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, mingo, bpf, linux-kernel, jolsa, paulmck,
	Andrii Nakryiko

Currently put_uprobe() might trigger mutex_lock()/mutex_unlock(), which
makes it unsuitable to be called from more restricted context like softirq.

Let's make put_uprobe() agnostic to the context in which it is called,
and use work queue to defer the mutex-protected clean up steps.

RB tree removal step is also moved into work-deferred callback to avoid
potential deadlock between softirq-based timer callback, added in the
next patch, and the rest of uprobe code.

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

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index a2e6a57f79f2..9d3ab472200d 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -27,6 +27,7 @@
 #include <linux/shmem_fs.h>
 #include <linux/khugepaged.h>
 #include <linux/rcupdate_trace.h>
+#include <linux/workqueue.h>
 
 #include <linux/uprobes.h>
 
@@ -61,7 +62,10 @@ struct uprobe {
 	struct list_head	pending_list;
 	struct list_head	consumers;
 	struct inode		*inode;		/* Also hold a ref to inode */
-	struct rcu_head		rcu;
+	union {
+		struct rcu_head		rcu;
+		struct work_struct	work;
+	};
 	loff_t			offset;
 	loff_t			ref_ctr_offset;
 	unsigned long		flags;
@@ -627,10 +631,9 @@ static void uprobe_free_rcu(struct rcu_head *rcu)
 	kfree(uprobe);
 }
 
-static void put_uprobe(struct uprobe *uprobe)
+static void uprobe_free_deferred(struct work_struct *work)
 {
-	if (!refcount_dec_and_test(&uprobe->ref))
-		return;
+	struct uprobe *uprobe = container_of(work, struct uprobe, work);
 
 	write_lock(&uprobes_treelock);
 
@@ -654,6 +657,15 @@ static void put_uprobe(struct uprobe *uprobe)
 	call_rcu_tasks_trace(&uprobe->rcu, uprobe_free_rcu);
 }
 
+static void put_uprobe(struct uprobe *uprobe)
+{
+	if (!refcount_dec_and_test(&uprobe->ref))
+		return;
+
+	INIT_WORK(&uprobe->work, uprobe_free_deferred);
+	schedule_work(&uprobe->work);
+}
+
 static __always_inline
 int uprobe_cmp(const struct inode *l_inode, const loff_t l_offset,
 	       const struct uprobe *r)
-- 
2.43.5


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

* [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-08  0:25 [PATCH v2 tip/perf/core 0/2] SRCU-protected uretprobes hot path Andrii Nakryiko
  2024-10-08  0:25 ` [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context Andrii Nakryiko
@ 2024-10-08  0:25 ` Andrii Nakryiko
  2024-10-18 10:16   ` Peter Zijlstra
  1 sibling, 1 reply; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-08  0:25 UTC (permalink / raw)
  To: linux-trace-kernel, peterz, oleg
  Cc: rostedt, mhiramat, mingo, bpf, linux-kernel, jolsa, paulmck,
	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 synchronization related issues behind a small number of
basic helpers: hprobe_expire() for "downgrading" uprobe from SRCU-protected
state to refcounted state, and a hprobe_consume() and hprobe_finalize()
pair of single-use consuming helpers. 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.

Just to give an impression of a kind of performance improvements this
change brings, below are benchmarking results with and without these
SRCU changes, assuming other uprobe optimizations (mainly RCU Tasks
Trace for entry uprobes, lockless RB-tree lookup, and lockless VMA to
uprobe lookup) are left intact:

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)

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)

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

diff --git a/include/linux/uprobes.h b/include/linux/uprobes.h
index 2b294bf1881f..05c95abea252 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;
@@ -56,6 +57,53 @@ enum uprobe_task_state {
 	UTASK_SSTEP_TRAPPED,
 };
 
+/* The state of hybrid-lifetime uprobe inside struct return_instance */
+enum hprobe_state {
+	HPROBE_LEASED,		/* uretprobes_srcu-protected uprobe */
+	HPROBE_STABLE,		/* refcounted uprobe */
+	HPROBE_GONE,		/* NULL uprobe, SRCU expired, refcount failed */
+	HPROBE_CONSUMED,	/* uprobe "consumed" by uretprobe handler */
+};
+
+/*
+ * 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 {
+	enum hprobe_state state;
+	int srcu_idx;
+	struct uprobe *uprobe;
+};
+
 /*
  * uprobe_task: Metadata of a task while it singlesteps.
  */
@@ -75,6 +123,7 @@ struct uprobe_task {
 	};
 
 	struct uprobe			*active_uprobe;
+	struct timer_list		ri_timer;
 	unsigned long			xol_vaddr;
 
 	struct arch_uprobe              *auprobe;
@@ -84,14 +133,15 @@ struct uprobe_task {
 };
 
 struct return_instance {
-	struct uprobe		*uprobe;
+	struct hprobe		hprobe;
 	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 */
-};
+	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 9d3ab472200d..18b5ab949804 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -28,6 +28,7 @@
 #include <linux/khugepaged.h>
 #include <linux/rcupdate_trace.h>
 #include <linux/workqueue.h>
+#include <linux/srcu.h>
 
 #include <linux/uprobes.h>
 
@@ -51,6 +52,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
 
@@ -599,13 +603,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);
@@ -624,13 +621,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 uprobe_free_deferred(struct work_struct *work)
 {
 	struct uprobe *uprobe = container_of(work, struct uprobe, work);
@@ -654,7 +658,8 @@ static void uprobe_free_deferred(struct work_struct *work)
 	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);
 }
 
 static void put_uprobe(struct uprobe *uprobe)
@@ -666,6 +671,150 @@ static void put_uprobe(struct uprobe *uprobe)
 	schedule_work(&uprobe->work);
 }
 
+/* Initialize hprobe as SRCU-protected "leased" uprobe */
+static void hprobe_init_leased(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
+{
+	hprobe->state = HPROBE_LEASED;
+	hprobe->uprobe = uprobe;
+	hprobe->srcu_idx = srcu_idx;
+}
+
+/* Initialize hprobe as refcounted ("stable") uprobe (uprobe can be NULL). */
+static void hprobe_init_stable(struct hprobe *hprobe, struct uprobe *uprobe)
+{
+	hprobe->state = HPROBE_STABLE;
+	hprobe->uprobe = uprobe;
+	hprobe->srcu_idx = -1;
+}
+
+/*
+ * hprobe_consume() fetches hprobe's underlying uprobe and detects whether
+ * uprobe is SRCU protected or is refcounted. hprobe_consume() can be
+ * used only once for a given hprobe.
+ *
+ * Caller has to call hprobe_finalize() and pass previous hprobe_state, so
+ * that hprobe_finalize() can perform SRCU unlock or put uprobe, whichever
+ * is appropriate.
+ */
+static inline struct uprobe *hprobe_consume(struct hprobe *hprobe, enum hprobe_state *hstate)
+{
+	enum hprobe_state state;
+
+	*hstate = xchg(&hprobe->state, HPROBE_CONSUMED);
+	switch (*hstate) {
+	case HPROBE_LEASED:
+	case HPROBE_STABLE:
+		return hprobe->uprobe;
+	case HPROBE_GONE:
+		return NULL; /* couldn't refcnt uprobe, it's effectively NULL */
+	case HPROBE_CONSUMED:
+		return NULL; /* uprobe was finalized already, do nothing */
+	default:
+		WARN(1, "hprobe invalid state %d", state);
+		return NULL;
+	}
+}
+
+/*
+ * Reset hprobe state and, if hprobe was LEASED, release SRCU lock.
+ * hprobe_finalize() can only be used from current context after
+ * hprobe_consume() call (which determines uprobe and hstate value).
+ */
+static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
+{
+	switch (hstate) {
+	case HPROBE_LEASED:
+		__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
+		break;
+	case HPROBE_STABLE:
+		if (hprobe->uprobe)
+			put_uprobe(hprobe->uprobe);
+		break;
+	case HPROBE_GONE:
+	case HPROBE_CONSUMED:
+		break;
+	default:
+		WARN(1, "hprobe invalid state %d", hstate);
+		break;
+	}
+}
+
+/*
+ * Attempt to switch (atomically) uprobe from being SRCU 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
+ * preserve uprobe beyond current SRCU lock region. See dup_utask().
+ */
+static struct uprobe* hprobe_expire(struct hprobe *hprobe)
+{
+	enum hprobe_state hstate;
+
+	/*
+	 * 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));
+
+	hstate = data_race(READ_ONCE(hprobe->state));
+	switch (hstate) {
+	case HPROBE_STABLE:
+		/* uprobe is properly refcounted, return it */
+		return hprobe->uprobe;
+	case HPROBE_GONE:
+		/*
+		 * SRCU was unlocked earlier and we didn't manage to take
+		 * uprobe refcnt, so it's effectively NULL
+		 */
+		return NULL;
+	case HPROBE_CONSUMED:
+		/*
+		 * uprobe was consumed, so it's effectively NULL as far as
+		 * uretprobe processing logic is concerned
+		 */
+		return NULL;
+	case HPROBE_LEASED: {
+		struct uprobe *uprobe = try_get_uprobe(hprobe->uprobe);
+		/*
+		 * Try to switch hprobe state, guarding against
+		 * hprobe_consume() or another hprobe_expire() racing with us.
+		 * Note, if we failed to get uprobe refcount, we use special
+		 * HPROBE_GONE state to signal that hprobe->uprobe shouldn't
+		 * be used as it will be freed after SRCU is unlocked.
+		 */
+		if (try_cmpxchg(&hprobe->state, &hstate, uprobe ? HPROBE_STABLE : HPROBE_GONE)) {
+			/* We won the race, we are the ones to unlock SRCU */
+			__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
+			return uprobe;
+		}
+
+		/* We lost the race, undo refcount bump (if it ever happened) */
+		if (uprobe)
+			put_uprobe(uprobe);
+		/*
+		 * Even if hprobe_consume() or another hprobe_expire() wins
+		 * the state update race and unlocks SRCU from under us, we
+		 * still have a guarantee that underyling uprobe won't be
+		 * freed due to ongoing caller's SRCU lock region, so we can
+		 * return it regardless. The caller then can attempt its own
+		 * try_get_uprobe() to preserve the instance, if necessary.
+		 * This is used in dup_utask().
+		 */
+		return uprobe;
+	}
+	default:
+		WARN(1, "unknown hprobe state %d", hstate);
+		return NULL;
+	}
+}
+
 static __always_inline
 int uprobe_cmp(const struct inode *l_inode, const loff_t l_offset,
 	       const struct uprobe *r)
@@ -1168,6 +1317,7 @@ void uprobe_unregister_sync(void)
 	 * handler_chain() or handle_uretprobe_chain() to do an use-after-free.
 	 */
 	synchronize_rcu_tasks_trace();
+	synchronize_srcu(&uretprobes_srcu);
 }
 EXPORT_SYMBOL_GPL(uprobe_unregister_sync);
 
@@ -1748,11 +1898,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);
+
+	if (cleanup_hprobe) {
+		enum hprobe_state hstate;
+
+		(void)hprobe_consume(&ri->hprobe, &hstate);
+		hprobe_finalize(&ri->hprobe, hstate);
+	}
+
+	kfree_rcu(ri, rcu);
 	return next;
 }
 
@@ -1768,18 +1925,53 @@ 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 RI_TIMER_PERIOD (HZ / 10) /* 100 ms */
+
+#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.
@@ -1791,7 +1983,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;
 }
 
@@ -1799,12 +1991,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);
@@ -1812,17 +2008,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++;
 	}
 
@@ -1898,10 +2101,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)
@@ -1910,6 +2113,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;
@@ -1925,10 +2129,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;
@@ -1958,20 +2158,26 @@ 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);
+
+	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. */
@@ -2166,11 +2372,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)
@@ -2195,6 +2404,8 @@ void uprobe_handle_trampoline(struct pt_regs *regs)
 {
 	struct uprobe_task *utask;
 	struct return_instance *ri, *next;
+	struct uprobe *uprobe;
+	enum hprobe_state hstate;
 	bool valid;
 
 	utask = current->utask;
@@ -2225,21 +2436,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, &hstate);
 			if (valid)
-				handle_uretprobe_chain(ri, regs);
-			ri = free_ret_instance(ri);
+				handle_uretprobe_chain(ri, uprobe, regs);
+			hprobe_finalize(&ri->hprobe, hstate);
+
+			/* 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] 13+ messages in thread

* Re: [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context
  2024-10-08  0:25 ` [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context Andrii Nakryiko
@ 2024-10-18  8:26   ` Peter Zijlstra
  2024-10-18 18:22     ` Andrii Nakryiko
  0 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2024-10-18  8:26 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, oleg, rostedt, mhiramat, mingo, bpf,
	linux-kernel, jolsa, paulmck

On Mon, Oct 07, 2024 at 05:25:55PM -0700, Andrii Nakryiko wrote:
> Currently put_uprobe() might trigger mutex_lock()/mutex_unlock(), which
> makes it unsuitable to be called from more restricted context like softirq.

This is delayed_uprobe_lock, right?

So can't we do something like so instead? 

---
 kernel/events/uprobes.c | 40 +++++++++++++++++++++++-----------------
 1 file changed, 23 insertions(+), 17 deletions(-)

diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 2a0059464383..d17a9046de35 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -83,9 +83,11 @@ struct delayed_uprobe {
 	struct list_head list;
 	struct uprobe *uprobe;
 	struct mm_struct *mm;
+	struct rcu_head rcu;
 };
 
-static DEFINE_MUTEX(delayed_uprobe_lock);
+/* XXX global state; use per mm list instead ? */
+static DEFINE_SPINLOCK(delayed_uprobe_lock);
 static LIST_HEAD(delayed_uprobe_list);
 
 /*
@@ -289,9 +291,11 @@ delayed_uprobe_check(struct uprobe *uprobe, struct mm_struct *mm)
 {
 	struct delayed_uprobe *du;
 
-	list_for_each_entry(du, &delayed_uprobe_list, list)
+	guard(rcu)();
+	list_for_each_entry_rcu(du, &delayed_uprobe_list, list) {
 		if (du->uprobe == uprobe && du->mm == mm)
 			return du;
+	}
 	return NULL;
 }
 
@@ -308,7 +312,8 @@ static int delayed_uprobe_add(struct uprobe *uprobe, struct mm_struct *mm)
 
 	du->uprobe = uprobe;
 	du->mm = mm;
-	list_add(&du->list, &delayed_uprobe_list);
+	scoped_guard(spinlock, &delayed_uprobe_lock)
+		list_add_rcu(&du->list, &delayed_uprobe_list);
 	return 0;
 }
 
@@ -316,19 +321,21 @@ static void delayed_uprobe_delete(struct delayed_uprobe *du)
 {
 	if (WARN_ON(!du))
 		return;
-	list_del(&du->list);
-	kfree(du);
+	scoped_guard(spinlock, &delayed_uprobe_lock)
+		list_del(&du->list);
+	kfree_rcu(du, rcu);
 }
 
 static void delayed_uprobe_remove(struct uprobe *uprobe, struct mm_struct *mm)
 {
-	struct list_head *pos, *q;
 	struct delayed_uprobe *du;
+	struct list_head *pos;
 
 	if (!uprobe && !mm)
 		return;
 
-	list_for_each_safe(pos, q, &delayed_uprobe_list) {
+	guard(rcu)();
+	list_for_each_rcu(pos, &delayed_uprobe_list) {
 		du = list_entry(pos, struct delayed_uprobe, list);
 
 		if (uprobe && du->uprobe != uprobe)
@@ -434,12 +441,10 @@ static int update_ref_ctr(struct uprobe *uprobe, struct mm_struct *mm,
 			return ret;
 	}
 
-	mutex_lock(&delayed_uprobe_lock);
 	if (d > 0)
 		ret = delayed_uprobe_add(uprobe, mm);
 	else
 		delayed_uprobe_remove(uprobe, mm);
-	mutex_unlock(&delayed_uprobe_lock);
 
 	return ret;
 }
@@ -645,9 +650,7 @@ static void put_uprobe(struct uprobe *uprobe)
 	 * 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);
 
 	call_rcu_tasks_trace(&uprobe->rcu, uprobe_free_rcu);
 }
@@ -1350,13 +1353,18 @@ static void build_probe_list(struct inode *inode,
 /* @vma contains reference counter, not the probed instruction. */
 static int delayed_ref_ctr_inc(struct vm_area_struct *vma)
 {
-	struct list_head *pos, *q;
 	struct delayed_uprobe *du;
+	struct list_head *pos;
 	unsigned long vaddr;
 	int ret = 0, err = 0;
 
-	mutex_lock(&delayed_uprobe_lock);
-	list_for_each_safe(pos, q, &delayed_uprobe_list) {
+	/*
+	 * delayed_uprobe_list is added to when the ref_ctr is not mapped
+	 * and is consulted (this function) when adding maps. And since
+	 * mmap_lock serializes these, it is not possible miss an entry.
+	 */
+	guard(rcu)();
+	list_for_each_rcu(pos, &delayed_uprobe_list) {
 		du = list_entry(pos, struct delayed_uprobe, list);
 
 		if (du->mm != vma->vm_mm ||
@@ -1370,9 +1378,9 @@ static int delayed_ref_ctr_inc(struct vm_area_struct *vma)
 			if (!err)
 				err = ret;
 		}
+
 		delayed_uprobe_delete(du);
 	}
-	mutex_unlock(&delayed_uprobe_lock);
 	return err;
 }
 
@@ -1596,9 +1604,7 @@ void uprobe_clear_state(struct mm_struct *mm)
 {
 	struct xol_area *area = mm->uprobes_state.xol_area;
 
-	mutex_lock(&delayed_uprobe_lock);
 	delayed_uprobe_remove(NULL, mm);
-	mutex_unlock(&delayed_uprobe_lock);
 
 	if (!area)
 		return;

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

* Re: [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-08  0:25 ` [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
@ 2024-10-18 10:16   ` Peter Zijlstra
  2024-10-18 18:22     ` Andrii Nakryiko
  0 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2024-10-18 10:16 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: linux-trace-kernel, oleg, rostedt, mhiramat, mingo, bpf,
	linux-kernel, jolsa, paulmck

On Mon, Oct 07, 2024 at 05:25:56PM -0700, Andrii Nakryiko wrote:

> +/* Initialize hprobe as SRCU-protected "leased" uprobe */
> +static void hprobe_init_leased(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
> +{
> +	hprobe->state = HPROBE_LEASED;
> +	hprobe->uprobe = uprobe;
> +	hprobe->srcu_idx = srcu_idx;
> +}
> +
> +/* Initialize hprobe as refcounted ("stable") uprobe (uprobe can be NULL). */
> +static void hprobe_init_stable(struct hprobe *hprobe, struct uprobe *uprobe)
> +{
> +	hprobe->state = HPROBE_STABLE;
> +	hprobe->uprobe = uprobe;
> +	hprobe->srcu_idx = -1;
> +}
> +
> +/*
> + * hprobe_consume() fetches hprobe's underlying uprobe and detects whether
> + * uprobe is SRCU protected or is refcounted. hprobe_consume() can be
> + * used only once for a given hprobe.
> + *
> + * Caller has to call hprobe_finalize() and pass previous hprobe_state, so
> + * that hprobe_finalize() can perform SRCU unlock or put uprobe, whichever
> + * is appropriate.
> + */
> +static inline struct uprobe *hprobe_consume(struct hprobe *hprobe, enum hprobe_state *hstate)
> +{
> +	enum hprobe_state state;
> +
> +	*hstate = xchg(&hprobe->state, HPROBE_CONSUMED);
> +	switch (*hstate) {
> +	case HPROBE_LEASED:
> +	case HPROBE_STABLE:
> +		return hprobe->uprobe;
> +	case HPROBE_GONE:
> +		return NULL; /* couldn't refcnt uprobe, it's effectively NULL */
> +	case HPROBE_CONSUMED:
> +		return NULL; /* uprobe was finalized already, do nothing */
> +	default:
> +		WARN(1, "hprobe invalid state %d", state);
> +		return NULL;
> +	}
> +}
> +
> +/*
> + * Reset hprobe state and, if hprobe was LEASED, release SRCU lock.
> + * hprobe_finalize() can only be used from current context after
> + * hprobe_consume() call (which determines uprobe and hstate value).
> + */
> +static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
> +{
> +	switch (hstate) {
> +	case HPROBE_LEASED:
> +		__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
> +		break;
> +	case HPROBE_STABLE:
> +		if (hprobe->uprobe)
> +			put_uprobe(hprobe->uprobe);
> +		break;
> +	case HPROBE_GONE:
> +	case HPROBE_CONSUMED:
> +		break;
> +	default:
> +		WARN(1, "hprobe invalid state %d", hstate);
> +		break;
> +	}
> +}
> +
> +/*
> + * Attempt to switch (atomically) uprobe from being SRCU 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
> + * preserve uprobe beyond current SRCU lock region. See dup_utask().
> + */
> +static struct uprobe* hprobe_expire(struct hprobe *hprobe)
> +{
> +	enum hprobe_state hstate;
> +
> +	/*
> +	 * 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));
> +
> +	hstate = data_race(READ_ONCE(hprobe->state));
> +	switch (hstate) {
> +	case HPROBE_STABLE:
> +		/* uprobe is properly refcounted, return it */
> +		return hprobe->uprobe;
> +	case HPROBE_GONE:
> +		/*
> +		 * SRCU was unlocked earlier and we didn't manage to take
> +		 * uprobe refcnt, so it's effectively NULL
> +		 */
> +		return NULL;
> +	case HPROBE_CONSUMED:
> +		/*
> +		 * uprobe was consumed, so it's effectively NULL as far as
> +		 * uretprobe processing logic is concerned
> +		 */
> +		return NULL;
> +	case HPROBE_LEASED: {
> +		struct uprobe *uprobe = try_get_uprobe(hprobe->uprobe);
> +		/*
> +		 * Try to switch hprobe state, guarding against
> +		 * hprobe_consume() or another hprobe_expire() racing with us.
> +		 * Note, if we failed to get uprobe refcount, we use special
> +		 * HPROBE_GONE state to signal that hprobe->uprobe shouldn't
> +		 * be used as it will be freed after SRCU is unlocked.
> +		 */
> +		if (try_cmpxchg(&hprobe->state, &hstate, uprobe ? HPROBE_STABLE : HPROBE_GONE)) {
> +			/* We won the race, we are the ones to unlock SRCU */
> +			__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
> +			return uprobe;
> +		}
> +
> +		/* We lost the race, undo refcount bump (if it ever happened) */
> +		if (uprobe)
> +			put_uprobe(uprobe);
> +		/*
> +		 * Even if hprobe_consume() or another hprobe_expire() wins
> +		 * the state update race and unlocks SRCU from under us, we
> +		 * still have a guarantee that underyling uprobe won't be
> +		 * freed due to ongoing caller's SRCU lock region, so we can
> +		 * return it regardless. The caller then can attempt its own
> +		 * try_get_uprobe() to preserve the instance, if necessary.
> +		 * This is used in dup_utask().
> +		 */
> +		return uprobe;
> +	}
> +	default:
> +		WARN(1, "unknown hprobe state %d", hstate);
> +		return NULL;
> +	}
> +}

So... after a few readings I think I'm mostly okay with this. But I got
annoyed by the whole HPROBE_STABLE with uprobe=NULL weirdness. Also,
that data_race() usage is weird, what is that about?

And then there's the case where we end up doing:

  try_get_uprobe()
  put_uprobe()
  try_get_uprobe()

in the dup path. Yes, it's unlikely, but gah.


So how about something like this?

---
diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
index 06ec41c75c45..efb4f5ee6212 100644
--- a/kernel/events/uprobes.c
+++ b/kernel/events/uprobes.c
@@ -657,20 +657,19 @@ static void put_uprobe(struct uprobe *uprobe)
 	call_srcu(&uretprobes_srcu, &uprobe->rcu, uprobe_free_srcu);
 }
 
-/* Initialize hprobe as SRCU-protected "leased" uprobe */
-static void hprobe_init_leased(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
+static void hprobe_init(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
 {
-	hprobe->state = HPROBE_LEASED;
-	hprobe->uprobe = uprobe;
-	hprobe->srcu_idx = srcu_idx;
-}
+	enum hprobe_state state = HPROBE_GONE;
 
-/* Initialize hprobe as refcounted ("stable") uprobe (uprobe can be NULL). */
-static void hprobe_init_stable(struct hprobe *hprobe, struct uprobe *uprobe)
-{
-	hprobe->state = HPROBE_STABLE;
+	if (uprobe) {
+		state = HPROBE_LEASED;
+		if (srcu_idx < 0)
+			state = HPROBE_STABLE;
+	}
+
+	hprobe->state = state;
 	hprobe->uprobe = uprobe;
-	hprobe->srcu_idx = -1;
+	hprobe->srcu_idx = srcu_idx;
 }
 
 /*
@@ -713,8 +712,7 @@ static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
 		__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
 		break;
 	case HPROBE_STABLE:
-		if (hprobe->uprobe)
-			put_uprobe(hprobe->uprobe);
+		put_uprobe(hprobe->uprobe);
 		break;
 	case HPROBE_GONE:
 	case HPROBE_CONSUMED:
@@ -739,8 +737,9 @@ static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
  * refcount, so caller has to attempt try_get_uprobe(), if it needs to
  * preserve uprobe beyond current SRCU lock region. See dup_utask().
  */
-static struct uprobe* hprobe_expire(struct hprobe *hprobe)
+static struct uprobe *hprobe_expire(struct hprobe *hprobe, bool get)
 {
+	struct uprobe *uprobe = NULL;
 	enum hprobe_state hstate;
 
 	/*
@@ -749,25 +748,18 @@ static struct uprobe* hprobe_expire(struct hprobe *hprobe)
 	 */
 	lockdep_assert(rcu_read_lock_held() && srcu_read_lock_held(&uretprobes_srcu));
 
-	hstate = data_race(READ_ONCE(hprobe->state));
+	hstate = READ_ONCE(hprobe->state);
 	switch (hstate) {
 	case HPROBE_STABLE:
-		/* uprobe is properly refcounted, return it */
-		return hprobe->uprobe;
+		uprobe = hprobe->uprobe;
+		break;
+
 	case HPROBE_GONE:
-		/*
-		 * SRCU was unlocked earlier and we didn't manage to take
-		 * uprobe refcnt, so it's effectively NULL
-		 */
-		return NULL;
 	case HPROBE_CONSUMED:
-		/*
-		 * uprobe was consumed, so it's effectively NULL as far as
-		 * uretprobe processing logic is concerned
-		 */
-		return NULL;
-	case HPROBE_LEASED: {
-		struct uprobe *uprobe = try_get_uprobe(hprobe->uprobe);
+		break;
+
+	case HPROBE_LEASED:
+		uprobe = try_get_uprobe(hprobe->uprobe);
 		/*
 		 * Try to switch hprobe state, guarding against
 		 * hprobe_consume() or another hprobe_expire() racing with us.
@@ -778,27 +770,26 @@ static struct uprobe* hprobe_expire(struct hprobe *hprobe)
 		if (try_cmpxchg(&hprobe->state, &hstate, uprobe ? HPROBE_STABLE : HPROBE_GONE)) {
 			/* We won the race, we are the ones to unlock SRCU */
 			__srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
-			return uprobe;
+			break;
 		}
 
 		/* We lost the race, undo refcount bump (if it ever happened) */
-		if (uprobe)
+		if (uprobe && !get) {
 			put_uprobe(uprobe);
-		/*
-		 * Even if hprobe_consume() or another hprobe_expire() wins
-		 * the state update race and unlocks SRCU from under us, we
-		 * still have a guarantee that underyling uprobe won't be
-		 * freed due to ongoing caller's SRCU lock region, so we can
-		 * return it regardless. The caller then can attempt its own
-		 * try_get_uprobe() to preserve the instance, if necessary.
-		 * This is used in dup_utask().
-		 */
+			uprobe = NULL;
+		}
+
 		return uprobe;
-	}
+
 	default:
 		WARN(1, "unknown hprobe state %d", hstate);
 		return NULL;
 	}
+
+	if (uprobe && get)
+		return try_get_uprobe(uprobe);
+
+	return uprobe;
 }
 
 static __always_inline
@@ -1920,9 +1911,8 @@ static void ri_timer(struct timer_list *timer)
 	/* RCU protects return_instance from freeing. */
 	guard(rcu)();
 
-	for_each_ret_instance_rcu(ri, utask->return_instances) {
-		hprobe_expire(&ri->hprobe);
-	}
+	for_each_ret_instance_rcu(ri, utask->return_instances)
+		hprobe_expire(&ri->hprobe, false);
 }
 
 static struct uprobe_task *alloc_utask(void)
@@ -1975,10 +1965,7 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
 
 		*n = *o;
 
-		/* see hprobe_expire() comments */
-		uprobe = hprobe_expire(&o->hprobe);
-		if (uprobe) /* refcount bump for new utask */
-			uprobe = try_get_uprobe(uprobe);
+		uprobe = hprobe_expire(&o->hprobe, true);
 
 		/*
 		 * New utask will have stable properly refcounted uprobe or
@@ -1986,7 +1973,7 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
 		 * need to preserve full set of return_instances for proper
 		 * uretprobe handling and nesting in forked task.
 		 */
-		hprobe_init_stable(&n->hprobe, uprobe);
+		hprobe_init(&n->hprobe, uprobe, -1);
 
 		n->next = NULL;
 		rcu_assign_pointer(*p, n);
@@ -2131,7 +2118,7 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
 
 	utask->depth++;
 
-	hprobe_init_leased(&ri->hprobe, uprobe, srcu_idx);
+	hprobe_init(&ri->hprobe, uprobe, srcu_idx);
 	ri->next = utask->return_instances;
 	rcu_assign_pointer(utask->return_instances, ri);
 

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

* Re: [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context
  2024-10-18  8:26   ` Peter Zijlstra
@ 2024-10-18 18:22     ` Andrii Nakryiko
  2024-10-21 10:31       ` Peter Zijlstra
  0 siblings, 1 reply; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-18 18:22 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Andrii Nakryiko, linux-trace-kernel, oleg, rostedt, mhiramat,
	mingo, bpf, linux-kernel, jolsa, paulmck

On Fri, Oct 18, 2024 at 1:26 AM Peter Zijlstra <peterz@infradead.org> wrote:
>
> On Mon, Oct 07, 2024 at 05:25:55PM -0700, Andrii Nakryiko wrote:
> > Currently put_uprobe() might trigger mutex_lock()/mutex_unlock(), which
> > makes it unsuitable to be called from more restricted context like softirq.
>
> This is delayed_uprobe_lock, right?

Not just delated_uprobe_lock, there is also uprobes_treelock (I forgot
to update the commit message to mention that). Oleg had concerns (see
[0]) with that being taken from the timer thread, so I just moved all
of the locking into deferred work callback.

  [0] https://lore.kernel.org/linux-trace-kernel/20240915144910.GA27726@redhat.com/

>
> So can't we do something like so instead?

I'll need to look at this more thoroughly (and hopefully Oleg will get
a chance as well), dropping lock from delayed_ref_ctr_inc() is a bit
scary, but might be ok.

But generally speaking, what's your concern with doing deferred work
in put_uprobe()? It's not a hot path by any means, worst case we'll
have maybe thousands of uprobes attached/detached.

>
> ---
>  kernel/events/uprobes.c | 40 +++++++++++++++++++++++-----------------
>  1 file changed, 23 insertions(+), 17 deletions(-)
>
> diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
> index 2a0059464383..d17a9046de35 100644
> --- a/kernel/events/uprobes.c
> +++ b/kernel/events/uprobes.c
> @@ -83,9 +83,11 @@ struct delayed_uprobe {
>         struct list_head list;
>         struct uprobe *uprobe;
>         struct mm_struct *mm;
> +       struct rcu_head rcu;
>  };
>
> -static DEFINE_MUTEX(delayed_uprobe_lock);
> +/* XXX global state; use per mm list instead ? */
> +static DEFINE_SPINLOCK(delayed_uprobe_lock);
>  static LIST_HEAD(delayed_uprobe_list);
>
>  /*
> @@ -289,9 +291,11 @@ delayed_uprobe_check(struct uprobe *uprobe, struct mm_struct *mm)
>  {
>         struct delayed_uprobe *du;
>
> -       list_for_each_entry(du, &delayed_uprobe_list, list)
> +       guard(rcu)();
> +       list_for_each_entry_rcu(du, &delayed_uprobe_list, list) {
>                 if (du->uprobe == uprobe && du->mm == mm)
>                         return du;
> +       }
>         return NULL;
>  }
>
> @@ -308,7 +312,8 @@ static int delayed_uprobe_add(struct uprobe *uprobe, struct mm_struct *mm)
>
>         du->uprobe = uprobe;
>         du->mm = mm;
> -       list_add(&du->list, &delayed_uprobe_list);
> +       scoped_guard(spinlock, &delayed_uprobe_lock)
> +               list_add_rcu(&du->list, &delayed_uprobe_list);
>         return 0;
>  }
>
> @@ -316,19 +321,21 @@ static void delayed_uprobe_delete(struct delayed_uprobe *du)
>  {
>         if (WARN_ON(!du))
>                 return;
> -       list_del(&du->list);
> -       kfree(du);
> +       scoped_guard(spinlock, &delayed_uprobe_lock)
> +               list_del(&du->list);
> +       kfree_rcu(du, rcu);
>  }
>
>  static void delayed_uprobe_remove(struct uprobe *uprobe, struct mm_struct *mm)
>  {
> -       struct list_head *pos, *q;
>         struct delayed_uprobe *du;
> +       struct list_head *pos;
>
>         if (!uprobe && !mm)
>                 return;
>
> -       list_for_each_safe(pos, q, &delayed_uprobe_list) {
> +       guard(rcu)();
> +       list_for_each_rcu(pos, &delayed_uprobe_list) {
>                 du = list_entry(pos, struct delayed_uprobe, list);
>
>                 if (uprobe && du->uprobe != uprobe)
> @@ -434,12 +441,10 @@ static int update_ref_ctr(struct uprobe *uprobe, struct mm_struct *mm,
>                         return ret;
>         }
>
> -       mutex_lock(&delayed_uprobe_lock);
>         if (d > 0)
>                 ret = delayed_uprobe_add(uprobe, mm);
>         else
>                 delayed_uprobe_remove(uprobe, mm);
> -       mutex_unlock(&delayed_uprobe_lock);
>
>         return ret;
>  }
> @@ -645,9 +650,7 @@ static void put_uprobe(struct uprobe *uprobe)
>          * 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);
>
>         call_rcu_tasks_trace(&uprobe->rcu, uprobe_free_rcu);
>  }
> @@ -1350,13 +1353,18 @@ static void build_probe_list(struct inode *inode,
>  /* @vma contains reference counter, not the probed instruction. */
>  static int delayed_ref_ctr_inc(struct vm_area_struct *vma)
>  {
> -       struct list_head *pos, *q;
>         struct delayed_uprobe *du;
> +       struct list_head *pos;
>         unsigned long vaddr;
>         int ret = 0, err = 0;
>
> -       mutex_lock(&delayed_uprobe_lock);
> -       list_for_each_safe(pos, q, &delayed_uprobe_list) {
> +       /*
> +        * delayed_uprobe_list is added to when the ref_ctr is not mapped
> +        * and is consulted (this function) when adding maps. And since
> +        * mmap_lock serializes these, it is not possible miss an entry.
> +        */
> +       guard(rcu)();
> +       list_for_each_rcu(pos, &delayed_uprobe_list) {
>                 du = list_entry(pos, struct delayed_uprobe, list);
>
>                 if (du->mm != vma->vm_mm ||
> @@ -1370,9 +1378,9 @@ static int delayed_ref_ctr_inc(struct vm_area_struct *vma)
>                         if (!err)
>                                 err = ret;
>                 }
> +
>                 delayed_uprobe_delete(du);
>         }
> -       mutex_unlock(&delayed_uprobe_lock);
>         return err;
>  }
>
> @@ -1596,9 +1604,7 @@ void uprobe_clear_state(struct mm_struct *mm)
>  {
>         struct xol_area *area = mm->uprobes_state.xol_area;
>
> -       mutex_lock(&delayed_uprobe_lock);
>         delayed_uprobe_remove(NULL, mm);
> -       mutex_unlock(&delayed_uprobe_lock);
>
>         if (!area)
>                 return;

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

* Re: [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-18 10:16   ` Peter Zijlstra
@ 2024-10-18 18:22     ` Andrii Nakryiko
  2024-10-19  0:09       ` Paul E. McKenney
  2024-10-21 10:48       ` Peter Zijlstra
  0 siblings, 2 replies; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-18 18:22 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Andrii Nakryiko, linux-trace-kernel, oleg, rostedt, mhiramat,
	mingo, bpf, linux-kernel, jolsa, paulmck

On Fri, Oct 18, 2024 at 3:16 AM Peter Zijlstra <peterz@infradead.org> wrote:
>
> On Mon, Oct 07, 2024 at 05:25:56PM -0700, Andrii Nakryiko wrote:
>

[...]

> > +
> > +             /* We lost the race, undo refcount bump (if it ever happened) */
> > +             if (uprobe)
> > +                     put_uprobe(uprobe);
> > +             /*
> > +              * Even if hprobe_consume() or another hprobe_expire() wins
> > +              * the state update race and unlocks SRCU from under us, we
> > +              * still have a guarantee that underyling uprobe won't be
> > +              * freed due to ongoing caller's SRCU lock region, so we can
> > +              * return it regardless. The caller then can attempt its own
> > +              * try_get_uprobe() to preserve the instance, if necessary.
> > +              * This is used in dup_utask().
> > +              */
> > +             return uprobe;
> > +     }
> > +     default:
> > +             WARN(1, "unknown hprobe state %d", hstate);
> > +             return NULL;
> > +     }
> > +}
>
> So... after a few readings I think I'm mostly okay with this. But I got
> annoyed by the whole HPROBE_STABLE with uprobe=NULL weirdness. Also,
> that data_race() usage is weird, what is that about?

People keep saying that evil KCSAN will come after me if I don't add
data_race() for values that can change under me, so I add it to make
it explicit that it's fine. But I can of course just drop data_race(),
as it has no bearing on correctness.

>
> And then there's the case where we end up doing:
>
>   try_get_uprobe()
>   put_uprobe()
>   try_get_uprobe()
>
> in the dup path. Yes, it's unlikely, but gah.
>
>
> So how about something like this?

Yep, it makes sense to start with HPROBE_GONE if it's already NULL, no
problem. I'll roll those changes in.

I'm fine with the `bool get` flag as well. Will incorporate all that
into the next revision, thanks!

The only problem I can see is in the assumption that `srcu_idx < 0` is
never going to be returned by srcu_read_lock(). Paul says that it can
only be 0 or 1, but it's not codified as part of a contract. So until
we change that, probably safer to pass an extra bool specifying
whether srcu_idx is valid or not, is that OK?

(and I assume you want me to drop verbose comments for various states, right?)



>
> ---
> diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
> index 06ec41c75c45..efb4f5ee6212 100644
> --- a/kernel/events/uprobes.c
> +++ b/kernel/events/uprobes.c
> @@ -657,20 +657,19 @@ static void put_uprobe(struct uprobe *uprobe)
>         call_srcu(&uretprobes_srcu, &uprobe->rcu, uprobe_free_srcu);
>  }
>
> -/* Initialize hprobe as SRCU-protected "leased" uprobe */
> -static void hprobe_init_leased(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
> +static void hprobe_init(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
>  {
> -       hprobe->state = HPROBE_LEASED;
> -       hprobe->uprobe = uprobe;
> -       hprobe->srcu_idx = srcu_idx;
> -}
> +       enum hprobe_state state = HPROBE_GONE;
>
> -/* Initialize hprobe as refcounted ("stable") uprobe (uprobe can be NULL). */
> -static void hprobe_init_stable(struct hprobe *hprobe, struct uprobe *uprobe)
> -{
> -       hprobe->state = HPROBE_STABLE;
> +       if (uprobe) {
> +               state = HPROBE_LEASED;
> +               if (srcu_idx < 0)
> +                       state = HPROBE_STABLE;
> +       }
> +
> +       hprobe->state = state;
>         hprobe->uprobe = uprobe;
> -       hprobe->srcu_idx = -1;
> +       hprobe->srcu_idx = srcu_idx;
>  }
>
>  /*
> @@ -713,8 +712,7 @@ static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
>                 __srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
>                 break;
>         case HPROBE_STABLE:
> -               if (hprobe->uprobe)
> -                       put_uprobe(hprobe->uprobe);
> +               put_uprobe(hprobe->uprobe);
>                 break;
>         case HPROBE_GONE:
>         case HPROBE_CONSUMED:
> @@ -739,8 +737,9 @@ static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
>   * refcount, so caller has to attempt try_get_uprobe(), if it needs to
>   * preserve uprobe beyond current SRCU lock region. See dup_utask().
>   */
> -static struct uprobe* hprobe_expire(struct hprobe *hprobe)
> +static struct uprobe *hprobe_expire(struct hprobe *hprobe, bool get)
>  {
> +       struct uprobe *uprobe = NULL;
>         enum hprobe_state hstate;
>
>         /*
> @@ -749,25 +748,18 @@ static struct uprobe* hprobe_expire(struct hprobe *hprobe)
>          */
>         lockdep_assert(rcu_read_lock_held() && srcu_read_lock_held(&uretprobes_srcu));
>
> -       hstate = data_race(READ_ONCE(hprobe->state));
> +       hstate = READ_ONCE(hprobe->state);
>         switch (hstate) {
>         case HPROBE_STABLE:
> -               /* uprobe is properly refcounted, return it */
> -               return hprobe->uprobe;
> +               uprobe = hprobe->uprobe;
> +               break;
> +
>         case HPROBE_GONE:
> -               /*
> -                * SRCU was unlocked earlier and we didn't manage to take
> -                * uprobe refcnt, so it's effectively NULL
> -                */
> -               return NULL;
>         case HPROBE_CONSUMED:
> -               /*
> -                * uprobe was consumed, so it's effectively NULL as far as
> -                * uretprobe processing logic is concerned
> -                */
> -               return NULL;
> -       case HPROBE_LEASED: {
> -               struct uprobe *uprobe = try_get_uprobe(hprobe->uprobe);
> +               break;
> +
> +       case HPROBE_LEASED:
> +               uprobe = try_get_uprobe(hprobe->uprobe);
>                 /*
>                  * Try to switch hprobe state, guarding against
>                  * hprobe_consume() or another hprobe_expire() racing with us.
> @@ -778,27 +770,26 @@ static struct uprobe* hprobe_expire(struct hprobe *hprobe)
>                 if (try_cmpxchg(&hprobe->state, &hstate, uprobe ? HPROBE_STABLE : HPROBE_GONE)) {
>                         /* We won the race, we are the ones to unlock SRCU */
>                         __srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
> -                       return uprobe;
> +                       break;
>                 }
>
>                 /* We lost the race, undo refcount bump (if it ever happened) */
> -               if (uprobe)
> +               if (uprobe && !get) {
>                         put_uprobe(uprobe);
> -               /*
> -                * Even if hprobe_consume() or another hprobe_expire() wins
> -                * the state update race and unlocks SRCU from under us, we
> -                * still have a guarantee that underyling uprobe won't be
> -                * freed due to ongoing caller's SRCU lock region, so we can
> -                * return it regardless. The caller then can attempt its own
> -                * try_get_uprobe() to preserve the instance, if necessary.
> -                * This is used in dup_utask().
> -                */
> +                       uprobe = NULL;
> +               }
> +
>                 return uprobe;
> -       }
> +
>         default:
>                 WARN(1, "unknown hprobe state %d", hstate);
>                 return NULL;
>         }
> +
> +       if (uprobe && get)
> +               return try_get_uprobe(uprobe);
> +
> +       return uprobe;
>  }
>
>  static __always_inline
> @@ -1920,9 +1911,8 @@ static void ri_timer(struct timer_list *timer)
>         /* RCU protects return_instance from freeing. */
>         guard(rcu)();
>
> -       for_each_ret_instance_rcu(ri, utask->return_instances) {
> -               hprobe_expire(&ri->hprobe);
> -       }
> +       for_each_ret_instance_rcu(ri, utask->return_instances)
> +               hprobe_expire(&ri->hprobe, false);
>  }
>
>  static struct uprobe_task *alloc_utask(void)
> @@ -1975,10 +1965,7 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
>
>                 *n = *o;
>
> -               /* see hprobe_expire() comments */
> -               uprobe = hprobe_expire(&o->hprobe);
> -               if (uprobe) /* refcount bump for new utask */
> -                       uprobe = try_get_uprobe(uprobe);
> +               uprobe = hprobe_expire(&o->hprobe, true);
>
>                 /*
>                  * New utask will have stable properly refcounted uprobe or
> @@ -1986,7 +1973,7 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
>                  * need to preserve full set of return_instances for proper
>                  * uretprobe handling and nesting in forked task.
>                  */
> -               hprobe_init_stable(&n->hprobe, uprobe);
> +               hprobe_init(&n->hprobe, uprobe, -1);
>
>                 n->next = NULL;
>                 rcu_assign_pointer(*p, n);
> @@ -2131,7 +2118,7 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
>
>         utask->depth++;
>
> -       hprobe_init_leased(&ri->hprobe, uprobe, srcu_idx);
> +       hprobe_init(&ri->hprobe, uprobe, srcu_idx);
>         ri->next = utask->return_instances;
>         rcu_assign_pointer(utask->return_instances, ri);
>

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

* Re: [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-18 18:22     ` Andrii Nakryiko
@ 2024-10-19  0:09       ` Paul E. McKenney
  2024-10-21 10:48       ` Peter Zijlstra
  1 sibling, 0 replies; 13+ messages in thread
From: Paul E. McKenney @ 2024-10-19  0:09 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: Peter Zijlstra, Andrii Nakryiko, linux-trace-kernel, oleg,
	rostedt, mhiramat, mingo, bpf, linux-kernel, jolsa

On Fri, Oct 18, 2024 at 11:22:09AM -0700, Andrii Nakryiko wrote:
> On Fri, Oct 18, 2024 at 3:16 AM Peter Zijlstra <peterz@infradead.org> wrote:
> >
> > On Mon, Oct 07, 2024 at 05:25:56PM -0700, Andrii Nakryiko wrote:
> >
> 
> [...]
> 
> > > +
> > > +             /* We lost the race, undo refcount bump (if it ever happened) */
> > > +             if (uprobe)
> > > +                     put_uprobe(uprobe);
> > > +             /*
> > > +              * Even if hprobe_consume() or another hprobe_expire() wins
> > > +              * the state update race and unlocks SRCU from under us, we
> > > +              * still have a guarantee that underyling uprobe won't be
> > > +              * freed due to ongoing caller's SRCU lock region, so we can
> > > +              * return it regardless. The caller then can attempt its own
> > > +              * try_get_uprobe() to preserve the instance, if necessary.
> > > +              * This is used in dup_utask().
> > > +              */
> > > +             return uprobe;
> > > +     }
> > > +     default:
> > > +             WARN(1, "unknown hprobe state %d", hstate);
> > > +             return NULL;
> > > +     }
> > > +}
> >
> > So... after a few readings I think I'm mostly okay with this. But I got
> > annoyed by the whole HPROBE_STABLE with uprobe=NULL weirdness. Also,
> > that data_race() usage is weird, what is that about?
> 
> People keep saying that evil KCSAN will come after me if I don't add
> data_race() for values that can change under me, so I add it to make
> it explicit that it's fine. But I can of course just drop data_race(),
> as it has no bearing on correctness.

Use data_race() if you know that the compiler cannot mess you up or when
its messing you up is acceptable.  An example of the latter is code that
gathers statistics or debug information, at least in some cases.

Something like READ_ONCE() or atomic_read() is better when the compiler
can mess you up.

For more information:  https://lwn.net/Articles/793253/
https://lwn.net/Articles/799218/

							Thanx, Paul

> > And then there's the case where we end up doing:
> >
> >   try_get_uprobe()
> >   put_uprobe()
> >   try_get_uprobe()
> >
> > in the dup path. Yes, it's unlikely, but gah.
> >
> >
> > So how about something like this?
> 
> Yep, it makes sense to start with HPROBE_GONE if it's already NULL, no
> problem. I'll roll those changes in.
> 
> I'm fine with the `bool get` flag as well. Will incorporate all that
> into the next revision, thanks!
> 
> The only problem I can see is in the assumption that `srcu_idx < 0` is
> never going to be returned by srcu_read_lock(). Paul says that it can
> only be 0 or 1, but it's not codified as part of a contract. So until
> we change that, probably safer to pass an extra bool specifying
> whether srcu_idx is valid or not, is that OK?
> 
> (and I assume you want me to drop verbose comments for various states, right?)
> 
> 
> 
> >
> > ---
> > diff --git a/kernel/events/uprobes.c b/kernel/events/uprobes.c
> > index 06ec41c75c45..efb4f5ee6212 100644
> > --- a/kernel/events/uprobes.c
> > +++ b/kernel/events/uprobes.c
> > @@ -657,20 +657,19 @@ static void put_uprobe(struct uprobe *uprobe)
> >         call_srcu(&uretprobes_srcu, &uprobe->rcu, uprobe_free_srcu);
> >  }
> >
> > -/* Initialize hprobe as SRCU-protected "leased" uprobe */
> > -static void hprobe_init_leased(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
> > +static void hprobe_init(struct hprobe *hprobe, struct uprobe *uprobe, int srcu_idx)
> >  {
> > -       hprobe->state = HPROBE_LEASED;
> > -       hprobe->uprobe = uprobe;
> > -       hprobe->srcu_idx = srcu_idx;
> > -}
> > +       enum hprobe_state state = HPROBE_GONE;
> >
> > -/* Initialize hprobe as refcounted ("stable") uprobe (uprobe can be NULL). */
> > -static void hprobe_init_stable(struct hprobe *hprobe, struct uprobe *uprobe)
> > -{
> > -       hprobe->state = HPROBE_STABLE;
> > +       if (uprobe) {
> > +               state = HPROBE_LEASED;
> > +               if (srcu_idx < 0)
> > +                       state = HPROBE_STABLE;
> > +       }
> > +
> > +       hprobe->state = state;
> >         hprobe->uprobe = uprobe;
> > -       hprobe->srcu_idx = -1;
> > +       hprobe->srcu_idx = srcu_idx;
> >  }
> >
> >  /*
> > @@ -713,8 +712,7 @@ static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
> >                 __srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
> >                 break;
> >         case HPROBE_STABLE:
> > -               if (hprobe->uprobe)
> > -                       put_uprobe(hprobe->uprobe);
> > +               put_uprobe(hprobe->uprobe);
> >                 break;
> >         case HPROBE_GONE:
> >         case HPROBE_CONSUMED:
> > @@ -739,8 +737,9 @@ static void hprobe_finalize(struct hprobe *hprobe, enum hprobe_state hstate)
> >   * refcount, so caller has to attempt try_get_uprobe(), if it needs to
> >   * preserve uprobe beyond current SRCU lock region. See dup_utask().
> >   */
> > -static struct uprobe* hprobe_expire(struct hprobe *hprobe)
> > +static struct uprobe *hprobe_expire(struct hprobe *hprobe, bool get)
> >  {
> > +       struct uprobe *uprobe = NULL;
> >         enum hprobe_state hstate;
> >
> >         /*
> > @@ -749,25 +748,18 @@ static struct uprobe* hprobe_expire(struct hprobe *hprobe)
> >          */
> >         lockdep_assert(rcu_read_lock_held() && srcu_read_lock_held(&uretprobes_srcu));
> >
> > -       hstate = data_race(READ_ONCE(hprobe->state));
> > +       hstate = READ_ONCE(hprobe->state);
> >         switch (hstate) {
> >         case HPROBE_STABLE:
> > -               /* uprobe is properly refcounted, return it */
> > -               return hprobe->uprobe;
> > +               uprobe = hprobe->uprobe;
> > +               break;
> > +
> >         case HPROBE_GONE:
> > -               /*
> > -                * SRCU was unlocked earlier and we didn't manage to take
> > -                * uprobe refcnt, so it's effectively NULL
> > -                */
> > -               return NULL;
> >         case HPROBE_CONSUMED:
> > -               /*
> > -                * uprobe was consumed, so it's effectively NULL as far as
> > -                * uretprobe processing logic is concerned
> > -                */
> > -               return NULL;
> > -       case HPROBE_LEASED: {
> > -               struct uprobe *uprobe = try_get_uprobe(hprobe->uprobe);
> > +               break;
> > +
> > +       case HPROBE_LEASED:
> > +               uprobe = try_get_uprobe(hprobe->uprobe);
> >                 /*
> >                  * Try to switch hprobe state, guarding against
> >                  * hprobe_consume() or another hprobe_expire() racing with us.
> > @@ -778,27 +770,26 @@ static struct uprobe* hprobe_expire(struct hprobe *hprobe)
> >                 if (try_cmpxchg(&hprobe->state, &hstate, uprobe ? HPROBE_STABLE : HPROBE_GONE)) {
> >                         /* We won the race, we are the ones to unlock SRCU */
> >                         __srcu_read_unlock(&uretprobes_srcu, hprobe->srcu_idx);
> > -                       return uprobe;
> > +                       break;
> >                 }
> >
> >                 /* We lost the race, undo refcount bump (if it ever happened) */
> > -               if (uprobe)
> > +               if (uprobe && !get) {
> >                         put_uprobe(uprobe);
> > -               /*
> > -                * Even if hprobe_consume() or another hprobe_expire() wins
> > -                * the state update race and unlocks SRCU from under us, we
> > -                * still have a guarantee that underyling uprobe won't be
> > -                * freed due to ongoing caller's SRCU lock region, so we can
> > -                * return it regardless. The caller then can attempt its own
> > -                * try_get_uprobe() to preserve the instance, if necessary.
> > -                * This is used in dup_utask().
> > -                */
> > +                       uprobe = NULL;
> > +               }
> > +
> >                 return uprobe;
> > -       }
> > +
> >         default:
> >                 WARN(1, "unknown hprobe state %d", hstate);
> >                 return NULL;
> >         }
> > +
> > +       if (uprobe && get)
> > +               return try_get_uprobe(uprobe);
> > +
> > +       return uprobe;
> >  }
> >
> >  static __always_inline
> > @@ -1920,9 +1911,8 @@ static void ri_timer(struct timer_list *timer)
> >         /* RCU protects return_instance from freeing. */
> >         guard(rcu)();
> >
> > -       for_each_ret_instance_rcu(ri, utask->return_instances) {
> > -               hprobe_expire(&ri->hprobe);
> > -       }
> > +       for_each_ret_instance_rcu(ri, utask->return_instances)
> > +               hprobe_expire(&ri->hprobe, false);
> >  }
> >
> >  static struct uprobe_task *alloc_utask(void)
> > @@ -1975,10 +1965,7 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
> >
> >                 *n = *o;
> >
> > -               /* see hprobe_expire() comments */
> > -               uprobe = hprobe_expire(&o->hprobe);
> > -               if (uprobe) /* refcount bump for new utask */
> > -                       uprobe = try_get_uprobe(uprobe);
> > +               uprobe = hprobe_expire(&o->hprobe, true);
> >
> >                 /*
> >                  * New utask will have stable properly refcounted uprobe or
> > @@ -1986,7 +1973,7 @@ static int dup_utask(struct task_struct *t, struct uprobe_task *o_utask)
> >                  * need to preserve full set of return_instances for proper
> >                  * uretprobe handling and nesting in forked task.
> >                  */
> > -               hprobe_init_stable(&n->hprobe, uprobe);
> > +               hprobe_init(&n->hprobe, uprobe, -1);
> >
> >                 n->next = NULL;
> >                 rcu_assign_pointer(*p, n);
> > @@ -2131,7 +2118,7 @@ static void prepare_uretprobe(struct uprobe *uprobe, struct pt_regs *regs)
> >
> >         utask->depth++;
> >
> > -       hprobe_init_leased(&ri->hprobe, uprobe, srcu_idx);
> > +       hprobe_init(&ri->hprobe, uprobe, srcu_idx);
> >         ri->next = utask->return_instances;
> >         rcu_assign_pointer(utask->return_instances, ri);
> >

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

* Re: [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context
  2024-10-18 18:22     ` Andrii Nakryiko
@ 2024-10-21 10:31       ` Peter Zijlstra
  2024-10-21 17:04         ` Andrii Nakryiko
  0 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2024-10-21 10:31 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: Andrii Nakryiko, linux-trace-kernel, oleg, rostedt, mhiramat,
	mingo, bpf, linux-kernel, jolsa, paulmck

On Fri, Oct 18, 2024 at 11:22:00AM -0700, Andrii Nakryiko wrote:
> On Fri, Oct 18, 2024 at 1:26 AM Peter Zijlstra <peterz@infradead.org> wrote:
> >
> > On Mon, Oct 07, 2024 at 05:25:55PM -0700, Andrii Nakryiko wrote:
> > > Currently put_uprobe() might trigger mutex_lock()/mutex_unlock(), which
> > > makes it unsuitable to be called from more restricted context like softirq.
> >
> > This is delayed_uprobe_lock, right?
> 
> Not just delated_uprobe_lock, there is also uprobes_treelock (I forgot
> to update the commit message to mention that). Oleg had concerns (see
> [0]) with that being taken from the timer thread, so I just moved all
> of the locking into deferred work callback.
> 
>   [0] https://lore.kernel.org/linux-trace-kernel/20240915144910.GA27726@redhat.com/

Right, but at least that's not a sleeping lock. He's right about it
needing to become a softirq-safe lock though. And yeah, unfortunate
that.

> > So can't we do something like so instead?
> 
> I'll need to look at this more thoroughly (and hopefully Oleg will get
> a chance as well), dropping lock from delayed_ref_ctr_inc() is a bit
> scary, but might be ok.

So I figured that update_ref_ctr() is already doing the
__update_ref_ctr() thing without holding the lock, so that lock really
is only there to manage the list.

And that list is super offensive... That really wants to be a per-mm
rb-tree or somesuch.

AFAICT the only reason it is a mutex, is because doing unbouded list
iteration under a spinlock is a really bad idea.

> But generally speaking, what's your concern with doing deferred work
> in put_uprobe()? It's not a hot path by any means, worst case we'll
> have maybe thousands of uprobes attached/detached.

Mostly I got offended by the level of crap in that code, and working
around crap instead of fixing crap just ain't right.


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

* Re: [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-18 18:22     ` Andrii Nakryiko
  2024-10-19  0:09       ` Paul E. McKenney
@ 2024-10-21 10:48       ` Peter Zijlstra
  2024-10-21 13:57         ` Paul E. McKenney
  2024-10-21 16:53         ` Andrii Nakryiko
  1 sibling, 2 replies; 13+ messages in thread
From: Peter Zijlstra @ 2024-10-21 10:48 UTC (permalink / raw)
  To: Andrii Nakryiko
  Cc: Andrii Nakryiko, linux-trace-kernel, oleg, rostedt, mhiramat,
	mingo, bpf, linux-kernel, jolsa, paulmck

On Fri, Oct 18, 2024 at 11:22:09AM -0700, Andrii Nakryiko wrote:

> > So... after a few readings I think I'm mostly okay with this. But I got
> > annoyed by the whole HPROBE_STABLE with uprobe=NULL weirdness. Also,
> > that data_race() usage is weird, what is that about?
> 
> People keep saying that evil KCSAN will come after me if I don't add
> data_race() for values that can change under me, so I add it to make
> it explicit that it's fine. But I can of course just drop data_race(),
> as it has no bearing on correctness.

AFAICT this was READ_ONCE() vs xchg(), and that should work. Otherwise I
have to yell at KCSAN people again :-)

> > And then there's the case where we end up doing:
> >
> >   try_get_uprobe()
> >   put_uprobe()
> >   try_get_uprobe()
> >
> > in the dup path. Yes, it's unlikely, but gah.
> >
> >
> > So how about something like this?
> 
> Yep, it makes sense to start with HPROBE_GONE if it's already NULL, no
> problem. I'll roll those changes in.
> 
> I'm fine with the `bool get` flag as well. Will incorporate all that
> into the next revision, thanks!
> 
> The only problem I can see is in the assumption that `srcu_idx < 0` is
> never going to be returned by srcu_read_lock(). Paul says that it can
> only be 0 or 1, but it's not codified as part of a contract.

Yeah, [0,1] is the current range. Fundamentally that thing is an array
index, so negative values are out and generally safe to use as 'error'
codes. Paul can't we simply document that the SRCU cookie is always a
positive integer (or zero) and the negative space shall not be used?

> So until we change that, probably safer to pass an extra bool
> specifying whether srcu_idx is valid or not, is that OK?

I think Changeing the SRCU documentation to provide us this guarantee
should be an achievable goal.

> (and I assume you want me to drop verbose comments for various states, right?)

I axed the comments because I made them invalid and didn't care enough
to fix them up. If you like them feel free to amend them to reflect the
new state of things.

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

* Re: [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-21 10:48       ` Peter Zijlstra
@ 2024-10-21 13:57         ` Paul E. McKenney
  2024-10-21 16:53         ` Andrii Nakryiko
  1 sibling, 0 replies; 13+ messages in thread
From: Paul E. McKenney @ 2024-10-21 13:57 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Andrii Nakryiko, Andrii Nakryiko, linux-trace-kernel, oleg,
	rostedt, mhiramat, mingo, bpf, linux-kernel, jolsa

On Mon, Oct 21, 2024 at 12:48:15PM +0200, Peter Zijlstra wrote:
> On Fri, Oct 18, 2024 at 11:22:09AM -0700, Andrii Nakryiko wrote:
> 
> > > So... after a few readings I think I'm mostly okay with this. But I got
> > > annoyed by the whole HPROBE_STABLE with uprobe=NULL weirdness. Also,
> > > that data_race() usage is weird, what is that about?
> > 
> > People keep saying that evil KCSAN will come after me if I don't add
> > data_race() for values that can change under me, so I add it to make
> > it explicit that it's fine. But I can of course just drop data_race(),
> > as it has no bearing on correctness.
> 
> AFAICT this was READ_ONCE() vs xchg(), and that should work. Otherwise I
> have to yell at KCSAN people again :-)
> 
> > > And then there's the case where we end up doing:
> > >
> > >   try_get_uprobe()
> > >   put_uprobe()
> > >   try_get_uprobe()
> > >
> > > in the dup path. Yes, it's unlikely, but gah.
> > >
> > >
> > > So how about something like this?
> > 
> > Yep, it makes sense to start with HPROBE_GONE if it's already NULL, no
> > problem. I'll roll those changes in.
> > 
> > I'm fine with the `bool get` flag as well. Will incorporate all that
> > into the next revision, thanks!
> > 
> > The only problem I can see is in the assumption that `srcu_idx < 0` is
> > never going to be returned by srcu_read_lock(). Paul says that it can
> > only be 0 or 1, but it's not codified as part of a contract.
> 
> Yeah, [0,1] is the current range. Fundamentally that thing is an array
> index, so negative values are out and generally safe to use as 'error'
> codes. Paul can't we simply document that the SRCU cookie is always a
> positive integer (or zero) and the negative space shall not be used?

We are looking at a few approaches, but they all guarantee that the
return value will be non-negative.  My current guess is that we will
just document this non-negative return value, but in all cases, you
should feel free to assume non-negative starting now.

							Thanx, Paul

> > So until we change that, probably safer to pass an extra bool
> > specifying whether srcu_idx is valid or not, is that OK?
> 
> I think Changeing the SRCU documentation to provide us this guarantee
> should be an achievable goal.
> 
> > (and I assume you want me to drop verbose comments for various states, right?)
> 
> I axed the comments because I made them invalid and didn't care enough
> to fix them up. If you like them feel free to amend them to reflect the
> new state of things.

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

* Re: [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout)
  2024-10-21 10:48       ` Peter Zijlstra
  2024-10-21 13:57         ` Paul E. McKenney
@ 2024-10-21 16:53         ` Andrii Nakryiko
  1 sibling, 0 replies; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-21 16:53 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Andrii Nakryiko, linux-trace-kernel, oleg, rostedt, mhiramat,
	mingo, bpf, linux-kernel, jolsa, paulmck

On Mon, Oct 21, 2024 at 3:48 AM Peter Zijlstra <peterz@infradead.org> wrote:
>
> On Fri, Oct 18, 2024 at 11:22:09AM -0700, Andrii Nakryiko wrote:
>
> > > So... after a few readings I think I'm mostly okay with this. But I got
> > > annoyed by the whole HPROBE_STABLE with uprobe=NULL weirdness. Also,
> > > that data_race() usage is weird, what is that about?
> >
> > People keep saying that evil KCSAN will come after me if I don't add
> > data_race() for values that can change under me, so I add it to make
> > it explicit that it's fine. But I can of course just drop data_race(),
> > as it has no bearing on correctness.
>
> AFAICT this was READ_ONCE() vs xchg(), and that should work. Otherwise I
> have to yell at KCSAN people again :-)
>

sounds good, READ_ONCE() it is :)

> > > And then there's the case where we end up doing:
> > >
> > >   try_get_uprobe()
> > >   put_uprobe()
> > >   try_get_uprobe()
> > >
> > > in the dup path. Yes, it's unlikely, but gah.
> > >
> > >
> > > So how about something like this?
> >
> > Yep, it makes sense to start with HPROBE_GONE if it's already NULL, no
> > problem. I'll roll those changes in.
> >
> > I'm fine with the `bool get` flag as well. Will incorporate all that
> > into the next revision, thanks!
> >
> > The only problem I can see is in the assumption that `srcu_idx < 0` is
> > never going to be returned by srcu_read_lock(). Paul says that it can
> > only be 0 or 1, but it's not codified as part of a contract.
>
> Yeah, [0,1] is the current range. Fundamentally that thing is an array
> index, so negative values are out and generally safe to use as 'error'
> codes. Paul can't we simply document that the SRCU cookie is always a
> positive integer (or zero) and the negative space shall not be used?
>
> > So until we change that, probably safer to pass an extra bool
> > specifying whether srcu_idx is valid or not, is that OK?
>
> I think Changeing the SRCU documentation to provide us this guarantee
> should be an achievable goal.

agreed, I'll let Paul handle that, but will assume srcu_idx < 0 can't
legally happen

>
> > (and I assume you want me to drop verbose comments for various states, right?)
>
> I axed the comments because I made them invalid and didn't care enough
> to fix them up. If you like them feel free to amend them to reflect the
> new state of things.

got it, I'll update where necessary

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

* Re: [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context
  2024-10-21 10:31       ` Peter Zijlstra
@ 2024-10-21 17:04         ` Andrii Nakryiko
  0 siblings, 0 replies; 13+ messages in thread
From: Andrii Nakryiko @ 2024-10-21 17:04 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Andrii Nakryiko, linux-trace-kernel, oleg, rostedt, mhiramat,
	mingo, bpf, linux-kernel, jolsa, paulmck

On Mon, Oct 21, 2024 at 3:31 AM Peter Zijlstra <peterz@infradead.org> wrote:
>
> On Fri, Oct 18, 2024 at 11:22:00AM -0700, Andrii Nakryiko wrote:
> > On Fri, Oct 18, 2024 at 1:26 AM Peter Zijlstra <peterz@infradead.org> wrote:
> > >
> > > On Mon, Oct 07, 2024 at 05:25:55PM -0700, Andrii Nakryiko wrote:
> > > > Currently put_uprobe() might trigger mutex_lock()/mutex_unlock(), which
> > > > makes it unsuitable to be called from more restricted context like softirq.
> > >
> > > This is delayed_uprobe_lock, right?
> >
> > Not just delated_uprobe_lock, there is also uprobes_treelock (I forgot
> > to update the commit message to mention that). Oleg had concerns (see
> > [0]) with that being taken from the timer thread, so I just moved all
> > of the locking into deferred work callback.
> >
> >   [0] https://lore.kernel.org/linux-trace-kernel/20240915144910.GA27726@redhat.com/
>
> Right, but at least that's not a sleeping lock. He's right about it
> needing to become a softirq-safe lock though. And yeah, unfortunate
> that.
>
> > > So can't we do something like so instead?
> >
> > I'll need to look at this more thoroughly (and hopefully Oleg will get
> > a chance as well), dropping lock from delayed_ref_ctr_inc() is a bit
> > scary, but might be ok.
>
> So I figured that update_ref_ctr() is already doing the
> __update_ref_ctr() thing without holding the lock, so that lock really
> is only there to manage the list.
>
> And that list is super offensive... That really wants to be a per-mm
> rb-tree or somesuch.

Probably hard to justify to add that to mm_struct, tbh, given that
uprobe+refcnt case (which is USDT with semaphore) isn't all that
frequent, and even then it will be active on a very small subset of
processes in the system, most probably. But, even if (see below),
probably should be a separate change.

>
> AFAICT the only reason it is a mutex, is because doing unbouded list
> iteration under a spinlock is a really bad idea.
>
> > But generally speaking, what's your concern with doing deferred work
> > in put_uprobe()? It's not a hot path by any means, worst case we'll
> > have maybe thousands of uprobes attached/detached.
>
> Mostly I got offended by the level of crap in that code, and working
> around crap instead of fixing crap just ain't right.
>

Ok, so where are we at? Do you insist on the delayed_ref_ctr_inc()
rework, switching uprobe_treelock to be softirq-safe and leaving
put_uprobe() mostly as is? Or is it ok, to do a quick deferred work
change for put_uprobe()  to unblock uretprobe+SRCU and land it sooner?
What if we split this work into two independent patch sets, go with
deferred work for uretprobe + SRCU, and then work with Oleg and you on
simplifying and improving delayed_uprobe_lock-related stuff?

After all, neither deferred work nor delayed_ref_ctr_inc() change has
much practical bearing on real-world performance. WDYT?

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

end of thread, other threads:[~2024-10-21 17:04 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2024-10-08  0:25 [PATCH v2 tip/perf/core 0/2] SRCU-protected uretprobes hot path Andrii Nakryiko
2024-10-08  0:25 ` [PATCH v2 tip/perf/core 1/2] uprobes: allow put_uprobe() from non-sleepable softirq context Andrii Nakryiko
2024-10-18  8:26   ` Peter Zijlstra
2024-10-18 18:22     ` Andrii Nakryiko
2024-10-21 10:31       ` Peter Zijlstra
2024-10-21 17:04         ` Andrii Nakryiko
2024-10-08  0:25 ` [PATCH v2 tip/perf/core 2/2] uprobes: SRCU-protect uretprobe lifetime (with timeout) Andrii Nakryiko
2024-10-18 10:16   ` Peter Zijlstra
2024-10-18 18:22     ` Andrii Nakryiko
2024-10-19  0:09       ` Paul E. McKenney
2024-10-21 10:48       ` Peter Zijlstra
2024-10-21 13:57         ` Paul E. McKenney
2024-10-21 16:53         ` 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).