linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting
@ 2025-07-10 11:00 Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 1/6] selftests/futex: Adapt the private hash test to RCU related changes Sebastian Andrzej Siewior
                   ` (6 more replies)
  0 siblings, 7 replies; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Sebastian Andrzej Siewior

I picked up PeterZ futex patch from
    https://lore.kernel.org/all/20250624190118.GB1490279@noisy.programming.kicks-ass.net/

and I am posting it here it now so it can be staged for v6.17.

This survived a few days on my machine and compile robot reported that
is passes its tests.

v1…v2 https://lore.kernel.org/all/20250707143623.70325-1-bigeasy@linutronix.de
 - Removed the IMMUTABLE bits
 - There was a race if the application exits while the RCU callback is
   pending. Stuffed with mmget()/ mmput_async().

Changes since its initial posting:
- A patch description has been added
- The testuite is "fixed" slightly different and has been split out
- futex_mm_init() is fixed up.
- The guard(preempt) has been replaced with guard(rcu) since there is
  no reason to disable preemption.

Since it was not yet released, should we rip out the IMMUTABLE bits and
just stick with GET/SET slots?

Peter Zijlstra (1):
  futex: Use RCU-based per-CPU reference counting instead of rcuref_t

Sebastian Andrzej Siewior (5):
  selftests/futex: Adapt the private hash test to RCU related changes
  futex: Make futex_private_hash_get() static
  futex: Remove support for IMMUTABLE
  selftests/futex: Remove support for IMMUTABLE
  perf bench futex: Remove support for IMMUTABLE

 include/linux/futex.h                         |  16 +-
 include/linux/mm_types.h                      |   5 +
 include/linux/sched/mm.h                      |   2 +-
 include/uapi/linux/prctl.h                    |   2 -
 init/Kconfig                                  |   4 -
 kernel/fork.c                                 |   8 +-
 kernel/futex/core.c                           | 281 ++++++++++++++----
 kernel/futex/futex.h                          |   2 -
 tools/include/uapi/linux/prctl.h              |   2 -
 tools/perf/bench/futex-hash.c                 |   1 -
 tools/perf/bench/futex-lock-pi.c              |   1 -
 tools/perf/bench/futex-requeue.c              |   1 -
 tools/perf/bench/futex-wake-parallel.c        |   1 -
 tools/perf/bench/futex-wake.c                 |   1 -
 tools/perf/bench/futex.c                      |  21 +-
 tools/perf/bench/futex.h                      |   1 -
 .../trace/beauty/include/uapi/linux/prctl.h   |   2 -
 .../futex/functional/futex_priv_hash.c        | 113 +++----
 18 files changed, 315 insertions(+), 149 deletions(-)

-- 
2.50.0


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

* [PATCH v2 1/6] selftests/futex: Adapt the private hash test to RCU related changes
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
@ 2025-07-10 11:00 ` Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t Sebastian Andrzej Siewior
                   ` (5 subsequent siblings)
  6 siblings, 1 reply; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Sebastian Andrzej Siewior

The auto scaling on create creation used to automatically assign the new
hash because there was the private hash was unused and could be replaced
right away.

This is already racy because if the private hash is in use by a thread
then the visibile resize will be delayed. With the upcoming change to
wait for a RCU grace period before the hash can be assigned, the test
will always fail.

If the reported number of hash buckets is not updated after an
auto scaling event, block on an acquired lock with a timeout. The timeout
is the delay to wait towards a grace period and locking and a locked
pthread_mutex_t ensure that glibc calls into kernel using futex
operation which will assign new private hash if available.
This will retry every 100ms up to 2 seconds in total.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 .../futex/functional/futex_priv_hash.c        | 42 ++++++++++++++++++-
 1 file changed, 41 insertions(+), 1 deletion(-)

diff --git a/tools/testing/selftests/futex/functional/futex_priv_hash.c b/tools/testing/selftests/futex/functional/futex_priv_hash.c
index 24a92dc94eb86..625e3be4129c3 100644
--- a/tools/testing/selftests/futex/functional/futex_priv_hash.c
+++ b/tools/testing/selftests/futex/functional/futex_priv_hash.c
@@ -111,6 +111,30 @@ static void join_max_threads(void)
 	}
 }
 
+#define SEC_IN_NSEC	1000000000
+#define MSEC_IN_NSEC	1000000
+
+static void futex_dummy_op(void)
+{
+	pthread_mutex_t lock = PTHREAD_MUTEX_INITIALIZER;
+	struct timespec timeout;
+	int ret;
+
+	pthread_mutex_lock(&lock);
+	clock_gettime(CLOCK_REALTIME, &timeout);
+	timeout.tv_nsec += 100 * MSEC_IN_NSEC;
+	if (timeout.tv_nsec >=  SEC_IN_NSEC) {
+		timeout.tv_nsec -= SEC_IN_NSEC;
+		timeout.tv_sec++;
+	}
+	ret = pthread_mutex_timedlock(&lock, &timeout);
+	if (ret == 0)
+		ksft_exit_fail_msg("Succeffuly locked an already locked mutex.\n");
+
+	if (ret != ETIMEDOUT)
+		ksft_exit_fail_msg("pthread_mutex_timedlock() did not timeout: %d.\n", ret);
+}
+
 static void usage(char *prog)
 {
 	printf("Usage: %s\n", prog);
@@ -129,7 +153,7 @@ int main(int argc, char *argv[])
 	int futex_slots1, futex_slotsn, online_cpus;
 	pthread_mutexattr_t mutex_attr_pi;
 	int use_global_hash = 0;
-	int ret;
+	int ret, retry = 20;
 	int c;
 
 	while ((c = getopt(argc, argv, "cghv:")) != -1) {
@@ -208,8 +232,24 @@ int main(int argc, char *argv[])
 	 */
 	ksft_print_msg("Online CPUs: %d\n", online_cpus);
 	if (online_cpus > 16) {
+retry_getslots:
 		futex_slotsn = futex_hash_slots_get();
 		if (futex_slotsn < 0 || futex_slots1 == futex_slotsn) {
+			retry--;
+			/*
+			 * Auto scaling on thread creation can be slightly delayed
+			 * because it waits for a RCU grace period twice. The new
+			 * private hash is assigned upon the first futex operation
+			 * after grace period.
+			 * To cover all this for testing purposes the function
+			 * below will acquire a lock and acquire it again with a
+			 * 100ms timeout which must timeout. This ensures we
+			 * sleep for 100ms and issue a futex operation.
+			 */
+			if (retry > 0) {
+				futex_dummy_op();
+				goto retry_getslots;
+			}
 			ksft_print_msg("Expected increase of hash buckets but got: %d -> %d\n",
 				       futex_slots1, futex_slotsn);
 			ksft_exit_fail_msg(test_msg_auto_inc);
-- 
2.50.0


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

* [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 1/6] selftests/futex: Adapt the private hash test to RCU related changes Sebastian Andrzej Siewior
@ 2025-07-10 11:00 ` Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Peter Zijlstra
  2025-07-30 12:20   ` [PATCH v2 2/6] " André Draszik
  2025-07-10 11:00 ` [PATCH v2 3/6] futex: Make futex_private_hash_get() static Sebastian Andrzej Siewior
                   ` (4 subsequent siblings)
  6 siblings, 2 replies; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Andrew Morton, David Hildenbrand, Liam R. Howlett,
	Lorenzo Stoakes, Michal Hocko, Mike Rapoport, Suren Baghdasaryan,
	Vlastimil Babka, linux-mm, Sebastian Andrzej Siewior

From: Peter Zijlstra <peterz@infradead.org>

The use of rcuref_t for reference counting introduces a performance bottleneck
when accessed concurrently by multiple threads during futex operations.

Replace rcuref_t with special crafted per-CPU reference counters. The
lifetime logic remains the same.

The newly allocate private hash starts in FR_PERCPU state. In this state, each
futex operation that requires the private hash uses a per-CPU counter (an
unsigned int) for incrementing or decrementing the reference count.

When the private hash is about to be replaced, the per-CPU counters are
migrated to a atomic_t counter mm_struct::futex_atomic.
The migration process:
- Waiting for one RCU grace period to ensure all users observe the
  current private hash. This can be skipped if a grace period elapsed
  since the private hash was assigned.

- futex_private_hash::state is set to FR_ATOMIC, forcing all users to
  use mm_struct::futex_atomic for reference counting.

- After a RCU grace period, all users are guaranteed to be using the
  atomic counter. The per-CPU counters can now be summed up and added to
  the atomic_t counter. If the resulting count is zero, the hash can be
  safely replaced. Otherwise, active users still hold a valid reference.

- Once the atomic reference count drops to zero, the next futex
  operation will switch to the new private hash.

call_rcu_hurry() is used to speed up transition which otherwise might be
delay with RCU_LAZY. There is nothing wrong with using call_rcu(). The
side effects would be that on auto scaling the new hash is used later
and the SET_SLOTS prctl() will block longer.

[bigeasy: commit description + mm get/ put_async]

Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liam R. Howlett <Liam.Howlett@oracle.com>
Cc: Lorenzo Stoakes <lorenzo.stoakes@oracle.com>
Cc: Michal Hocko <mhocko@suse.com>
Cc: Mike Rapoport <rppt@kernel.org>
Cc: Suren Baghdasaryan <surenb@google.com>
Cc: Vlastimil Babka <vbabka@suse.cz>
Cc: linux-mm@kvack.org
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 include/linux/futex.h    |  16 +--
 include/linux/mm_types.h |   5 +
 include/linux/sched/mm.h |   2 +-
 init/Kconfig             |   4 -
 kernel/fork.c            |   8 +-
 kernel/futex/core.c      | 243 ++++++++++++++++++++++++++++++++++++---
 6 files changed, 243 insertions(+), 35 deletions(-)

diff --git a/include/linux/futex.h b/include/linux/futex.h
index b37193653e6b5..9e9750f049805 100644
--- a/include/linux/futex.h
+++ b/include/linux/futex.h
@@ -85,18 +85,12 @@ int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
 int futex_hash_allocate_default(void);
 void futex_hash_free(struct mm_struct *mm);
-
-static inline void futex_mm_init(struct mm_struct *mm)
-{
-	RCU_INIT_POINTER(mm->futex_phash, NULL);
-	mm->futex_phash_new = NULL;
-	mutex_init(&mm->futex_hash_lock);
-}
+int futex_mm_init(struct mm_struct *mm);
 
 #else /* !CONFIG_FUTEX_PRIVATE_HASH */
 static inline int futex_hash_allocate_default(void) { return 0; }
-static inline void futex_hash_free(struct mm_struct *mm) { }
-static inline void futex_mm_init(struct mm_struct *mm) { }
+static inline int futex_hash_free(struct mm_struct *mm) { return 0; }
+static inline int futex_mm_init(struct mm_struct *mm) { return 0; }
 #endif /* CONFIG_FUTEX_PRIVATE_HASH */
 
 #else /* !CONFIG_FUTEX */
@@ -118,8 +112,8 @@ static inline int futex_hash_allocate_default(void)
 {
 	return 0;
 }
-static inline void futex_hash_free(struct mm_struct *mm) { }
-static inline void futex_mm_init(struct mm_struct *mm) { }
+static inline int futex_hash_free(struct mm_struct *mm) { return 0; }
+static inline int futex_mm_init(struct mm_struct *mm) { return 0; }
 
 #endif
 
diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index d6b91e8a66d6d..0f0662157066a 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -1070,6 +1070,11 @@ struct mm_struct {
 		struct mutex			futex_hash_lock;
 		struct futex_private_hash	__rcu *futex_phash;
 		struct futex_private_hash	*futex_phash_new;
+		/* futex-ref */
+		unsigned long			futex_batches;
+		struct rcu_head			futex_rcu;
+		atomic_long_t			futex_atomic;
+		unsigned int			__percpu *futex_ref;
 #endif
 
 		unsigned long hiwater_rss; /* High-watermark of RSS usage */
diff --git a/include/linux/sched/mm.h b/include/linux/sched/mm.h
index b13474825130f..2201da0afecc5 100644
--- a/include/linux/sched/mm.h
+++ b/include/linux/sched/mm.h
@@ -140,7 +140,7 @@ static inline bool mmget_not_zero(struct mm_struct *mm)
 
 /* mmput gets rid of the mappings and all user-space */
 extern void mmput(struct mm_struct *);
-#ifdef CONFIG_MMU
+#if defined(CONFIG_MMU) || defined(CONFIG_FUTEX_PRIVATE_HASH)
 /* same as above but performs the slow path from the async context. Can
  * be called from the atomic context as well
  */
diff --git a/init/Kconfig b/init/Kconfig
index 666783eb50abd..af4c2f0854554 100644
--- a/init/Kconfig
+++ b/init/Kconfig
@@ -1716,13 +1716,9 @@ config FUTEX_PI
 	depends on FUTEX && RT_MUTEXES
 	default y
 
-#
-# marked broken for performance reasons; gives us one more cycle to sort things out.
-#
 config FUTEX_PRIVATE_HASH
 	bool
 	depends on FUTEX && !BASE_SMALL && MMU
-	depends on BROKEN
 	default y
 
 config FUTEX_MPOL
diff --git a/kernel/fork.c b/kernel/fork.c
index 1ee8eb11f38ba..0b885dcbde9af 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -1046,7 +1046,6 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
 	RCU_INIT_POINTER(mm->exe_file, NULL);
 	mmu_notifier_subscriptions_init(mm);
 	init_tlb_flush_pending(mm);
-	futex_mm_init(mm);
 #if defined(CONFIG_TRANSPARENT_HUGEPAGE) && !defined(CONFIG_SPLIT_PMD_PTLOCKS)
 	mm->pmd_huge_pte = NULL;
 #endif
@@ -1061,6 +1060,9 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
 		mm->def_flags = 0;
 	}
 
+	if (futex_mm_init(mm))
+		goto fail_mm_init;
+
 	if (mm_alloc_pgd(mm))
 		goto fail_nopgd;
 
@@ -1090,6 +1092,8 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
 fail_noid:
 	mm_free_pgd(mm);
 fail_nopgd:
+	futex_hash_free(mm);
+fail_mm_init:
 	free_mm(mm);
 	return NULL;
 }
@@ -1145,7 +1149,7 @@ void mmput(struct mm_struct *mm)
 }
 EXPORT_SYMBOL_GPL(mmput);
 
-#ifdef CONFIG_MMU
+#if defined(CONFIG_MMU) || defined(CONFIG_FUTEX_PRIVATE_HASH)
 static void mmput_async_fn(struct work_struct *work)
 {
 	struct mm_struct *mm = container_of(work, struct mm_struct,
diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index 90d53fb0ee9e1..1dcb4c8a2585d 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -42,7 +42,6 @@
 #include <linux/fault-inject.h>
 #include <linux/slab.h>
 #include <linux/prctl.h>
-#include <linux/rcuref.h>
 #include <linux/mempolicy.h>
 #include <linux/mmap_lock.h>
 
@@ -65,7 +64,7 @@ static struct {
 #define futex_queues	(__futex_data.queues)
 
 struct futex_private_hash {
-	rcuref_t	users;
+	int		state;
 	unsigned int	hash_mask;
 	struct rcu_head	rcu;
 	void		*mm;
@@ -129,6 +128,12 @@ static struct futex_hash_bucket *
 __futex_hash(union futex_key *key, struct futex_private_hash *fph);
 
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
+static bool futex_ref_get(struct futex_private_hash *fph);
+static bool futex_ref_put(struct futex_private_hash *fph);
+static bool futex_ref_is_dead(struct futex_private_hash *fph);
+
+enum { FR_PERCPU = 0, FR_ATOMIC };
+
 static inline bool futex_key_is_private(union futex_key *key)
 {
 	/*
@@ -142,15 +147,14 @@ bool futex_private_hash_get(struct futex_private_hash *fph)
 {
 	if (fph->immutable)
 		return true;
-	return rcuref_get(&fph->users);
+	return futex_ref_get(fph);
 }
 
 void futex_private_hash_put(struct futex_private_hash *fph)
 {
-	/* Ignore return value, last put is verified via rcuref_is_dead() */
 	if (fph->immutable)
 		return;
-	if (rcuref_put(&fph->users))
+	if (futex_ref_put(fph))
 		wake_up_var(fph->mm);
 }
 
@@ -243,14 +247,18 @@ static bool __futex_pivot_hash(struct mm_struct *mm,
 	fph = rcu_dereference_protected(mm->futex_phash,
 					lockdep_is_held(&mm->futex_hash_lock));
 	if (fph) {
-		if (!rcuref_is_dead(&fph->users)) {
+		if (!futex_ref_is_dead(fph)) {
 			mm->futex_phash_new = new;
 			return false;
 		}
 
 		futex_rehash_private(fph, new);
 	}
-	rcu_assign_pointer(mm->futex_phash, new);
+	new->state = FR_PERCPU;
+	scoped_guard(rcu) {
+		mm->futex_batches = get_state_synchronize_rcu();
+		rcu_assign_pointer(mm->futex_phash, new);
+	}
 	kvfree_rcu(fph, rcu);
 	return true;
 }
@@ -289,9 +297,7 @@ struct futex_private_hash *futex_private_hash(void)
 		if (!fph)
 			return NULL;
 
-		if (fph->immutable)
-			return fph;
-		if (rcuref_get(&fph->users))
+		if (futex_private_hash_get(fph))
 			return fph;
 	}
 	futex_pivot_hash(mm);
@@ -1527,16 +1533,219 @@ static void futex_hash_bucket_init(struct futex_hash_bucket *fhb,
 #define FH_IMMUTABLE	0x02
 
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
+
+/*
+ * futex-ref
+ *
+ * Heavily inspired by percpu-rwsem/percpu-refcount; not reusing any of that
+ * code because it just doesn't fit right.
+ *
+ * Dual counter, per-cpu / atomic approach like percpu-refcount, except it
+ * re-initializes the state automatically, such that the fph swizzle is also a
+ * transition back to per-cpu.
+ */
+
+static void futex_ref_rcu(struct rcu_head *head);
+
+static void __futex_ref_atomic_begin(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	/*
+	 * The counter we're about to switch to must have fully switched;
+	 * otherwise it would be impossible for it to have reported success
+	 * from futex_ref_is_dead().
+	 */
+	WARN_ON_ONCE(atomic_long_read(&mm->futex_atomic) != 0);
+
+	/*
+	 * Set the atomic to the bias value such that futex_ref_{get,put}()
+	 * will never observe 0. Will be fixed up in __futex_ref_atomic_end()
+	 * when folding in the percpu count.
+	 */
+	atomic_long_set(&mm->futex_atomic, LONG_MAX);
+	smp_store_release(&fph->state, FR_ATOMIC);
+
+	call_rcu_hurry(&mm->futex_rcu, futex_ref_rcu);
+}
+
+static void __futex_ref_atomic_end(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+	unsigned int count = 0;
+	long ret;
+	int cpu;
+
+	/*
+	 * Per __futex_ref_atomic_begin() the state of the fph must be ATOMIC
+	 * and per this RCU callback, everybody must now observe this state and
+	 * use the atomic variable.
+	 */
+	WARN_ON_ONCE(fph->state != FR_ATOMIC);
+
+	/*
+	 * Therefore the per-cpu counter is now stable, sum and reset.
+	 */
+	for_each_possible_cpu(cpu) {
+		unsigned int *ptr = per_cpu_ptr(mm->futex_ref, cpu);
+		count += *ptr;
+		*ptr = 0;
+	}
+
+	/*
+	 * Re-init for the next cycle.
+	 */
+	this_cpu_inc(*mm->futex_ref); /* 0 -> 1 */
+
+	/*
+	 * Add actual count, subtract bias and initial refcount.
+	 *
+	 * The moment this atomic operation happens, futex_ref_is_dead() can
+	 * become true.
+	 */
+	ret = atomic_long_add_return(count - LONG_MAX - 1, &mm->futex_atomic);
+	if (!ret)
+		wake_up_var(mm);
+
+	WARN_ON_ONCE(ret < 0);
+	mmput_async(mm);
+}
+
+static void futex_ref_rcu(struct rcu_head *head)
+{
+	struct mm_struct *mm = container_of(head, struct mm_struct, futex_rcu);
+	struct futex_private_hash *fph = rcu_dereference_raw(mm->futex_phash);
+
+	if (fph->state == FR_PERCPU) {
+		/*
+		 * Per this extra grace-period, everybody must now observe
+		 * fph as the current fph and no previously observed fph's
+		 * are in-flight.
+		 *
+		 * Notably, nobody will now rely on the atomic
+		 * futex_ref_is_dead() state anymore so we can begin the
+		 * migration of the per-cpu counter into the atomic.
+		 */
+		__futex_ref_atomic_begin(fph);
+		return;
+	}
+
+	__futex_ref_atomic_end(fph);
+}
+
+/*
+ * Drop the initial refcount and transition to atomics.
+ */
+static void futex_ref_drop(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	/*
+	 * Can only transition the current fph;
+	 */
+	WARN_ON_ONCE(rcu_dereference_raw(mm->futex_phash) != fph);
+	/*
+	 * We enqueue at least one RCU callback. Ensure mm stays if the task
+	 * exits before the transition is completed.
+	 */
+	mmget(mm);
+
+	/*
+	 * In order to avoid the following scenario:
+	 *
+	 * futex_hash()			__futex_pivot_hash()
+	 *   guard(rcu);		  guard(mm->futex_hash_lock);
+	 *   fph = mm->futex_phash;
+	 *				  rcu_assign_pointer(&mm->futex_phash, new);
+	 *				futex_hash_allocate()
+	 *				  futex_ref_drop()
+	 *				    fph->state = FR_ATOMIC;
+	 *				    atomic_set(, BIAS);
+	 *
+	 *   futex_private_hash_get(fph); // OOPS
+	 *
+	 * Where an old fph (which is FR_ATOMIC) and should fail on
+	 * inc_not_zero, will succeed because a new transition is started and
+	 * the atomic is bias'ed away from 0.
+	 *
+	 * There must be at least one full grace-period between publishing a
+	 * new fph and trying to replace it.
+	 */
+	if (poll_state_synchronize_rcu(mm->futex_batches)) {
+		/*
+		 * There was a grace-period, we can begin now.
+		 */
+		__futex_ref_atomic_begin(fph);
+		return;
+	}
+
+	call_rcu_hurry(&mm->futex_rcu, futex_ref_rcu);
+}
+
+static bool futex_ref_get(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	guard(rcu)();
+
+	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
+		this_cpu_inc(*mm->futex_ref);
+		return true;
+	}
+
+	return atomic_long_inc_not_zero(&mm->futex_atomic);
+}
+
+static bool futex_ref_put(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	guard(rcu)();
+
+	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
+		this_cpu_dec(*mm->futex_ref);
+		return false;
+	}
+
+	return atomic_long_dec_and_test(&mm->futex_atomic);
+}
+
+static bool futex_ref_is_dead(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	guard(rcu)();
+
+	if (smp_load_acquire(&fph->state) == FR_PERCPU)
+		return false;
+
+	return atomic_long_read(&mm->futex_atomic) == 0;
+}
+
+int futex_mm_init(struct mm_struct *mm)
+{
+	mutex_init(&mm->futex_hash_lock);
+	RCU_INIT_POINTER(mm->futex_phash, NULL);
+	mm->futex_phash_new = NULL;
+	/* futex-ref */
+	atomic_long_set(&mm->futex_atomic, 0);
+	mm->futex_batches = get_state_synchronize_rcu();
+	mm->futex_ref = alloc_percpu(unsigned int);
+	if (!mm->futex_ref)
+		return -ENOMEM;
+	this_cpu_inc(*mm->futex_ref); /* 0 -> 1 */
+	return 0;
+}
+
 void futex_hash_free(struct mm_struct *mm)
 {
 	struct futex_private_hash *fph;
 
+	free_percpu(mm->futex_ref);
 	kvfree(mm->futex_phash_new);
 	fph = rcu_dereference_raw(mm->futex_phash);
-	if (fph) {
-		WARN_ON_ONCE(rcuref_read(&fph->users) > 1);
+	if (fph)
 		kvfree(fph);
-	}
 }
 
 static bool futex_pivot_pending(struct mm_struct *mm)
@@ -1549,7 +1758,7 @@ static bool futex_pivot_pending(struct mm_struct *mm)
 		return true;
 
 	fph = rcu_dereference(mm->futex_phash);
-	return rcuref_is_dead(&fph->users);
+	return futex_ref_is_dead(fph);
 }
 
 static bool futex_hash_less(struct futex_private_hash *a,
@@ -1598,11 +1807,11 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 		}
 	}
 
-	fph = kvzalloc(struct_size(fph, queues, hash_slots), GFP_KERNEL_ACCOUNT | __GFP_NOWARN);
+	fph = kvzalloc(struct_size(fph, queues, hash_slots),
+		       GFP_KERNEL_ACCOUNT | __GFP_NOWARN);
 	if (!fph)
 		return -ENOMEM;
 
-	rcuref_init(&fph->users, 1);
 	fph->hash_mask = hash_slots ? hash_slots - 1 : 0;
 	fph->custom = custom;
 	fph->immutable = !!(flags & FH_IMMUTABLE);
@@ -1645,7 +1854,7 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 				 * allocated a replacement hash, drop the initial
 				 * reference on the existing hash.
 				 */
-				futex_private_hash_put(cur);
+				futex_ref_drop(cur);
 			}
 
 			if (new) {
-- 
2.50.0


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

* [PATCH v2 3/6] futex: Make futex_private_hash_get() static
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 1/6] selftests/futex: Adapt the private hash test to RCU related changes Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t Sebastian Andrzej Siewior
@ 2025-07-10 11:00 ` Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 4/6] futex: Remove support for IMMUTABLE Sebastian Andrzej Siewior
                   ` (3 subsequent siblings)
  6 siblings, 1 reply; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Sebastian Andrzej Siewior

futex_private_hash_get() is not used outside if its compilation unit.
Make it static.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 kernel/futex/core.c  | 2 +-
 kernel/futex/futex.h | 2 --
 2 files changed, 1 insertion(+), 3 deletions(-)

diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index 1dcb4c8a2585d..1981574a459d5 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -143,7 +143,7 @@ static inline bool futex_key_is_private(union futex_key *key)
 	return !(key->both.offset & (FUT_OFF_INODE | FUT_OFF_MMSHARED));
 }
 
-bool futex_private_hash_get(struct futex_private_hash *fph)
+static bool futex_private_hash_get(struct futex_private_hash *fph)
 {
 	if (fph->immutable)
 		return true;
diff --git a/kernel/futex/futex.h b/kernel/futex/futex.h
index fcd1617212eed..c74eac572acd7 100644
--- a/kernel/futex/futex.h
+++ b/kernel/futex/futex.h
@@ -228,14 +228,12 @@ extern void futex_hash_get(struct futex_hash_bucket *hb);
 extern void futex_hash_put(struct futex_hash_bucket *hb);
 
 extern struct futex_private_hash *futex_private_hash(void);
-extern bool futex_private_hash_get(struct futex_private_hash *fph);
 extern void futex_private_hash_put(struct futex_private_hash *fph);
 
 #else /* !CONFIG_FUTEX_PRIVATE_HASH */
 static inline void futex_hash_get(struct futex_hash_bucket *hb) { }
 static inline void futex_hash_put(struct futex_hash_bucket *hb) { }
 static inline struct futex_private_hash *futex_private_hash(void) { return NULL; }
-static inline bool futex_private_hash_get(void) { return false; }
 static inline void futex_private_hash_put(struct futex_private_hash *fph) { }
 #endif
 
-- 
2.50.0


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

* [PATCH v2 4/6] futex: Remove support for IMMUTABLE
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
                   ` (2 preceding siblings ...)
  2025-07-10 11:00 ` [PATCH v2 3/6] futex: Make futex_private_hash_get() static Sebastian Andrzej Siewior
@ 2025-07-10 11:00 ` Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 5/6] selftests/futex: " Sebastian Andrzej Siewior
                   ` (2 subsequent siblings)
  6 siblings, 1 reply; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Sebastian Andrzej Siewior

The FH_FLAG_IMMUTABLE flag was meant to avoid the reference counting on
the private hash and so to avoid the performance regression on big
machines.
With the switch to per-CPU counter this is no longer needed. That flag
was never useable on any released kernel.

Remove any support for IMMUTABLE while preserve the flags argument and
enforce it to be zero.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 include/uapi/linux/prctl.h |  2 --
 kernel/futex/core.c        | 36 +++---------------------------------
 2 files changed, 3 insertions(+), 35 deletions(-)

diff --git a/include/uapi/linux/prctl.h b/include/uapi/linux/prctl.h
index 43dec6eed559a..3b93fb906e3c5 100644
--- a/include/uapi/linux/prctl.h
+++ b/include/uapi/linux/prctl.h
@@ -367,8 +367,6 @@ struct prctl_mm_map {
 /* FUTEX hash management */
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 
 #endif /* _LINUX_PRCTL_H */
diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index 1981574a459d5..d9bb5567af0c5 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -69,7 +69,6 @@ struct futex_private_hash {
 	struct rcu_head	rcu;
 	void		*mm;
 	bool		custom;
-	bool		immutable;
 	struct futex_hash_bucket queues[];
 };
 
@@ -145,15 +144,11 @@ static inline bool futex_key_is_private(union futex_key *key)
 
 static bool futex_private_hash_get(struct futex_private_hash *fph)
 {
-	if (fph->immutable)
-		return true;
 	return futex_ref_get(fph);
 }
 
 void futex_private_hash_put(struct futex_private_hash *fph)
 {
-	if (fph->immutable)
-		return;
 	if (futex_ref_put(fph))
 		wake_up_var(fph->mm);
 }
@@ -1530,7 +1525,6 @@ static void futex_hash_bucket_init(struct futex_hash_bucket *fhb,
 }
 
 #define FH_CUSTOM	0x01
-#define FH_IMMUTABLE	0x02
 
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
 
@@ -1800,7 +1794,7 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 	 */
 	scoped_guard(rcu) {
 		fph = rcu_dereference(mm->futex_phash);
-		if (fph && (!fph->hash_mask || fph->immutable)) {
+		if (fph && !fph->hash_mask) {
 			if (custom)
 				return -EBUSY;
 			return 0;
@@ -1814,7 +1808,6 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 
 	fph->hash_mask = hash_slots ? hash_slots - 1 : 0;
 	fph->custom = custom;
-	fph->immutable = !!(flags & FH_IMMUTABLE);
 	fph->mm = mm;
 
 	for (i = 0; i < hash_slots; i++)
@@ -1838,7 +1831,7 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 		mm->futex_phash_new = NULL;
 
 		if (fph) {
-			if (cur && (!cur->hash_mask || cur->immutable)) {
+			if (cur && !cur->hash_mask) {
 				/*
 				 * If two threads simultaneously request the global
 				 * hash then the first one performs the switch,
@@ -1931,19 +1924,6 @@ static int futex_hash_get_slots(void)
 	return 0;
 }
 
-static int futex_hash_get_immutable(void)
-{
-	struct futex_private_hash *fph;
-
-	guard(rcu)();
-	fph = rcu_dereference(current->mm->futex_phash);
-	if (fph && fph->immutable)
-		return 1;
-	if (fph && !fph->hash_mask)
-		return 1;
-	return 0;
-}
-
 #else
 
 static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
@@ -1956,10 +1936,6 @@ static int futex_hash_get_slots(void)
 	return 0;
 }
 
-static int futex_hash_get_immutable(void)
-{
-	return 0;
-}
 #endif
 
 int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
@@ -1969,10 +1945,8 @@ int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
 
 	switch (arg2) {
 	case PR_FUTEX_HASH_SET_SLOTS:
-		if (arg4 & ~FH_FLAG_IMMUTABLE)
+		if (arg4)
 			return -EINVAL;
-		if (arg4 & FH_FLAG_IMMUTABLE)
-			flags |= FH_IMMUTABLE;
 		ret = futex_hash_allocate(arg3, flags);
 		break;
 
@@ -1980,10 +1954,6 @@ int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
 		ret = futex_hash_get_slots();
 		break;
 
-	case PR_FUTEX_HASH_GET_IMMUTABLE:
-		ret = futex_hash_get_immutable();
-		break;
-
 	default:
 		ret = -EINVAL;
 		break;
-- 
2.50.0


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

* [PATCH v2 5/6] selftests/futex: Remove support for IMMUTABLE
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
                   ` (3 preceding siblings ...)
  2025-07-10 11:00 ` [PATCH v2 4/6] futex: Remove support for IMMUTABLE Sebastian Andrzej Siewior
@ 2025-07-10 11:00 ` Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
  2025-07-10 11:00 ` [PATCH v2 6/6] perf bench futex: " Sebastian Andrzej Siewior
  2025-07-15 15:59 ` [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Shrikanth Hegde
  6 siblings, 1 reply; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Sebastian Andrzej Siewior

Testing for the IMMUTABLE part of the futex interface is not needed
after the removal of the interface.

Remove support for IMMUTABLE from the sefltest.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 .../futex/functional/futex_priv_hash.c        | 71 ++++++-------------
 1 file changed, 22 insertions(+), 49 deletions(-)

diff --git a/tools/testing/selftests/futex/functional/futex_priv_hash.c b/tools/testing/selftests/futex/functional/futex_priv_hash.c
index 625e3be4129c3..a9cedc365102e 100644
--- a/tools/testing/selftests/futex/functional/futex_priv_hash.c
+++ b/tools/testing/selftests/futex/functional/futex_priv_hash.c
@@ -26,14 +26,12 @@ static int counter;
 #ifndef PR_FUTEX_HASH
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 #endif
 
-static int futex_hash_slots_set(unsigned int slots, int flags)
+static int futex_hash_slots_set(unsigned int slots)
 {
-	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, slots, flags);
+	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, slots, 0);
 }
 
 static int futex_hash_slots_get(void)
@@ -41,16 +39,11 @@ static int futex_hash_slots_get(void)
 	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_GET_SLOTS);
 }
 
-static int futex_hash_immutable_get(void)
-{
-	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_GET_IMMUTABLE);
-}
-
 static void futex_hash_slots_set_verify(int slots)
 {
 	int ret;
 
-	ret = futex_hash_slots_set(slots, 0);
+	ret = futex_hash_slots_set(slots);
 	if (ret != 0) {
 		ksft_test_result_fail("Failed to set slots to %d: %m\n", slots);
 		ksft_finished();
@@ -64,13 +57,13 @@ static void futex_hash_slots_set_verify(int slots)
 	ksft_test_result_pass("SET and GET slots %d passed\n", slots);
 }
 
-static void futex_hash_slots_set_must_fail(int slots, int flags)
+static void futex_hash_slots_set_must_fail(int slots)
 {
 	int ret;
 
-	ret = futex_hash_slots_set(slots, flags);
-	ksft_test_result(ret < 0, "futex_hash_slots_set(%d, %d)\n",
-			 slots, flags);
+	ret = futex_hash_slots_set(slots);
+	ksft_test_result(ret < 0, "futex_hash_slots_set(%d)\n",
+			 slots);
 }
 
 static void *thread_return_fn(void *arg)
@@ -152,18 +145,14 @@ int main(int argc, char *argv[])
 {
 	int futex_slots1, futex_slotsn, online_cpus;
 	pthread_mutexattr_t mutex_attr_pi;
-	int use_global_hash = 0;
 	int ret, retry = 20;
 	int c;
 
-	while ((c = getopt(argc, argv, "cghv:")) != -1) {
+	while ((c = getopt(argc, argv, "chv:")) != -1) {
 		switch (c) {
 		case 'c':
 			log_color(1);
 			break;
-		case 'g':
-			use_global_hash = 1;
-			break;
 		case 'h':
 			usage(basename(argv[0]));
 			exit(0);
@@ -178,7 +167,7 @@ int main(int argc, char *argv[])
 	}
 
 	ksft_print_header();
-	ksft_set_plan(22);
+	ksft_set_plan(21);
 
 	ret = pthread_mutexattr_init(&mutex_attr_pi);
 	ret |= pthread_mutexattr_setprotocol(&mutex_attr_pi, PTHREAD_PRIO_INHERIT);
@@ -191,10 +180,6 @@ int main(int argc, char *argv[])
 	if (ret != 0)
 		ksft_exit_fail_msg("futex_hash_slots_get() failed: %d, %m\n", ret);
 
-	ret = futex_hash_immutable_get();
-	if (ret != 0)
-		ksft_exit_fail_msg("futex_hash_immutable_get() failed: %d, %m\n", ret);
-
 	ksft_test_result_pass("Basic get slots and immutable status.\n");
 	ret = pthread_create(&threads[0], NULL, thread_return_fn, NULL);
 	if (ret != 0)
@@ -267,7 +252,7 @@ int main(int argc, char *argv[])
 	futex_hash_slots_set_verify(32);
 	futex_hash_slots_set_verify(16);
 
-	ret = futex_hash_slots_set(15, 0);
+	ret = futex_hash_slots_set(15);
 	ksft_test_result(ret < 0, "Use 15 slots\n");
 
 	futex_hash_slots_set_verify(2);
@@ -285,28 +270,23 @@ int main(int argc, char *argv[])
 	ksft_test_result(ret == 2, "No more auto-resize after manaul setting, got %d\n",
 			 ret);
 
-	futex_hash_slots_set_must_fail(1 << 29, 0);
+	futex_hash_slots_set_must_fail(1 << 29);
+	futex_hash_slots_set_verify(4);
 
 	/*
-	 * Once the private hash has been made immutable or global hash has been requested,
-	 * then this requested can not be undone.
+	 * Once the global hash has been requested, then this requested can not
+	 * be undone.
 	 */
-	if (use_global_hash) {
-		ret = futex_hash_slots_set(0, 0);
-		ksft_test_result(ret == 0, "Global hash request\n");
-	} else {
-		ret = futex_hash_slots_set(4, FH_FLAG_IMMUTABLE);
-		ksft_test_result(ret == 0, "Immutable resize to 4\n");
-	}
+	ret = futex_hash_slots_set(0);
+	ksft_test_result(ret == 0, "Global hash request\n");
 	if (ret != 0)
 		goto out;
 
-	futex_hash_slots_set_must_fail(4, 0);
-	futex_hash_slots_set_must_fail(4, FH_FLAG_IMMUTABLE);
-	futex_hash_slots_set_must_fail(8, 0);
-	futex_hash_slots_set_must_fail(8, FH_FLAG_IMMUTABLE);
-	futex_hash_slots_set_must_fail(0, FH_FLAG_IMMUTABLE);
-	futex_hash_slots_set_must_fail(6, FH_FLAG_IMMUTABLE);
+	futex_hash_slots_set_must_fail(4);
+	futex_hash_slots_set_must_fail(8);
+	futex_hash_slots_set_must_fail(8);
+	futex_hash_slots_set_must_fail(0);
+	futex_hash_slots_set_must_fail(6);
 
 	ret = pthread_barrier_init(&barrier_main, NULL, MAX_THREADS);
 	if (ret != 0) {
@@ -317,14 +297,7 @@ int main(int argc, char *argv[])
 	join_max_threads();
 
 	ret = futex_hash_slots_get();
-	if (use_global_hash) {
-		ksft_test_result(ret == 0, "Continue to use global hash\n");
-	} else {
-		ksft_test_result(ret == 4, "Continue to use the 4 hash buckets\n");
-	}
-
-	ret = futex_hash_immutable_get();
-	ksft_test_result(ret == 1, "Hash reports to be immutable\n");
+	ksft_test_result(ret == 0, "Continue to use global hash\n");
 
 out:
 	ksft_finished();
-- 
2.50.0


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

* [PATCH v2 6/6] perf bench futex: Remove support for IMMUTABLE
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
                   ` (4 preceding siblings ...)
  2025-07-10 11:00 ` [PATCH v2 5/6] selftests/futex: " Sebastian Andrzej Siewior
@ 2025-07-10 11:00 ` Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
  2025-07-15 15:59 ` [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Shrikanth Hegde
  6 siblings, 1 reply; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-10 11:00 UTC (permalink / raw)
  To: linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Sebastian Andrzej Siewior, Liang, Kan, Adrian Hunter,
	Alexander Shishkin, Arnaldo Carvalho de Melo, Ian Rogers,
	Jiri Olsa, Mark Rutland, Namhyung Kim, linux-perf-users

It has been decided to remove the support IMMUTABLE futex.
perf bench was one of the eary users for testing purposes. Now that the
API is removed before it could be used in an official release, remove
the bits from perf, too.

Remove Remove support for IMMUTABLE futex.

Cc: "Liang, Kan" <kan.liang@linux.intel.com>
Cc: Adrian Hunter <adrian.hunter@intel.com>
Cc: Alexander Shishkin <alexander.shishkin@linux.intel.com>
Cc: Arnaldo Carvalho de Melo <acme@kernel.org>
Cc: Ian Rogers <irogers@google.com>
Cc: Jiri Olsa <jolsa@kernel.org>
Cc: Mark Rutland <mark.rutland@arm.com>
Cc: Namhyung Kim <namhyung@kernel.org>
Cc: linux-perf-users@vger.kernel.org
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
---
 tools/include/uapi/linux/prctl.h              |  2 --
 tools/perf/bench/futex-hash.c                 |  1 -
 tools/perf/bench/futex-lock-pi.c              |  1 -
 tools/perf/bench/futex-requeue.c              |  1 -
 tools/perf/bench/futex-wake-parallel.c        |  1 -
 tools/perf/bench/futex-wake.c                 |  1 -
 tools/perf/bench/futex.c                      | 21 +++++--------------
 tools/perf/bench/futex.h                      |  1 -
 .../trace/beauty/include/uapi/linux/prctl.h   |  2 --
 9 files changed, 5 insertions(+), 26 deletions(-)

diff --git a/tools/include/uapi/linux/prctl.h b/tools/include/uapi/linux/prctl.h
index 43dec6eed559a..3b93fb906e3c5 100644
--- a/tools/include/uapi/linux/prctl.h
+++ b/tools/include/uapi/linux/prctl.h
@@ -367,8 +367,6 @@ struct prctl_mm_map {
 /* FUTEX hash management */
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 
 #endif /* _LINUX_PRCTL_H */
diff --git a/tools/perf/bench/futex-hash.c b/tools/perf/bench/futex-hash.c
index d2d6d7f3ea331..7e29f04da7449 100644
--- a/tools/perf/bench/futex-hash.c
+++ b/tools/perf/bench/futex-hash.c
@@ -56,7 +56,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('r', "runtime", &params.runtime, "Specify runtime (in seconds)"),
 	OPT_UINTEGER('f', "futexes", &params.nfutexes, "Specify amount of futexes per threads"),
diff --git a/tools/perf/bench/futex-lock-pi.c b/tools/perf/bench/futex-lock-pi.c
index 5144a158512cc..40640b6744279 100644
--- a/tools/perf/bench/futex-lock-pi.c
+++ b/tools/perf/bench/futex-lock-pi.c
@@ -47,7 +47,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('r', "runtime", &params.runtime, "Specify runtime (in seconds)"),
 	OPT_BOOLEAN( 'M', "multi",   &params.multi, "Use multiple futexes"),
diff --git a/tools/perf/bench/futex-requeue.c b/tools/perf/bench/futex-requeue.c
index a2f91ee1950b3..0748b0fd689e8 100644
--- a/tools/perf/bench/futex-requeue.c
+++ b/tools/perf/bench/futex-requeue.c
@@ -52,7 +52,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads",  &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('q', "nrequeue", &params.nrequeue, "Specify amount of threads to requeue at once"),
 	OPT_BOOLEAN( 's', "silent",   &params.silent, "Silent mode: do not display data/details"),
diff --git a/tools/perf/bench/futex-wake-parallel.c b/tools/perf/bench/futex-wake-parallel.c
index ee66482c29fd1..6aede7c46b337 100644
--- a/tools/perf/bench/futex-wake-parallel.c
+++ b/tools/perf/bench/futex-wake-parallel.c
@@ -63,7 +63,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('w', "nwakers", &params.nwakes, "Specify amount of waking threads"),
 	OPT_BOOLEAN( 's', "silent",  &params.silent, "Silent mode: do not display data/details"),
diff --git a/tools/perf/bench/futex-wake.c b/tools/perf/bench/futex-wake.c
index 8d6107f7cd941..a31fc1563862e 100644
--- a/tools/perf/bench/futex-wake.c
+++ b/tools/perf/bench/futex-wake.c
@@ -52,7 +52,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('w', "nwakes",  &params.nwakes, "Specify amount of threads to wake at once"),
 	OPT_BOOLEAN( 's', "silent",  &params.silent, "Silent mode: do not display data/details"),
diff --git a/tools/perf/bench/futex.c b/tools/perf/bench/futex.c
index 4c4fee107e591..1481196a22f0c 100644
--- a/tools/perf/bench/futex.c
+++ b/tools/perf/bench/futex.c
@@ -9,21 +9,17 @@
 #ifndef PR_FUTEX_HASH
 #define PR_FUTEX_HASH                   78
 # define PR_FUTEX_HASH_SET_SLOTS        1
-# define FH_FLAG_IMMUTABLE              (1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS        2
-# define PR_FUTEX_HASH_GET_IMMUTABLE    3
 #endif // PR_FUTEX_HASH
 
 void futex_set_nbuckets_param(struct bench_futex_parameters *params)
 {
-	unsigned long flags;
 	int ret;
 
 	if (params->nbuckets < 0)
 		return;
 
-	flags = params->buckets_immutable ? FH_FLAG_IMMUTABLE : 0;
-	ret = prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, params->nbuckets, flags);
+	ret = prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, params->nbuckets, 0);
 	if (ret) {
 		printf("Requesting %d hash buckets failed: %d/%m\n",
 		       params->nbuckets, ret);
@@ -47,18 +43,11 @@ void futex_print_nbuckets(struct bench_futex_parameters *params)
 			printf("Requested: %d in usage: %d\n", params->nbuckets, ret);
 			err(EXIT_FAILURE, "prctl(PR_FUTEX_HASH)");
 		}
-		if (params->nbuckets == 0) {
+		if (params->nbuckets == 0)
 			ret = asprintf(&futex_hash_mode, "Futex hashing: global hash");
-		} else {
-			ret = prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_GET_IMMUTABLE);
-			if (ret < 0) {
-				printf("Can't check if the hash is immutable: %m\n");
-				err(EXIT_FAILURE, "prctl(PR_FUTEX_HASH)");
-			}
-			ret = asprintf(&futex_hash_mode, "Futex hashing: %d hash buckets %s",
-				       params->nbuckets,
-				       ret == 1 ? "(immutable)" : "");
-		}
+		else
+			ret = asprintf(&futex_hash_mode, "Futex hashing: %d hash buckets",
+				       params->nbuckets);
 	} else {
 		if (ret <= 0) {
 			ret = asprintf(&futex_hash_mode, "Futex hashing: global hash");
diff --git a/tools/perf/bench/futex.h b/tools/perf/bench/futex.h
index 9c9a73f9d865e..dd295d27044ac 100644
--- a/tools/perf/bench/futex.h
+++ b/tools/perf/bench/futex.h
@@ -26,7 +26,6 @@ struct bench_futex_parameters {
 	unsigned int nwakes;
 	unsigned int nrequeue;
 	int nbuckets;
-	bool buckets_immutable;
 };
 
 /**
diff --git a/tools/perf/trace/beauty/include/uapi/linux/prctl.h b/tools/perf/trace/beauty/include/uapi/linux/prctl.h
index 43dec6eed559a..3b93fb906e3c5 100644
--- a/tools/perf/trace/beauty/include/uapi/linux/prctl.h
+++ b/tools/perf/trace/beauty/include/uapi/linux/prctl.h
@@ -367,8 +367,6 @@ struct prctl_mm_map {
 /* FUTEX hash management */
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 
 #endif /* _LINUX_PRCTL_H */
-- 
2.50.0


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

* [tip: locking/futex] perf bench futex: Remove support for IMMUTABLE
  2025-07-10 11:00 ` [PATCH v2 6/6] perf bench futex: " Sebastian Andrzej Siewior
@ 2025-07-11 18:33   ` tip-bot2 for Sebastian Andrzej Siewior
  0 siblings, 0 replies; 25+ messages in thread
From: tip-bot2 for Sebastian Andrzej Siewior @ 2025-07-11 18:33 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: Sebastian Andrzej Siewior, Peter Zijlstra (Intel), x86,
	linux-kernel

The following commit has been merged into the locking/futex branch of tip:

Commit-ID:     7497e947bc1d3f761b46c2105c8ae37af98add54
Gitweb:        https://git.kernel.org/tip/7497e947bc1d3f761b46c2105c8ae37af98add54
Author:        Sebastian Andrzej Siewior <bigeasy@linutronix.de>
AuthorDate:    Thu, 10 Jul 2025 13:00:11 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Fri, 11 Jul 2025 16:02:01 +02:00

perf bench futex: Remove support for IMMUTABLE

It has been decided to remove the support IMMUTABLE futex.
perf bench was one of the eary users for testing purposes. Now that the
API is removed before it could be used in an official release, remove
the bits from perf, too.

Remove Remove support for IMMUTABLE futex.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lore.kernel.org/r/20250710110011.384614-7-bigeasy@linutronix.de
---
 tools/include/uapi/linux/prctl.h                   |  2 +-
 tools/perf/bench/futex-hash.c                      |  1 +-
 tools/perf/bench/futex-lock-pi.c                   |  1 +-
 tools/perf/bench/futex-requeue.c                   |  1 +-
 tools/perf/bench/futex-wake-parallel.c             |  1 +-
 tools/perf/bench/futex-wake.c                      |  1 +-
 tools/perf/bench/futex.c                           | 21 +++----------
 tools/perf/bench/futex.h                           |  1 +-
 tools/perf/trace/beauty/include/uapi/linux/prctl.h |  2 +-
 9 files changed, 5 insertions(+), 26 deletions(-)

diff --git a/tools/include/uapi/linux/prctl.h b/tools/include/uapi/linux/prctl.h
index 43dec6e..3b93fb9 100644
--- a/tools/include/uapi/linux/prctl.h
+++ b/tools/include/uapi/linux/prctl.h
@@ -367,8 +367,6 @@ struct prctl_mm_map {
 /* FUTEX hash management */
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 
 #endif /* _LINUX_PRCTL_H */
diff --git a/tools/perf/bench/futex-hash.c b/tools/perf/bench/futex-hash.c
index d2d6d7f..7e29f04 100644
--- a/tools/perf/bench/futex-hash.c
+++ b/tools/perf/bench/futex-hash.c
@@ -56,7 +56,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('r', "runtime", &params.runtime, "Specify runtime (in seconds)"),
 	OPT_UINTEGER('f', "futexes", &params.nfutexes, "Specify amount of futexes per threads"),
diff --git a/tools/perf/bench/futex-lock-pi.c b/tools/perf/bench/futex-lock-pi.c
index 5144a15..40640b6 100644
--- a/tools/perf/bench/futex-lock-pi.c
+++ b/tools/perf/bench/futex-lock-pi.c
@@ -47,7 +47,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('r', "runtime", &params.runtime, "Specify runtime (in seconds)"),
 	OPT_BOOLEAN( 'M', "multi",   &params.multi, "Use multiple futexes"),
diff --git a/tools/perf/bench/futex-requeue.c b/tools/perf/bench/futex-requeue.c
index a2f91ee..0748b0f 100644
--- a/tools/perf/bench/futex-requeue.c
+++ b/tools/perf/bench/futex-requeue.c
@@ -52,7 +52,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads",  &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('q', "nrequeue", &params.nrequeue, "Specify amount of threads to requeue at once"),
 	OPT_BOOLEAN( 's', "silent",   &params.silent, "Silent mode: do not display data/details"),
diff --git a/tools/perf/bench/futex-wake-parallel.c b/tools/perf/bench/futex-wake-parallel.c
index ee66482..6aede7c 100644
--- a/tools/perf/bench/futex-wake-parallel.c
+++ b/tools/perf/bench/futex-wake-parallel.c
@@ -63,7 +63,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('w', "nwakers", &params.nwakes, "Specify amount of waking threads"),
 	OPT_BOOLEAN( 's', "silent",  &params.silent, "Silent mode: do not display data/details"),
diff --git a/tools/perf/bench/futex-wake.c b/tools/perf/bench/futex-wake.c
index 8d6107f..a31fc15 100644
--- a/tools/perf/bench/futex-wake.c
+++ b/tools/perf/bench/futex-wake.c
@@ -52,7 +52,6 @@ static struct bench_futex_parameters params = {
 
 static const struct option options[] = {
 	OPT_INTEGER( 'b', "buckets", &params.nbuckets, "Specify amount of hash buckets"),
-	OPT_BOOLEAN( 'I', "immutable", &params.buckets_immutable, "Make the hash buckets immutable"),
 	OPT_UINTEGER('t', "threads", &params.nthreads, "Specify amount of threads"),
 	OPT_UINTEGER('w', "nwakes",  &params.nwakes, "Specify amount of threads to wake at once"),
 	OPT_BOOLEAN( 's', "silent",  &params.silent, "Silent mode: do not display data/details"),
diff --git a/tools/perf/bench/futex.c b/tools/perf/bench/futex.c
index 4c4fee1..1481196 100644
--- a/tools/perf/bench/futex.c
+++ b/tools/perf/bench/futex.c
@@ -9,21 +9,17 @@
 #ifndef PR_FUTEX_HASH
 #define PR_FUTEX_HASH                   78
 # define PR_FUTEX_HASH_SET_SLOTS        1
-# define FH_FLAG_IMMUTABLE              (1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS        2
-# define PR_FUTEX_HASH_GET_IMMUTABLE    3
 #endif // PR_FUTEX_HASH
 
 void futex_set_nbuckets_param(struct bench_futex_parameters *params)
 {
-	unsigned long flags;
 	int ret;
 
 	if (params->nbuckets < 0)
 		return;
 
-	flags = params->buckets_immutable ? FH_FLAG_IMMUTABLE : 0;
-	ret = prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, params->nbuckets, flags);
+	ret = prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, params->nbuckets, 0);
 	if (ret) {
 		printf("Requesting %d hash buckets failed: %d/%m\n",
 		       params->nbuckets, ret);
@@ -47,18 +43,11 @@ void futex_print_nbuckets(struct bench_futex_parameters *params)
 			printf("Requested: %d in usage: %d\n", params->nbuckets, ret);
 			err(EXIT_FAILURE, "prctl(PR_FUTEX_HASH)");
 		}
-		if (params->nbuckets == 0) {
+		if (params->nbuckets == 0)
 			ret = asprintf(&futex_hash_mode, "Futex hashing: global hash");
-		} else {
-			ret = prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_GET_IMMUTABLE);
-			if (ret < 0) {
-				printf("Can't check if the hash is immutable: %m\n");
-				err(EXIT_FAILURE, "prctl(PR_FUTEX_HASH)");
-			}
-			ret = asprintf(&futex_hash_mode, "Futex hashing: %d hash buckets %s",
-				       params->nbuckets,
-				       ret == 1 ? "(immutable)" : "");
-		}
+		else
+			ret = asprintf(&futex_hash_mode, "Futex hashing: %d hash buckets",
+				       params->nbuckets);
 	} else {
 		if (ret <= 0) {
 			ret = asprintf(&futex_hash_mode, "Futex hashing: global hash");
diff --git a/tools/perf/bench/futex.h b/tools/perf/bench/futex.h
index 9c9a73f..dd295d2 100644
--- a/tools/perf/bench/futex.h
+++ b/tools/perf/bench/futex.h
@@ -26,7 +26,6 @@ struct bench_futex_parameters {
 	unsigned int nwakes;
 	unsigned int nrequeue;
 	int nbuckets;
-	bool buckets_immutable;
 };
 
 /**
diff --git a/tools/perf/trace/beauty/include/uapi/linux/prctl.h b/tools/perf/trace/beauty/include/uapi/linux/prctl.h
index 43dec6e..3b93fb9 100644
--- a/tools/perf/trace/beauty/include/uapi/linux/prctl.h
+++ b/tools/perf/trace/beauty/include/uapi/linux/prctl.h
@@ -367,8 +367,6 @@ struct prctl_mm_map {
 /* FUTEX hash management */
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 
 #endif /* _LINUX_PRCTL_H */

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

* [tip: locking/futex] selftests/futex: Remove support for IMMUTABLE
  2025-07-10 11:00 ` [PATCH v2 5/6] selftests/futex: " Sebastian Andrzej Siewior
@ 2025-07-11 18:33   ` tip-bot2 for Sebastian Andrzej Siewior
  0 siblings, 0 replies; 25+ messages in thread
From: tip-bot2 for Sebastian Andrzej Siewior @ 2025-07-11 18:33 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: Sebastian Andrzej Siewior, Peter Zijlstra (Intel), x86,
	linux-kernel

The following commit has been merged into the locking/futex branch of tip:

Commit-ID:     16adc7f136dc143fdaa0d465172d7a1e6d5ae3c5
Gitweb:        https://git.kernel.org/tip/16adc7f136dc143fdaa0d465172d7a1e6d5ae3c5
Author:        Sebastian Andrzej Siewior <bigeasy@linutronix.de>
AuthorDate:    Thu, 10 Jul 2025 13:00:10 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Fri, 11 Jul 2025 16:02:01 +02:00

selftests/futex: Remove support for IMMUTABLE

Testing for the IMMUTABLE part of the futex interface is not needed
after the removal of the interface.

Remove support for IMMUTABLE from the sefltest.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lore.kernel.org/r/20250710110011.384614-6-bigeasy@linutronix.de
---
 tools/testing/selftests/futex/functional/futex_priv_hash.c | 71 ++-----
 1 file changed, 22 insertions(+), 49 deletions(-)

diff --git a/tools/testing/selftests/futex/functional/futex_priv_hash.c b/tools/testing/selftests/futex/functional/futex_priv_hash.c
index 625e3be..a9cedc3 100644
--- a/tools/testing/selftests/futex/functional/futex_priv_hash.c
+++ b/tools/testing/selftests/futex/functional/futex_priv_hash.c
@@ -26,14 +26,12 @@ static int counter;
 #ifndef PR_FUTEX_HASH
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 #endif
 
-static int futex_hash_slots_set(unsigned int slots, int flags)
+static int futex_hash_slots_set(unsigned int slots)
 {
-	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, slots, flags);
+	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_SET_SLOTS, slots, 0);
 }
 
 static int futex_hash_slots_get(void)
@@ -41,16 +39,11 @@ static int futex_hash_slots_get(void)
 	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_GET_SLOTS);
 }
 
-static int futex_hash_immutable_get(void)
-{
-	return prctl(PR_FUTEX_HASH, PR_FUTEX_HASH_GET_IMMUTABLE);
-}
-
 static void futex_hash_slots_set_verify(int slots)
 {
 	int ret;
 
-	ret = futex_hash_slots_set(slots, 0);
+	ret = futex_hash_slots_set(slots);
 	if (ret != 0) {
 		ksft_test_result_fail("Failed to set slots to %d: %m\n", slots);
 		ksft_finished();
@@ -64,13 +57,13 @@ static void futex_hash_slots_set_verify(int slots)
 	ksft_test_result_pass("SET and GET slots %d passed\n", slots);
 }
 
-static void futex_hash_slots_set_must_fail(int slots, int flags)
+static void futex_hash_slots_set_must_fail(int slots)
 {
 	int ret;
 
-	ret = futex_hash_slots_set(slots, flags);
-	ksft_test_result(ret < 0, "futex_hash_slots_set(%d, %d)\n",
-			 slots, flags);
+	ret = futex_hash_slots_set(slots);
+	ksft_test_result(ret < 0, "futex_hash_slots_set(%d)\n",
+			 slots);
 }
 
 static void *thread_return_fn(void *arg)
@@ -152,18 +145,14 @@ int main(int argc, char *argv[])
 {
 	int futex_slots1, futex_slotsn, online_cpus;
 	pthread_mutexattr_t mutex_attr_pi;
-	int use_global_hash = 0;
 	int ret, retry = 20;
 	int c;
 
-	while ((c = getopt(argc, argv, "cghv:")) != -1) {
+	while ((c = getopt(argc, argv, "chv:")) != -1) {
 		switch (c) {
 		case 'c':
 			log_color(1);
 			break;
-		case 'g':
-			use_global_hash = 1;
-			break;
 		case 'h':
 			usage(basename(argv[0]));
 			exit(0);
@@ -178,7 +167,7 @@ int main(int argc, char *argv[])
 	}
 
 	ksft_print_header();
-	ksft_set_plan(22);
+	ksft_set_plan(21);
 
 	ret = pthread_mutexattr_init(&mutex_attr_pi);
 	ret |= pthread_mutexattr_setprotocol(&mutex_attr_pi, PTHREAD_PRIO_INHERIT);
@@ -191,10 +180,6 @@ int main(int argc, char *argv[])
 	if (ret != 0)
 		ksft_exit_fail_msg("futex_hash_slots_get() failed: %d, %m\n", ret);
 
-	ret = futex_hash_immutable_get();
-	if (ret != 0)
-		ksft_exit_fail_msg("futex_hash_immutable_get() failed: %d, %m\n", ret);
-
 	ksft_test_result_pass("Basic get slots and immutable status.\n");
 	ret = pthread_create(&threads[0], NULL, thread_return_fn, NULL);
 	if (ret != 0)
@@ -267,7 +252,7 @@ retry_getslots:
 	futex_hash_slots_set_verify(32);
 	futex_hash_slots_set_verify(16);
 
-	ret = futex_hash_slots_set(15, 0);
+	ret = futex_hash_slots_set(15);
 	ksft_test_result(ret < 0, "Use 15 slots\n");
 
 	futex_hash_slots_set_verify(2);
@@ -285,28 +270,23 @@ retry_getslots:
 	ksft_test_result(ret == 2, "No more auto-resize after manaul setting, got %d\n",
 			 ret);
 
-	futex_hash_slots_set_must_fail(1 << 29, 0);
+	futex_hash_slots_set_must_fail(1 << 29);
+	futex_hash_slots_set_verify(4);
 
 	/*
-	 * Once the private hash has been made immutable or global hash has been requested,
-	 * then this requested can not be undone.
+	 * Once the global hash has been requested, then this requested can not
+	 * be undone.
 	 */
-	if (use_global_hash) {
-		ret = futex_hash_slots_set(0, 0);
-		ksft_test_result(ret == 0, "Global hash request\n");
-	} else {
-		ret = futex_hash_slots_set(4, FH_FLAG_IMMUTABLE);
-		ksft_test_result(ret == 0, "Immutable resize to 4\n");
-	}
+	ret = futex_hash_slots_set(0);
+	ksft_test_result(ret == 0, "Global hash request\n");
 	if (ret != 0)
 		goto out;
 
-	futex_hash_slots_set_must_fail(4, 0);
-	futex_hash_slots_set_must_fail(4, FH_FLAG_IMMUTABLE);
-	futex_hash_slots_set_must_fail(8, 0);
-	futex_hash_slots_set_must_fail(8, FH_FLAG_IMMUTABLE);
-	futex_hash_slots_set_must_fail(0, FH_FLAG_IMMUTABLE);
-	futex_hash_slots_set_must_fail(6, FH_FLAG_IMMUTABLE);
+	futex_hash_slots_set_must_fail(4);
+	futex_hash_slots_set_must_fail(8);
+	futex_hash_slots_set_must_fail(8);
+	futex_hash_slots_set_must_fail(0);
+	futex_hash_slots_set_must_fail(6);
 
 	ret = pthread_barrier_init(&barrier_main, NULL, MAX_THREADS);
 	if (ret != 0) {
@@ -317,14 +297,7 @@ retry_getslots:
 	join_max_threads();
 
 	ret = futex_hash_slots_get();
-	if (use_global_hash) {
-		ksft_test_result(ret == 0, "Continue to use global hash\n");
-	} else {
-		ksft_test_result(ret == 4, "Continue to use the 4 hash buckets\n");
-	}
-
-	ret = futex_hash_immutable_get();
-	ksft_test_result(ret == 1, "Hash reports to be immutable\n");
+	ksft_test_result(ret == 0, "Continue to use global hash\n");
 
 out:
 	ksft_finished();

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

* [tip: locking/futex] futex: Remove support for IMMUTABLE
  2025-07-10 11:00 ` [PATCH v2 4/6] futex: Remove support for IMMUTABLE Sebastian Andrzej Siewior
@ 2025-07-11 18:33   ` tip-bot2 for Sebastian Andrzej Siewior
  0 siblings, 0 replies; 25+ messages in thread
From: tip-bot2 for Sebastian Andrzej Siewior @ 2025-07-11 18:33 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: Sebastian Andrzej Siewior, Peter Zijlstra (Intel), x86,
	linux-kernel

The following commit has been merged into the locking/futex branch of tip:

Commit-ID:     760e6f7befbab9a84c54457a8ee45313b7b91ee5
Gitweb:        https://git.kernel.org/tip/760e6f7befbab9a84c54457a8ee45313b7b91ee5
Author:        Sebastian Andrzej Siewior <bigeasy@linutronix.de>
AuthorDate:    Thu, 10 Jul 2025 13:00:09 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Fri, 11 Jul 2025 16:02:01 +02:00

futex: Remove support for IMMUTABLE

The FH_FLAG_IMMUTABLE flag was meant to avoid the reference counting on
the private hash and so to avoid the performance regression on big
machines.
With the switch to per-CPU counter this is no longer needed. That flag
was never useable on any released kernel.

Remove any support for IMMUTABLE while preserve the flags argument and
enforce it to be zero.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lore.kernel.org/r/20250710110011.384614-5-bigeasy@linutronix.de
---
 include/uapi/linux/prctl.h |  2 --
 kernel/futex/core.c        | 36 +++---------------------------------
 2 files changed, 3 insertions(+), 35 deletions(-)

diff --git a/include/uapi/linux/prctl.h b/include/uapi/linux/prctl.h
index 43dec6e..3b93fb9 100644
--- a/include/uapi/linux/prctl.h
+++ b/include/uapi/linux/prctl.h
@@ -367,8 +367,6 @@ struct prctl_mm_map {
 /* FUTEX hash management */
 #define PR_FUTEX_HASH			78
 # define PR_FUTEX_HASH_SET_SLOTS	1
-# define FH_FLAG_IMMUTABLE		(1ULL << 0)
 # define PR_FUTEX_HASH_GET_SLOTS	2
-# define PR_FUTEX_HASH_GET_IMMUTABLE	3
 
 #endif /* _LINUX_PRCTL_H */
diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index 1981574..d9bb556 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -69,7 +69,6 @@ struct futex_private_hash {
 	struct rcu_head	rcu;
 	void		*mm;
 	bool		custom;
-	bool		immutable;
 	struct futex_hash_bucket queues[];
 };
 
@@ -145,15 +144,11 @@ static inline bool futex_key_is_private(union futex_key *key)
 
 static bool futex_private_hash_get(struct futex_private_hash *fph)
 {
-	if (fph->immutable)
-		return true;
 	return futex_ref_get(fph);
 }
 
 void futex_private_hash_put(struct futex_private_hash *fph)
 {
-	if (fph->immutable)
-		return;
 	if (futex_ref_put(fph))
 		wake_up_var(fph->mm);
 }
@@ -1530,7 +1525,6 @@ static void futex_hash_bucket_init(struct futex_hash_bucket *fhb,
 }
 
 #define FH_CUSTOM	0x01
-#define FH_IMMUTABLE	0x02
 
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
 
@@ -1800,7 +1794,7 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 	 */
 	scoped_guard(rcu) {
 		fph = rcu_dereference(mm->futex_phash);
-		if (fph && (!fph->hash_mask || fph->immutable)) {
+		if (fph && !fph->hash_mask) {
 			if (custom)
 				return -EBUSY;
 			return 0;
@@ -1814,7 +1808,6 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 
 	fph->hash_mask = hash_slots ? hash_slots - 1 : 0;
 	fph->custom = custom;
-	fph->immutable = !!(flags & FH_IMMUTABLE);
 	fph->mm = mm;
 
 	for (i = 0; i < hash_slots; i++)
@@ -1838,7 +1831,7 @@ again:
 		mm->futex_phash_new = NULL;
 
 		if (fph) {
-			if (cur && (!cur->hash_mask || cur->immutable)) {
+			if (cur && !cur->hash_mask) {
 				/*
 				 * If two threads simultaneously request the global
 				 * hash then the first one performs the switch,
@@ -1931,19 +1924,6 @@ static int futex_hash_get_slots(void)
 	return 0;
 }
 
-static int futex_hash_get_immutable(void)
-{
-	struct futex_private_hash *fph;
-
-	guard(rcu)();
-	fph = rcu_dereference(current->mm->futex_phash);
-	if (fph && fph->immutable)
-		return 1;
-	if (fph && !fph->hash_mask)
-		return 1;
-	return 0;
-}
-
 #else
 
 static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
@@ -1956,10 +1936,6 @@ static int futex_hash_get_slots(void)
 	return 0;
 }
 
-static int futex_hash_get_immutable(void)
-{
-	return 0;
-}
 #endif
 
 int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
@@ -1969,10 +1945,8 @@ int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
 
 	switch (arg2) {
 	case PR_FUTEX_HASH_SET_SLOTS:
-		if (arg4 & ~FH_FLAG_IMMUTABLE)
+		if (arg4)
 			return -EINVAL;
-		if (arg4 & FH_FLAG_IMMUTABLE)
-			flags |= FH_IMMUTABLE;
 		ret = futex_hash_allocate(arg3, flags);
 		break;
 
@@ -1980,10 +1954,6 @@ int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
 		ret = futex_hash_get_slots();
 		break;
 
-	case PR_FUTEX_HASH_GET_IMMUTABLE:
-		ret = futex_hash_get_immutable();
-		break;
-
 	default:
 		ret = -EINVAL;
 		break;

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

* [tip: locking/futex] futex: Make futex_private_hash_get() static
  2025-07-10 11:00 ` [PATCH v2 3/6] futex: Make futex_private_hash_get() static Sebastian Andrzej Siewior
@ 2025-07-11 18:33   ` tip-bot2 for Sebastian Andrzej Siewior
  0 siblings, 0 replies; 25+ messages in thread
From: tip-bot2 for Sebastian Andrzej Siewior @ 2025-07-11 18:33 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: Sebastian Andrzej Siewior, Peter Zijlstra (Intel), x86,
	linux-kernel

The following commit has been merged into the locking/futex branch of tip:

Commit-ID:     fb3c553da7fa9991f9b1436d91dbb78c7477c86a
Gitweb:        https://git.kernel.org/tip/fb3c553da7fa9991f9b1436d91dbb78c7477c86a
Author:        Sebastian Andrzej Siewior <bigeasy@linutronix.de>
AuthorDate:    Thu, 10 Jul 2025 13:00:08 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Fri, 11 Jul 2025 16:02:00 +02:00

futex: Make futex_private_hash_get() static

futex_private_hash_get() is not used outside if its compilation unit.
Make it static.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lore.kernel.org/r/20250710110011.384614-4-bigeasy@linutronix.de
---
 kernel/futex/core.c  | 2 +-
 kernel/futex/futex.h | 2 --
 2 files changed, 1 insertion(+), 3 deletions(-)

diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index 1dcb4c8..1981574 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -143,7 +143,7 @@ static inline bool futex_key_is_private(union futex_key *key)
 	return !(key->both.offset & (FUT_OFF_INODE | FUT_OFF_MMSHARED));
 }
 
-bool futex_private_hash_get(struct futex_private_hash *fph)
+static bool futex_private_hash_get(struct futex_private_hash *fph)
 {
 	if (fph->immutable)
 		return true;
diff --git a/kernel/futex/futex.h b/kernel/futex/futex.h
index fcd1617..c74eac5 100644
--- a/kernel/futex/futex.h
+++ b/kernel/futex/futex.h
@@ -228,14 +228,12 @@ extern void futex_hash_get(struct futex_hash_bucket *hb);
 extern void futex_hash_put(struct futex_hash_bucket *hb);
 
 extern struct futex_private_hash *futex_private_hash(void);
-extern bool futex_private_hash_get(struct futex_private_hash *fph);
 extern void futex_private_hash_put(struct futex_private_hash *fph);
 
 #else /* !CONFIG_FUTEX_PRIVATE_HASH */
 static inline void futex_hash_get(struct futex_hash_bucket *hb) { }
 static inline void futex_hash_put(struct futex_hash_bucket *hb) { }
 static inline struct futex_private_hash *futex_private_hash(void) { return NULL; }
-static inline bool futex_private_hash_get(void) { return false; }
 static inline void futex_private_hash_put(struct futex_private_hash *fph) { }
 #endif
 

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

* [tip: locking/futex] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-07-10 11:00 ` [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t Sebastian Andrzej Siewior
@ 2025-07-11 18:33   ` tip-bot2 for Peter Zijlstra
  2025-08-16  2:38     ` Sean Christopherson
  2025-07-30 12:20   ` [PATCH v2 2/6] " André Draszik
  1 sibling, 1 reply; 25+ messages in thread
From: tip-bot2 for Peter Zijlstra @ 2025-07-11 18:33 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: Peter Zijlstra (Intel), Sebastian Andrzej Siewior, x86,
	linux-kernel

The following commit has been merged into the locking/futex branch of tip:

Commit-ID:     56180dd20c19e5b0fa34822997a9ac66b517e7b3
Gitweb:        https://git.kernel.org/tip/56180dd20c19e5b0fa34822997a9ac66b517e7b3
Author:        Peter Zijlstra <peterz@infradead.org>
AuthorDate:    Thu, 10 Jul 2025 13:00:07 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Fri, 11 Jul 2025 16:02:00 +02:00

futex: Use RCU-based per-CPU reference counting instead of rcuref_t

The use of rcuref_t for reference counting introduces a performance bottleneck
when accessed concurrently by multiple threads during futex operations.

Replace rcuref_t with special crafted per-CPU reference counters. The
lifetime logic remains the same.

The newly allocate private hash starts in FR_PERCPU state. In this state, each
futex operation that requires the private hash uses a per-CPU counter (an
unsigned int) for incrementing or decrementing the reference count.

When the private hash is about to be replaced, the per-CPU counters are
migrated to a atomic_t counter mm_struct::futex_atomic.
The migration process:
- Waiting for one RCU grace period to ensure all users observe the
  current private hash. This can be skipped if a grace period elapsed
  since the private hash was assigned.

- futex_private_hash::state is set to FR_ATOMIC, forcing all users to
  use mm_struct::futex_atomic for reference counting.

- After a RCU grace period, all users are guaranteed to be using the
  atomic counter. The per-CPU counters can now be summed up and added to
  the atomic_t counter. If the resulting count is zero, the hash can be
  safely replaced. Otherwise, active users still hold a valid reference.

- Once the atomic reference count drops to zero, the next futex
  operation will switch to the new private hash.

call_rcu_hurry() is used to speed up transition which otherwise might be
delay with RCU_LAZY. There is nothing wrong with using call_rcu(). The
side effects would be that on auto scaling the new hash is used later
and the SET_SLOTS prctl() will block longer.

[bigeasy: commit description + mm get/ put_async]

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lore.kernel.org/r/20250710110011.384614-3-bigeasy@linutronix.de
---
 include/linux/futex.h    |  16 +---
 include/linux/mm_types.h |   5 +-
 include/linux/sched/mm.h |   2 +-
 init/Kconfig             |   4 +-
 kernel/fork.c            |   8 +-
 kernel/futex/core.c      | 243 +++++++++++++++++++++++++++++++++++---
 6 files changed, 243 insertions(+), 35 deletions(-)

diff --git a/include/linux/futex.h b/include/linux/futex.h
index b371936..9e9750f 100644
--- a/include/linux/futex.h
+++ b/include/linux/futex.h
@@ -85,18 +85,12 @@ int futex_hash_prctl(unsigned long arg2, unsigned long arg3, unsigned long arg4)
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
 int futex_hash_allocate_default(void);
 void futex_hash_free(struct mm_struct *mm);
-
-static inline void futex_mm_init(struct mm_struct *mm)
-{
-	RCU_INIT_POINTER(mm->futex_phash, NULL);
-	mm->futex_phash_new = NULL;
-	mutex_init(&mm->futex_hash_lock);
-}
+int futex_mm_init(struct mm_struct *mm);
 
 #else /* !CONFIG_FUTEX_PRIVATE_HASH */
 static inline int futex_hash_allocate_default(void) { return 0; }
-static inline void futex_hash_free(struct mm_struct *mm) { }
-static inline void futex_mm_init(struct mm_struct *mm) { }
+static inline int futex_hash_free(struct mm_struct *mm) { return 0; }
+static inline int futex_mm_init(struct mm_struct *mm) { return 0; }
 #endif /* CONFIG_FUTEX_PRIVATE_HASH */
 
 #else /* !CONFIG_FUTEX */
@@ -118,8 +112,8 @@ static inline int futex_hash_allocate_default(void)
 {
 	return 0;
 }
-static inline void futex_hash_free(struct mm_struct *mm) { }
-static inline void futex_mm_init(struct mm_struct *mm) { }
+static inline int futex_hash_free(struct mm_struct *mm) { return 0; }
+static inline int futex_mm_init(struct mm_struct *mm) { return 0; }
 
 #endif
 
diff --git a/include/linux/mm_types.h b/include/linux/mm_types.h
index d6b91e8..0f06621 100644
--- a/include/linux/mm_types.h
+++ b/include/linux/mm_types.h
@@ -1070,6 +1070,11 @@ struct mm_struct {
 		struct mutex			futex_hash_lock;
 		struct futex_private_hash	__rcu *futex_phash;
 		struct futex_private_hash	*futex_phash_new;
+		/* futex-ref */
+		unsigned long			futex_batches;
+		struct rcu_head			futex_rcu;
+		atomic_long_t			futex_atomic;
+		unsigned int			__percpu *futex_ref;
 #endif
 
 		unsigned long hiwater_rss; /* High-watermark of RSS usage */
diff --git a/include/linux/sched/mm.h b/include/linux/sched/mm.h
index b134748..2201da0 100644
--- a/include/linux/sched/mm.h
+++ b/include/linux/sched/mm.h
@@ -140,7 +140,7 @@ static inline bool mmget_not_zero(struct mm_struct *mm)
 
 /* mmput gets rid of the mappings and all user-space */
 extern void mmput(struct mm_struct *);
-#ifdef CONFIG_MMU
+#if defined(CONFIG_MMU) || defined(CONFIG_FUTEX_PRIVATE_HASH)
 /* same as above but performs the slow path from the async context. Can
  * be called from the atomic context as well
  */
diff --git a/init/Kconfig b/init/Kconfig
index 666783e..af4c2f0 100644
--- a/init/Kconfig
+++ b/init/Kconfig
@@ -1716,13 +1716,9 @@ config FUTEX_PI
 	depends on FUTEX && RT_MUTEXES
 	default y
 
-#
-# marked broken for performance reasons; gives us one more cycle to sort things out.
-#
 config FUTEX_PRIVATE_HASH
 	bool
 	depends on FUTEX && !BASE_SMALL && MMU
-	depends on BROKEN
 	default y
 
 config FUTEX_MPOL
diff --git a/kernel/fork.c b/kernel/fork.c
index 1ee8eb1..0b885dc 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -1046,7 +1046,6 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
 	RCU_INIT_POINTER(mm->exe_file, NULL);
 	mmu_notifier_subscriptions_init(mm);
 	init_tlb_flush_pending(mm);
-	futex_mm_init(mm);
 #if defined(CONFIG_TRANSPARENT_HUGEPAGE) && !defined(CONFIG_SPLIT_PMD_PTLOCKS)
 	mm->pmd_huge_pte = NULL;
 #endif
@@ -1061,6 +1060,9 @@ static struct mm_struct *mm_init(struct mm_struct *mm, struct task_struct *p,
 		mm->def_flags = 0;
 	}
 
+	if (futex_mm_init(mm))
+		goto fail_mm_init;
+
 	if (mm_alloc_pgd(mm))
 		goto fail_nopgd;
 
@@ -1090,6 +1092,8 @@ fail_nocontext:
 fail_noid:
 	mm_free_pgd(mm);
 fail_nopgd:
+	futex_hash_free(mm);
+fail_mm_init:
 	free_mm(mm);
 	return NULL;
 }
@@ -1145,7 +1149,7 @@ void mmput(struct mm_struct *mm)
 }
 EXPORT_SYMBOL_GPL(mmput);
 
-#ifdef CONFIG_MMU
+#if defined(CONFIG_MMU) || defined(CONFIG_FUTEX_PRIVATE_HASH)
 static void mmput_async_fn(struct work_struct *work)
 {
 	struct mm_struct *mm = container_of(work, struct mm_struct,
diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index 90d53fb..1dcb4c8 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -42,7 +42,6 @@
 #include <linux/fault-inject.h>
 #include <linux/slab.h>
 #include <linux/prctl.h>
-#include <linux/rcuref.h>
 #include <linux/mempolicy.h>
 #include <linux/mmap_lock.h>
 
@@ -65,7 +64,7 @@ static struct {
 #define futex_queues	(__futex_data.queues)
 
 struct futex_private_hash {
-	rcuref_t	users;
+	int		state;
 	unsigned int	hash_mask;
 	struct rcu_head	rcu;
 	void		*mm;
@@ -129,6 +128,12 @@ static struct futex_hash_bucket *
 __futex_hash(union futex_key *key, struct futex_private_hash *fph);
 
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
+static bool futex_ref_get(struct futex_private_hash *fph);
+static bool futex_ref_put(struct futex_private_hash *fph);
+static bool futex_ref_is_dead(struct futex_private_hash *fph);
+
+enum { FR_PERCPU = 0, FR_ATOMIC };
+
 static inline bool futex_key_is_private(union futex_key *key)
 {
 	/*
@@ -142,15 +147,14 @@ bool futex_private_hash_get(struct futex_private_hash *fph)
 {
 	if (fph->immutable)
 		return true;
-	return rcuref_get(&fph->users);
+	return futex_ref_get(fph);
 }
 
 void futex_private_hash_put(struct futex_private_hash *fph)
 {
-	/* Ignore return value, last put is verified via rcuref_is_dead() */
 	if (fph->immutable)
 		return;
-	if (rcuref_put(&fph->users))
+	if (futex_ref_put(fph))
 		wake_up_var(fph->mm);
 }
 
@@ -243,14 +247,18 @@ static bool __futex_pivot_hash(struct mm_struct *mm,
 	fph = rcu_dereference_protected(mm->futex_phash,
 					lockdep_is_held(&mm->futex_hash_lock));
 	if (fph) {
-		if (!rcuref_is_dead(&fph->users)) {
+		if (!futex_ref_is_dead(fph)) {
 			mm->futex_phash_new = new;
 			return false;
 		}
 
 		futex_rehash_private(fph, new);
 	}
-	rcu_assign_pointer(mm->futex_phash, new);
+	new->state = FR_PERCPU;
+	scoped_guard(rcu) {
+		mm->futex_batches = get_state_synchronize_rcu();
+		rcu_assign_pointer(mm->futex_phash, new);
+	}
 	kvfree_rcu(fph, rcu);
 	return true;
 }
@@ -289,9 +297,7 @@ again:
 		if (!fph)
 			return NULL;
 
-		if (fph->immutable)
-			return fph;
-		if (rcuref_get(&fph->users))
+		if (futex_private_hash_get(fph))
 			return fph;
 	}
 	futex_pivot_hash(mm);
@@ -1527,16 +1533,219 @@ static void futex_hash_bucket_init(struct futex_hash_bucket *fhb,
 #define FH_IMMUTABLE	0x02
 
 #ifdef CONFIG_FUTEX_PRIVATE_HASH
+
+/*
+ * futex-ref
+ *
+ * Heavily inspired by percpu-rwsem/percpu-refcount; not reusing any of that
+ * code because it just doesn't fit right.
+ *
+ * Dual counter, per-cpu / atomic approach like percpu-refcount, except it
+ * re-initializes the state automatically, such that the fph swizzle is also a
+ * transition back to per-cpu.
+ */
+
+static void futex_ref_rcu(struct rcu_head *head);
+
+static void __futex_ref_atomic_begin(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	/*
+	 * The counter we're about to switch to must have fully switched;
+	 * otherwise it would be impossible for it to have reported success
+	 * from futex_ref_is_dead().
+	 */
+	WARN_ON_ONCE(atomic_long_read(&mm->futex_atomic) != 0);
+
+	/*
+	 * Set the atomic to the bias value such that futex_ref_{get,put}()
+	 * will never observe 0. Will be fixed up in __futex_ref_atomic_end()
+	 * when folding in the percpu count.
+	 */
+	atomic_long_set(&mm->futex_atomic, LONG_MAX);
+	smp_store_release(&fph->state, FR_ATOMIC);
+
+	call_rcu_hurry(&mm->futex_rcu, futex_ref_rcu);
+}
+
+static void __futex_ref_atomic_end(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+	unsigned int count = 0;
+	long ret;
+	int cpu;
+
+	/*
+	 * Per __futex_ref_atomic_begin() the state of the fph must be ATOMIC
+	 * and per this RCU callback, everybody must now observe this state and
+	 * use the atomic variable.
+	 */
+	WARN_ON_ONCE(fph->state != FR_ATOMIC);
+
+	/*
+	 * Therefore the per-cpu counter is now stable, sum and reset.
+	 */
+	for_each_possible_cpu(cpu) {
+		unsigned int *ptr = per_cpu_ptr(mm->futex_ref, cpu);
+		count += *ptr;
+		*ptr = 0;
+	}
+
+	/*
+	 * Re-init for the next cycle.
+	 */
+	this_cpu_inc(*mm->futex_ref); /* 0 -> 1 */
+
+	/*
+	 * Add actual count, subtract bias and initial refcount.
+	 *
+	 * The moment this atomic operation happens, futex_ref_is_dead() can
+	 * become true.
+	 */
+	ret = atomic_long_add_return(count - LONG_MAX - 1, &mm->futex_atomic);
+	if (!ret)
+		wake_up_var(mm);
+
+	WARN_ON_ONCE(ret < 0);
+	mmput_async(mm);
+}
+
+static void futex_ref_rcu(struct rcu_head *head)
+{
+	struct mm_struct *mm = container_of(head, struct mm_struct, futex_rcu);
+	struct futex_private_hash *fph = rcu_dereference_raw(mm->futex_phash);
+
+	if (fph->state == FR_PERCPU) {
+		/*
+		 * Per this extra grace-period, everybody must now observe
+		 * fph as the current fph and no previously observed fph's
+		 * are in-flight.
+		 *
+		 * Notably, nobody will now rely on the atomic
+		 * futex_ref_is_dead() state anymore so we can begin the
+		 * migration of the per-cpu counter into the atomic.
+		 */
+		__futex_ref_atomic_begin(fph);
+		return;
+	}
+
+	__futex_ref_atomic_end(fph);
+}
+
+/*
+ * Drop the initial refcount and transition to atomics.
+ */
+static void futex_ref_drop(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	/*
+	 * Can only transition the current fph;
+	 */
+	WARN_ON_ONCE(rcu_dereference_raw(mm->futex_phash) != fph);
+	/*
+	 * We enqueue at least one RCU callback. Ensure mm stays if the task
+	 * exits before the transition is completed.
+	 */
+	mmget(mm);
+
+	/*
+	 * In order to avoid the following scenario:
+	 *
+	 * futex_hash()			__futex_pivot_hash()
+	 *   guard(rcu);		  guard(mm->futex_hash_lock);
+	 *   fph = mm->futex_phash;
+	 *				  rcu_assign_pointer(&mm->futex_phash, new);
+	 *				futex_hash_allocate()
+	 *				  futex_ref_drop()
+	 *				    fph->state = FR_ATOMIC;
+	 *				    atomic_set(, BIAS);
+	 *
+	 *   futex_private_hash_get(fph); // OOPS
+	 *
+	 * Where an old fph (which is FR_ATOMIC) and should fail on
+	 * inc_not_zero, will succeed because a new transition is started and
+	 * the atomic is bias'ed away from 0.
+	 *
+	 * There must be at least one full grace-period between publishing a
+	 * new fph and trying to replace it.
+	 */
+	if (poll_state_synchronize_rcu(mm->futex_batches)) {
+		/*
+		 * There was a grace-period, we can begin now.
+		 */
+		__futex_ref_atomic_begin(fph);
+		return;
+	}
+
+	call_rcu_hurry(&mm->futex_rcu, futex_ref_rcu);
+}
+
+static bool futex_ref_get(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	guard(rcu)();
+
+	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
+		this_cpu_inc(*mm->futex_ref);
+		return true;
+	}
+
+	return atomic_long_inc_not_zero(&mm->futex_atomic);
+}
+
+static bool futex_ref_put(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	guard(rcu)();
+
+	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
+		this_cpu_dec(*mm->futex_ref);
+		return false;
+	}
+
+	return atomic_long_dec_and_test(&mm->futex_atomic);
+}
+
+static bool futex_ref_is_dead(struct futex_private_hash *fph)
+{
+	struct mm_struct *mm = fph->mm;
+
+	guard(rcu)();
+
+	if (smp_load_acquire(&fph->state) == FR_PERCPU)
+		return false;
+
+	return atomic_long_read(&mm->futex_atomic) == 0;
+}
+
+int futex_mm_init(struct mm_struct *mm)
+{
+	mutex_init(&mm->futex_hash_lock);
+	RCU_INIT_POINTER(mm->futex_phash, NULL);
+	mm->futex_phash_new = NULL;
+	/* futex-ref */
+	atomic_long_set(&mm->futex_atomic, 0);
+	mm->futex_batches = get_state_synchronize_rcu();
+	mm->futex_ref = alloc_percpu(unsigned int);
+	if (!mm->futex_ref)
+		return -ENOMEM;
+	this_cpu_inc(*mm->futex_ref); /* 0 -> 1 */
+	return 0;
+}
+
 void futex_hash_free(struct mm_struct *mm)
 {
 	struct futex_private_hash *fph;
 
+	free_percpu(mm->futex_ref);
 	kvfree(mm->futex_phash_new);
 	fph = rcu_dereference_raw(mm->futex_phash);
-	if (fph) {
-		WARN_ON_ONCE(rcuref_read(&fph->users) > 1);
+	if (fph)
 		kvfree(fph);
-	}
 }
 
 static bool futex_pivot_pending(struct mm_struct *mm)
@@ -1549,7 +1758,7 @@ static bool futex_pivot_pending(struct mm_struct *mm)
 		return true;
 
 	fph = rcu_dereference(mm->futex_phash);
-	return rcuref_is_dead(&fph->users);
+	return futex_ref_is_dead(fph);
 }
 
 static bool futex_hash_less(struct futex_private_hash *a,
@@ -1598,11 +1807,11 @@ static int futex_hash_allocate(unsigned int hash_slots, unsigned int flags)
 		}
 	}
 
-	fph = kvzalloc(struct_size(fph, queues, hash_slots), GFP_KERNEL_ACCOUNT | __GFP_NOWARN);
+	fph = kvzalloc(struct_size(fph, queues, hash_slots),
+		       GFP_KERNEL_ACCOUNT | __GFP_NOWARN);
 	if (!fph)
 		return -ENOMEM;
 
-	rcuref_init(&fph->users, 1);
 	fph->hash_mask = hash_slots ? hash_slots - 1 : 0;
 	fph->custom = custom;
 	fph->immutable = !!(flags & FH_IMMUTABLE);
@@ -1645,7 +1854,7 @@ again:
 				 * allocated a replacement hash, drop the initial
 				 * reference on the existing hash.
 				 */
-				futex_private_hash_put(cur);
+				futex_ref_drop(cur);
 			}
 
 			if (new) {

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

* [tip: locking/futex] selftests/futex: Adapt the private hash test to RCU related changes
  2025-07-10 11:00 ` [PATCH v2 1/6] selftests/futex: Adapt the private hash test to RCU related changes Sebastian Andrzej Siewior
@ 2025-07-11 18:33   ` tip-bot2 for Sebastian Andrzej Siewior
  0 siblings, 0 replies; 25+ messages in thread
From: tip-bot2 for Sebastian Andrzej Siewior @ 2025-07-11 18:33 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: Sebastian Andrzej Siewior, Peter Zijlstra (Intel), x86,
	linux-kernel

The following commit has been merged into the locking/futex branch of tip:

Commit-ID:     a255b78d14324f8a4a49f88e983b9f00818d1194
Gitweb:        https://git.kernel.org/tip/a255b78d14324f8a4a49f88e983b9f00818d1194
Author:        Sebastian Andrzej Siewior <bigeasy@linutronix.de>
AuthorDate:    Thu, 10 Jul 2025 13:00:06 +02:00
Committer:     Peter Zijlstra <peterz@infradead.org>
CommitterDate: Fri, 11 Jul 2025 16:02:00 +02:00

selftests/futex: Adapt the private hash test to RCU related changes

The auto scaling on create creation used to automatically assign the new
hash because there was the private hash was unused and could be replaced
right away.

This is already racy because if the private hash is in use by a thread
then the visibile resize will be delayed. With the upcoming change to
wait for a RCU grace period before the hash can be assigned, the test
will always fail.

If the reported number of hash buckets is not updated after an
auto scaling event, block on an acquired lock with a timeout. The timeout
is the delay to wait towards a grace period and locking and a locked
pthread_mutex_t ensure that glibc calls into kernel using futex
operation which will assign new private hash if available.
This will retry every 100ms up to 2 seconds in total.

Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lore.kernel.org/r/20250710110011.384614-2-bigeasy@linutronix.de
---
 tools/testing/selftests/futex/functional/futex_priv_hash.c | 42 ++++++-
 1 file changed, 41 insertions(+), 1 deletion(-)

diff --git a/tools/testing/selftests/futex/functional/futex_priv_hash.c b/tools/testing/selftests/futex/functional/futex_priv_hash.c
index 24a92dc..625e3be 100644
--- a/tools/testing/selftests/futex/functional/futex_priv_hash.c
+++ b/tools/testing/selftests/futex/functional/futex_priv_hash.c
@@ -111,6 +111,30 @@ static void join_max_threads(void)
 	}
 }
 
+#define SEC_IN_NSEC	1000000000
+#define MSEC_IN_NSEC	1000000
+
+static void futex_dummy_op(void)
+{
+	pthread_mutex_t lock = PTHREAD_MUTEX_INITIALIZER;
+	struct timespec timeout;
+	int ret;
+
+	pthread_mutex_lock(&lock);
+	clock_gettime(CLOCK_REALTIME, &timeout);
+	timeout.tv_nsec += 100 * MSEC_IN_NSEC;
+	if (timeout.tv_nsec >=  SEC_IN_NSEC) {
+		timeout.tv_nsec -= SEC_IN_NSEC;
+		timeout.tv_sec++;
+	}
+	ret = pthread_mutex_timedlock(&lock, &timeout);
+	if (ret == 0)
+		ksft_exit_fail_msg("Succeffuly locked an already locked mutex.\n");
+
+	if (ret != ETIMEDOUT)
+		ksft_exit_fail_msg("pthread_mutex_timedlock() did not timeout: %d.\n", ret);
+}
+
 static void usage(char *prog)
 {
 	printf("Usage: %s\n", prog);
@@ -129,7 +153,7 @@ int main(int argc, char *argv[])
 	int futex_slots1, futex_slotsn, online_cpus;
 	pthread_mutexattr_t mutex_attr_pi;
 	int use_global_hash = 0;
-	int ret;
+	int ret, retry = 20;
 	int c;
 
 	while ((c = getopt(argc, argv, "cghv:")) != -1) {
@@ -208,8 +232,24 @@ int main(int argc, char *argv[])
 	 */
 	ksft_print_msg("Online CPUs: %d\n", online_cpus);
 	if (online_cpus > 16) {
+retry_getslots:
 		futex_slotsn = futex_hash_slots_get();
 		if (futex_slotsn < 0 || futex_slots1 == futex_slotsn) {
+			retry--;
+			/*
+			 * Auto scaling on thread creation can be slightly delayed
+			 * because it waits for a RCU grace period twice. The new
+			 * private hash is assigned upon the first futex operation
+			 * after grace period.
+			 * To cover all this for testing purposes the function
+			 * below will acquire a lock and acquire it again with a
+			 * 100ms timeout which must timeout. This ensures we
+			 * sleep for 100ms and issue a futex operation.
+			 */
+			if (retry > 0) {
+				futex_dummy_op();
+				goto retry_getslots;
+			}
 			ksft_print_msg("Expected increase of hash buckets but got: %d -> %d\n",
 				       futex_slots1, futex_slotsn);
 			ksft_exit_fail_msg(test_msg_auto_inc);

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

* Re: [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting
  2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
                   ` (5 preceding siblings ...)
  2025-07-10 11:00 ` [PATCH v2 6/6] perf bench futex: " Sebastian Andrzej Siewior
@ 2025-07-15 15:59 ` Shrikanth Hegde
  2025-07-15 16:31   ` Sebastian Andrzej Siewior
  6 siblings, 1 reply; 25+ messages in thread
From: Shrikanth Hegde @ 2025-07-15 15:59 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior, Peter Zijlstra
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Thomas Gleixner, linux-kernel, Valentin Schneider,
	Waiman Long



On 7/10/25 16:30, Sebastian Andrzej Siewior wrote:
> I picked up PeterZ futex patch from
>      https://lore.kernel.org/all/20250624190118.GB1490279@noisy.programming.kicks-ass.net/
> 
> and I am posting it here it now so it can be staged for v6.17.
> 
> This survived a few days on my machine and compile robot reported that
> is passes its tests.
> 
> v1…v2 https://lore.kernel.org/all/20250707143623.70325-1-bigeasy@linutronix.de
>   - Removed the IMMUTABLE bits
>   - There was a race if the application exits while the RCU callback is
>     pending. Stuffed with mmget()/ mmput_async().
> 
> Changes since its initial posting:
> - A patch description has been added
> - The testuite is "fixed" slightly different and has been split out
> - futex_mm_init() is fixed up.
> - The guard(preempt) has been replaced with guard(rcu) since there is
>    no reason to disable preemption.
> 
> Since it was not yet released, should we rip out the IMMUTABLE bits and
> just stick with GET/SET slots?
> 
> Peter Zijlstra (1):
>    futex: Use RCU-based per-CPU reference counting instead of rcuref_t
> 
> Sebastian Andrzej Siewior (5):
>    selftests/futex: Adapt the private hash test to RCU related changes
>    futex: Make futex_private_hash_get() static
>    futex: Remove support for IMMUTABLE
>    selftests/futex: Remove support for IMMUTABLE
>    perf bench futex: Remove support for IMMUTABLE
> 
>   include/linux/futex.h                         |  16 +-
>   include/linux/mm_types.h                      |   5 +
>   include/linux/sched/mm.h                      |   2 +-
>   include/uapi/linux/prctl.h                    |   2 -
>   init/Kconfig                                  |   4 -
>   kernel/fork.c                                 |   8 +-
>   kernel/futex/core.c                           | 281 ++++++++++++++----
>   kernel/futex/futex.h                          |   2 -
>   tools/include/uapi/linux/prctl.h              |   2 -
>   tools/perf/bench/futex-hash.c                 |   1 -
>   tools/perf/bench/futex-lock-pi.c              |   1 -
>   tools/perf/bench/futex-requeue.c              |   1 -
>   tools/perf/bench/futex-wake-parallel.c        |   1 -
>   tools/perf/bench/futex-wake.c                 |   1 -
>   tools/perf/bench/futex.c                      |  21 +-
>   tools/perf/bench/futex.h                      |   1 -
>   .../trace/beauty/include/uapi/linux/prctl.h   |   2 -
>   .../futex/functional/futex_priv_hash.c        | 113 +++----
>   18 files changed, 315 insertions(+), 149 deletions(-)
> 

Hi. Sorry for not stumble upon this earlier. Saw these now.

Since perf bench had shown a significant regression last time around, and
for which immutable option was added, gave perf futex a try again.

Below are the results: Ran on 5 core LPAR(VM) on power. perf was compiled from tools/perf.

===========
baseline:
===========
tip/master at
commit 8784fb5fa2e0042fe3b1632d4876e1037b695f56 (HEAD)
Author: Borislav Petkov (AMD) <bp@alien8.de>

./perf bench futex hash
Averaged 1559643 operations/sec (+- 0.09%), total secs = 10
Futex hashing: global hash

schbench -t 64 -r 5 -i 5
current rps: 2629.85

schbench -L -m 4 -M auto -t 64 -n 0 -r 5 -i 5
current rps: 1538674.22

=================
baseline + series
=================

./perf bench futex hash
Averaged 306403 operations/sec (+- 0.29%), total secs = 10    <<<  around 1/5th of baseline.
Futex hashing: auto resized to 256 buckets                    <<<  maybe resize doesn't happen fast?


./perf bench futex hash -b 512                                <<< Gave 512 buckets,
Averaged 1412543 operations/sec (+- 0.14%), total secs = 10   <<< much better numbers, still off by 8-10%.
Futex hashing: 512 hash buckets

(512 is the number of buckets that baseline would have used, increased the buckets to 8192 for trial)

./perf bench futex hash -b 8192
Averaged 1441627 operations/sec (+- 0.14%), total secs = 10
Futex hashing: 8192 hash buckets


schbench -t 64 -r 5 -i 5
current rps: 2656.85                                          <<< schbench seems good.

schbench -L -m 4 -M auto -t 64 -n 0 -r 5 -i 5
current rps: 1539273.79

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

* Re: [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting
  2025-07-15 15:59 ` [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Shrikanth Hegde
@ 2025-07-15 16:31   ` Sebastian Andrzej Siewior
  2025-07-15 17:04     ` Shrikanth Hegde
  0 siblings, 1 reply; 25+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-07-15 16:31 UTC (permalink / raw)
  To: Shrikanth Hegde
  Cc: Peter Zijlstra, André Almeida, Darren Hart, Davidlohr Bueso,
	Ingo Molnar, Juri Lelli, Thomas Gleixner, linux-kernel,
	Valentin Schneider, Waiman Long

On 2025-07-15 21:29:34 [+0530], Shrikanth Hegde wrote:
> Hi. Sorry for not stumble upon this earlier. Saw these now.
> 
> Since perf bench had shown a significant regression last time around, and
> for which immutable option was added, gave perf futex a try again.
> 
> Below are the results: Ran on 5 core LPAR(VM) on power. perf was compiled from tools/perf.
Thank you.

If you use perf-bench with -b then the buckets are applied
"immediately". It mostly works also with auto scaling. The problem is
that perf creates the threads and immediately after it starts the test.
While the RCU kicks in shortly after there is no transition happening
until after all the test completes/ the threads terminate. The reason is
that several private-hash references are in use because a some threads
are always in the futex() syscall.

It would require something like commit
    a255b78d14324 ("selftests/futex: Adapt the private hash test to RCU related changes")

to have this transition before the test starts.
Your schbench seems not affected?

If you use -b, is it better than or equal compared to the immutable
option? This isn't quite clear.

Sebastian

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

* Re: [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting
  2025-07-15 16:31   ` Sebastian Andrzej Siewior
@ 2025-07-15 17:04     ` Shrikanth Hegde
  2025-07-16 14:29       ` Peter Zijlstra
  0 siblings, 1 reply; 25+ messages in thread
From: Shrikanth Hegde @ 2025-07-15 17:04 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Peter Zijlstra, André Almeida, Darren Hart, Davidlohr Bueso,
	Ingo Molnar, Juri Lelli, Thomas Gleixner, linux-kernel,
	Valentin Schneider, Waiman Long



On 7/15/25 22:01, Sebastian Andrzej Siewior wrote:
> On 2025-07-15 21:29:34 [+0530], Shrikanth Hegde wrote:
>> Hi. Sorry for not stumble upon this earlier. Saw these now.
>>
>> Since perf bench had shown a significant regression last time around, and
>> for which immutable option was added, gave perf futex a try again.
>>
>> Below are the results: Ran on 5 core LPAR(VM) on power. perf was compiled from tools/perf.
> Thank you.
> 
> If you use perf-bench with -b then the buckets are applied
> "immediately". It mostly works also with auto scaling. The problem is
> that perf creates the threads and immediately after it starts the test.
> While the RCU kicks in shortly after there is no transition happening
> until after all the test completes/ the threads terminate. The reason is
> that several private-hash references are in use because a some threads
> are always in the futex() syscall.
> 
> It would require something like commit
>      a255b78d14324 ("selftests/futex: Adapt the private hash test to RCU related changes")
> 
> to have this transition before the test starts.
> Your schbench seems not affected?

Yes. schbench shows similar number.

> 
> If you use -b, is it better than or equal compared to the immutable
> option? This isn't quite clear.
> n


I did try again by going to baseline, removed BROKEN and ran below. Which gives us immutable numbers.
./perf bench futex hash -Ib512
Averaged 1536035 operations/sec (+- 0.11%), total secs = 10
Futex hashing: 512 hash buckets (immutable)

So, with -b 512 option, it is around 8-10% less compared to immutable.


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

* Re: [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting
  2025-07-15 17:04     ` Shrikanth Hegde
@ 2025-07-16 14:29       ` Peter Zijlstra
  2025-07-16 18:21         ` Shrikanth Hegde
  0 siblings, 1 reply; 25+ messages in thread
From: Peter Zijlstra @ 2025-07-16 14:29 UTC (permalink / raw)
  To: Shrikanth Hegde
  Cc: Sebastian Andrzej Siewior, André Almeida, Darren Hart,
	Davidlohr Bueso, Ingo Molnar, Juri Lelli, Thomas Gleixner,
	linux-kernel, Valentin Schneider, Waiman Long

On Tue, Jul 15, 2025 at 10:34:24PM +0530, Shrikanth Hegde wrote:

> I did try again by going to baseline, removed BROKEN and ran below. Which gives us immutable numbers.
> ./perf bench futex hash -Ib512
> Averaged 1536035 operations/sec (+- 0.11%), total secs = 10
> Futex hashing: 512 hash buckets (immutable)
> 
> So, with -b 512 option, it is around 8-10% less compared to immutable.

Urgh, can you run perf on that and tell me if this is due to
this_cpu_{inc,dec}() doing local_irq_disable() or the smp_load_acquire()
doing LWSYNC ?

Anyway, I think we can improve both. Does the below help?


---
diff --git a/kernel/futex/core.c b/kernel/futex/core.c
index d9bb5567af0c..8c41d050bd1f 100644
--- a/kernel/futex/core.c
+++ b/kernel/futex/core.c
@@ -1680,10 +1680,10 @@ static bool futex_ref_get(struct futex_private_hash *fph)
 {
 	struct mm_struct *mm = fph->mm;
 
-	guard(rcu)();
+	guard(preempt)();
 
-	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
-		this_cpu_inc(*mm->futex_ref);
+	if (READ_ONCE(fph->state) == FR_PERCPU) {
+		__this_cpu_inc(*mm->futex_ref);
 		return true;
 	}
 
@@ -1694,10 +1694,10 @@ static bool futex_ref_put(struct futex_private_hash *fph)
 {
 	struct mm_struct *mm = fph->mm;
 
-	guard(rcu)();
+	guard(preempt)();
 
-	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
-		this_cpu_dec(*mm->futex_ref);
+	if (READ_ONCE(fph->state) == FR_PERCPU) {
+		__this_cpu_dec(*mm->futex_ref);
 		return false;
 	}
 

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

* Re: [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting
  2025-07-16 14:29       ` Peter Zijlstra
@ 2025-07-16 18:21         ` Shrikanth Hegde
  0 siblings, 0 replies; 25+ messages in thread
From: Shrikanth Hegde @ 2025-07-16 18:21 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Sebastian Andrzej Siewior, André Almeida, Darren Hart,
	Davidlohr Bueso, Ingo Molnar, Juri Lelli, Thomas Gleixner,
	linux-kernel, Valentin Schneider, Waiman Long



On 7/16/25 19:59, Peter Zijlstra wrote:
> On Tue, Jul 15, 2025 at 10:34:24PM +0530, Shrikanth Hegde wrote:
> 
>> I did try again by going to baseline, removed BROKEN and ran below. Which gives us immutable numbers.
>> ./perf bench futex hash -Ib512
>> Averaged 1536035 operations/sec (+- 0.11%), total secs = 10
>> Futex hashing: 512 hash buckets (immutable)
>>
>> So, with -b 512 option, it is around 8-10% less compared to immutable.
> 
> Urgh, can you run perf on that and tell me if this is due to
> this_cpu_{inc,dec}() doing local_irq_disable() or the smp_load_acquire()
> doing LWSYNC ?

It seems like due to rcu and irq enable.
Both perf records are collected with -b512.


base_futex_immutable_b512 - perf record collected with baseline + remove BROKEN + ./perf bench futex hash -Ib512
per_cpu_futex_hash_b_512 - baseline + series + ./perf bench futex hash -b512


perf diff base_futex_immutable_b512 per_cpu_futex_hash_b_512
# Event 'cycles'
#
# Baseline  Delta Abs  Shared Object               Symbol
# ........  .........  ..........................  ....................................................
#
     21.62%     -2.26%  [kernel.vmlinux]            [k] futex_get_value_locked
      0.16%     +2.01%  [kernel.vmlinux]            [k] __rcu_read_unlock
      1.35%     +1.63%  [kernel.vmlinux]            [k] arch_local_irq_restore.part.0
                +1.48%  [kernel.vmlinux]            [k] futex_private_hash_put
                +1.16%  [kernel.vmlinux]            [k] futex_ref_get
     10.41%     -0.78%  [kernel.vmlinux]            [k] system_call_vectored_common
      1.24%     +0.72%  perf                        [.] workerfn
      5.32%     -0.66%  [kernel.vmlinux]            [k] futex_q_lock
      2.48%     -0.43%  [kernel.vmlinux]            [k] futex_wait
      2.47%     -0.40%  [kernel.vmlinux]            [k] _raw_spin_lock
      2.98%     -0.35%  [kernel.vmlinux]            [k] futex_q_unlock
      2.42%     -0.34%  [kernel.vmlinux]            [k] __futex_wait
      5.47%     -0.32%  libc.so.6                   [.] syscall
      4.03%     -0.32%  [kernel.vmlinux]            [k] memcpy_power7
      0.16%     +0.22%  [kernel.vmlinux]            [k] arch_local_irq_restore
      5.93%     -0.18%  [kernel.vmlinux]            [k] futex_hash
      1.72%     -0.17%  [kernel.vmlinux]            [k] sys_futex


> 
> Anyway, I think we can improve both. Does the below help?
> 
> 
> ---
> diff --git a/kernel/futex/core.c b/kernel/futex/core.c
> index d9bb5567af0c..8c41d050bd1f 100644
> --- a/kernel/futex/core.c
> +++ b/kernel/futex/core.c
> @@ -1680,10 +1680,10 @@ static bool futex_ref_get(struct futex_private_hash *fph)
>   {
>   	struct mm_struct *mm = fph->mm;
>   
> -	guard(rcu)();
> +	guard(preempt)();
>   
> -	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
> -		this_cpu_inc(*mm->futex_ref);
> +	if (READ_ONCE(fph->state) == FR_PERCPU) {
> +		__this_cpu_inc(*mm->futex_ref);
>   		return true;
>   	}
>   
> @@ -1694,10 +1694,10 @@ static bool futex_ref_put(struct futex_private_hash *fph)
>   {
>   	struct mm_struct *mm = fph->mm;
>   
> -	guard(rcu)();
> +	guard(preempt)();
>   
> -	if (smp_load_acquire(&fph->state) == FR_PERCPU) {
> -		this_cpu_dec(*mm->futex_ref);
> +	if (READ_ONCE(fph->state) == FR_PERCPU) {
> +		__this_cpu_dec(*mm->futex_ref);
>   		return false;
>   	}
>   

Yes. It helps. It improves "-b 512" numbers by at-least 5%.

baseline + series:
Averaged 1412543 operations/sec (+- 0.14%), total secs = 10
Futex hashing: 512 hash buckets


baseline + series+ above_patch:
Averaged 1482733 operations/sec (+- 0.26%), total secs = 10   <<< 5% improvement
Futex hashing: 512 hash buckets


Now we are closer baseline/immutable by 4-5%.
baseline:
commit 8784fb5fa2e0042fe3b1632d4876e1037b695f56 (HEAD)

./perf bench futex hash
Averaged 1559643 operations/sec (+- 0.09%), total secs = 10
Futex hashing: global hash
  

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

* Re: [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-07-10 11:00 ` [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t Sebastian Andrzej Siewior
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Peter Zijlstra
@ 2025-07-30 12:20   ` André Draszik
  2025-07-30 19:44     ` Thomas Gleixner
  1 sibling, 1 reply; 25+ messages in thread
From: André Draszik @ 2025-07-30 12:20 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior, linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Thomas Gleixner, Valentin Schneider,
	Waiman Long, Andrew Morton, David Hildenbrand, Liam R. Howlett,
	Lorenzo Stoakes, Michal Hocko, Mike Rapoport, Suren Baghdasaryan,
	Vlastimil Babka, linux-mm

On Thu, 2025-07-10 at 13:00 +0200, Sebastian Andrzej Siewior wrote:
> From: Peter Zijlstra <peterz@infradead.org>
> 
> The use of rcuref_t for reference counting introduces a performance bottleneck
> when accessed concurrently by multiple threads during futex operations.
> 
> Replace rcuref_t with special crafted per-CPU reference counters. The
> lifetime logic remains the same.
> 
> The newly allocate private hash starts in FR_PERCPU state. In this state, each
> futex operation that requires the private hash uses a per-CPU counter (an
> unsigned int) for incrementing or decrementing the reference count.
> 
> When the private hash is about to be replaced, the per-CPU counters are
> migrated to a atomic_t counter mm_struct::futex_atomic.
> The migration process:
> - Waiting for one RCU grace period to ensure all users observe the
>   current private hash. This can be skipped if a grace period elapsed
>   since the private hash was assigned.
> 
> - futex_private_hash::state is set to FR_ATOMIC, forcing all users to
>   use mm_struct::futex_atomic for reference counting.
> 
> - After a RCU grace period, all users are guaranteed to be using the
>   atomic counter. The per-CPU counters can now be summed up and added to
>   the atomic_t counter. If the resulting count is zero, the hash can be
>   safely replaced. Otherwise, active users still hold a valid reference.
> 
> - Once the atomic reference count drops to zero, the next futex
>   operation will switch to the new private hash.
> 
> call_rcu_hurry() is used to speed up transition which otherwise might be
> delay with RCU_LAZY. There is nothing wrong with using call_rcu(). The
> side effects would be that on auto scaling the new hash is used later
> and the SET_SLOTS prctl() will block longer.
> 
> [bigeasy: commit description + mm get/ put_async]

kmemleak complains about a new memleak with this commit:

[  680.179004][  T101] kmemleak: 1 new suspected memory leaks (see /sys/kernel/debug/kmemleak)

$ cat /sys/kernel/debug/kmemleak
unreferenced object (percpu) 0xc22ec0eface8 (size 4):
  comm "swapper/0", pid 1, jiffies 4294893115
  hex dump (first 4 bytes on cpu 7):
    01 00 00 00                                      ....
  backtrace (crc b8bc6765):
    kmemleak_alloc_percpu+0x48/0xb8
    pcpu_alloc_noprof+0x6ac/0xb68
    futex_mm_init+0x60/0xe0
    mm_init+0x1e8/0x3c0
    mm_alloc+0x5c/0x78
    init_args+0x74/0x4b0
    debug_vm_pgtable+0x60/0x2d8
    do_one_initcall+0x128/0x3e0
    do_initcall_level+0xb4/0xe8
    do_initcalls+0x60/0xb0
    do_basic_setup+0x28/0x40
    kernel_init_freeable+0x158/0x1f8
    kernel_init+0x2c/0x1e0
    ret_from_fork+0x10/0x20

And futex_mm_init+0x60/0xe0 resolves to
    mm->futex_ref = alloc_percpu(unsigned int);
in futex_mm_init().

Reverting this commit (and patches 3 and 4 in this series due to context),
makes kmemleak happy again.

Cheers,
Andre'

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

* Re: [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-07-30 12:20   ` [PATCH v2 2/6] " André Draszik
@ 2025-07-30 19:44     ` Thomas Gleixner
  2025-08-01 14:59       ` André Draszik
  2025-08-02 13:22       ` [tip: locking/urgent] futex: Move futex cleanup to __mmdrop() tip-bot2 for Thomas Gleixner
  0 siblings, 2 replies; 25+ messages in thread
From: Thomas Gleixner @ 2025-07-30 19:44 UTC (permalink / raw)
  To: André Draszik, Sebastian Andrzej Siewior, linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Valentin Schneider, Waiman Long,
	Andrew Morton, David Hildenbrand, Liam R. Howlett,
	Lorenzo Stoakes, Michal Hocko, Mike Rapoport, Suren Baghdasaryan,
	Vlastimil Babka, linux-mm

On Wed, Jul 30 2025 at 13:20, André Draszik wrote:
> kmemleak complains about a new memleak with this commit:
>
> [  680.179004][  T101] kmemleak: 1 new suspected memory leaks (see /sys/kernel/debug/kmemleak)
>
> $ cat /sys/kernel/debug/kmemleak
> unreferenced object (percpu) 0xc22ec0eface8 (size 4):
>   comm "swapper/0", pid 1, jiffies 4294893115
>   hex dump (first 4 bytes on cpu 7):
>     01 00 00 00                                      ....
>   backtrace (crc b8bc6765):
>     kmemleak_alloc_percpu+0x48/0xb8
>     pcpu_alloc_noprof+0x6ac/0xb68
>     futex_mm_init+0x60/0xe0
>     mm_init+0x1e8/0x3c0
>     mm_alloc+0x5c/0x78
>     init_args+0x74/0x4b0
>     debug_vm_pgtable+0x60/0x2d8
>
> Reverting this commit (and patches 3 and 4 in this series due to context),
> makes kmemleak happy again.

Unsurprisingly ...

debug_vm_pgtable() allocates it via mm_alloc() -> mm->init() and then
after the selftest it invokes mmdrop(), which does not free it, as it is
only freed in __mmput().

The patch below should fix it.

Thanks,

        tglx
---
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -686,6 +686,7 @@ void __mmdrop(struct mm_struct *mm)
 	mm_pasid_drop(mm);
 	mm_destroy_cid(mm);
 	percpu_counter_destroy_many(mm->rss_stat, NR_MM_COUNTERS);
+	futex_hash_free(mm);
 
 	free_mm(mm);
 }
@@ -1133,7 +1134,6 @@ static inline void __mmput(struct mm_str
 	if (mm->binfmt)
 		module_put(mm->binfmt->module);
 	lru_gen_del_mm(mm);
-	futex_hash_free(mm);
 	mmdrop(mm);
 }
 

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

* Re: [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-07-30 19:44     ` Thomas Gleixner
@ 2025-08-01 14:59       ` André Draszik
  2025-08-02 13:22       ` [tip: locking/urgent] futex: Move futex cleanup to __mmdrop() tip-bot2 for Thomas Gleixner
  1 sibling, 0 replies; 25+ messages in thread
From: André Draszik @ 2025-08-01 14:59 UTC (permalink / raw)
  To: Thomas Gleixner, Sebastian Andrzej Siewior, linux-kernel
  Cc: André Almeida, Darren Hart, Davidlohr Bueso, Ingo Molnar,
	Juri Lelli, Peter Zijlstra, Valentin Schneider, Waiman Long,
	Andrew Morton, David Hildenbrand, Liam R. Howlett,
	Lorenzo Stoakes, Michal Hocko, Mike Rapoport, Suren Baghdasaryan,
	Vlastimil Babka, linux-mm

On Wed, 2025-07-30 at 21:44 +0200, Thomas Gleixner wrote:
> On Wed, Jul 30 2025 at 13:20, André Draszik wrote:
> > kmemleak complains about a new memleak with this commit:
> > 
> > [  680.179004][  T101] kmemleak: 1 new suspected memory leaks (see /sys/kernel/debug/kmemleak)
> > 
> > $ cat /sys/kernel/debug/kmemleak
> > unreferenced object (percpu) 0xc22ec0eface8 (size 4):
> >   comm "swapper/0", pid 1, jiffies 4294893115
> >   hex dump (first 4 bytes on cpu 7):
> >     01 00 00 00                                      ....
> >   backtrace (crc b8bc6765):
> >     kmemleak_alloc_percpu+0x48/0xb8
> >     pcpu_alloc_noprof+0x6ac/0xb68
> >     futex_mm_init+0x60/0xe0
> >     mm_init+0x1e8/0x3c0
> >     mm_alloc+0x5c/0x78
> >     init_args+0x74/0x4b0
> >     debug_vm_pgtable+0x60/0x2d8
> > 
> > Reverting this commit (and patches 3 and 4 in this series due to context),
> > makes kmemleak happy again.
> 
> Unsurprisingly ...
> 
> debug_vm_pgtable() allocates it via mm_alloc() -> mm->init() and then
> after the selftest it invokes mmdrop(), which does not free it, as it is
> only freed in __mmput().
> 
> The patch below should fix it.

It does. Thanks Thomas!

A.

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

* [tip: locking/urgent] futex: Move futex cleanup to __mmdrop()
  2025-07-30 19:44     ` Thomas Gleixner
  2025-08-01 14:59       ` André Draszik
@ 2025-08-02 13:22       ` tip-bot2 for Thomas Gleixner
  2025-08-21 17:39         ` Breno Leitao
  1 sibling, 1 reply; 25+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2025-08-02 13:22 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: andre.draszik, Thomas Gleixner, x86, linux-kernel

The following commit has been merged into the locking/urgent branch of tip:

Commit-ID:     e703b7e247503b8bf87b62c02a4392749b09eca8
Gitweb:        https://git.kernel.org/tip/e703b7e247503b8bf87b62c02a4392749b09eca8
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Wed, 30 Jul 2025 21:44:55 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sat, 02 Aug 2025 15:11:52 +02:00

futex: Move futex cleanup to __mmdrop()

Futex hash allocations are done in mm_init() and the cleanup happens in
__mmput(). That works most of the time, but there are mm instances which
are instantiated via mm_alloc() and freed via mmdrop(), which causes the
futex hash to be leaked.

Move the cleanup to __mmdrop().

Fixes: 56180dd20c19 ("futex: Use RCU-based per-CPU reference counting instead of rcuref_t")
Reported-by: André Draszik <andre.draszik@linaro.org>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Tested-by: André Draszik <andre.draszik@linaro.org>
Link: https://lore.kernel.org/all/87ldo5ihu0.ffs@tglx
Closes: https://lore.kernel.org/all/0c8cc83bb73abf080faf584f319008b67d0931db.camel@linaro.org

---
 kernel/fork.c | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/kernel/fork.c b/kernel/fork.c
index f82b77e..1b0535e 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -686,6 +686,7 @@ void __mmdrop(struct mm_struct *mm)
 	mm_pasid_drop(mm);
 	mm_destroy_cid(mm);
 	percpu_counter_destroy_many(mm->rss_stat, NR_MM_COUNTERS);
+	futex_hash_free(mm);
 
 	free_mm(mm);
 }
@@ -1133,7 +1134,6 @@ static inline void __mmput(struct mm_struct *mm)
 	if (mm->binfmt)
 		module_put(mm->binfmt->module);
 	lru_gen_del_mm(mm);
-	futex_hash_free(mm);
 	mmdrop(mm);
 }
 

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

* Re: [tip: locking/futex] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Peter Zijlstra
@ 2025-08-16  2:38     ` Sean Christopherson
  2025-08-26  0:38       ` Sean Christopherson
  0 siblings, 1 reply; 25+ messages in thread
From: Sean Christopherson @ 2025-08-16  2:38 UTC (permalink / raw)
  To: linux-kernel
  Cc: linux-tip-commits, Peter Zijlstra (Intel),
	Sebastian Andrzej Siewior, x86

On Fri, Jul 11, 2025, tip-bot2 for Peter Zijlstra wrote:
> The following commit has been merged into the locking/futex branch of tip:
> 
> Commit-ID:     56180dd20c19e5b0fa34822997a9ac66b517e7b3
> Gitweb:        https://git.kernel.org/tip/56180dd20c19e5b0fa34822997a9ac66b517e7b3
> Author:        Peter Zijlstra <peterz@infradead.org>
> AuthorDate:    Thu, 10 Jul 2025 13:00:07 +02:00
> Committer:     Peter Zijlstra <peterz@infradead.org>
> CommitterDate: Fri, 11 Jul 2025 16:02:00 +02:00
> 
> futex: Use RCU-based per-CPU reference counting instead of rcuref_t
> 
> The use of rcuref_t for reference counting introduces a performance bottleneck
> when accessed concurrently by multiple threads during futex operations.
> 
> Replace rcuref_t with special crafted per-CPU reference counters. The
> lifetime logic remains the same.
> 
> The newly allocate private hash starts in FR_PERCPU state. In this state, each
> futex operation that requires the private hash uses a per-CPU counter (an
> unsigned int) for incrementing or decrementing the reference count.
> 
> When the private hash is about to be replaced, the per-CPU counters are
> migrated to a atomic_t counter mm_struct::futex_atomic.
> The migration process:
> - Waiting for one RCU grace period to ensure all users observe the
>   current private hash. This can be skipped if a grace period elapsed
>   since the private hash was assigned.
> 
> - futex_private_hash::state is set to FR_ATOMIC, forcing all users to
>   use mm_struct::futex_atomic for reference counting.
> 
> - After a RCU grace period, all users are guaranteed to be using the
>   atomic counter. The per-CPU counters can now be summed up and added to
>   the atomic_t counter. If the resulting count is zero, the hash can be
>   safely replaced. Otherwise, active users still hold a valid reference.
> 
> - Once the atomic reference count drops to zero, the next futex
>   operation will switch to the new private hash.
> 
> call_rcu_hurry() is used to speed up transition which otherwise might be
> delay with RCU_LAZY. There is nothing wrong with using call_rcu(). The
> side effects would be that on auto scaling the new hash is used later
> and the SET_SLOTS prctl() will block longer.
> 
> [bigeasy: commit description + mm get/ put_async]
> 
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> Link: https://lore.kernel.org/r/20250710110011.384614-3-bigeasy@linutronix.de
> ---

This is causing explosions on my test systems, in code that doesn't obviously
have anything to do with futex.

The most common symptom is a #GP on this code in try_to_wake_up():

		/* Link @node into the waitqueue. */
		WRITE_ONCE(prev->next, node);

although on systems with 5-level paging I _think_ it just manifests as hard
hanges (I assume because prev->next is corrupted, but is still canonical with
LA57?  But that's a wild guess).

The failure always occurs when userspace writes /sys/module/kvm/parameters/nx_huge_pages,
but I don't think there's anything KVM specific about the issue.  Simply writing
the param doesn't explode, the problem only arises when I'm running tests in
parallel (but then failure is almost immediate), so presumably there's a task
migration angle or something?

Manually disabling CONFIG_FUTEX_PRIVATE_HASH makes the problem go away, and
running with CONFIG_FUTEX_PRIVATE_HASH=y prior to this rework is also fine.  So
it appears that the problem is specifically in the new code.

I can provide more info as needed next week.

Oops: general protection fault, probably for non-canonical address 0xff0e899fa1566052: 0000 [#1] SMP
CPU: 51 UID: 0 PID: 53807 Comm: tee Tainted: G S         O        6.17.0-smp--38183c31756a-next #826 NONE 
Tainted: [S]=CPU_OUT_OF_SPEC, [O]=OOT_MODULE
Hardware name: Google LLC Indus/Indus_QC_03, BIOS 30.110.0 09/13/2024
RIP: 0010:queued_spin_lock_slowpath+0x123/0x250
Code: d8 c1 e8 10 66 87 47 02 66 85 c0 74 40 0f b7 c0 89 c2 83 e2 03 c1 e2 04 83 e0 fc 48 c7 c6 f8 ff ff ff 48 8b 84 46 a0 e2 a3 a0 <48> 89 8c 02 c0 da 47 a2 83 79 08 00 75 08 f3 90 83 79 08 00 74 f8
RSP: 0018:ffffbf55cffe7cf8 EFLAGS: 00010006
RAX: ff0e899fff0e8562 RBX: 0000000000d00000 RCX: ffffa39b40aefac0
RDX: 0000000000000030 RSI: fffffffffffffff8 RDI: ffffa39d0592e68c
RBP: 0000000000d00000 R08: 00000000ffffff80 R09: 0000000400000000
R10: ffffa36cce4fe401 R11: 0000000000000800 R12: 0000000000000003
R13: 0000000000000000 R14: ffffa39d0592e68c R15: ffffa39b9e672000
FS:  00007f233b2e9740(0000) GS:ffffa39b9e672000(0000) knlGS:0000000000000000
CS:  0010 DS: 0000 ES: 0000 CR0: 0000000080050033
CR2: 00007f233b39fda0 CR3: 00000004d031f002 CR4: 00000000007726f0
PKRU: 55555554
Call Trace:
 <TASK>
 _raw_spin_lock_irqsave+0x50/0x60
 try_to_wake_up+0x4f/0x5d0
 set_nx_huge_pages+0xe4/0x1c0 [kvm]
 param_attr_store+0x89/0xf0
 module_attr_store+0x1e/0x30
 kernfs_fop_write_iter+0xe4/0x160
 vfs_write+0x2cb/0x420
 ksys_write+0x7f/0xf0
 do_syscall_64+0x6f/0x1f0
 ? arch_exit_to_user_mode_prepare+0x9/0x50
 entry_SYSCALL_64_after_hwframe+0x4b/0x53
RIP: 0033:0x7f233b4178b3
Code: cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc cc 48 8b 05 99 91 07 00 83 38 00 75 10 b8 01 00 00 00 0f 05 <48> 3d 01 f0 ff ff 73 4d c3 55 48 89 e5 41 57 41 56 53 50 48 89 d3
RSP: 002b:00007ffffbdbf3b8 EFLAGS: 00000246 ORIG_RAX: 0000000000000001
RAX: ffffffffffffffda RBX: 0000000000000002 RCX: 00007f233b4178b3
RDX: 0000000000000002 RSI: 00007ffffbdbf4a0 RDI: 0000000000000005
RBP: 00007ffffbdbf3e0 R08: 0000000000001004 R09: 0000000000000000
R10: 00000000000001b6 R11: 0000000000000246 R12: 00007ffffbdbf4a0
R13: 0000000000000002 R14: 00000000226ff3d0 R15: 0000000000000002
 </TASK>
Modules linked in: kvm_intel kvm irqbypass vfat fat i2c_mux_pca954x i2c_mux spidev cdc_acm xhci_pci xhci_hcd gq(O) sha3_generic
gsmi: Log Shutdown Reason 0x03
---[ end trace 0000000000000000 ]---
RIP: 0010:queued_spin_lock_slowpath+0x123/0x250
Code: d8 c1 e8 10 66 87 47 02 66 85 c0 74 40 0f b7 c0 89 c2 83 e2 03 c1 e2 04 83 e0 fc 48 c7 c6 f8 ff ff ff 48 8b 84 46 a0 e2 a3 a0 <48> 89 8c 02 c0 da 47 a2 83 79 08 00 75 08 f3 90 83 79 08 00 74 f8
RSP: 0018:ffffbf55cffe7cf8 EFLAGS: 00010006
RAX: ff0e899fff0e8562 RBX: 0000000000d00000 RCX: ffffa39b40aefac0
RDX: 0000000000000030 RSI: fffffffffffffff8 RDI: ffffa39d0592e68c
RBP: 0000000000d00000 R08: 00000000ffffff80 R09: 0000000400000000
R10: ffffa36cce4fe401 R11: 0000000000000800 R12: 0000000000000003
R13: 0000000000000000 R14: ffffa39d0592e68c R15: ffffa39b9e672000
FS:  00007f233b2e9740(0000) GS:ffffa39b9e672000(0000) knlGS:0000000000000000
CS:  0010 DS: 0000 ES: 0000 CR0: 0000000080050033
CR2: 00007f233b39fda0 CR3: 00000004d031f002 CR4: 00000000007726f0
PKRU: 55555554
Kernel panic - not syncing: Fatal exception
Kernel Offset: 0x1e600000 from 0xffffffff81000000 (relocation range: 0xffffffff80000000-0xffffffffbfffffff)
gsmi: Log Shutdown Reason 0x02
Rebooting in 10 seconds..


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

* Re: [tip: locking/urgent] futex: Move futex cleanup to __mmdrop()
  2025-08-02 13:22       ` [tip: locking/urgent] futex: Move futex cleanup to __mmdrop() tip-bot2 for Thomas Gleixner
@ 2025-08-21 17:39         ` Breno Leitao
  0 siblings, 0 replies; 25+ messages in thread
From: Breno Leitao @ 2025-08-21 17:39 UTC (permalink / raw)
  To: tglx; +Cc: linux-tip-commits, andre.draszik, Thomas Gleixner, x86,
	linux-kernel

On Sat, Aug 02, 2025 at 01:22:10PM +0000, tip-bot2 for Thomas Gleixner wrote:
> The following commit has been merged into the locking/urgent branch of tip:
> 
> Commit-ID:     e703b7e247503b8bf87b62c02a4392749b09eca8
> Gitweb:        https://git.kernel.org/tip/e703b7e247503b8bf87b62c02a4392749b09eca8
> Author:        Thomas Gleixner <tglx@linutronix.de>
> AuthorDate:    Wed, 30 Jul 2025 21:44:55 +02:00
> Committer:     Thomas Gleixner <tglx@linutronix.de>
> CommitterDate: Sat, 02 Aug 2025 15:11:52 +02:00
> 
> futex: Move futex cleanup to __mmdrop()
> 
> Futex hash allocations are done in mm_init() and the cleanup happens in
> __mmput(). That works most of the time, but there are mm instances which
> are instantiated via mm_alloc() and freed via mmdrop(), which causes the
> futex hash to be leaked.
> 
> Move the cleanup to __mmdrop().
> 
> Fixes: 56180dd20c19 ("futex: Use RCU-based per-CPU reference counting instead of rcuref_t")
> Reported-by: André Draszik <andre.draszik@linaro.org>
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> Tested-by: André Draszik <andre.draszik@linaro.org>
> Link: https://lore.kernel.org/all/87ldo5ihu0.ffs@tglx
> Closes: https://lore.kernel.org/all/0c8cc83bb73abf080faf584f319008b67d0931db.camel@linaro.org

Thomas,

it seems this change caused the bug being reported here:

https://lore.kernel.org/all/20250818131902.5039-1-hdanton@sina.com/

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

* Re: [tip: locking/futex] futex: Use RCU-based per-CPU reference counting instead of rcuref_t
  2025-08-16  2:38     ` Sean Christopherson
@ 2025-08-26  0:38       ` Sean Christopherson
  0 siblings, 0 replies; 25+ messages in thread
From: Sean Christopherson @ 2025-08-26  0:38 UTC (permalink / raw)
  To: linux-kernel
  Cc: linux-tip-commits, Peter Zijlstra (Intel),
	Sebastian Andrzej Siewior, x86

On Fri, Aug 15, 2025, Sean Christopherson wrote:
> On Fri, Jul 11, 2025, tip-bot2 for Peter Zijlstra wrote:
> > The following commit has been merged into the locking/futex branch of tip:
> > 
> > Commit-ID:     56180dd20c19e5b0fa34822997a9ac66b517e7b3
> > Gitweb:        https://git.kernel.org/tip/56180dd20c19e5b0fa34822997a9ac66b517e7b3
> > Author:        Peter Zijlstra <peterz@infradead.org>
> > AuthorDate:    Thu, 10 Jul 2025 13:00:07 +02:00
> > Committer:     Peter Zijlstra <peterz@infradead.org>
> > CommitterDate: Fri, 11 Jul 2025 16:02:00 +02:00
> > 
> > futex: Use RCU-based per-CPU reference counting instead of rcuref_t
> > 
> > The use of rcuref_t for reference counting introduces a performance bottleneck
> > when accessed concurrently by multiple threads during futex operations.
> > 
> > Replace rcuref_t with special crafted per-CPU reference counters. The
> > lifetime logic remains the same.
> > 
> > The newly allocate private hash starts in FR_PERCPU state. In this state, each
> > futex operation that requires the private hash uses a per-CPU counter (an
> > unsigned int) for incrementing or decrementing the reference count.
> > 
> > When the private hash is about to be replaced, the per-CPU counters are
> > migrated to a atomic_t counter mm_struct::futex_atomic.
> > The migration process:
> > - Waiting for one RCU grace period to ensure all users observe the
> >   current private hash. This can be skipped if a grace period elapsed
> >   since the private hash was assigned.
> > 
> > - futex_private_hash::state is set to FR_ATOMIC, forcing all users to
> >   use mm_struct::futex_atomic for reference counting.
> > 
> > - After a RCU grace period, all users are guaranteed to be using the
> >   atomic counter. The per-CPU counters can now be summed up and added to
> >   the atomic_t counter. If the resulting count is zero, the hash can be
> >   safely replaced. Otherwise, active users still hold a valid reference.
> > 
> > - Once the atomic reference count drops to zero, the next futex
> >   operation will switch to the new private hash.
> > 
> > call_rcu_hurry() is used to speed up transition which otherwise might be
> > delay with RCU_LAZY. There is nothing wrong with using call_rcu(). The
> > side effects would be that on auto scaling the new hash is used later
> > and the SET_SLOTS prctl() will block longer.
> > 
> > [bigeasy: commit description + mm get/ put_async]
> > 
> > Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> > Signed-off-by: Sebastian Andrzej Siewior <bigeasy@linutronix.de>
> > Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> > Link: https://lore.kernel.org/r/20250710110011.384614-3-bigeasy@linutronix.de
> > ---
> 
> This is causing explosions on my test systems, in code that doesn't obviously
> have anything to do with futex.

Closing the loop, this turned out to be a KVM bug[*].  Why the futex changes
exposed the bug and caused explosions, I have no idea, but nothing suggests that
this patch is buggy.

[*] https://lore.kernel.org/all/20250825160406.ZVcVPStz@linutronix.de


> The most common symptom is a #GP on this code in try_to_wake_up():
> 
> 		/* Link @node into the waitqueue. */
> 		WRITE_ONCE(prev->next, node);
> 
> although on systems with 5-level paging I _think_ it just manifests as hard
> hanges (I assume because prev->next is corrupted, but is still canonical with
> LA57?  But that's a wild guess).
> 
> The failure always occurs when userspace writes /sys/module/kvm/parameters/nx_huge_pages,
> but I don't think there's anything KVM specific about the issue.  Simply writing
> the param doesn't explode, the problem only arises when I'm running tests in
> parallel (but then failure is almost immediate), so presumably there's a task
> migration angle or something?
> 
> Manually disabling CONFIG_FUTEX_PRIVATE_HASH makes the problem go away, and
> running with CONFIG_FUTEX_PRIVATE_HASH=y prior to this rework is also fine.  So
> it appears that the problem is specifically in the new code.
> 
> I can provide more info as needed next week.
> 
> Oops: general protection fault, probably for non-canonical address 0xff0e899fa1566052: 0000 [#1] SMP

...

> Call Trace:
>  <TASK>
>  _raw_spin_lock_irqsave+0x50/0x60
>  try_to_wake_up+0x4f/0x5d0
>  set_nx_huge_pages+0xe4/0x1c0 [kvm]
>  param_attr_store+0x89/0xf0
>  module_attr_store+0x1e/0x30
>  kernfs_fop_write_iter+0xe4/0x160
>  vfs_write+0x2cb/0x420
>  ksys_write+0x7f/0xf0
>  do_syscall_64+0x6f/0x1f0
>  ? arch_exit_to_user_mode_prepare+0x9/0x50
>  entry_SYSCALL_64_after_hwframe+0x4b/0x53

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

end of thread, other threads:[~2025-08-26  0:38 UTC | newest]

Thread overview: 25+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2025-07-10 11:00 [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Sebastian Andrzej Siewior
2025-07-10 11:00 ` [PATCH v2 1/6] selftests/futex: Adapt the private hash test to RCU related changes Sebastian Andrzej Siewior
2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
2025-07-10 11:00 ` [PATCH v2 2/6] futex: Use RCU-based per-CPU reference counting instead of rcuref_t Sebastian Andrzej Siewior
2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Peter Zijlstra
2025-08-16  2:38     ` Sean Christopherson
2025-08-26  0:38       ` Sean Christopherson
2025-07-30 12:20   ` [PATCH v2 2/6] " André Draszik
2025-07-30 19:44     ` Thomas Gleixner
2025-08-01 14:59       ` André Draszik
2025-08-02 13:22       ` [tip: locking/urgent] futex: Move futex cleanup to __mmdrop() tip-bot2 for Thomas Gleixner
2025-08-21 17:39         ` Breno Leitao
2025-07-10 11:00 ` [PATCH v2 3/6] futex: Make futex_private_hash_get() static Sebastian Andrzej Siewior
2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
2025-07-10 11:00 ` [PATCH v2 4/6] futex: Remove support for IMMUTABLE Sebastian Andrzej Siewior
2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
2025-07-10 11:00 ` [PATCH v2 5/6] selftests/futex: " Sebastian Andrzej Siewior
2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
2025-07-10 11:00 ` [PATCH v2 6/6] perf bench futex: " Sebastian Andrzej Siewior
2025-07-11 18:33   ` [tip: locking/futex] " tip-bot2 for Sebastian Andrzej Siewior
2025-07-15 15:59 ` [PATCH v2 0/6] futex: Use RCU-based per-CPU reference counting Shrikanth Hegde
2025-07-15 16:31   ` Sebastian Andrzej Siewior
2025-07-15 17:04     ` Shrikanth Hegde
2025-07-16 14:29       ` Peter Zijlstra
2025-07-16 18:21         ` Shrikanth Hegde

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