All of lore.kernel.org
 help / color / mirror / Atom feed
* [patch 00/20] posix-timers: Fixes and cleanups
@ 2023-04-25 18:48 Thomas Gleixner
  2023-04-25 18:48 ` [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete() Thomas Gleixner
                   ` (20 more replies)
  0 siblings, 21 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:48 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Hi!

A recent syzcaller/KCSAN report about a data race and an actually broken
loop termination due to that race made me look deeper into the posix timer
code.

Aside of the loop termination issue this unearthed another issue in the
exit() path where timer deletion might livelock on RT enabled kernels.

While going through the code with a fine comb, I stumbled over another
place which needs READ/WRITE_ONCE() annotations, tons of outdated and/or
uncomprehensible comments and a bunch of silly code.

The series has therefore a larger set of cleanups on top of the two fixes.

It is based on

   git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git timers/core

and also availble from git:

   git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git timers/posix

Thanks,

	tglx
---
 include/linux/sched/signal.h |    2 
 kernel/time/posix-timers.c   |  523 ++++++++++++++++++++++++++-----------------
 2 files changed, 321 insertions(+), 204 deletions(-)



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

* [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
@ 2023-04-25 18:48 ` Thomas Gleixner
  2023-05-04 17:06   ` Frederic Weisbecker
  2023-04-25 18:48 ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Thomas Gleixner
                   ` (19 subsequent siblings)
  20 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:48 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

itimer_delete() has a retry loop when the timer is concurrently expired. On
non-RT kernels this just spin-waits until the timer callback has
completed. On RT kernels this is a potential livelock when the exiting task
preempted the hrtimer soft interrupt.

This only affects hrtimer based timers as Posix CPU timers cannot be
concurrently expired. For CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this is
obviously impossible as the task cannot run task work and exit at the same
time. The CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n (only non-RT) is prevented
because interrupts are disabled.

Replace spin_unlock() with an invocation of timer_wait_running() to handle
it the same way as the other retry loops in the posix timer code.

Fixes: ec8f954a40da ("posix-timers: Use a callback for cancel synchronization on PREEMPT_RT")
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Cc: Sebastian Siewior <bigeasy@linutronix.de>
---
 kernel/time/posix-timers.c |   50 +++++++++++++++++++++++++++++++++++++--------
 1 file changed, 42 insertions(+), 8 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1037,27 +1037,59 @@ SYSCALL_DEFINE1(timer_delete, timer_t, t
 }
 
 /*
- * return timer owned by the process, used by exit_itimers
+ * Delete a timer if it is armed, remove it from the hash and schedule it
+ * for RCU freeing.
  */
 static void itimer_delete(struct k_itimer *timer)
 {
-retry_delete:
-	spin_lock_irq(&timer->it_lock);
+	unsigned long flags;
 
+retry_delete:
+	/*
+	 * irqsave is required to make timer_wait_running() work.
+	 */
+	spin_lock_irqsave(&timer->it_lock, flags);
+
+	/*
+	 * Even if the timer is not longer accessible from other tasks
+	 * it still might be armed and queued in the underlying timer
+	 * mechanism. Worse, that timer mechanism might run the expiry
+	 * function concurrently.
+	 */
 	if (timer_delete_hook(timer) == TIMER_RETRY) {
-		spin_unlock_irq(&timer->it_lock);
+		/*
+		 * Timer is expired concurrently, prevent livelocks
+		 * and pointless spinning on RT.
+		 *
+		 * The CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y case is
+		 * irrelevant here because obviously the exiting task
+		 * cannot be expiring timer in task work concurrently.
+		 * Ditto for CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n as the
+		 * tick interrupt cannot run on this CPU because the above
+		 * spin_lock disabled interrupts.
+		 *
+		 * timer_wait_running() drops timer::it_lock, which opens
+		 * the possibility for another task to delete the timer.
+		 *
+		 * That's not possible here because this is invoked from
+		 * do_exit() only for the last thread of the thread group.
+		 * So no other task can access that timer.
+		 */
+		if (WARN_ON_ONCE(timer_wait_running(timer, &flags) != timer))
+			return;
+
 		goto retry_delete;
 	}
 	list_del(&timer->list);
 
-	spin_unlock_irq(&timer->it_lock);
+	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }
 
 /*
- * This is called by do_exit or de_thread, only when nobody else can
- * modify the signal->posix_timers list. Yet we need sighand->siglock
- * to prevent the race with /proc/pid/timers.
+ * Invoked from do_exit() when the last thread of a thread group exits.
+ * At that point no other task can access the timers of the dying
+ * task anymore.
  */
 void exit_itimers(struct task_struct *tsk)
 {
@@ -1067,10 +1099,12 @@ void exit_itimers(struct task_struct *ts
 	if (list_empty(&tsk->signal->posix_timers))
 		return;
 
+	/* Protect against concurrent read via /proc/$PID/timers */
 	spin_lock_irq(&tsk->sighand->siglock);
 	list_replace_init(&tsk->signal->posix_timers, &timers);
 	spin_unlock_irq(&tsk->sighand->siglock);
 
+	/* The timers are not longer accessible via tsk::signal */
 	while (!list_empty(&timers)) {
 		tmr = list_first_entry(&timers, struct k_itimer, list);
 		itimer_delete(tmr);


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

* [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
  2023-04-25 18:48 ` [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete() Thomas Gleixner
@ 2023-04-25 18:48 ` Thomas Gleixner
  2023-05-05 14:50   ` Frederic Weisbecker
  2023-04-25 18:49 ` [patch 03/20] posix-timers: Clarify timer_wait_running() comment Thomas Gleixner
                   ` (18 subsequent siblings)
  20 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:48 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

posix_timer_add() tries to allocate a posix timer ID by starting from the
cached ID which was stored by the last successful allocation.

This is done in a loop searching the ID space for a free slot one by
one. The loop has to terminate when the search wrapped around to the
starting point.

But that's racy vs. establishing the starting point. That is read out
lockless, which leads to the following problem:

CPU0	  	      	     	   CPU1
posix_timer_add()
  start = sig->posix_timer_id;
  lock(hash_lock);
  ...				   posix_timer_add()
  if (++sig->posix_timer_id < 0)
      			             start = sig->posix_timer_id;
     sig->posix_timer_id = 0;

So CPU1 can observe a negative start value, i.e. -1, and the loop break
never happens because the condition can never be true:

  if (sig->posix_timer_id == start)
     break;

While this is unlikely to ever turn into an endless loop as the ID space is
huge (INT_MAX), the racy read of the start value caught the attention of
KCSAN and Dmitry unearthed that incorrectness.

Rewrite it so that the start condition can never observe the negative value
and annotate the read and the write with READ_ONCE()/WRITE_ONCE().

Reported-by: syzbot+5c54bd3eb218bb595aa9@syzkaller.appspotmail.com
Reported-by: Dmitry Vyukov <dvyukov@google.com>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 include/linux/sched/signal.h |    2 +-
 kernel/time/posix-timers.c   |   30 +++++++++++++++++-------------
 2 files changed, 18 insertions(+), 14 deletions(-)

--- a/include/linux/sched/signal.h
+++ b/include/linux/sched/signal.h
@@ -135,7 +135,7 @@ struct signal_struct {
 #ifdef CONFIG_POSIX_TIMERS
 
 	/* POSIX.1b Interval Timers */
-	int			posix_timer_id;
+	unsigned int		next_posix_timer_id;
 	struct list_head	posix_timers;
 
 	/* ITIMER_REAL timer for the process */
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -140,25 +140,29 @@ static struct k_itimer *posix_timer_by_i
 static int posix_timer_add(struct k_itimer *timer)
 {
 	struct signal_struct *sig = current->signal;
-	int first_free_id = sig->posix_timer_id;
 	struct hlist_head *head;
-	int ret = -ENOENT;
+	unsigned int start, id;
 
-	do {
+	/* Can be written by a different task concurrently in the loop below */
+	start = READ_ONCE(sig->next_posix_timer_id);
+
+	for (id = ~start; start != id; id++) {
 		spin_lock(&hash_lock);
-		head = &posix_timers_hashtable[hash(sig, sig->posix_timer_id)];
-		if (!__posix_timers_find(head, sig, sig->posix_timer_id)) {
+		id = sig->next_posix_timer_id;
+
+		/* Write the next ID back. Clamp it to the positive space */
+		WRITE_ONCE(sig->next_posix_timer_id, (id + 1) & INT_MAX);
+
+		head = &posix_timers_hashtable[hash(sig, id)];
+		if (!__posix_timers_find(head, sig, id)) {
 			hlist_add_head_rcu(&timer->t_hash, head);
-			ret = sig->posix_timer_id;
+			spin_unlock(&hash_lock);
+			return id;
 		}
-		if (++sig->posix_timer_id < 0)
-			sig->posix_timer_id = 0;
-		if ((sig->posix_timer_id == first_free_id) && (ret == -ENOENT))
-			/* Loop over all possible ids completed */
-			ret = -EAGAIN;
 		spin_unlock(&hash_lock);
-	} while (ret == -ENOENT);
-	return ret;
+	}
+	/* POSIX return code when no timer ID could be allocated */
+	return -EAGAIN;
 }
 
 static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)


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

* [patch 03/20] posix-timers: Clarify timer_wait_running() comment
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
  2023-04-25 18:48 ` [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete() Thomas Gleixner
  2023-04-25 18:48 ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-05-09  9:50   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 04/20] posix-timers: Cleanup comments about timer ID tracking Thomas Gleixner
                   ` (17 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Explain it better and add the CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y aspect
for completeness.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -835,10 +835,18 @@ static void common_timer_wait_running(st
 }
 
 /*
- * On PREEMPT_RT this prevent priority inversion against softirq kthread in
- * case it gets preempted while executing a timer callback. See comments in
- * hrtimer_cancel_wait_running. For PREEMPT_RT=n this just results in a
- * cpu_relax().
+ * On PREEMPT_RT this prevents priority inversion and a potential livelock
+ * against the ksoftirqd thread in case that ksoftirqd gets preempted while
+ * executing a hrtimer callback.
+ *
+ * See the comments in hrtimer_cancel_wait_running(). For PREEMPT_RT=n this
+ * just results in a cpu_relax().
+ *
+ * For POSIX CPU timers with CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n this is
+ * just a cpu_relax(). With CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this
+ * prevents spinning on an eventually scheduled out task and a livelock
+ * when the task which tries to delete or disarm the timer has preempted
+ * the task which runs the expiry in task work context.
  */
 static struct k_itimer *timer_wait_running(struct k_itimer *timer,
 					   unsigned long *flags)


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

* [patch 04/20] posix-timers: Cleanup comments about timer ID tracking
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (2 preceding siblings ...)
  2023-04-25 18:49 ` [patch 03/20] posix-timers: Clarify timer_wait_running() comment Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-05-09  9:58   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 05/20] posix-timers: Add comments about timer lookup Thomas Gleixner
                   ` (16 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Describe the hash table properly and remove the IDR leftover comments.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   28 ++++++++--------------------
 1 file changed, 8 insertions(+), 20 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -35,20 +35,17 @@
 #include "timekeeping.h"
 #include "posix-timers.h"
 
-/*
- * Management arrays for POSIX timers. Timers are now kept in static hash table
- * with 512 entries.
- * Timer ids are allocated by local routine, which selects proper hash head by
- * key, constructed from current->signal address and per signal struct counter.
- * This keeps timer ids unique per process, but now they can intersect between
- * processes.
- */
+static struct kmem_cache *posix_timers_cache;
 
 /*
- * Lets keep our timers in a slab cache :-)
+ * Timers are managed in a hash table for lockless lookup. The hash key is
+ * constructed from current::signal and the timer ID and the timer is
+ * matched against current::signal and the timer ID when walking the hash
+ * bucket list.
+ *
+ * This allows checkpoint/restore to reconstruct the exact timer IDs for
+ * a process.
  */
-static struct kmem_cache *posix_timers_cache;
-
 static DEFINE_HASHTABLE(posix_timers_hashtable, 9);
 static DEFINE_SPINLOCK(hash_lock);
 
@@ -66,15 +63,6 @@ static const struct k_clock clock_realti
 #endif
 
 /*
- * The timer ID is turned into a timer address by idr_find().
- * Verifying a valid ID consists of:
- *
- * a) checking that idr_find() returns other than -1.
- * b) checking that the timer id matches the one in the timer itself.
- * c) that the timer owner is in the callers thread group.
- */
-
-/*
  * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
  *	    to implement others.  This structure defines the various
  *	    clocks.


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

* [patch 05/20] posix-timers: Add comments about timer lookup
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (3 preceding siblings ...)
  2023-04-25 18:49 ` [patch 04/20] posix-timers: Cleanup comments about timer ID tracking Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-05-09 10:58   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal Thomas Gleixner
                   ` (15 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Document how the timer ID validation in the hash table works.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   39 ++++++++++++++++++++++++++++++++-------
 1 file changed, 32 insertions(+), 7 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -505,6 +505,12 @@ static int do_timer_create(clockid_t whi
 		return -EAGAIN;
 
 	spin_lock_init(&new_timer->it_lock);
+
+	/*
+	 * Add the timer to the hash table. The timer is not yet valid
+	 * because new_timer::it_signal is still NULL. The timer id is also
+	 * not yet visible to user space.
+	 */
 	new_timer_id = posix_timer_add(new_timer);
 	if (new_timer_id < 0) {
 		error = new_timer_id;
@@ -550,6 +556,7 @@ static int do_timer_create(clockid_t whi
 		goto out;
 
 	spin_lock_irq(&current->sighand->siglock);
+	/* This makes the timer valid in the hash table */
 	new_timer->it_signal = current->signal;
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
@@ -596,13 +603,6 @@ COMPAT_SYSCALL_DEFINE3(timer_create, clo
 }
 #endif
 
-/*
- * Locking issues: We need to protect the result of the id look up until
- * we get the timer locked down so it is not deleted under us.  The
- * removal is done under the idr spinlock so we use that here to bridge
- * the find to the timer lock.  To avoid a dead lock, the timer id MUST
- * be release with out holding the timer lock.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags)
 {
 	struct k_itimer *timr;
@@ -614,10 +614,35 @@ static struct k_itimer *__lock_timer(tim
 	if ((unsigned long long)timer_id > INT_MAX)
 		return NULL;
 
+	/*
+	 * The hash lookup and the timers are RCU protected.
+	 *
+	 * Timers are added to the hash in invalid state where
+	 * timr::it_signal == NULL. timer::it_signal is only set after the
+	 * rest of the initialization succeeded.
+	 *
+	 * Timer destruction happens in steps:
+	 *  1) Set timr::it_signal to NULL with timr::it_lock held
+	 *  2) Release timr::it_lock
+	 *  3) Remove from the hash under hash_lock
+	 *  4) Call RCU for removal after the grace period
+	 *
+	 * Holding rcu_read_lock() accross the lookup ensures that
+	 * the timer cannot be freed.
+	 *
+	 * The lookup validates locklessly that timr::it_signal ==
+	 * current::it_signal and timr::it_id == @timer_id. timr::it_id
+	 * can't change, but timr::it_signal becomes NULL during
+	 * destruction.
+	 */
 	rcu_read_lock();
 	timr = posix_timer_by_id(timer_id);
 	if (timr) {
 		spin_lock_irqsave(&timr->it_lock, *flags);
+		/*
+		 * Validate under timr::it_lock that timr::it_signal is
+		 * still valid. Pairs with #1 above.
+		 */
 		if (timr->it_signal == current->signal) {
 			rcu_read_unlock();
 			return timr;


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

* [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (4 preceding siblings ...)
  2023-04-25 18:49 ` [patch 05/20] posix-timers: Add comments about timer lookup Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-05-09 11:04   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit() Thomas Gleixner
                   ` (14 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

k_itimer::it_signal is read lockless in the RCU protected hash lookup, but
it can be written concurrently in the timer_create() and timer_delete()
path. Annotate these places with READ_ONCE() and WRITE_ONCE()

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -109,9 +109,9 @@ static struct k_itimer *__posix_timers_f
 {
 	struct k_itimer *timer;
 
-	hlist_for_each_entry_rcu(timer, head, t_hash,
-				 lockdep_is_held(&hash_lock)) {
-		if ((timer->it_signal == sig) && (timer->it_id == id))
+	hlist_for_each_entry_rcu(timer, head, t_hash, lockdep_is_held(&hash_lock)) {
+		/* timer->it_signal can be set concurrently */
+		if ((READ_ONCE(timer->it_signal) == sig) && (timer->it_id == id))
 			return timer;
 	}
 	return NULL;
@@ -557,7 +557,7 @@ static int do_timer_create(clockid_t whi
 
 	spin_lock_irq(&current->sighand->siglock);
 	/* This makes the timer valid in the hash table */
-	new_timer->it_signal = current->signal;
+	WRITE_ONCE(new_timer->it_signal, current->signal);
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
 
@@ -1051,10 +1051,10 @@ SYSCALL_DEFINE1(timer_delete, timer_t, t
 	list_del(&timer->list);
 	spin_unlock(&current->sighand->siglock);
 	/*
-	 * This keeps any tasks waiting on the spin lock from thinking
-	 * they got something (see the lock code above).
+	 * A concurrent lookup could check timer::it_signal lockless. It
+	 * will reevaluate with timer::it_lock held and observe the NULL.
 	 */
-	timer->it_signal = NULL;
+	WRITE_ONCE(timer->it_signal, NULL);
 
 	unlock_timer(timer, flags);
 	release_posix_timer(timer, IT_ID_SET);


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

* [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit()
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (5 preceding siblings ...)
  2023-04-25 18:49 ` [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 10:09   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock Thomas Gleixner
                   ` (13 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Technically it's not required to set k_itimer::it_signal to NULL on exit()
because there is no other thread anymore which could lookup the timer
concurrently.

Set it to NULL for consistency sake and add a comment to that effect.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |    8 ++++++++
 1 file changed, 8 insertions(+)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1107,6 +1107,14 @@ static void itimer_delete(struct k_itime
 	}
 	list_del(&timer->list);
 
+	/*
+	 * Setting timer::it_signal to NULL is technically not required
+	 * here as nothing can access the timer anymore legitimately via
+	 * the hash table. Set it to NULL nevertheless so that all deletion
+	 * paths are consistent.
+	 */
+	WRITE_ONCE(timer->it_signal, NULL);
+
 	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }


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

* [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (6 preceding siblings ...)
  2023-04-25 18:49 ` [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit() Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 10:12   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 09/20] posix-timers: Split release_posix_timers() Thomas Gleixner
                   ` (12 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

All usage of hash_lock is in thread context. No point in using
spin_lock_irqsave()/irqrestore() for a single usage site.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |    5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -470,10 +470,9 @@ static void k_itimer_rcu_free(struct rcu
 static void release_posix_timer(struct k_itimer *tmr, int it_id_set)
 {
 	if (it_id_set) {
-		unsigned long flags;
-		spin_lock_irqsave(&hash_lock, flags);
+		spin_lock(&hash_lock, flags);
 		hlist_del_rcu(&tmr->t_hash);
-		spin_unlock_irqrestore(&hash_lock, flags);
+		spin_unlock(&hash_lock, flags);
 	}
 	put_pid(tmr->it_pid);
 	sigqueue_free(tmr->sigq);


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

* [patch 09/20] posix-timers: Split release_posix_timers()
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (7 preceding siblings ...)
  2023-04-25 18:49 ` [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 10:25   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 10/20] posix-timers: Document sys_clock_getres() correctly Thomas Gleixner
                   ` (11 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

release_posix_timers() is called for cleaning up both hashed and unhashed
timers. The cases are differentiated by an argument and the usage is
hideous.

Seperate the actual free path out and use it for unhashed timers. Provide a
function for hashed timers.

No functional change.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   31 +++++++++++++++----------------
 1 file changed, 15 insertions(+), 16 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -465,20 +465,21 @@ static void k_itimer_rcu_free(struct rcu
 	kmem_cache_free(posix_timers_cache, tmr);
 }
 
-#define IT_ID_SET	1
-#define IT_ID_NOT_SET	0
-static void release_posix_timer(struct k_itimer *tmr, int it_id_set)
-{
-	if (it_id_set) {
-		spin_lock(&hash_lock, flags);
-		hlist_del_rcu(&tmr->t_hash);
-		spin_unlock(&hash_lock, flags);
-	}
+static void posix_timer_free(struct k_itimer *tmr)
+{
 	put_pid(tmr->it_pid);
 	sigqueue_free(tmr->sigq);
 	call_rcu(&tmr->rcu, k_itimer_rcu_free);
 }
 
+static void posix_timer_unhash_and_free(struct k_itimer *tmr)
+{
+	spin_lock(&hash_lock);
+	hlist_del_rcu(&tmr->t_hash);
+	spin_unlock(&hash_lock);
+	posix_timer_free(tmr);
+}
+
 static int common_timer_create(struct k_itimer *new_timer)
 {
 	hrtimer_init(&new_timer->it.real.timer, new_timer->it_clock, 0);
@@ -492,7 +493,6 @@ static int do_timer_create(clockid_t whi
 	const struct k_clock *kc = clockid_to_kclock(which_clock);
 	struct k_itimer *new_timer;
 	int error, new_timer_id;
-	int it_id_set = IT_ID_NOT_SET;
 
 	if (!kc)
 		return -EINVAL;
@@ -512,11 +512,10 @@ static int do_timer_create(clockid_t whi
 	 */
 	new_timer_id = posix_timer_add(new_timer);
 	if (new_timer_id < 0) {
-		error = new_timer_id;
-		goto out;
+		posix_timer_free(new_timer);
+		return new_timer_id;
 	}
 
-	it_id_set = IT_ID_SET;
 	new_timer->it_id = (timer_t) new_timer_id;
 	new_timer->it_clock = which_clock;
 	new_timer->kclock = kc;
@@ -568,7 +567,7 @@ static int do_timer_create(clockid_t whi
 	 * new_timer after the unlock call.
 	 */
 out:
-	release_posix_timer(new_timer, it_id_set);
+	posix_timer_unhash_and_free(new_timer);
 	return error;
 }
 
@@ -1056,7 +1055,7 @@ SYSCALL_DEFINE1(timer_delete, timer_t, t
 	WRITE_ONCE(timer->it_signal, NULL);
 
 	unlock_timer(timer, flags);
-	release_posix_timer(timer, IT_ID_SET);
+	posix_timer_unhash_and_free(timer);
 	return 0;
 }
 
@@ -1115,7 +1114,7 @@ static void itimer_delete(struct k_itime
 	WRITE_ONCE(timer->it_signal, NULL);
 
 	spin_unlock_irqrestore(&timer->it_lock, flags);
-	release_posix_timer(timer, IT_ID_SET);
+	posix_timer_unhash_and_free(timer);
 }
 
 /*


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

* [patch 10/20] posix-timers: Document sys_clock_getres() correctly
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (8 preceding siblings ...)
  2023-04-25 18:49 ` [patch 09/20] posix-timers: Split release_posix_timers() Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 10:44   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 11/20] posix-timers: Document common_clock_get() correctly Thomas Gleixner
                   ` (10 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Michael Kerrisk, Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9,
	Dmitry Vyukov

The decades old comment about Posix clock resolution is confusing at best.

Remove it and add a proper explanation to sys_clock_getres().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Cc: Michael Kerrisk <mtk.manpages@gmail.com>
---
 kernel/time/posix-timers.c |   81 ++++++++++++++++++++++++++++++++++++++++-----
 1 file changed, 73 insertions(+), 8 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -67,14 +67,6 @@ static const struct k_clock clock_realti
  *	    to implement others.  This structure defines the various
  *	    clocks.
  *
- * RESOLUTION: Clock resolution is used to round up timer and interval
- *	    times, NOT to report clock times, which are reported with as
- *	    much resolution as the system can muster.  In some cases this
- *	    resolution may depend on the underlying clock hardware and
- *	    may not be quantifiable until run time, and only then is the
- *	    necessary code is written.	The standard says we should say
- *	    something about this issue in the documentation...
- *
  * FUNCTIONS: The CLOCKs structure defines possible functions to
  *	    handle various clock functions.
  *
@@ -1204,6 +1196,79 @@ SYSCALL_DEFINE2(clock_adjtime, const clo
 	return err;
 }
 
+/**
+ * sys_clock_getres - Get the resolution of a clock
+ * @which_clock:	The clock to get the resolution for
+ * @tp:			Pointer to a a user space timespec64 for storage
+ *
+ * POSIX defines:
+ *
+ * "The clock_getres() function shall return the resolution of any
+ * clock. Clock resolutions are implementation-defined and cannot be set by
+ * a process. If the argument res is not NULL, the resolution of the
+ * specified clock shall be stored in the location pointed to by res. If
+ * res is NULL, the clock resolution is not returned. If the time argument
+ * of clock_settime() is not a multiple of res, then the value is truncated
+ * to a multiple of res."
+ *
+ * Due to the various hardware constraints the real resolution can vary
+ * wildly and even change during runtime when the underlying devices are
+ * replaced. The kernel also can use hardware devices with different
+ * resolutions for reading the time and for arming timers.
+ *
+ * The kernel therefore deviates from the POSIX spec in various aspects:
+ *
+ * 1) The resolution returned to user space
+ *
+ *    For CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME, CLOCK_TAI,
+ *    CLOCK_REALTIME_ALARM, CLOCK_BOOTTIME_ALAREM and CLOCK_MONOTONIC_RAW
+ *    the kernel differentiates only two cases:
+ *
+ *    I)  Low resolution mode:
+ *
+ *	  When high resolution timers are disabled at compile or runtime
+ *	  the resolution returned is nanoseconds per tick, which represents
+ *	  the precision at which timers expire.
+ *
+ *    II) High resolution mode:
+ *
+ *	  When high resolution timers are enabled the resolution returned
+ *	  is always one nanosecond independent of the actual resolution of
+ *	  the underlying hardware devices.
+ *
+ *	  For CLOCK_*_ALARM the actual resolution depends on system
+ *	  state. When system is running the resolution is the same as the
+ *	  resolution of the other clocks. During suspend the actual
+ *	  resolution is the resolution of the underlying RTC device which
+ *	  might be way less precise than the clockevent device used during
+ *	  running state.
+ *
+ *   For CLOCK_REALTIME_COARSE and CLOCK_MONOTONIC_COARSE the resolution
+ *   returned is always nanoseconds per tick.
+ *
+ *   For CLOCK_PROCESS_CPUTIME and CLOCK_THREAD_CPUTIME the resolution
+ *   returned is always one nanosecond under the assumption that the
+ *   underlying scheduler clock has a better resolution than nanoseconds
+ *   per tick.
+ *
+ *   For dynamic POSIX clocks (PTP devices) the resolution returned is
+ *   always one nanosecond.
+ *
+ * 2) Affect on sys_clock_settime()
+ *
+ *    The kernel does not truncate the time which is handed in to
+ *    sys_clock_settime(). The kernel internal timekeeping is always using
+ *    nanoseconds precision independent of the clocksource device which is
+ *    used to read the time from. The resolution of that device only
+ *    affects the presicion of the time returned by sys_clock_gettime().
+ *
+ * Returns:
+ *	0		Success. @tp contains the resolution
+ *	-EINVAL		@which_clock is not a valid clock ID
+ *	-EFAULT		Copying the resolution to @tp faulted
+ *	-ENODEV		Dynamic POSIX clock is not backed by a device
+ *	-EOPNOTSUPP	Dynamic POSIX clock does not support getres()
+ */
 SYSCALL_DEFINE2(clock_getres, const clockid_t, which_clock,
 		struct __kernel_timespec __user *, tp)
 {


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

* [patch 11/20] posix-timers: Document common_clock_get() correctly
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (9 preceding siblings ...)
  2023-04-25 18:49 ` [patch 10/20] posix-timers: Document sys_clock_getres() correctly Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 11:00   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 12/20] posix-timers: Document sys_clock_getoverrun() Thomas Gleixner
                   ` (9 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Replace another confusing and inaccurate set of comments.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   50 +++++++++++++++++++++++++++------------------
 1 file changed, 30 insertions(+), 20 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -659,20 +659,16 @@ static s64 common_hrtimer_forward(struct
 }
 
 /*
- * Get the time remaining on a POSIX.1b interval timer.  This function
- * is ALWAYS called with spin_lock_irq on the timer, thus it must not
- * mess with irq.
+ * Get the time remaining on a POSIX.1b interval timer.
  *
- * We have a couple of messes to clean up here.  First there is the case
- * of a timer that has a requeue pending.  These timers should appear to
- * be in the timer list with an expiry as if we were to requeue them
- * now.
+ * Two issues to handle here:
  *
- * The second issue is the SIGEV_NONE timer which may be active but is
- * not really ever put in the timer list (to save system resources).
- * This timer may be expired, and if so, we will do it here.  Otherwise
- * it is the same as a requeue pending timer WRT to what we should
- * report.
+ *  1) The timer has a requeue pending. The return value must appear as
+ *     if the timer has been requeued right now.
+ *
+ *  2) The timer is a SIGEV_NONE timer. These timers are never enqueued
+ *     into the hrtimer queue and therefore never expired. Emulate expiry
+ *     here taking #1 into account.
  */
 void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 {
@@ -688,8 +684,12 @@ void common_timer_get(struct k_itimer *t
 		cur_setting->it_interval = ktime_to_timespec64(iv);
 	} else if (!timr->it_active) {
 		/*
-		 * SIGEV_NONE oneshot timers are never queued. Check them
-		 * below.
+		 * SIGEV_NONE oneshot timers are never queued and therefore
+		 * timr->it_active is always false. The check below
+		 * vs. remaining time will handle this case.
+		 *
+		 * For all other timers there is nothing to update here, so
+		 * return.
 		 */
 		if (!sig_none)
 			return;
@@ -698,18 +698,29 @@ void common_timer_get(struct k_itimer *t
 	now = kc->clock_get_ktime(timr->it_clock);
 
 	/*
-	 * When a requeue is pending or this is a SIGEV_NONE timer move the
-	 * expiry time forward by intervals, so expiry is > now.
+	 * If this is an interval timer and either has requeue pending or
+	 * is a SIGEV_NONE timer move the expiry time forward by intervals,
+	 * so expiry is > now.
 	 */
 	if (iv && (timr->it_requeue_pending & REQUEUE_PENDING || sig_none))
 		timr->it_overrun += kc->timer_forward(timr, now);
 
 	remaining = kc->timer_remaining(timr, now);
-	/* Return 0 only, when the timer is expired and not pending */
+	/*
+	 * As @now is retrieved before a possible timer_forward() and
+	 * cannot be reevaluated by the compiler @remaining is based on the
+	 * same @now value. Therefore @remaining is consistent vs. @now.
+	 *
+	 * Consequently all interval timers, i.e. @iv > 0, cannot have a
+	 * remaining time <= 0 because timer_forward() guarantees to move
+	 * them forward so that the next timer expiry is > @now.
+	 */
 	if (remaining <= 0) {
 		/*
-		 * A single shot SIGEV_NONE timer must return 0, when
-		 * it is expired !
+		 * A single shot SIGEV_NONE timer must return 0, when it is
+		 * expired! Timers which have a real signal delivery mode
+		 * must return a remaining time greater than 0 because the
+		 * signal has not yet been delivered.
 		 */
 		if (!sig_none)
 			cur_setting->it_value.tv_nsec = 1;
@@ -718,7 +729,6 @@ void common_timer_get(struct k_itimer *t
 	}
 }
 
-/* Get the time remaining on a POSIX.1b interval timer. */
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
 	struct k_itimer *timr;


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

* [patch 12/20] posix-timers: Document sys_clock_getoverrun()
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (10 preceding siblings ...)
  2023-04-25 18:49 ` [patch 11/20] posix-timers: Document common_clock_get() correctly Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 11:06   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place Thomas Gleixner
                   ` (8 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Michael Kerrisk, Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9,
	Dmitry Vyukov

Document the syscall in detail and with coherent sentences.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Cc: Michael Kerrisk <mtk.manpages@gmail.com>
---
 kernel/time/posix-timers.c |   25 +++++++++++++++++--------
 1 file changed, 17 insertions(+), 8 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -782,14 +782,23 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t
 
 #endif
 
-/*
- * Get the number of overruns of a POSIX.1b interval timer.  This is to
- * be the overrun of the timer last delivered.  At the same time we are
- * accumulating overruns on the next timer.  The overrun is frozen when
- * the signal is delivered, either at the notify time (if the info block
- * is not queued) or at the actual delivery time (as we are informed by
- * the call back to posixtimer_rearm().  So all we need to do is
- * to pick up the frozen overrun.
+/**
+ * sys_timer_getoverrun - Get the number of overruns of a POSIX.1b interval timer
+ * @timer_id:	The timer ID which identifies the timer
+ *
+ * The "overrun count" of a timer is one plus the number of expiration
+ * intervals which have elapsed between the first expiry, which queues the
+ * signal and the actual signal delivery. On signal delivery the "overrun
+ * count" is calculated and cached, so it can be returned directly here.
+ *
+ * As this is relative to the last queued signal the returned overrun count
+ * is meaningless outside of the signal delivery path and even there it
+ * does not accurately reflect the current state when user space evaluates
+ * it.
+ *
+ * Returns:
+ *	-EINVAL		@timer_id is invalid
+ *	1..INT_MAX	The number of overruns related to the last delivered signal
  */
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {


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

* [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (11 preceding siblings ...)
  2023-04-25 18:49 ` [patch 12/20] posix-timers: Document sys_clock_getoverrun() Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 11:22   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 14/20] posix-timers: Document nanosleep() details Thomas Gleixner
                   ` (7 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

The documentation of sys_clock_settime() permissions is at a random place
and mostly word salad.

Remove it and add a concise comment into sys_clock_settime().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -74,13 +74,6 @@ static const struct k_clock clock_realti
  *	    following: 1.) The k_itimer struct (sched.h) is used for
  *	    the timer.  2.) The list, it_lock, it_clock, it_id and
  *	    it_pid fields are not modified by timer code.
- *
- * Permissions: It is assumed that the clock_settime() function defined
- *	    for each clock will take care of permission checks.	 Some
- *	    clocks may be set able by any user (i.e. local process
- *	    clocks) others not.	 Currently the only set able clock we
- *	    have is CLOCK_REALTIME and its high res counter part, both of
- *	    which we beg off on and pass to do_sys_settimeofday().
  */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
@@ -1165,6 +1158,10 @@ SYSCALL_DEFINE2(clock_settime, const clo
 	if (get_timespec64(&new_tp, tp))
 		return -EFAULT;
 
+	/*
+	 * Permission checks have to be done inside the clock specific
+	 * setter callback.
+	 */
 	return kc->clock_set(which_clock, &new_tp);
 }
 


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

* [patch 14/20] posix-timers: Document nanosleep() details
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (12 preceding siblings ...)
  2023-04-25 18:49 ` [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 12:30   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 15/20] posix-timers: Add proper comments in do_timer_create() Thomas Gleixner
                   ` (6 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

The descriptions for common_nsleep() is wrong and common_nsleep_timens()
lacks any form of comment.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |    9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1376,7 +1376,7 @@ SYSCALL_DEFINE2(clock_getres_time32, clo
 #endif
 
 /*
- * nanosleep for monotonic and realtime clocks
+ * sys_clock_nanosleep() for CLOCK_REALTIME and CLOCK_TAI
  */
 static int common_nsleep(const clockid_t which_clock, int flags,
 			 const struct timespec64 *rqtp)
@@ -1388,8 +1388,13 @@ static int common_nsleep(const clockid_t
 				 which_clock);
 }
 
+/*
+ * sys_clock_nanosleep() for CLOCK_MONOTONIC and CLOCK_BOOTTIME
+ *
+ * Absolute nanosleeps for these clocks are time-namespace adjusted.
+ */
 static int common_nsleep_timens(const clockid_t which_clock, int flags,
-			 const struct timespec64 *rqtp)
+				const struct timespec64 *rqtp)
 {
 	ktime_t texp = timespec64_to_ktime(*rqtp);
 


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

* [patch 15/20] posix-timers: Add proper comments in do_timer_create()
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (13 preceding siblings ...)
  2023-04-25 18:49 ` [patch 14/20] posix-timers: Document nanosleep() details Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 12:43   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly Thomas Gleixner
                   ` (5 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

The comment about timer lifetime at the end of the function is misplaced
and uncomprehensible.

Make it understandable and put it at the right place. Add a new comment
about the visibility of the new timer ID to user space.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -528,12 +528,17 @@ static int do_timer_create(clockid_t whi
 	new_timer->sigq->info.si_tid   = new_timer->it_id;
 	new_timer->sigq->info.si_code  = SI_TIMER;
 
-	if (copy_to_user(created_timer_id,
-			 &new_timer_id, sizeof (new_timer_id))) {
+	if (copy_to_user(created_timer_id, &new_timer_id, sizeof (new_timer_id))) {
 		error = -EFAULT;
 		goto out;
 	}
-
+	/*
+	 * After succesful copy out, the timer ID is visible to user space
+	 * now but not yet valid because new_timer::signal is still NULL.
+	 *
+	 * Complete the initialization with the clock specific create
+	 * callback.
+	 */
 	error = kc->timer_create(new_timer);
 	if (error)
 		goto out;
@@ -543,14 +548,11 @@ static int do_timer_create(clockid_t whi
 	WRITE_ONCE(new_timer->it_signal, current->signal);
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
-
-	return 0;
 	/*
-	 * In the case of the timer belonging to another task, after
-	 * the task is unlocked, the timer is owned by the other task
-	 * and may cease to exist at any time.  Don't use or modify
-	 * new_timer after the unlock call.
+	 * After unlocking sighand::siglock @new_timer is subject to
+	 * concurrent removal and cannot be touched anymore
 	 */
+	return 0;
 out:
 	posix_timer_unhash_and_free(new_timer);
 	return error;


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

* [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (14 preceding siblings ...)
  2023-04-25 18:49 ` [patch 15/20] posix-timers: Add proper comments in do_timer_create() Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 12:47   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment Thomas Gleixner
                   ` (4 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Replace the word salad.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |    7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -53,12 +53,9 @@ static const struct k_clock * const posi
 static const struct k_clock *clockid_to_kclock(const clockid_t id);
 static const struct k_clock clock_realtime, clock_monotonic;
 
-/*
- * we assume that the new SIGEV_THREAD_ID shares no bits with the other
- * SIGEV values.  Here we put out an error if this assumption fails.
- */
+/* SIGEV_THREAD_ID cannot share a bit with the other SIGEV values. */
 #if SIGEV_THREAD_ID != (SIGEV_THREAD_ID & \
-                       ~(SIGEV_SIGNAL | SIGEV_NONE | SIGEV_THREAD))
+			~(SIGEV_SIGNAL | SIGEV_NONE | SIGEV_THREAD))
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 


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

* [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (15 preceding siblings ...)
  2023-04-25 18:49 ` [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 12:52   ` Frederic Weisbecker
                     ` (2 more replies)
  2023-04-25 18:49 ` [patch 18/20] posix-timers: Clarify posix_timer_fn() comments Thomas Gleixner
                   ` (3 subsequent siblings)
  20 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Yet another incomprehensible piece of art.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   12 +++---------
 1 file changed, 3 insertions(+), 9 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -274,15 +274,9 @@ static void common_hrtimer_rearm(struct
 }
 
 /*
- * This function is exported for use by the signal deliver code.  It is
- * called just prior to the info block being released and passes that
- * block to us.  It's function is to update the overrun entry AND to
- * restart the timer.  It should only be called if the timer is to be
- * restarted (i.e. we have flagged this in the sys_private entry of the
- * info block).
- *
- * To protect against the timer going away while the interrupt is queued,
- * we require that the it_requeue_pending flag be set.
+ * This function is called from the signal delivery code if
+ * info->si_sys_private is not zero, which indicates that the timer has to
+ * be rearmed. Restart the timer and update info::si_overrun.
  */
 void posixtimer_rearm(struct kernel_siginfo *info)
 {


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

* [patch 18/20] posix-timers: Clarify posix_timer_fn() comments
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (16 preceding siblings ...)
  2023-04-25 18:49 ` [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 13:21   ` Frederic Weisbecker
  2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
                   ` (2 subsequent siblings)
  20 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Make the issues vs. SIG_IGN understandable and remove the 15 years old
promise that a proper solution is already on the horizon.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   56 +++++++++++++++++++++------------------------
 1 file changed, 27 insertions(+), 29 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -325,11 +325,11 @@ int posix_timer_event(struct k_itimer *t
 }
 
 /*
- * This function gets called when a POSIX.1b interval timer expires.  It
- * is used as a callback from the kernel internal timer.  The
- * run_timer_list code ALWAYS calls with interrupts on.
-
- * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
+ * This function gets called when a POSIX.1b interval timer expires from
+ * the HRTIMER soft interrupt with interrupts enabled.
+ *
+ * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
+ * based timers.
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
@@ -347,9 +347,10 @@ static enum hrtimer_restart posix_timer_
 
 	if (posix_timer_event(timr, si_private)) {
 		/*
-		 * signal was not sent because of sig_ignor
-		 * we will not get a call back to restart it AND
-		 * it should be restarted.
+		 * The signal was not queued due to SIG_IGN. As a
+		 * consequence the timer is not going to be rearmed from
+		 * the signal delivery path. But as a real signal handler
+		 * can be installed later the timer must be rearmed here.
 		 */
 		if (timr->it_interval != 0) {
 			ktime_t now = hrtimer_cb_get_time(timer);
@@ -358,34 +359,31 @@ static enum hrtimer_restart posix_timer_
 			 * FIXME: What we really want, is to stop this
 			 * timer completely and restart it in case the
 			 * SIG_IGN is removed. This is a non trivial
-			 * change which involves sighand locking
-			 * (sigh !), which we don't want to do late in
-			 * the release cycle.
+			 * change to the signal handling code.
+			 *
+			 * For now let timers with an interval less than a
+			 * jiffie expire every jiffie to avoid softirq
+			 * starvation in case of SIG_IGN and a very small
+			 * interval, which would put the timer right back
+			 * on the softirq pending list. Moving now ahead of
+			 * time tricks hrtimer_forward() to expire the
+			 * timer later, while it still maintains the
+			 * overrun accuracy for the price of a slightly
+			 * inconsistency in the timer_gettime() case. This
+			 * is at least better than a starved softirq.
 			 *
-			 * For now we just let timers with an interval
-			 * less than a jiffie expire every jiffie to
-			 * avoid softirq starvation in case of SIG_IGN
-			 * and a very small interval, which would put
-			 * the timer right back on the softirq pending
-			 * list. By moving now ahead of time we trick
-			 * hrtimer_forward() to expire the timer
-			 * later, while we still maintain the overrun
-			 * accuracy, but have some inconsistency in
-			 * the timer_gettime() case. This is at least
-			 * better than a starved softirq. A more
-			 * complex fix which solves also another related
-			 * inconsistency is already in the pipeline.
+			 * Only required when high resolution timers are
+			 * enabled as the periodic tick based timers are
+			 * automatically aligned to the next tick.
 			 */
-#ifdef CONFIG_HIGH_RES_TIMERS
-			{
+			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
 				ktime_t kj = NSEC_PER_SEC / HZ;
 
 				if (timr->it_interval < kj)
 					now = ktime_add(now, kj);
 			}
-#endif
-			timr->it_overrun += hrtimer_forward(timer, now,
-							    timr->it_interval);
+
+			timr->it_overrun += hrtimer_forward(timer, now, timr->it_interval);
 			ret = HRTIMER_RESTART;
 			++timr->it_requeue_pending;
 			timr->it_active = 1;


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

* [patch 19/20] posix-timers: Remove pointless comments
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (17 preceding siblings ...)
  2023-04-25 18:49 ` [patch 18/20] posix-timers: Clarify posix_timer_fn() comments Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 13:48   ` Frederic Weisbecker
                     ` (3 more replies)
  2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
  2023-06-05 14:32 ` [patch 00/20] posix-timers: Fixes and cleanups Frederic Weisbecker
  20 siblings, 4 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Documenting the obvious is just consuming space for no value.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   25 -------------------------
 1 file changed, 25 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -59,19 +59,6 @@ static const struct k_clock clock_realti
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 
-/*
- * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
- *	    to implement others.  This structure defines the various
- *	    clocks.
- *
- * FUNCTIONS: The CLOCKs structure defines possible functions to
- *	    handle various clock functions.
- *
- *	    The standard POSIX timer management code assumes the
- *	    following: 1.) The k_itimer struct (sched.h) is used for
- *	    the timer.  2.) The list, it_lock, it_clock, it_id and
- *	    it_pid fields are not modified by timer code.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
 #define lock_timer(tid, flags)						   \
@@ -140,7 +127,6 @@ static inline void unlock_timer(struct k
 	spin_unlock_irqrestore(&timr->it_lock, flags);
 }
 
-/* Get clock_realtime */
 static int posix_get_realtime_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_real_ts64(tp);
@@ -152,7 +138,6 @@ static ktime_t posix_get_realtime_ktime(
 	return ktime_get_real();
 }
 
-/* Set clock_realtime */
 static int posix_clock_realtime_set(const clockid_t which_clock,
 				    const struct timespec64 *tp)
 {
@@ -165,9 +150,6 @@ static int posix_clock_realtime_adj(cons
 	return do_adjtimex(t);
 }
 
-/*
- * Get monotonic time for posix timers
- */
 static int posix_get_monotonic_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_ts64(tp);
@@ -180,9 +162,6 @@ static ktime_t posix_get_monotonic_ktime
 	return ktime_get();
 }
 
-/*
- * Get monotonic-raw time for posix timers
- */
 static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_raw_ts64(tp);
@@ -190,7 +169,6 @@ static int posix_get_monotonic_raw(clock
 	return 0;
 }
 
-
 static int posix_get_realtime_coarse(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_coarse_real_ts64(tp);
@@ -241,9 +219,6 @@ static int posix_get_hrtimer_res(clockid
 	return 0;
 }
 
-/*
- * Initialize everything, well, just everything in Posix clocks/timers ;)
- */
 static __init int init_posix_timers(void)
 {
 	posix_timers_cache = kmem_cache_create("posix_timers_cache",


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

* [patch 20/20] posix-timers: Polish coding style in a few places
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (18 preceding siblings ...)
  2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
@ 2023-04-25 18:49 ` Thomas Gleixner
  2023-06-01 13:50   ` Frederic Weisbecker
                     ` (3 more replies)
  2023-06-05 14:32 ` [patch 00/20] posix-timers: Fixes and cleanups Frederic Weisbecker
  20 siblings, 4 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-04-25 18:49 UTC (permalink / raw)
  To: LKML
  Cc: Frederic Weisbecker, Anna-Maria Behnsen, Peter Zijlstra,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Michael Kerrisk

Make it consistent with the TIP tree documentation.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
 kernel/time/posix-timers.c |   14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -308,10 +308,10 @@ int posix_timer_event(struct k_itimer *t
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
+	enum hrtimer_restart ret = HRTIMER_NORESTART;
 	struct k_itimer *timr;
 	unsigned long flags;
 	int si_private = 0;
-	enum hrtimer_restart ret = HRTIMER_NORESTART;
 
 	timr = container_of(timer, struct k_itimer, it.real.timer);
 	spin_lock_irqsave(&timr->it_lock, flags);
@@ -395,8 +395,8 @@ static struct pid *good_sigevent(sigeven
 
 static struct k_itimer * alloc_posix_timer(void)
 {
-	struct k_itimer *tmr;
-	tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+	struct k_itimer *tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+
 	if (!tmr)
 		return tmr;
 	if (unlikely(!(tmr->sigq = sigqueue_alloc()))) {
@@ -690,8 +690,8 @@ void common_timer_get(struct k_itimer *t
 
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
-	struct k_itimer *timr;
 	const struct k_clock *kc;
+	struct k_itimer *timr;
 	unsigned long flags;
 	int ret = 0;
 
@@ -762,8 +762,8 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {
 	struct k_itimer *timr;
-	int overrun;
 	unsigned long flags;
+	int overrun;
 
 	timr = lock_timer(timer_id, &flags);
 	if (!timr)
@@ -936,8 +936,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t,
 		const struct __kernel_itimerspec __user *, new_setting,
 		struct __kernel_itimerspec __user *, old_setting)
 {
-	struct itimerspec64 new_spec, old_spec;
-	struct itimerspec64 *rtn = old_setting ? &old_spec : NULL;
+	struct itimerspec64 new_spec, old_spec, *rtn;
 	int error = 0;
 
 	if (!new_setting)
@@ -946,6 +945,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t,
 	if (get_itimerspec64(&new_spec, new_setting))
 		return -EFAULT;
 
+	rtn = old_setting ? &old_spec : NULL;
 	error = do_timer_settime(timer_id, flags, &new_spec, rtn);
 	if (!error && old_setting) {
 		if (put_itimerspec64(&old_spec, old_setting))


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

* Re: [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-04-25 18:48 ` [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete() Thomas Gleixner
@ 2023-05-04 17:06   ` Frederic Weisbecker
  2023-05-04 18:20     ` Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-04 17:06 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

Le Tue, Apr 25, 2023 at 08:48:56PM +0200, Thomas Gleixner a écrit :
> itimer_delete() has a retry loop when the timer is concurrently expired. On
> non-RT kernels this just spin-waits until the timer callback has
> completed. On RT kernels this is a potential livelock when the exiting task
> preempted the hrtimer soft interrupt.
> 
> This only affects hrtimer based timers as Posix CPU timers cannot be
> concurrently expired. For CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this is
> obviously impossible as the task cannot run task work and exit at the same
> time. The CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n (only non-RT) is prevented
> because interrupts are disabled.

But the owner of the timer is not the same as the target of the timer, right?

Though I seem to remember that we forbid setting a timer to a target outside
the current process, in which case the owner and the target are the same at
this exit stage. But I can't remember what enforces that permission in pid_for_clock()...

Thanks.

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

* Re: [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-05-04 17:06   ` Frederic Weisbecker
@ 2023-05-04 18:20     ` Thomas Gleixner
  2023-05-05  7:57       ` Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-04 18:20 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Thu, May 04 2023 at 19:06, Frederic Weisbecker wrote:
> Le Tue, Apr 25, 2023 at 08:48:56PM +0200, Thomas Gleixner a écrit :
>> itimer_delete() has a retry loop when the timer is concurrently expired. On
>> non-RT kernels this just spin-waits until the timer callback has
>> completed. On RT kernels this is a potential livelock when the exiting task
>> preempted the hrtimer soft interrupt.
>> 
>> This only affects hrtimer based timers as Posix CPU timers cannot be
>> concurrently expired. For CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this is
>> obviously impossible as the task cannot run task work and exit at the same
>> time. The CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n (only non-RT) is prevented
>> because interrupts are disabled.
>
> But the owner of the timer is not the same as the target of the timer, right?
>
> Though I seem to remember that we forbid setting a timer to a target outside
> the current process, in which case the owner and the target are the same at
> this exit stage. But I can't remember what enforces that permission in
> pid_for_clock()..

The owner of the timer is always the one which needs to find the entity
to synchronize on, whether that's the right hrtimer base or the task
which runs the expiry code.

wait_for_running_timer() is taking that into account:

  - The hrtimer timer based posix timers lock the hrtimer base expiry
    lock on the base to which the timer is currently associated

  - Posix CPU timers can be armed on a differnet process (only per
    thread timers are restricted to currents threadgroup) but the
    wait_for_running() callback "knows" how to find that process:

    When the timer is moved to the expiry list it gets:

         cputimer->firing = 1;
         rcu_assign_pointer(ctmr->handling, current);

   and the wait for running side does:

       rcu_read_lock()
       tsk = rcu_dereference(timr->it.cpu.handling);
       ....
       mutex_lock(&tsk->posix_cputimers_work.mutex);

   See collect_timerqueue(), handle_posix_cpu_timers() and
   posix_cpu_timer_wait_running() for details.

   commit f7abf14f0001 ("posix-cpu-timers: Implement the missing
   timer_wait_running callback") has quite some prose in the changelog.

Thanks,

        tglx


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

* Re: [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-05-04 18:20     ` Thomas Gleixner
@ 2023-05-05  7:57       ` Thomas Gleixner
  2023-06-01 19:00         ` [patch v2 " Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-05  7:57 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Thu, May 04 2023 at 20:20, Thomas Gleixner wrote:
> On Thu, May 04 2023 at 19:06, Frederic Weisbecker wrote:
>> Le Tue, Apr 25, 2023 at 08:48:56PM +0200, Thomas Gleixner a écrit :
>>> itimer_delete() has a retry loop when the timer is concurrently expired. On
>>> non-RT kernels this just spin-waits until the timer callback has
>>> completed. On RT kernels this is a potential livelock when the exiting task
>>> preempted the hrtimer soft interrupt.
>>> 
>>> This only affects hrtimer based timers as Posix CPU timers cannot be
>>> concurrently expired. For CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this is
>>> obviously impossible as the task cannot run task work and exit at the same
>>> time. The CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n (only non-RT) is prevented
>>> because interrupts are disabled.
>>
>> But the owner of the timer is not the same as the target of the timer, right?
>>
>> Though I seem to remember that we forbid setting a timer to a target outside
>> the current process, in which case the owner and the target are the same at
>> this exit stage. But I can't remember what enforces that permission in
>> pid_for_clock()..
>
> The owner of the timer is always the one which needs to find the entity
> to synchronize on, whether that's the right hrtimer base or the task
> which runs the expiry code.
>
> wait_for_running_timer() is taking that into account:
>
>   - The hrtimer timer based posix timers lock the hrtimer base expiry
>     lock on the base to which the timer is currently associated
>
>   - Posix CPU timers can be armed on a differnet process (only per
>     thread timers are restricted to currents threadgroup) but the
>     wait_for_running() callback "knows" how to find that process:
>
>     When the timer is moved to the expiry list it gets:
>
>          cputimer->firing = 1;
>          rcu_assign_pointer(ctmr->handling, current);
>
>    and the wait for running side does:
>
>        rcu_read_lock()
>        tsk = rcu_dereference(timr->it.cpu.handling);
>        ....
>        mutex_lock(&tsk->posix_cputimers_work.mutex);
>
>    See collect_timerqueue(), handle_posix_cpu_timers() and
>    posix_cpu_timer_wait_running() for details.
>
>    commit f7abf14f0001 ("posix-cpu-timers: Implement the missing
>    timer_wait_running callback") has quite some prose in the changelog.

But you have a point. The comment I added in itimer_delete() vs. CPU
timers is wrong for timers which are armed on a different process.
Needs to be removed.

Thanks,

        tglx

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-04-25 18:48 ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Thomas Gleixner
@ 2023-05-05 14:50   ` Frederic Weisbecker
  2023-05-05 22:58     ` Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-05 14:50 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Tue, Apr 25, 2023 at 08:48:58PM +0200, Thomas Gleixner wrote:
> posix_timer_add() tries to allocate a posix timer ID by starting from the
> cached ID which was stored by the last successful allocation.
> 
> This is done in a loop searching the ID space for a free slot one by
> one. The loop has to terminate when the search wrapped around to the
> starting point.
> 
> But that's racy vs. establishing the starting point. That is read out
> lockless, which leads to the following problem:
> 
> CPU0	  	      	     	   CPU1
> posix_timer_add()
>   start = sig->posix_timer_id;
>   lock(hash_lock);
>   ...				   posix_timer_add()
>   if (++sig->posix_timer_id < 0)
>       			             start = sig->posix_timer_id;
>      sig->posix_timer_id = 0;
> 
> So CPU1 can observe a negative start value, i.e. -1, and the loop break
> never happens because the condition can never be true:
> 
>   if (sig->posix_timer_id == start)
>      break;
> 
> While this is unlikely to ever turn into an endless loop as the ID space is
> huge (INT_MAX), the racy read of the start value caught the attention of
> KCSAN and Dmitry unearthed that incorrectness.
> 
> Rewrite it so that the start condition can never observe the negative value
> and annotate the read and the write with READ_ONCE()/WRITE_ONCE().
> 
> Reported-by: syzbot+5c54bd3eb218bb595aa9@syzkaller.appspotmail.com
> Reported-by: Dmitry Vyukov <dvyukov@google.com>
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> ---
>  include/linux/sched/signal.h |    2 +-
>  kernel/time/posix-timers.c   |   30 +++++++++++++++++-------------
>  2 files changed, 18 insertions(+), 14 deletions(-)
> 
> --- a/include/linux/sched/signal.h
> +++ b/include/linux/sched/signal.h
> @@ -135,7 +135,7 @@ struct signal_struct {
>  #ifdef CONFIG_POSIX_TIMERS
>  
>  	/* POSIX.1b Interval Timers */
> -	int			posix_timer_id;
> +	unsigned int		next_posix_timer_id;
>  	struct list_head	posix_timers;
>  
>  	/* ITIMER_REAL timer for the process */
> --- a/kernel/time/posix-timers.c
> +++ b/kernel/time/posix-timers.c
> @@ -140,25 +140,29 @@ static struct k_itimer *posix_timer_by_i
>  static int posix_timer_add(struct k_itimer *timer)
>  {
>  	struct signal_struct *sig = current->signal;
> -	int first_free_id = sig->posix_timer_id;
>  	struct hlist_head *head;
> -	int ret = -ENOENT;
> +	unsigned int start, id;
>  
> -	do {
> +	/* Can be written by a different task concurrently in the loop below */
> +	start = READ_ONCE(sig->next_posix_timer_id);
> +
> +	for (id = ~start; start != id; id++) {
>  		spin_lock(&hash_lock);
> -		head = &posix_timers_hashtable[hash(sig, sig->posix_timer_id)];
> -		if (!__posix_timers_find(head, sig, sig->posix_timer_id)) {
> +		id = sig->next_posix_timer_id;
> +
> +		/* Write the next ID back. Clamp it to the positive space */
> +		WRITE_ONCE(sig->next_posix_timer_id, (id + 1) & INT_MAX);

Isn't that looping forever?

Thanks.

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-05 14:50   ` Frederic Weisbecker
@ 2023-05-05 22:58     ` Thomas Gleixner
  2023-05-05 23:36       ` Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-05 22:58 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Fri, May 05 2023 at 16:50, Frederic Weisbecker wrote:
> On Tue, Apr 25, 2023 at 08:48:58PM +0200, Thomas Gleixner wrote:
>>  
>> -	do {
>> +	/* Can be written by a different task concurrently in the loop below */
>> +	start = READ_ONCE(sig->next_posix_timer_id);
>> +
>> +	for (id = ~start; start != id; id++) {
>>  		spin_lock(&hash_lock);
>> -		head = &posix_timers_hashtable[hash(sig, sig->posix_timer_id)];
>> -		if (!__posix_timers_find(head, sig, sig->posix_timer_id)) {
>> +		id = sig->next_posix_timer_id;
>> +
>> +		/* Write the next ID back. Clamp it to the positive space */
>> +		WRITE_ONCE(sig->next_posix_timer_id, (id + 1) & INT_MAX);
>
> Isn't that looping forever?

No. The loop breaks when @id reaches the locklessly read out @start
value again.

I admit that the 'id = ~start' part in the for() expression is confusing
without a comment. That initial @id value is in the invalid space to
make sure that the termination condition 'start != id' does not trigger
right away. But that value gets immediately overwritten after acquiring
hash_lock by the real sig->next_posix_timer_id value.

The clamp to the positive space has nothing to do with that. That's
required because the ID must be positive as a negative value would be an
error when returned, right?

So the whole thing works like this:

   start = READ_LOCKLESS(sig->next_id);

   // Enfore that id and start are different to not terminate right away
   id = ~start;

loop:
   if (id == start)
   	goto fail;
   lock()
        id = sig->next_id;                      <-- stable readout
        sig->next_id = (id + 1) & INT_MAX;      <-- prevent going negative

        if (unused_id(id)) {
           add_timer_to_hash(timer, id);
           unlock();
           return id;
        }
   id++;
   unlock();
   goto loop;

As the initial lockless readout is guaranteed to be in the positive
space, how is that supposed to be looping forever?

Admittedly this can be written less obscure, but not tonight :)

Thanks,

        tglx



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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-05 22:58     ` Thomas Gleixner
@ 2023-05-05 23:36       ` Thomas Gleixner
  2023-05-08 21:57         ` Thomas Gleixner
  2023-05-09  9:42         ` Frederic Weisbecker
  0 siblings, 2 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-05 23:36 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Sat, May 06 2023 at 00:58, Thomas Gleixner wrote:
> On Fri, May 05 2023 at 16:50, Frederic Weisbecker wrote:
> So the whole thing works like this:
>
>    start = READ_LOCKLESS(sig->next_id);
>
>    // Enfore that id and start are different to not terminate right away
>    id = ~start;
>
> loop:
>    if (id == start)
>    	goto fail;
>    lock()
>         id = sig->next_id;                      <-- stable readout
>         sig->next_id = (id + 1) & INT_MAX;      <-- prevent going negative
>
>         if (unused_id(id)) {
>            add_timer_to_hash(timer, id);
>            unlock();
>            return id;
>         }
>    id++;
>    unlock();
>    goto loop;
>
> As the initial lockless readout is guaranteed to be in the positive
> space, how is that supposed to be looping forever?

Unless you think about the theoretical case of an unlimited number of
threads sharing the signal_struct which all concurrently try to allocate
a timer id and then releasing it immediately again (to avoid resource
limit exhaustion). Theoretically possible, but is this a real concern
with a timer ID space of 2G?

I'm sure that it's incredibly hard to exploit this, but what's really
bothering me is the hash table itself. The only reason why we have that
is CRIU.

The only alternative solution I could come up with is a paritioned
xarray where the index space would be segmented for each TGID, i.e.

       segment.start = TGID * MAX_TIMERS_PER_PROCESS
       segment.end    = segment.start + MAX_TIMERS_PER_PROCESS - 1

where MAX_TIMERS_PER_PROCESS could be a copius 2^16 which would work for
both 32bit and 64bit TID limits.

That would avoid the hash table lookups and the related issues, but OTH
it would require to allocate one extra page per TGID if the application
uses a single posix timer.

Not sure whether that's worth it though.

Thanks,

        tglx

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-05 23:36       ` Thomas Gleixner
@ 2023-05-08 21:57         ` Thomas Gleixner
  2023-05-09  9:30           ` Thomas Gleixner
  2023-05-09  9:42         ` Frederic Weisbecker
  1 sibling, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-08 21:57 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Sat, May 06 2023 at 01:36, Thomas Gleixner wrote:
> On Sat, May 06 2023 at 00:58, Thomas Gleixner wrote:
>> On Fri, May 05 2023 at 16:50, Frederic Weisbecker wrote:
>> As the initial lockless readout is guaranteed to be in the positive
>> space, how is that supposed to be looping forever?
>
> Unless you think about the theoretical case of an unlimited number of
> threads sharing the signal_struct which all concurrently try to allocate
> a timer id and then releasing it immediately again (to avoid resource
> limit exhaustion). Theoretically possible, but is this a real concern
> with a timer ID space of 2G?

It only hurts the process which does this and does not inflict any
latencies by holding hash_lock over the full search for a free spot.

> The only alternative solution I could come up with is a paritioned
> xarray where the index space would be segmented for each TGID, i.e.
>
>        segment.start = TGID * MAX_TIMERS_PER_PROCESS
>        segment.end    = segment.start + MAX_TIMERS_PER_PROCESS - 1
>
> where MAX_TIMERS_PER_PROCESS could be a copius 2^16 which would work for
> both 32bit and 64bit TID limits.
>
> That would avoid the hash table lookups and the related issues, but OTH
> it would require to allocate one extra page per TGID if the application
> uses a single posix timer.
>
> Not sure whether that's worth it though.

More thoughts on this. If we go there and accept the extra page of
memory then we can just go all the way and make the xarray per process,
actually per signal.

That would also require another change, namely making the preallocated
sigqueue part of struct k_itimer, which in turn would not be the worst
of all ideas as it gets rid of the lookup in posixtimer_rearm() and
would also allow for some clever handling of the nasty SIG_IGN issues.

Though that's separate from the problem at hand.

Thanks,

        tglx

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-08 21:57         ` Thomas Gleixner
@ 2023-05-09  9:30           ` Thomas Gleixner
  2023-05-09 12:50             ` Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-09  9:30 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin

On Mon, May 08 2023 at 23:57, Thomas Gleixner wrote:
> On Sat, May 06 2023 at 01:36, Thomas Gleixner wrote:
>> The only alternative solution I could come up with is a paritioned
>> xarray where the index space would be segmented for each TGID, i.e.
>>
>>        segment.start = TGID * MAX_TIMERS_PER_PROCESS
>>        segment.end    = segment.start + MAX_TIMERS_PER_PROCESS - 1
>>
>> where MAX_TIMERS_PER_PROCESS could be a copius 2^16 which would work for
>> both 32bit and 64bit TID limits.
>>
>> That would avoid the hash table lookups and the related issues, but OTH
>> it would require to allocate one extra page per TGID if the application
>> uses a single posix timer.
>>
>> Not sure whether that's worth it though.
>
> More thoughts on this. If we go there and accept the extra page of
> memory then we can just go all the way and make the xarray per process,
> actually per signal.

Thinking more about it. The current scheme how timer ID allocation works
is really interesting vs. CRIU.

Assume a process creates/deletes timers frequently. It's not hard to
move the next ID close to INT_MAX, i.e. 2G

Now checkpoint that thing and restore it which means to do the
create/delete dance to move next ID up to the last one-1. Will only take
a couple of hours....

Thanks,

        tglx

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-05 23:36       ` Thomas Gleixner
  2023-05-08 21:57         ` Thomas Gleixner
@ 2023-05-09  9:42         ` Frederic Weisbecker
  2023-05-09 12:04           ` Thomas Gleixner
  2023-05-09 12:38           ` Thomas Gleixner
  1 sibling, 2 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-09  9:42 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Sat, May 06, 2023 at 01:36:22AM +0200, Thomas Gleixner wrote:
> On Sat, May 06 2023 at 00:58, Thomas Gleixner wrote:
> > On Fri, May 05 2023 at 16:50, Frederic Weisbecker wrote:
> > So the whole thing works like this:
> >
> >    start = READ_LOCKLESS(sig->next_id);
> >
> >    // Enfore that id and start are different to not terminate right away
> >    id = ~start;
> >
> > loop:
> >    if (id == start)
> >    	goto fail;
> >    lock()
> >         id = sig->next_id;                      <-- stable readout
> >         sig->next_id = (id + 1) & INT_MAX;      <-- prevent going negative
> >
> >         if (unused_id(id)) {
> >            add_timer_to_hash(timer, id);
> >            unlock();
> >            return id;
> >         }
> >    id++;
> >    unlock();
> >    goto loop;
> >
> > As the initial lockless readout is guaranteed to be in the positive
> > space, how is that supposed to be looping forever?
> 
> Unless you think about the theoretical case of an unlimited number of
> threads sharing the signal_struct which all concurrently try to allocate
> a timer id and then releasing it immediately again (to avoid resource
> limit exhaustion). Theoretically possible, but is this a real concern
> with a timer ID space of 2G?

I didn't go that far actually, it was just me misunderstanding that loop and
especially the (id =~start) part. Now I got it.

I guess the for statement can just be:

for (; start != id; id++)

> 
> I'm sure that it's incredibly hard to exploit this, but what's really
> bothering me is the hash table itself. The only reason why we have that
> is CRIU.
> 
> The only alternative solution I could come up with is a paritioned
> xarray where the index space would be segmented for each TGID, i.e.
> 
>        segment.start = TGID * MAX_TIMERS_PER_PROCESS
>        segment.end    = segment.start + MAX_TIMERS_PER_PROCESS - 1
> 
> where MAX_TIMERS_PER_PROCESS could be a copius 2^16 which would work for
> both 32bit and 64bit TID limits.
> 
> That would avoid the hash table lookups and the related issues, but OTH
> it would require to allocate one extra page per TGID if the application
> uses a single posix timer.
> 
> Not sure whether that's worth it though.

Not sure either...

Thanks.

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

* Re: [patch 03/20] posix-timers: Clarify timer_wait_running() comment
  2023-04-25 18:49 ` [patch 03/20] posix-timers: Clarify timer_wait_running() comment Thomas Gleixner
@ 2023-05-09  9:50   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-09  9:50 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:00PM +0200, Thomas Gleixner wrote:
> Explain it better and add the CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y aspect
> for completeness.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 04/20] posix-timers: Cleanup comments about timer ID tracking
  2023-04-25 18:49 ` [patch 04/20] posix-timers: Cleanup comments about timer ID tracking Thomas Gleixner
@ 2023-05-09  9:58   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-09  9:58 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:01PM +0200, Thomas Gleixner wrote:
> Describe the hash table properly and remove the IDR leftover comments.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 05/20] posix-timers: Add comments about timer lookup
  2023-04-25 18:49 ` [patch 05/20] posix-timers: Add comments about timer lookup Thomas Gleixner
@ 2023-05-09 10:58   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-09 10:58 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:03PM +0200, Thomas Gleixner wrote:
> Document how the timer ID validation in the hash table works.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal
  2023-04-25 18:49 ` [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal Thomas Gleixner
@ 2023-05-09 11:04   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] posix-timers: Annotate concurrent access to k_itimer:: It_signal tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-09 11:04 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:05PM +0200, Thomas Gleixner wrote:
> k_itimer::it_signal is read lockless in the RCU protected hash lookup, but
> it can be written concurrently in the timer_create() and timer_delete()
> path. Annotate these places with READ_ONCE() and WRITE_ONCE()
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-09  9:42         ` Frederic Weisbecker
@ 2023-05-09 12:04           ` Thomas Gleixner
  2023-05-09 12:38           ` Thomas Gleixner
  1 sibling, 0 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-09 12:04 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Tue, May 09 2023 at 11:42, Frederic Weisbecker wrote:
> On Sat, May 06, 2023 at 01:36:22AM +0200, Thomas Gleixner wrote:
>> That would avoid the hash table lookups and the related issues, but OTH
>> it would require to allocate one extra page per TGID if the application
>> uses a single posix timer.
>> 
>> Not sure whether that's worth it though.
>
> Not sure either...

See my other reply on that...

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-09  9:42         ` Frederic Weisbecker
  2023-05-09 12:04           ` Thomas Gleixner
@ 2023-05-09 12:38           ` Thomas Gleixner
  2023-05-09 14:18             ` Frederic Weisbecker
  1 sibling, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-09 12:38 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Tue, May 09 2023 at 11:42, Frederic Weisbecker wrote:
> On Sat, May 06, 2023 at 01:36:22AM +0200, Thomas Gleixner wrote:
>> Unless you think about the theoretical case of an unlimited number of
>> threads sharing the signal_struct which all concurrently try to allocate
>> a timer id and then releasing it immediately again (to avoid resource
>> limit exhaustion). Theoretically possible, but is this a real concern
>> with a timer ID space of 2G?
>
> I didn't go that far actually, it was just me misunderstanding that loop and
> especially the (id =~start) part. Now I got it.
>
> I guess the for statement can just be:
>
> for (; start != id; id++)

My brain based compiler complains about uninitialized usage of @id. I'm
pretty sure it's rightfully complaining and a real compiler would agree,
no?

Thanks,

        tglx

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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-09  9:30           ` Thomas Gleixner
@ 2023-05-09 12:50             ` Thomas Gleixner
  2023-05-09 21:42               ` [RFD] posix-timers: CRIU woes Thomas Gleixner
  2023-05-10  0:42               ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Andrey Vagin
  0 siblings, 2 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-09 12:50 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin

On Tue, May 09 2023 at 11:30, Thomas Gleixner wrote:
> On Mon, May 08 2023 at 23:57, Thomas Gleixner wrote:
>> More thoughts on this. If we go there and accept the extra page of
>> memory then we can just go all the way and make the xarray per process,
>> actually per signal.
>
> Thinking more about it. The current scheme how timer ID allocation works
> is really interesting vs. CRIU.
>
> Assume a process creates/deletes timers frequently. It's not hard to
> move the next ID close to INT_MAX, i.e. 2G
>
> Now checkpoint that thing and restore it which means to do the
> create/delete dance to move next ID up to the last one-1. Will only take
> a couple of hours....

I'm cursing myself for overlooking this back then when the CRIU changes
to the timer ID management were made. Why?

   Because that created an ABI which CRIU relies on.

The proper solution for this would be to make it possible to create a
timer with a given ID. That's not rocket science, but we need buy in
from the CRIU folks. Otherwise we are up the regression creek without a
paddle.

Thanks,

        tglx




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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-09 12:38           ` Thomas Gleixner
@ 2023-05-09 14:18             ` Frederic Weisbecker
  2023-06-01 18:58               ` [patch v2 " Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Frederic Weisbecker @ 2023-05-09 14:18 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Tue, May 09, 2023 at 02:38:50PM +0200, Thomas Gleixner wrote:
> On Tue, May 09 2023 at 11:42, Frederic Weisbecker wrote:
> > On Sat, May 06, 2023 at 01:36:22AM +0200, Thomas Gleixner wrote:
> >> Unless you think about the theoretical case of an unlimited number of
> >> threads sharing the signal_struct which all concurrently try to allocate
> >> a timer id and then releasing it immediately again (to avoid resource
> >> limit exhaustion). Theoretically possible, but is this a real concern
> >> with a timer ID space of 2G?
> >
> > I didn't go that far actually, it was just me misunderstanding that loop and
> > especially the (id =~start) part. Now I got it.
> >
> > I guess the for statement can just be:
> >
> > for (; start != id; id++)
> 
> My brain based compiler complains about uninitialized usage of @id. I'm
> pretty sure it's rightfully complaining and a real compiler would agree,
> no?

*sigh* I should think more before pressing answer these days :)

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

* [RFD] posix-timers: CRIU woes
  2023-05-09 12:50             ` Thomas Gleixner
@ 2023-05-09 21:42               ` Thomas Gleixner
  2023-05-10  4:36                 ` Pavel Tikhomirov
  2023-05-10  8:16                 ` Andrey Vagin
  2023-05-10  0:42               ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Andrey Vagin
  1 sibling, 2 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-09 21:42 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Tikhomirov, Pavel Emelyanov

Hi!

This is a summary of several mails so that the CRIU people have the full
picture.

A recent syzbot report made me look at the timer ID management, which
was modified 7 years ago to accomodate CRIU:

    5ed67f05f66c ("posix timers: Allocate timer id per process (v2)")

and introduced that reported issue along with a bogus loop termination
problem. See

    https://lore.kernel.org/lkml/000000000000a3723305f9d98fc3@google.com/
    https://lore.kernel.org/lkml/20230425183312.932345089@linutronix.de

for details.

The intent to make the timer IDs per process is definitely correct, but
the implementation is beyond suboptimal. I really regret that I did not
catch this back then when picking those changes up.

The way it works is that each process keeps a 'next_timer_id' which it
uses to allocate the next timer. That allows CRIU to reconstruct timers
with the same ID by walking the ID space via

  do {
     timer_create(&timer, ...., &id);
     if (id == original_id)
        goto success;
     timer_delete(&timer);
  } while (idspace_not_exhausted());

That works by some definition of works, but it is problematic in two ways:

 1) As the timer ID space is up to INT_MAX, a process which creates and
    deletes timers frequently, can easily move up close to the INT_MAX
    id space over time.

    If such a process is checkpointed and restored, then the above loop
    will run for at least an hour to restore a single timer.

    And no, this is not only a hypothetical issue. There are legitimate
    scenarios where threads are created and the control thread arms a
    posix CPU timer on them. Such threads can be torn down on a regular
    base due to thread pool consolidations. As CRIU does not happen
    every 5 minutes it's not completely unlikely that such a process
    surives quite some time on a particular host and thereby approaches
    the ID space limit.

    Sure we can restrict the ID space to a way smaller number so the
    search wraps around earlier, but what's a sensible limit?

    Though restricting the ID space has its own issue vs. backwards
    compability. A process which created a timer on an older kernel with
    an ID larger than the newer kernels ID limit cannot longer be
    restored on that newer kernel.

    Aside of that it does not solve the other problem this created:

 2) That change created an user space ABI, which means that the kernel
    side has to stick with this next ID search mechanism forever.

    That prevents to get rid of that global lock and hash table by
    sticking an xarray into task::signal which makes a lot of sense in
    terms of cache locality and gets rid of the extra timer list
    management in task::signal. Making this change would be very useful
    to address some other issues in the posix-timer code without
    creating yet more duct tape horrors.

    Such a change obviously has to aim for a dense ID space to keep the
    memory overhead low, but that breaks existing CRIU userspace because
    dense ID space and next ID search does not fit together.

    Next ID search is obviously creating non-recoverable holes in the
    case that timers are deleted afterwards.

    A dense ID space approach can create holes too, but they are
    recoverable and well within the resource limits, because the process
    has to be able to create enough timers in the first place in order
    to release those in the middle.

    With the next ID search brute force recovery is not possible on a
    kernel with dense ID as there is no way to create all intermediate
    timers first before reaching the one at the far end due to resource
    limits.

So because of that half thought out user space ABI we are now up the
regression creek without a paddle, unless CRIU can accomodate to a
different restore mechanism to lift this restriction from the kernel.

Thoughts?

Thanks,

        tglx



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

* Re: [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-09 12:50             ` Thomas Gleixner
  2023-05-09 21:42               ` [RFD] posix-timers: CRIU woes Thomas Gleixner
@ 2023-05-10  0:42               ` Andrey Vagin
  1 sibling, 0 replies; 122+ messages in thread
From: Andrey Vagin @ 2023-05-10  0:42 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

On Tue, May 9, 2023 at 5:50 AM Thomas Gleixner <tglx@linutronix.de> wrote:
>
> On Tue, May 09 2023 at 11:30, Thomas Gleixner wrote:
> > On Mon, May 08 2023 at 23:57, Thomas Gleixner wrote:
> >> More thoughts on this. If we go there and accept the extra page of
> >> memory then we can just go all the way and make the xarray per process,
> >> actually per signal.
> >
> > Thinking more about it. The current scheme how timer ID allocation works
> > is really interesting vs. CRIU.
> >
> > Assume a process creates/deletes timers frequently. It's not hard to
> > move the next ID close to INT_MAX, i.e. 2G
> >
> > Now checkpoint that thing and restore it which means to do the
> > create/delete dance to move next ID up to the last one-1. Will only take
> > a couple of hours....
>
> I'm cursing myself for overlooking this back then when the CRIU changes
> to the timer ID management were made. Why?
>
>    Because that created an ABI which CRIU relies on.
>
> The proper solution for this would be to make it possible to create a
> timer with a given ID. That's not rocket science, but we need buy in
> from the CRIU folks. Otherwise we are up the regression creek without a
> paddle.

Let's go with the proper solution. I will prepare CRIU changes when
kernel patches are ready.

Thanks,
Andrei

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-09 21:42               ` [RFD] posix-timers: CRIU woes Thomas Gleixner
@ 2023-05-10  4:36                 ` Pavel Tikhomirov
  2023-05-10  8:30                   ` Thomas Gleixner
  2023-05-11 10:13                   ` David Laight
  2023-05-10  8:16                 ` Andrey Vagin
  1 sibling, 2 replies; 122+ messages in thread
From: Pavel Tikhomirov @ 2023-05-10  4:36 UTC (permalink / raw)
  To: Thomas Gleixner, Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov



On 10.05.2023 05:42, Thomas Gleixner wrote:
> Hi!
> 
> This is a summary of several mails so that the CRIU people have the full
> picture.
> 
> A recent syzbot report made me look at the timer ID management, which
> was modified 7 years ago to accomodate CRIU:
> 
>      5ed67f05f66c ("posix timers: Allocate timer id per process (v2)")
> 
> and introduced that reported issue along with a bogus loop termination
> problem. See
> 
>      https://lore.kernel.org/lkml/000000000000a3723305f9d98fc3@google.com/
>      https://lore.kernel.org/lkml/20230425183312.932345089@linutronix.de
> 
> for details.
> 
> The intent to make the timer IDs per process is definitely correct, but
> the implementation is beyond suboptimal. I really regret that I did not
> catch this back then when picking those changes up.
> 
> The way it works is that each process keeps a 'next_timer_id' which it
> uses to allocate the next timer. That allows CRIU to reconstruct timers
> with the same ID by walking the ID space via
> 
>    do {
>       timer_create(&timer, ...., &id);
>       if (id == original_id)
>          goto success;
>       timer_delete(&timer);
>    } while (idspace_not_exhausted());
> 
> That works by some definition of works, but it is problematic in two ways:
> 
>   1) As the timer ID space is up to INT_MAX, a process which creates and
>      deletes timers frequently, can easily move up close to the INT_MAX
>      id space over time.
> 
>      If such a process is checkpointed and restored, then the above loop
>      will run for at least an hour to restore a single timer.
> 
>      And no, this is not only a hypothetical issue. There are legitimate
>      scenarios where threads are created and the control thread arms a
>      posix CPU timer on them. Such threads can be torn down on a regular
>      base due to thread pool consolidations. As CRIU does not happen
>      every 5 minutes it's not completely unlikely that such a process
>      surives quite some time on a particular host and thereby approaches
>      the ID space limit.
> 
>      Sure we can restrict the ID space to a way smaller number so the
>      search wraps around earlier, but what's a sensible limit?
> 
>      Though restricting the ID space has its own issue vs. backwards
>      compability. A process which created a timer on an older kernel with
>      an ID larger than the newer kernels ID limit cannot longer be
>      restored on that newer kernel.
> 
>      Aside of that it does not solve the other problem this created:
> 
>   2) That change created an user space ABI, which means that the kernel
>      side has to stick with this next ID search mechanism forever.
> 
>      That prevents to get rid of that global lock and hash table by
>      sticking an xarray into task::signal which makes a lot of sense in
>      terms of cache locality and gets rid of the extra timer list
>      management in task::signal. Making this change would be very useful
>      to address some other issues in the posix-timer code without
>      creating yet more duct tape horrors.
> 
>      Such a change obviously has to aim for a dense ID space to keep the
>      memory overhead low, but that breaks existing CRIU userspace because
>      dense ID space and next ID search does not fit together.
> 
>      Next ID search is obviously creating non-recoverable holes in the
>      case that timers are deleted afterwards.
> 
>      A dense ID space approach can create holes too, but they are
>      recoverable and well within the resource limits, because the process
>      has to be able to create enough timers in the first place in order
>      to release those in the middle.
> 
>      With the next ID search brute force recovery is not possible on a
>      kernel with dense ID as there is no way to create all intermediate
>      timers first before reaching the one at the far end due to resource
>      limits.
> 
> So because of that half thought out user space ABI we are now up the
> regression creek without a paddle, unless CRIU can accomodate to a
> different restore mechanism to lift this restriction from the kernel.
> 
> Thoughts?

Maybe we can do something similar to /proc/sys/kernel/ns_last_pid? 
Switch to per-(process->signal) idr based approach with idr_set_cursor 
to set next id for next posix timer from new sysctl?

> 
> Thanks,
> 
>          tglx
> 
> 

-- 
Best regards, Tikhomirov Pavel
Senior Software Developer, Virtuozzo.

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-09 21:42               ` [RFD] posix-timers: CRIU woes Thomas Gleixner
  2023-05-10  4:36                 ` Pavel Tikhomirov
@ 2023-05-10  8:16                 ` Andrey Vagin
  2023-05-11  3:17                   ` Pavel Tikhomirov
  2023-05-11  7:49                   ` Cyrill Gorcunov
  1 sibling, 2 replies; 122+ messages in thread
From: Andrey Vagin @ 2023-05-10  8:16 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Tikhomirov, Pavel Emelyanov

On Tue, May 9, 2023 at 2:42 PM Thomas Gleixner <tglx@linutronix.de> wrote:
>
> Hi!
>
> This is a summary of several mails so that the CRIU people have the full
> picture.
>
> A recent syzbot report made me look at the timer ID management, which
> was modified 7 years ago to accomodate CRIU:
>
>     5ed67f05f66c ("posix timers: Allocate timer id per process (v2)")
>
> and introduced that reported issue along with a bogus loop termination
> problem. See
>
>     https://lore.kernel.org/lkml/000000000000a3723305f9d98fc3@google.com/
>     https://lore.kernel.org/lkml/20230425183312.932345089@linutronix.de
>
> for details.
>
> The intent to make the timer IDs per process is definitely correct, but
> the implementation is beyond suboptimal. I really regret that I did not
> catch this back then when picking those changes up.
>
> The way it works is that each process keeps a 'next_timer_id' which it
> uses to allocate the next timer. That allows CRIU to reconstruct timers
> with the same ID by walking the ID space via
>
>   do {
>      timer_create(&timer, ...., &id);
>      if (id == original_id)
>         goto success;
>      timer_delete(&timer);
>   } while (idspace_not_exhausted());
>
> That works by some definition of works, but it is problematic in two ways:
>
>  1) As the timer ID space is up to INT_MAX, a process which creates and
>     deletes timers frequently, can easily move up close to the INT_MAX
>     id space over time.
>
>     If such a process is checkpointed and restored, then the above loop
>     will run for at least an hour to restore a single timer.
>
>     And no, this is not only a hypothetical issue. There are legitimate
>     scenarios where threads are created and the control thread arms a
>     posix CPU timer on them. Such threads can be torn down on a regular
>     base due to thread pool consolidations. As CRIU does not happen
>     every 5 minutes it's not completely unlikely that such a process
>     surives quite some time on a particular host and thereby approaches
>     the ID space limit.
>
>     Sure we can restrict the ID space to a way smaller number so the
>     search wraps around earlier, but what's a sensible limit?
>
>     Though restricting the ID space has its own issue vs. backwards
>     compability. A process which created a timer on an older kernel with
>     an ID larger than the newer kernels ID limit cannot longer be
>     restored on that newer kernel.
>
>     Aside of that it does not solve the other problem this created:
>
>  2) That change created an user space ABI, which means that the kernel
>     side has to stick with this next ID search mechanism forever.
>
>     That prevents to get rid of that global lock and hash table by
>     sticking an xarray into task::signal which makes a lot of sense in
>     terms of cache locality and gets rid of the extra timer list
>     management in task::signal. Making this change would be very useful
>     to address some other issues in the posix-timer code without
>     creating yet more duct tape horrors.
>
>     Such a change obviously has to aim for a dense ID space to keep the
>     memory overhead low, but that breaks existing CRIU userspace because
>     dense ID space and next ID search does not fit together.
>
>     Next ID search is obviously creating non-recoverable holes in the
>     case that timers are deleted afterwards.
>
>     A dense ID space approach can create holes too, but they are
>     recoverable and well within the resource limits, because the process
>     has to be able to create enough timers in the first place in order
>     to release those in the middle.
>
>     With the next ID search brute force recovery is not possible on a
>     kernel with dense ID as there is no way to create all intermediate
>     timers first before reaching the one at the far end due to resource
>     limits.
>
> So because of that half thought out user space ABI we are now up the
> regression creek without a paddle, unless CRIU can accomodate to a
> different restore mechanism to lift this restriction from the kernel.
>
> Thoughts?

Hi Thomas,

If you give us a new API to create timers with specified id-s, we will
figure out how to live with it. It isn't good to ask users to update
CRIU to work on new kernels, but here are reasons and event improvements
for CRIU, so I think it's worth it.

As for API, we can use one bit of sigevent.sigev_notify to request a
timer with a specified id.

Thanks,
Andrei

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-10  4:36                 ` Pavel Tikhomirov
@ 2023-05-10  8:30                   ` Thomas Gleixner
  2023-05-11  4:12                     ` Pavel Tikhomirov
  2023-05-11 10:13                   ` David Laight
  1 sibling, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-10  8:30 UTC (permalink / raw)
  To: Pavel Tikhomirov, Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov

Pavel!

On Wed, May 10 2023 at 12:36, Pavel Tikhomirov wrote:
> On 10.05.2023 05:42, Thomas Gleixner wrote:
>> So because of that half thought out user space ABI we are now up the
>> regression creek without a paddle, unless CRIU can accomodate to a
>> different restore mechanism to lift this restriction from the kernel.
>> 
>> Thoughts?
>
> Maybe we can do something similar to /proc/sys/kernel/ns_last_pid? 
> Switch to per-(process->signal) idr based approach with idr_set_cursor 
> to set next id for next posix timer from new sysctl?

I'm not a fan of such sysctls. We have already too many of them and that
particular one does not buy much.

We can simply let timer_create() or a new syscall create a timer at a
given ID.

That allows CRIU to restore any checkpointed process no matter which
kernel version it came from without doing this insane create/delete
dance.

The downside is that this allows to create stupidly sparse timer IDs
even for the non CRIU case, which increases per process kernel memory
consumption and creates slightly more overhead in the signal delivery
path. The latter is a burden on the process owning the timer and not
affecting expiry, which is a context stealing operation. The memory part
needs eventually some thoughts vs. accounting.

If the 'explicit at ID' option is not used then the ID mechanism is
optimzied for dense IDs by using the first available ID in a bottom up
search, which recovers holes created by a timer_delete() operation.

Thanks,

        tglx

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-10  8:16                 ` Andrey Vagin
@ 2023-05-11  3:17                   ` Pavel Tikhomirov
  2023-05-11  9:36                     ` Thomas Gleixner
  2023-05-11  7:49                   ` Cyrill Gorcunov
  1 sibling, 1 reply; 122+ messages in thread
From: Pavel Tikhomirov @ 2023-05-11  3:17 UTC (permalink / raw)
  To: Andrey Vagin, Thomas Gleixner
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Emelyanov



On 10.05.2023 16:16, Andrey Vagin wrote:
> On Tue, May 9, 2023 at 2:42 PM Thomas Gleixner <tglx@linutronix.de> wrote:
>>
>> Hi!
>>
>> This is a summary of several mails so that the CRIU people have the full
>> picture.
>>
>> A recent syzbot report made me look at the timer ID management, which
>> was modified 7 years ago to accomodate CRIU:
>>
>>      5ed67f05f66c ("posix timers: Allocate timer id per process (v2)")
>>
>> and introduced that reported issue along with a bogus loop termination
>> problem. See
>>
>>      https://lore.kernel.org/lkml/000000000000a3723305f9d98fc3@google.com/
>>      https://lore.kernel.org/lkml/20230425183312.932345089@linutronix.de
>>
>> for details.
>>
>> The intent to make the timer IDs per process is definitely correct, but
>> the implementation is beyond suboptimal. I really regret that I did not
>> catch this back then when picking those changes up.
>>
>> The way it works is that each process keeps a 'next_timer_id' which it
>> uses to allocate the next timer. That allows CRIU to reconstruct timers
>> with the same ID by walking the ID space via
>>
>>    do {
>>       timer_create(&timer, ...., &id);
>>       if (id == original_id)
>>          goto success;
>>       timer_delete(&timer);
>>    } while (idspace_not_exhausted());
>>
>> That works by some definition of works, but it is problematic in two ways:
>>
>>   1) As the timer ID space is up to INT_MAX, a process which creates and
>>      deletes timers frequently, can easily move up close to the INT_MAX
>>      id space over time.
>>
>>      If such a process is checkpointed and restored, then the above loop
>>      will run for at least an hour to restore a single timer.
>>
>>      And no, this is not only a hypothetical issue. There are legitimate
>>      scenarios where threads are created and the control thread arms a
>>      posix CPU timer on them. Such threads can be torn down on a regular
>>      base due to thread pool consolidations. As CRIU does not happen
>>      every 5 minutes it's not completely unlikely that such a process
>>      surives quite some time on a particular host and thereby approaches
>>      the ID space limit.
>>
>>      Sure we can restrict the ID space to a way smaller number so the
>>      search wraps around earlier, but what's a sensible limit?
>>
>>      Though restricting the ID space has its own issue vs. backwards
>>      compability. A process which created a timer on an older kernel with
>>      an ID larger than the newer kernels ID limit cannot longer be
>>      restored on that newer kernel.
>>
>>      Aside of that it does not solve the other problem this created:
>>
>>   2) That change created an user space ABI, which means that the kernel
>>      side has to stick with this next ID search mechanism forever.
>>
>>      That prevents to get rid of that global lock and hash table by
>>      sticking an xarray into task::signal which makes a lot of sense in
>>      terms of cache locality and gets rid of the extra timer list
>>      management in task::signal. Making this change would be very useful
>>      to address some other issues in the posix-timer code without
>>      creating yet more duct tape horrors.
>>
>>      Such a change obviously has to aim for a dense ID space to keep the
>>      memory overhead low, but that breaks existing CRIU userspace because
>>      dense ID space and next ID search does not fit together.
>>
>>      Next ID search is obviously creating non-recoverable holes in the
>>      case that timers are deleted afterwards.
>>
>>      A dense ID space approach can create holes too, but they are
>>      recoverable and well within the resource limits, because the process
>>      has to be able to create enough timers in the first place in order
>>      to release those in the middle.
>>
>>      With the next ID search brute force recovery is not possible on a
>>      kernel with dense ID as there is no way to create all intermediate
>>      timers first before reaching the one at the far end due to resource
>>      limits.
>>
>> So because of that half thought out user space ABI we are now up the
>> regression creek without a paddle, unless CRIU can accomodate to a
>> different restore mechanism to lift this restriction from the kernel.
>>
>> Thoughts?
> 
> Hi Thomas,
> 
> If you give us a new API to create timers with specified id-s, we will
> figure out how to live with it. It isn't good to ask users to update
> CRIU to work on new kernels, but here are reasons and event improvements
> for CRIU, so I think it's worth it.

I agree, any API to create timers with specified id-s would work for new 
CRIU versions.

> 
> As for API, we can use one bit of sigevent.sigev_notify to request a
> timer with a specified id.

Yes, I agree, this kind of API is a good option.

> 
> Thanks,
> Andrei



-- 
Best regards, Tikhomirov Pavel
Senior Software Developer, Virtuozzo.

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-10  8:30                   ` Thomas Gleixner
@ 2023-05-11  4:12                     ` Pavel Tikhomirov
  2023-05-11  7:56                       ` Peter Zijlstra
  2023-05-11  9:32                       ` Thomas Gleixner
  0 siblings, 2 replies; 122+ messages in thread
From: Pavel Tikhomirov @ 2023-05-11  4:12 UTC (permalink / raw)
  To: Thomas Gleixner, Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov



On 10.05.2023 16:30, Thomas Gleixner wrote:
> Pavel!
> 
> On Wed, May 10 2023 at 12:36, Pavel Tikhomirov wrote:
>> On 10.05.2023 05:42, Thomas Gleixner wrote:
>>> So because of that half thought out user space ABI we are now up the
>>> regression creek without a paddle, unless CRIU can accomodate to a
>>> different restore mechanism to lift this restriction from the kernel.
>>>
>>> Thoughts?
>>
>> Maybe we can do something similar to /proc/sys/kernel/ns_last_pid?
>> Switch to per-(process->signal) idr based approach with idr_set_cursor
>> to set next id for next posix timer from new sysctl?
> 
> I'm not a fan of such sysctls. We have already too many of them and that
> particular one does not buy much.

Sorry, it was a bad idea, what you suggest below is much better.

> 
> We can simply let timer_create() or a new syscall create a timer at a
> given ID.

Yes this would work for CRIU. (note: in neighbor thread Andrei writes 
about adding a bit to sigevent.sigev_notify to request a timer with a 
specified id, new syscall is also a good option)

> 
> That allows CRIU to restore any checkpointed process no matter which
> kernel version it came from without doing this insane create/delete
> dance.

Yes, for CRIU this kind of API change is a big improvement.

> 
> The downside is that this allows to create stupidly sparse timer IDs
> even for the non CRIU case, which increases per process kernel memory
> consumption and creates slightly more overhead in the signal delivery
> path. The latter is a burden on the process owning the timer and not
> affecting expiry, which is a context stealing operation. The memory part
> needs eventually some thoughts vs. accounting.
> 
> If the 'explicit at ID' option is not used then the ID mechanism is
> optimzied for dense IDs by using the first available ID in a bottom up
> search, which recovers holes created by a timer_delete() operation.

Not sure how kernel memory consumption increases with sparse timer IDs, 
global hashtable (posix_timers_hashtable) is the same size anyway, 
entries in hlists can be distributed differently as hash depends on id 
directly but we have same number of entries. Probably I miss something, 
why do we need dense IDs?

> 
> Thanks,
> 
>          tglx

-- 
Best regards, Tikhomirov Pavel
Senior Software Developer, Virtuozzo.

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-10  8:16                 ` Andrey Vagin
  2023-05-11  3:17                   ` Pavel Tikhomirov
@ 2023-05-11  7:49                   ` Cyrill Gorcunov
  1 sibling, 0 replies; 122+ messages in thread
From: Cyrill Gorcunov @ 2023-05-11  7:49 UTC (permalink / raw)
  To: Andrey Vagin
  Cc: Thomas Gleixner, Frederic Weisbecker, LKML, Anna-Maria Behnsen,
	Peter Zijlstra, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Sebastian Siewior, Michael Kerrisk, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Tikhomirov, Pavel Emelyanov

On Wed, May 10, 2023 at 01:16:26AM -0700, Andrey Vagin wrote:
... 
> Hi Thomas,
> 
> If you give us a new API to create timers with specified id-s, we will
> figure out how to live with it. It isn't good to ask users to update
> CRIU to work on new kernels, but here are reasons and event improvements
> for CRIU, so I think it's worth it.
> 
> As for API, we can use one bit of sigevent.sigev_notify to request a
> timer with a specified id.

Which will do the trick but would look somehow strange I think, since signals
are not some how related to timer's ID. Another option might be to use output
`created_timer_id` parameter as an input cookie.

Say we describe input as

struct {
	u32 magic;
	timer_t timer_id;
};

Then if magic doesn't match we use `created_timer_id` for output only, and
otherwise we read `timer_id` from input and use it. Of course there is a
chance that some unitialized memory passed with existing old programs but
i think false positive gonna be very-very low if ever. Just IMHO.

	Cyrill

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11  4:12                     ` Pavel Tikhomirov
@ 2023-05-11  7:56                       ` Peter Zijlstra
  2023-05-11  9:32                       ` Thomas Gleixner
  1 sibling, 0 replies; 122+ messages in thread
From: Peter Zijlstra @ 2023-05-11  7:56 UTC (permalink / raw)
  To: Pavel Tikhomirov
  Cc: Thomas Gleixner, Frederic Weisbecker, LKML, Anna-Maria Behnsen,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov

On Thu, May 11, 2023 at 12:12:32PM +0800, Pavel Tikhomirov wrote:
> Not sure how kernel memory consumption increases with sparse timer IDs,
> global hashtable (posix_timers_hashtable) is the same size anyway, entries
> in hlists can be distributed differently as hash depends on id directly but
> we have same number of entries. Probably I miss something, why do we need
> dense IDs?

The proposal was to remove the global hash and use a signal_struct based
xarray instead.

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11  4:12                     ` Pavel Tikhomirov
  2023-05-11  7:56                       ` Peter Zijlstra
@ 2023-05-11  9:32                       ` Thomas Gleixner
  1 sibling, 0 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-11  9:32 UTC (permalink / raw)
  To: Pavel Tikhomirov, Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov

On Thu, May 11 2023 at 12:12, Pavel Tikhomirov wrote:
> On 10.05.2023 16:30, Thomas Gleixner wrote:
>> The downside is that this allows to create stupidly sparse timer IDs
>> even for the non CRIU case, which increases per process kernel memory
>> consumption and creates slightly more overhead in the signal delivery
>> path. The latter is a burden on the process owning the timer and not
>> affecting expiry, which is a context stealing operation. The memory part
>> needs eventually some thoughts vs. accounting.
>> 
>> If the 'explicit at ID' option is not used then the ID mechanism is
>> optimzied for dense IDs by using the first available ID in a bottom up
>> search, which recovers holes created by a timer_delete() operation.
>
> Not sure how kernel memory consumption increases with sparse timer IDs, 
> global hashtable (posix_timers_hashtable) is the same size anyway, 
> entries in hlists can be distributed differently as hash depends on id 
> directly but we have same number of entries. Probably I miss something, 
> why do we need dense IDs?

Because I want to get rid of the global hash table as I explained in my
summary mail.

Thanks,

        tglx

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11  3:17                   ` Pavel Tikhomirov
@ 2023-05-11  9:36                     ` Thomas Gleixner
  2023-05-11  9:52                       ` Pavel Tikhomirov
  2023-05-12  1:21                       ` Andrey Vagin
  0 siblings, 2 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-11  9:36 UTC (permalink / raw)
  To: Pavel Tikhomirov, Andrey Vagin
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Emelyanov

On Thu, May 11 2023 at 11:17, Pavel Tikhomirov wrote:
> On 10.05.2023 16:16, Andrey Vagin wrote:
>>>
>>> So because of that half thought out user space ABI we are now up the
>>> regression creek without a paddle, unless CRIU can accomodate to a
>>> different restore mechanism to lift this restriction from the kernel.
>>>
>> If you give us a new API to create timers with specified id-s, we will
>> figure out how to live with it. It isn't good to ask users to update
>> CRIU to work on new kernels, but here are reasons and event improvements
>> for CRIU, so I think it's worth it.
>
> I agree, any API to create timers with specified id-s would work for new 
> CRIU versions.

The real question is whether this will cause any upheaval when a new
kernel meets a non-updated CRIU stack.

You know the UABI regression rules of the kernel...

Thanks,

        tglx


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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11  9:36                     ` Thomas Gleixner
@ 2023-05-11  9:52                       ` Pavel Tikhomirov
  2023-05-11 13:42                         ` Thomas Gleixner
  2023-05-12  1:21                       ` Andrey Vagin
  1 sibling, 1 reply; 122+ messages in thread
From: Pavel Tikhomirov @ 2023-05-11  9:52 UTC (permalink / raw)
  To: Thomas Gleixner, Andrey Vagin
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Emelyanov



On 11.05.2023 17:36, Thomas Gleixner wrote:
> On Thu, May 11 2023 at 11:17, Pavel Tikhomirov wrote:
>> On 10.05.2023 16:16, Andrey Vagin wrote:
>>>>
>>>> So because of that half thought out user space ABI we are now up the
>>>> regression creek without a paddle, unless CRIU can accomodate to a
>>>> different restore mechanism to lift this restriction from the kernel.
>>>>
>>> If you give us a new API to create timers with specified id-s, we will
>>> figure out how to live with it. It isn't good to ask users to update
>>> CRIU to work on new kernels, but here are reasons and event improvements
>>> for CRIU, so I think it's worth it.
>>
>> I agree, any API to create timers with specified id-s would work for new
>> CRIU versions.
> 
> The real question is whether this will cause any upheaval when a new
> kernel meets a non-updated CRIU stack.

Creation of posix timer would hang forever in this loop 
https://github.com/checkpoint-restore/criu/blob/33dd66c6fc93c47213aaa0447a94d97ba1fa56ba/criu/pie/restorer.c#L1185 
if old criu is run on new kernel (without consecutive id allocation) AFAICS.

> You know the UABI regression rules of the kernel...
> 
> Thanks,
> 
>          tglx
> 

-- 
Best regards, Tikhomirov Pavel
Senior Software Developer, Virtuozzo.

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

* RE: [RFD] posix-timers: CRIU woes
  2023-05-10  4:36                 ` Pavel Tikhomirov
  2023-05-10  8:30                   ` Thomas Gleixner
@ 2023-05-11 10:13                   ` David Laight
  1 sibling, 0 replies; 122+ messages in thread
From: David Laight @ 2023-05-11 10:13 UTC (permalink / raw)
  To: 'Pavel Tikhomirov', Thomas Gleixner, Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Andrei Vagin, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov

From: Pavel Tikhomirov
> Sent: 10 May 2023 05:37
...
> >      A dense ID space approach can create holes too, but they are
> >      recoverable and well within the resource limits, because the process
> >      has to be able to create enough timers in the first place in order
> >      to release those in the middle.

While it doesn't help at all for creating items with fixed ids,
my 'favourite' scheme for allocating ids it to allocate a number
that will be a perfect hash onto an empty hash table slot.

The lookup check is then just array[id & mask].id == id.
A FIFO freelist can be run through the free entries and
the high bits incremented each time a slot is used.
So allocation is usually fixed cost.

If the table is full it's size can easily be doubled.
If the number of unused entries is doubled each time
the table size is doubled then the you (more or less)
guarantee that an id won't get reused any time soon
after it is freed.

This would be ok for restoring ids allocated by the same
scheme. But would need a fallback for restoring pathological
list of ids.

	David

-
Registered Address Lakeside, Bramley Road, Mount Farm, Milton Keynes, MK1 1PT, UK
Registration No: 1397386 (Wales)

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11  9:52                       ` Pavel Tikhomirov
@ 2023-05-11 13:42                         ` Thomas Gleixner
  2023-05-11 14:54                           ` Pavel Tikhomirov
  2023-05-11 15:25                           ` Pavel Tikhomirov
  0 siblings, 2 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-11 13:42 UTC (permalink / raw)
  To: Pavel Tikhomirov, Andrey Vagin
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Emelyanov

On Thu, May 11 2023 at 17:52, Pavel Tikhomirov wrote:
> On 11.05.2023 17:36, Thomas Gleixner wrote:
>> On Thu, May 11 2023 at 11:17, Pavel Tikhomirov wrote:
>>> On 10.05.2023 16:16, Andrey Vagin wrote:
>>>>>
>>>>> So because of that half thought out user space ABI we are now up the
>>>>> regression creek without a paddle, unless CRIU can accomodate to a
>>>>> different restore mechanism to lift this restriction from the kernel.
>>>>>
>>>> If you give us a new API to create timers with specified id-s, we will
>>>> figure out how to live with it. It isn't good to ask users to update
>>>> CRIU to work on new kernels, but here are reasons and event improvements
>>>> for CRIU, so I think it's worth it.
>>>
>>> I agree, any API to create timers with specified id-s would work for new
>>> CRIU versions.
>> 
>> The real question is whether this will cause any upheaval when a new
>> kernel meets a non-updated CRIU stack.
>
> Creation of posix timer would hang forever in this loop 
> https://github.com/checkpoint-restore/criu/blob/33dd66c6fc93c47213aaa0447a94d97ba1fa56ba/criu/pie/restorer.c#L1185 
> if old criu is run on new kernel (without consecutive id allocation) AFAICS.

Yes, because that "sanity" check

     if ((long)next_id > args->posix_timers[i].spt.it_id)

which tries to establish whether the kernel provides timer IDs in strict
increasing order does not work for that case.

It "works" to detect the IDR case on older kernels by chance, but not
under all circumstances. Assume the following case:

      Global IDR has a free slot at index 1

      Restore tries to create a timer for index 2

That will also loop forever, unless some other process creates a timer
and occupies the free slot at index 1, right?

So this needs a fix anyway, which should be done so that the new kernel
case is at least properly detected.

But even then there is still the problem of "it worked before I upgraded
the kernel".

IOW, we are still up a creek without a paddle, unless you would be
willing to utilize the existing CRIU bug to distribute the 'deal with
new kernel' mechanics as a bug bounty :)

Fix for the loop termination below.

Thanks,

        tglx
---
 criu/pie/restorer.c |   24 +++++++++++++-----------
 1 file changed, 13 insertions(+), 11 deletions(-)

--- a/criu/pie/restorer.c
+++ b/criu/pie/restorer.c
@@ -1169,10 +1169,10 @@ static int timerfd_arm(struct task_resto
 static int create_posix_timers(struct task_restore_args *args)
 {
 	int ret, i;
-	kernel_timer_t next_id;
+	kernel_timer_t next_id, timer_id;
 	struct sigevent sev;
 
-	for (i = 0; i < args->posix_timers_n; i++) {
+	for (i = 0, next_id = 0; i < args->posix_timers_n; i++) {
 		sev.sigev_notify = args->posix_timers[i].spt.it_sigev_notify;
 		sev.sigev_signo = args->posix_timers[i].spt.si_signo;
 #ifdef __GLIBC__
@@ -1183,25 +1183,27 @@ static int create_posix_timers(struct ta
 		sev.sigev_value.sival_ptr = args->posix_timers[i].spt.sival_ptr;
 
 		while (1) {
-			ret = sys_timer_create(args->posix_timers[i].spt.clock_id, &sev, &next_id);
+			ret = sys_timer_create(args->posix_timers[i].spt.clock_id, &sev, &timer_id);
 			if (ret < 0) {
 				pr_err("Can't create posix timer - %d\n", i);
 				return ret;
 			}
 
-			if (next_id == args->posix_timers[i].spt.it_id)
+			if (timer_id != next_id) {
+				pr_err("Can't create timers, kernel don't give them consequently\n");
+				return -1;
+			}
+
+			next_id++;
+
+			if (timer_id == args->posix_timers[i].spt.it_id)
 				break;
 
-			ret = sys_timer_delete(next_id);
+			ret = sys_timer_delete(timer_id);
 			if (ret < 0) {
-				pr_err("Can't remove temporaty posix timer 0x%x\n", next_id);
+				pr_err("Can't remove temporaty posix timer 0x%x\n", timer_id);
 				return ret;
 			}
-
-			if ((long)next_id > args->posix_timers[i].spt.it_id) {
-				pr_err("Can't create timers, kernel don't give them consequently\n");
-				return -1;
-			}
 		}
 	}
 



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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11 13:42                         ` Thomas Gleixner
@ 2023-05-11 14:54                           ` Pavel Tikhomirov
  2023-05-11 15:25                           ` Pavel Tikhomirov
  1 sibling, 0 replies; 122+ messages in thread
From: Pavel Tikhomirov @ 2023-05-11 14:54 UTC (permalink / raw)
  To: Thomas Gleixner, Andrey Vagin
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Emelyanov



On 11.05.2023 21:42, Thomas Gleixner wrote:
> On Thu, May 11 2023 at 17:52, Pavel Tikhomirov wrote:
>> On 11.05.2023 17:36, Thomas Gleixner wrote:
>>> On Thu, May 11 2023 at 11:17, Pavel Tikhomirov wrote:
>>>> On 10.05.2023 16:16, Andrey Vagin wrote:
>>>>>>
>>>>>> So because of that half thought out user space ABI we are now up the
>>>>>> regression creek without a paddle, unless CRIU can accomodate to a
>>>>>> different restore mechanism to lift this restriction from the kernel.
>>>>>>
>>>>> If you give us a new API to create timers with specified id-s, we will
>>>>> figure out how to live with it. It isn't good to ask users to update
>>>>> CRIU to work on new kernels, but here are reasons and event improvements
>>>>> for CRIU, so I think it's worth it.
>>>>
>>>> I agree, any API to create timers with specified id-s would work for new
>>>> CRIU versions.
>>>
>>> The real question is whether this will cause any upheaval when a new
>>> kernel meets a non-updated CRIU stack.
>>
>> Creation of posix timer would hang forever in this loop
>> https://github.com/checkpoint-restore/criu/blob/33dd66c6fc93c47213aaa0447a94d97ba1fa56ba/criu/pie/restorer.c#L1185
>> if old criu is run on new kernel (without consecutive id allocation) AFAICS.
> 
> Yes, because that "sanity" check
> 
>       if ((long)next_id > args->posix_timers[i].spt.it_id)
> 
> which tries to establish whether the kernel provides timer IDs in strict
> increasing order does not work for that case.

Yes, this check is not perfect, but it does at least something: It 
detects that posix timer creation missed needed id (if you start from 0 
and increase by 1 each time you can not reach number > N before reaching 
N).

> 
> It "works" to detect the IDR case on older kernels by chance, but not
> under all circumstances. Assume the following case:
> 
>        Global IDR has a free slot at index 1
> 
>        Restore tries to create a timer for index 2
> 
> That will also loop forever, unless some other process creates a timer
> and occupies the free slot at index 1, right?

Yes on old-old kernel, where there were no ids increasing on 
create/delete, CRIU is broken, but CRIU does not support kernels earlier 
than 3.11 (https://criu.org/Check_the_kernel#Basic) so probably we are fine.

git describe --contains 5ed67f05f66c
v3.10-rc1~171^2~9

> 
> So this needs a fix anyway, which should be done so that the new kernel
> case is at least properly detected.
> 
> But even then there is still the problem of "it worked before I upgraded
> the kernel".
> 
> IOW, we are still up a creek without a paddle, unless you would be
> willing to utilize the existing CRIU bug to distribute the 'deal with
> new kernel' mechanics as a bug bounty :) >
> Fix for the loop termination below.

It fixes the loop for new kernel, I agree.

> 
> Thanks,
> 
>          tglx
> ---
>   criu/pie/restorer.c |   24 +++++++++++++-----------
>   1 file changed, 13 insertions(+), 11 deletions(-)
> 
> --- a/criu/pie/restorer.c
> +++ b/criu/pie/restorer.c
> @@ -1169,10 +1169,10 @@ static int timerfd_arm(struct task_resto
>   static int create_posix_timers(struct task_restore_args *args)
>   {
>   	int ret, i;
> -	kernel_timer_t next_id;
> +	kernel_timer_t next_id, timer_id;
>   	struct sigevent sev;
>   
> -	for (i = 0; i < args->posix_timers_n; i++) {
> +	for (i = 0, next_id = 0; i < args->posix_timers_n; i++) {
>   		sev.sigev_notify = args->posix_timers[i].spt.it_sigev_notify;
>   		sev.sigev_signo = args->posix_timers[i].spt.si_signo;
>   #ifdef __GLIBC__
> @@ -1183,25 +1183,27 @@ static int create_posix_timers(struct ta
>   		sev.sigev_value.sival_ptr = args->posix_timers[i].spt.sival_ptr;
>   
>   		while (1) {
> -			ret = sys_timer_create(args->posix_timers[i].spt.clock_id, &sev, &next_id);
> +			ret = sys_timer_create(args->posix_timers[i].spt.clock_id, &sev, &timer_id);
>   			if (ret < 0) {
>   				pr_err("Can't create posix timer - %d\n", i);
>   				return ret;
>   			}
>   
> -			if (next_id == args->posix_timers[i].spt.it_id)
> +			if (timer_id != next_id) {
> +				pr_err("Can't create timers, kernel don't give them consequently\n");
> +				return -1;
> +			}
> +
> +			next_id++;
> +
> +			if (timer_id == args->posix_timers[i].spt.it_id)
>   				break;
>   
> -			ret = sys_timer_delete(next_id);
> +			ret = sys_timer_delete(timer_id);
>   			if (ret < 0) {
> -				pr_err("Can't remove temporaty posix timer 0x%x\n", next_id);
> +				pr_err("Can't remove temporaty posix timer 0x%x\n", timer_id);
>   				return ret;
>   			}
> -
> -			if ((long)next_id > args->posix_timers[i].spt.it_id) {
> -				pr_err("Can't create timers, kernel don't give them consequently\n");
> -				return -1;
> -			}
>   		}
>   	}
>   
> 
> 

-- 
Best regards, Tikhomirov Pavel
Senior Software Developer, Virtuozzo.

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11 13:42                         ` Thomas Gleixner
  2023-05-11 14:54                           ` Pavel Tikhomirov
@ 2023-05-11 15:25                           ` Pavel Tikhomirov
  1 sibling, 0 replies; 122+ messages in thread
From: Pavel Tikhomirov @ 2023-05-11 15:25 UTC (permalink / raw)
  To: Thomas Gleixner, Andrey Vagin
  Cc: Frederic Weisbecker, LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk, Christian Brauner, Alexander Mikhalitsyn,
	Pavel Emelyanov



On 11.05.2023 21:42, Thomas Gleixner wrote:
> On Thu, May 11 2023 at 17:52, Pavel Tikhomirov wrote:
>> On 11.05.2023 17:36, Thomas Gleixner wrote:
>>> On Thu, May 11 2023 at 11:17, Pavel Tikhomirov wrote:
>>>> On 10.05.2023 16:16, Andrey Vagin wrote:
>>>>>>
>>>>>> So because of that half thought out user space ABI we are now up the
>>>>>> regression creek without a paddle, unless CRIU can accomodate to a
>>>>>> different restore mechanism to lift this restriction from the kernel.
>>>>>>
>>>>> If you give us a new API to create timers with specified id-s, we will
>>>>> figure out how to live with it. It isn't good to ask users to update
>>>>> CRIU to work on new kernels, but here are reasons and event improvements
>>>>> for CRIU, so I think it's worth it.
>>>>
>>>> I agree, any API to create timers with specified id-s would work for new
>>>> CRIU versions.
>>>
>>> The real question is whether this will cause any upheaval when a new
>>> kernel meets a non-updated CRIU stack.
>>
>> Creation of posix timer would hang forever in this loop
>> https://github.com/checkpoint-restore/criu/blob/33dd66c6fc93c47213aaa0447a94d97ba1fa56ba/criu/pie/restorer.c#L1185
>> if old criu is run on new kernel (without consecutive id allocation) AFAICS.
> 
> Yes, because that "sanity" check
> 
>       if ((long)next_id > args->posix_timers[i].spt.it_id)
> 
> which tries to establish whether the kernel provides timer IDs in strict
> increasing order does not work for that case.
> 
> It "works" to detect the IDR case on older kernels by chance, but not
> under all circumstances. Assume the following case:
> 
>        Global IDR has a free slot at index 1
> 
>        Restore tries to create a timer for index 2
> 
> That will also loop forever, unless some other process creates a timer
> and occupies the free slot at index 1, right?
> 
> So this needs a fix anyway, which should be done so that the new kernel
> case is at least properly detected.
> 
> But even then there is still the problem of "it worked before I upgraded
> the kernel".
> 
> IOW, we are still up a creek without a paddle, unless you would be
> willing to utilize the existing CRIU bug to distribute the 'deal with
> new kernel' mechanics as a bug bounty :)
> 
> Fix for the loop termination below.

I've prepared a PR with your patch (with minimal change) and added you 
Signed-off-by:, hope it's ok:
https://github.com/checkpoint-restore/criu/pull/2174

> 
> Thanks,
> 
>          tglx
> ---
>   criu/pie/restorer.c |   24 +++++++++++++-----------
>   1 file changed, 13 insertions(+), 11 deletions(-)
> 
> --- a/criu/pie/restorer.c
> +++ b/criu/pie/restorer.c
> @@ -1169,10 +1169,10 @@ static int timerfd_arm(struct task_resto
>   static int create_posix_timers(struct task_restore_args *args)
>   {
>   	int ret, i;
> -	kernel_timer_t next_id;
> +	kernel_timer_t next_id, timer_id;
>   	struct sigevent sev;
>   
> -	for (i = 0; i < args->posix_timers_n; i++) {
> +	for (i = 0, next_id = 0; i < args->posix_timers_n; i++) {
>   		sev.sigev_notify = args->posix_timers[i].spt.it_sigev_notify;
>   		sev.sigev_signo = args->posix_timers[i].spt.si_signo;
>   #ifdef __GLIBC__
> @@ -1183,25 +1183,27 @@ static int create_posix_timers(struct ta
>   		sev.sigev_value.sival_ptr = args->posix_timers[i].spt.sival_ptr;
>   
>   		while (1) {
> -			ret = sys_timer_create(args->posix_timers[i].spt.clock_id, &sev, &next_id);
> +			ret = sys_timer_create(args->posix_timers[i].spt.clock_id, &sev, &timer_id);
>   			if (ret < 0) {
>   				pr_err("Can't create posix timer - %d\n", i);
>   				return ret;
>   			}
>   
> -			if (next_id == args->posix_timers[i].spt.it_id)
> +			if (timer_id != next_id) {
> +				pr_err("Can't create timers, kernel don't give them consequently\n");
> +				return -1;
> +			}
> +
> +			next_id++;
> +
> +			if (timer_id == args->posix_timers[i].spt.it_id)
>   				break;
>   
> -			ret = sys_timer_delete(next_id);
> +			ret = sys_timer_delete(timer_id);
>   			if (ret < 0) {
> -				pr_err("Can't remove temporaty posix timer 0x%x\n", next_id);
> +				pr_err("Can't remove temporaty posix timer 0x%x\n", timer_id);
>   				return ret;
>   			}
> -
> -			if ((long)next_id > args->posix_timers[i].spt.it_id) {
> -				pr_err("Can't create timers, kernel don't give them consequently\n");
> -				return -1;
> -			}
>   		}
>   	}
>   
> 
> 

-- 
Best regards, Tikhomirov Pavel
Senior Software Developer, Virtuozzo.

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-11  9:36                     ` Thomas Gleixner
  2023-05-11  9:52                       ` Pavel Tikhomirov
@ 2023-05-12  1:21                       ` Andrey Vagin
  2023-05-31 17:38                         ` Thomas Gleixner
  1 sibling, 1 reply; 122+ messages in thread
From: Andrey Vagin @ 2023-05-12  1:21 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: Pavel Tikhomirov, Frederic Weisbecker, LKML, Anna-Maria Behnsen,
	Peter Zijlstra, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Sebastian Siewior, Michael Kerrisk, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov, Mike Rapoport,
	Dmitry Safonov, Adrian Reber

On Thu, May 11, 2023 at 2:36 AM Thomas Gleixner <tglx@linutronix.de> wrote:
>
> On Thu, May 11 2023 at 11:17, Pavel Tikhomirov wrote:
> > On 10.05.2023 16:16, Andrey Vagin wrote:
> >>>
> >>> So because of that half thought out user space ABI we are now up the
> >>> regression creek without a paddle, unless CRIU can accomodate to a
> >>> different restore mechanism to lift this restriction from the kernel.
> >>>
> >> If you give us a new API to create timers with specified id-s, we will
> >> figure out how to live with it. It isn't good to ask users to update
> >> CRIU to work on new kernels, but here are reasons and event improvements
> >> for CRIU, so I think it's worth it.
> >
> > I agree, any API to create timers with specified id-s would work for new
> > CRIU versions.
>
> The real question is whether this will cause any upheaval when a new
> kernel meets a non-updated CRIU stack.


It depends on what you mean by upheaval. We found that CRIU can be stuck
in a busy loop with the new changes. I suggest thinking about how to
work around this case and make sure that CRIU reports an error. The
error should minimize the time that users will need to spend to find the
reason and ways to resolve the problem.

One of the ways to fix the problem is to return indexes in a backward
direction from INT_MAX to zero. But in the kernel, user indices can be
converted back to "normal" values:
kernel_timer_id = INT_MAX - user_timer_id;

I have one idea of how to make these changes without breaking CRIU. CRIU
does a few special things. First, it does all timer operations from a
thread leader.  Second, it calls timer_settime only after creating all
timers. Third, it calls timer_delete for the last timer only. Any of
these events can be a trigger to switch to the new algo of allocating
timer id-s, but new processes allocate indices according to old rules.
It seems unfortunate that a real application will create a set of very
sparse indices without triggering one of these events.
I don't think that it is worth doing something like this, but if we want to
strictly follow the rules, it is the choice.

>
> You know the UABI regression rules of the kernel...

There is no rule without exceptions... With all pros and cons, we may
consider this case as an exception. From our side, we will try to make
everything to minimize the impact. Here are steps off the top of my
head:
* releasing the criu fix before the kernel release.
* update packages in Linux distros (Debian, Ubuntu, Fedora, and
  others that we will find).
* send an announcement to the criu mailing list and to users that we know.
* add the error to FAQ.
* create a GitHub issue with a full description.

Thanks,
Andrei

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

* Re: [RFD] posix-timers: CRIU woes
  2023-05-12  1:21                       ` Andrey Vagin
@ 2023-05-31 17:38                         ` Thomas Gleixner
  0 siblings, 0 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-05-31 17:38 UTC (permalink / raw)
  To: Andrey Vagin
  Cc: Pavel Tikhomirov, Frederic Weisbecker, LKML, Anna-Maria Behnsen,
	Peter Zijlstra, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov,
	Sebastian Siewior, Michael Kerrisk, Christian Brauner,
	Alexander Mikhalitsyn, Pavel Emelyanov, Mike Rapoport,
	Dmitry Safonov, Adrian Reber

Andrey!

On Thu, May 11 2023 at 18:21, Andrey Vagin wrote:
> On Thu, May 11, 2023 at 2:36 AM Thomas Gleixner <tglx@linutronix.de> wrote:
>>
>> You know the UABI regression rules of the kernel...
>
> There is no rule without exceptions... With all pros and cons, we may
> consider this case as an exception. From our side, we will try to make
> everything to minimize the impact. Here are steps off the top of my
> head:
> * releasing the criu fix before the kernel release.
> * update packages in Linux distros (Debian, Ubuntu, Fedora, and
>   others that we will find).
> * send an announcement to the criu mailing list and to users that we know.
> * add the error to FAQ.
> * create a GitHub issue with a full description.

Thanks for this plan. After digging deeper I managed to resolve the
actual problem I was chasing without changing that ID generator at
all.

The main pain point of having to do that lookup from the signal delivery
path is gone, which made it trivial to do the fix for the SIG_IGN mess
w/o these global lookups too.

Addressing this global ID issues I pointed out becomes therefore an
orthogonal issue which we can handle completely independent of the
kernel internal problems I'm trying to address.

I still think we should do that for sanity sake, but we can stage that
properly without dependencies outside of this particular ABI
problem. That makes me way more comfortable as that's something which
can be in the worst case reverted without doing any other damage.

Thanks,

        tglx

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

* Re: [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit()
  2023-04-25 18:49 ` [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit() Thomas Gleixner
@ 2023-06-01 10:09   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] posix-timers: Set k_itimer:: It_signal " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 10:09 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:06PM +0200, Thomas Gleixner wrote:
> Technically it's not required to set k_itimer::it_signal to NULL on exit()
> because there is no other thread anymore which could lookup the timer
> concurrently.
> 
> Set it to NULL for consistency sake and add a comment to that effect.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock
  2023-04-25 18:49 ` [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock Thomas Gleixner
@ 2023-06-01 10:12   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 10:12 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:08PM +0200, Thomas Gleixner wrote:
> All usage of hash_lock is in thread context. No point in using
> spin_lock_irqsave()/irqrestore() for a single usage site.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 09/20] posix-timers: Split release_posix_timers()
  2023-04-25 18:49 ` [patch 09/20] posix-timers: Split release_posix_timers() Thomas Gleixner
@ 2023-06-01 10:25   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 10:25 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:09PM +0200, Thomas Gleixner wrote:
> release_posix_timers() is called for cleaning up both hashed and unhashed
> timers. The cases are differentiated by an argument and the usage is
> hideous.
> 
> Seperate the actual free path out and use it for unhashed timers. Provide a
> function for hashed timers.
> 
> No functional change.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 10/20] posix-timers: Document sys_clock_getres() correctly
  2023-04-25 18:49 ` [patch 10/20] posix-timers: Document sys_clock_getres() correctly Thomas Gleixner
@ 2023-06-01 10:44   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 10:44 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Michael Kerrisk,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov

On Tue, Apr 25, 2023 at 08:49:11PM +0200, Thomas Gleixner wrote:
> +/**
> + * sys_clock_getres - Get the resolution of a clock
> + * @which_clock:	The clock to get the resolution for
> + * @tp:			Pointer to a a user space timespec64 for storage
                                           ^^^
a_a

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 11/20] posix-timers: Document common_clock_get() correctly
  2023-04-25 18:49 ` [patch 11/20] posix-timers: Document common_clock_get() correctly Thomas Gleixner
@ 2023-06-01 11:00   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 11:00 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:12PM +0200, Thomas Gleixner wrote:
> Replace another confusing and inaccurate set of comments.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 12/20] posix-timers: Document sys_clock_getoverrun()
  2023-04-25 18:49 ` [patch 12/20] posix-timers: Document sys_clock_getoverrun() Thomas Gleixner
@ 2023-06-01 11:06   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 11:06 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Michael Kerrisk,
	Sebastian Siewior, syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov

On Tue, Apr 25, 2023 at 08:49:14PM +0200, Thomas Gleixner wrote:
> Document the syscall in detail and with coherent sentences.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> Cc: Michael Kerrisk <mtk.manpages@gmail.com>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place
  2023-04-25 18:49 ` [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place Thomas Gleixner
@ 2023-06-01 11:22   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 11:22 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:16PM +0200, Thomas Gleixner wrote:
> The documentation of sys_clock_settime() permissions is at a random place
> and mostly word salad.
> 
> Remove it and add a concise comment into sys_clock_settime().
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 14/20] posix-timers: Document nanosleep() details
  2023-04-25 18:49 ` [patch 14/20] posix-timers: Document nanosleep() details Thomas Gleixner
@ 2023-06-01 12:30   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 12:30 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:17PM +0200, Thomas Gleixner wrote:
> The descriptions for common_nsleep() is wrong and common_nsleep_timens()
> lacks any form of comment.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 15/20] posix-timers: Add proper comments in do_timer_create()
  2023-04-25 18:49 ` [patch 15/20] posix-timers: Add proper comments in do_timer_create() Thomas Gleixner
@ 2023-06-01 12:43   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 12:43 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:19PM +0200, Thomas Gleixner wrote:
> The comment about timer lifetime at the end of the function is misplaced
> and uncomprehensible.
> 
> Make it understandable and put it at the right place. Add a new comment
> about the visibility of the new timer ID to user space.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly
  2023-04-25 18:49 ` [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly Thomas Gleixner
@ 2023-06-01 12:47   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 12:47 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:20PM +0200, Thomas Gleixner wrote:
> Replace the word salad.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment
  2023-04-25 18:49 ` [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment Thomas Gleixner
@ 2023-06-01 12:52   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 12:52 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:22PM +0200, Thomas Gleixner wrote:
> Yet another incomprehensible piece of art.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 18/20] posix-timers: Clarify posix_timer_fn() comments
  2023-04-25 18:49 ` [patch 18/20] posix-timers: Clarify posix_timer_fn() comments Thomas Gleixner
@ 2023-06-01 13:21   ` Frederic Weisbecker
  2023-06-01 18:43     ` Thomas Gleixner
  2023-06-01 19:07     ` Thomas Gleixner
  0 siblings, 2 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 13:21 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:24PM +0200, Thomas Gleixner wrote:
> Make the issues vs. SIG_IGN understandable and remove the 15 years old
> promise that a proper solution is already on the horizon.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> ---
>  kernel/time/posix-timers.c |   56 +++++++++++++++++++++------------------------
>  1 file changed, 27 insertions(+), 29 deletions(-)
> 
> --- a/kernel/time/posix-timers.c
> +++ b/kernel/time/posix-timers.c
> @@ -325,11 +325,11 @@ int posix_timer_event(struct k_itimer *t
>  }
>  
>  /*
> - * This function gets called when a POSIX.1b interval timer expires.  It
> - * is used as a callback from the kernel internal timer.  The
> - * run_timer_list code ALWAYS calls with interrupts on.
> -
> - * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
> + * This function gets called when a POSIX.1b interval timer expires from
> + * the HRTIMER soft interrupt with interrupts enabled.

BTW, what arranges for this to be called in softirq with interrupts enabled?
The modes I see used here are HRTIMER_MODE_ABS or HRTIMER_MODE_REL and not
their _SOFT counterparts.

> + *
> + * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
> + * based timers.
>   */
>  static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
>  {
> @@ -358,34 +359,31 @@ static enum hrtimer_restart posix_timer_
>  			 * FIXME: What we really want, is to stop this
>  			 * timer completely and restart it in case the
>  			 * SIG_IGN is removed. This is a non trivial
> -			 * change which involves sighand locking
> -			 * (sigh !), which we don't want to do late in
> -			 * the release cycle.
> +			 * change to the signal handling code.
> +			 *
> +			 * For now let timers with an interval less than a
> +			 * jiffie expire every jiffie to avoid softirq

Or rather at least to the next jiffie, right? Because then in the next jiffie
it gets re-evaluated in case a real signal handler might have been set
in-between.

Or it could be:

 +                      * For now let timers with an interval less than a
 +                      * jiffie expire every jiffie (until a real sig handler
 +			* is found set) to avoid softirq...

> +			 * starvation in case of SIG_IGN and a very small
> +			 * interval, which would put the timer right back
> +			 * on the softirq pending list. Moving now ahead of
> +			 * time tricks hrtimer_forward() to expire the
> +			 * timer later, while it still maintains the
> +			 * overrun accuracy for the price of a slightly
> +			 * inconsistency in the timer_gettime() case. This
> +			 * is at least better than a starved softirq.
[...]
>  			 */
> -#ifdef CONFIG_HIGH_RES_TIMERS
> -			{
> +			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
>  				ktime_t kj = NSEC_PER_SEC / HZ;

Could be TICK_NSECS?

Thanks!

>  
>  				if (timr->it_interval < kj)
>  					now = ktime_add(now, kj);
>  			}
> -#endif
> -			timr->it_overrun += hrtimer_forward(timer, now,
> -							    timr->it_interval);
> +
> +			timr->it_overrun += hrtimer_forward(timer, now, timr->it_interval);
>  			ret = HRTIMER_RESTART;
>  			++timr->it_requeue_pending;
>  			timr->it_active = 1;
> 

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

* Re: [patch 19/20] posix-timers: Remove pointless comments
  2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
@ 2023-06-01 13:48   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
                     ` (2 subsequent siblings)
  3 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 13:48 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:25PM +0200, Thomas Gleixner wrote:
> Documenting the obvious is just consuming space for no value.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 20/20] posix-timers: Polish coding style in a few places
  2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
@ 2023-06-01 13:50   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
                     ` (2 subsequent siblings)
  3 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-01 13:50 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Tue, Apr 25, 2023 at 08:49:27PM +0200, Thomas Gleixner wrote:
> Make it consistent with the TIP tree documentation.
> 
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 18/20] posix-timers: Clarify posix_timer_fn() comments
  2023-06-01 13:21   ` Frederic Weisbecker
@ 2023-06-01 18:43     ` Thomas Gleixner
  2023-06-01 19:07     ` Thomas Gleixner
  1 sibling, 0 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-06-01 18:43 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

On Thu, Jun 01 2023 at 15:21, Frederic Weisbecker wrote:
> On Tue, Apr 25, 2023 at 08:49:24PM +0200, Thomas Gleixner wrote:
>> Make the issues vs. SIG_IGN understandable and remove the 15 years old
>> promise that a proper solution is already on the horizon.
>> 
>> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
>> ---
>>  kernel/time/posix-timers.c |   56 +++++++++++++++++++++------------------------
>>  1 file changed, 27 insertions(+), 29 deletions(-)
>> 
>> --- a/kernel/time/posix-timers.c
>> +++ b/kernel/time/posix-timers.c
>> @@ -325,11 +325,11 @@ int posix_timer_event(struct k_itimer *t
>>  }
>>  
>>  /*
>> - * This function gets called when a POSIX.1b interval timer expires.  It
>> - * is used as a callback from the kernel internal timer.  The
>> - * run_timer_list code ALWAYS calls with interrupts on.
>> -
>> - * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
>> + * This function gets called when a POSIX.1b interval timer expires from
>> + * the HRTIMER soft interrupt with interrupts enabled.
>
> BTW, what arranges for this to be called in softirq with interrupts enabled?
> The modes I see used here are HRTIMER_MODE_ABS or HRTIMER_MODE_REL and not
> their _SOFT counterparts.

Duh. My RT biased brain tricked me.

>> + *
>> + * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
>> + * based timers.
>>   */
>>  static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
>>  {
>> @@ -358,34 +359,31 @@ static enum hrtimer_restart posix_timer_
>>  			 * FIXME: What we really want, is to stop this
>>  			 * timer completely and restart it in case the
>>  			 * SIG_IGN is removed. This is a non trivial
>> -			 * change which involves sighand locking
>> -			 * (sigh !), which we don't want to do late in
>> -			 * the release cycle.
>> +			 * change to the signal handling code.
>> +			 *
>> +			 * For now let timers with an interval less than a
>> +			 * jiffie expire every jiffie to avoid softirq
>
> Or rather at least to the next jiffie, right? Because then in the next jiffie
> it gets re-evaluated in case a real signal handler might have been set
> in-between.
>
> Or it could be:
>
>  +                      * For now let timers with an interval less than a
>  +                      * jiffie expire every jiffie (until a real sig handler
>  +			* is found set) to avoid softirq...

Let me rephrase that.

>> +			 * starvation in case of SIG_IGN and a very small
>> +			 * interval, which would put the timer right back
>> +			 * on the softirq pending list. Moving now ahead of
>> +			 * time tricks hrtimer_forward() to expire the
>> +			 * timer later, while it still maintains the
>> +			 * overrun accuracy for the price of a slightly
>> +			 * inconsistency in the timer_gettime() case. This
>> +			 * is at least better than a starved softirq.
> [...]
>>  			 */
>> -#ifdef CONFIG_HIGH_RES_TIMERS
>> -			{
>> +			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
>>  				ktime_t kj = NSEC_PER_SEC / HZ;
>
> Could be TICK_NSECS?

Yep. Fixed it up.

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

* [patch v2 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-05-09 14:18             ` Frederic Weisbecker
@ 2023-06-01 18:58               ` Thomas Gleixner
  2023-06-05 14:17                 ` Frederic Weisbecker
                                   ` (2 more replies)
  0 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-06-01 18:58 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

posix_timer_add() tries to allocate a posix timer ID by starting from the
cached ID which was stored by the last successful allocation.

This is done in a loop searching the ID space for a free slot one by
one. The loop has to terminate when the search wrapped around to the
starting point.

But that's racy vs. establishing the starting point. That is read out
lockless, which leads to the following problem:

CPU0	  	      	     	   CPU1
posix_timer_add()
  start = sig->posix_timer_id;
  lock(hash_lock);
  ...				   posix_timer_add()
  if (++sig->posix_timer_id < 0)
      			             start = sig->posix_timer_id;
     sig->posix_timer_id = 0;

So CPU1 can observe a negative start value, i.e. -1, and the loop break
never happens because the condition can never be true:

  if (sig->posix_timer_id == start)
     break;

While this is unlikely to ever turn into an endless loop as the ID space is
huge (INT_MAX), the racy read of the start value caught the attention of
KCSAN and Dmitry unearthed that incorrectness.

Rewrite it so that all id operations are under the hash lock.

Reported-by: syzbot+5c54bd3eb218bb595aa9@syzkaller.appspotmail.com
Reported-by: Dmitry Vyukov <dvyukov@google.com>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2: Make the loop less hideous.
---
 include/linux/sched/signal.h |    2 +-
 kernel/time/posix-timers.c   |   31 ++++++++++++++++++-------------
 2 files changed, 19 insertions(+), 14 deletions(-)

--- a/include/linux/sched/signal.h
+++ b/include/linux/sched/signal.h
@@ -135,7 +135,7 @@ struct signal_struct {
 #ifdef CONFIG_POSIX_TIMERS
 
 	/* POSIX.1b Interval Timers */
-	int			posix_timer_id;
+	unsigned int		next_posix_timer_id;
 	struct list_head	posix_timers;
 
 	/* ITIMER_REAL timer for the process */
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -140,25 +140,30 @@ static struct k_itimer *posix_timer_by_i
 static int posix_timer_add(struct k_itimer *timer)
 {
 	struct signal_struct *sig = current->signal;
-	int first_free_id = sig->posix_timer_id;
 	struct hlist_head *head;
-	int ret = -ENOENT;
+	unsigned int cnt, id;
 
-	do {
+	/*
+	 * FIXME: Replace this by a per signal struct xarray once there is
+	 * a plan to handle the resulting CRIU regression gracefully.
+	 */
+	for (cnt = 0; cnt <= INT_MAX; cnt++) {
 		spin_lock(&hash_lock);
-		head = &posix_timers_hashtable[hash(sig, sig->posix_timer_id)];
-		if (!__posix_timers_find(head, sig, sig->posix_timer_id)) {
+		id = sig->next_posix_timer_id;
+
+		/* Write the next ID back. Clamp it to the positive space */
+		sig->next_posix_timer_id = (id + 1) & INT_MAX;
+
+		head = &posix_timers_hashtable[hash(sig, id)];
+		if (!__posix_timers_find(head, sig, id)) {
 			hlist_add_head_rcu(&timer->t_hash, head);
-			ret = sig->posix_timer_id;
+			spin_unlock(&hash_lock);
+			return id;
 		}
-		if (++sig->posix_timer_id < 0)
-			sig->posix_timer_id = 0;
-		if ((sig->posix_timer_id == first_free_id) && (ret == -ENOENT))
-			/* Loop over all possible ids completed */
-			ret = -EAGAIN;
 		spin_unlock(&hash_lock);
-	} while (ret == -ENOENT);
-	return ret;
+	}
+	/* POSIX return code when no timer ID could be allocated */
+	return -EAGAIN;
 }
 
 static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)

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

* [patch v2 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-05-05  7:57       ` Thomas Gleixner
@ 2023-06-01 19:00         ` Thomas Gleixner
  2023-06-01 20:16           ` [patch v2a " Thomas Gleixner
  0 siblings, 1 reply; 122+ messages in thread
From: Thomas Gleixner @ 2023-06-01 19:00 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

itimer_delete() has a retry loop when the timer is concurrently expired. On
non-RT kernels this just spin-waits until the timer callback has completed.
On RT kernels this is a potential livelock when the exiting task preempted
the hrtimer soft interrupt.

Replace spin_unlock() with an invocation of timer_wait_running() to handle
it the same way as the other retry loops in the posix timer code.

Fixes: ec8f954a40da ("posix-timers: Use a callback for cancel synchronization on PREEMPT_RT")
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2: Remove the bogus claims about posix CPU timers - Frederic
---
 kernel/time/posix-timers.c |   50 +++++++++++++++++++++++++++++++++++++--------
 1 file changed, 42 insertions(+), 8 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1037,27 +1037,59 @@ SYSCALL_DEFINE1(timer_delete, timer_t, t
 }
 
 /*
- * return timer owned by the process, used by exit_itimers
+ * Delete a timer if it is armed, remove it from the hash and schedule it
+ * for RCU freeing.
  */
 static void itimer_delete(struct k_itimer *timer)
 {
-retry_delete:
-	spin_lock_irq(&timer->it_lock);
+	unsigned long flags;
 
+retry_delete:
+	/*
+	 * irqsave is required to make timer_wait_running() work.
+	 */
+	spin_lock_irqsave(&timer->it_lock, flags);
+
+	/*
+	 * Even if the timer is not longer accessible from other tasks
+	 * it still might be armed and queued in the underlying timer
+	 * mechanism. Worse, that timer mechanism might run the expiry
+	 * function concurrently.
+	 */
 	if (timer_delete_hook(timer) == TIMER_RETRY) {
-		spin_unlock_irq(&timer->it_lock);
+		/*
+		 * Timer is expired concurrently, prevent livelocks
+		 * and pointless spinning on RT.
+		 *
+		 * The CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y case is
+		 * irrelevant here because obviously the exiting task
+		 * cannot be expiring timer in task work concurrently.
+		 * Ditto for CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n as the
+		 * tick interrupt cannot run on this CPU because the above
+		 * spin_lock disabled interrupts.
+		 *
+		 * timer_wait_running() drops timer::it_lock, which opens
+		 * the possibility for another task to delete the timer.
+		 *
+		 * That's not possible here because this is invoked from
+		 * do_exit() only for the last thread of the thread group.
+		 * So no other task can access that timer.
+		 */
+		if (WARN_ON_ONCE(timer_wait_running(timer, &flags) != timer))
+			return;
+
 		goto retry_delete;
 	}
 	list_del(&timer->list);
 
-	spin_unlock_irq(&timer->it_lock);
+	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }
 
 /*
- * This is called by do_exit or de_thread, only when nobody else can
- * modify the signal->posix_timers list. Yet we need sighand->siglock
- * to prevent the race with /proc/pid/timers.
+ * Invoked from do_exit() when the last thread of a thread group exits.
+ * At that point no other task can access the timers of the dying
+ * task anymore.
  */
 void exit_itimers(struct task_struct *tsk)
 {
@@ -1067,10 +1099,12 @@ void exit_itimers(struct task_struct *ts
 	if (list_empty(&tsk->signal->posix_timers))
 		return;
 
+	/* Protect against concurrent read via /proc/$PID/timers */
 	spin_lock_irq(&tsk->sighand->siglock);
 	list_replace_init(&tsk->signal->posix_timers, &timers);
 	spin_unlock_irq(&tsk->sighand->siglock);
 
+	/* The timers are not longer accessible via tsk::signal */
 	while (!list_empty(&timers)) {
 		tmr = list_first_entry(&timers, struct k_itimer, list);
 		itimer_delete(tmr);


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

* Re: [patch 18/20] posix-timers: Clarify posix_timer_fn() comments
  2023-06-01 13:21   ` Frederic Weisbecker
  2023-06-01 18:43     ` Thomas Gleixner
@ 2023-06-01 19:07     ` Thomas Gleixner
  2023-06-05 14:26       ` Frederic Weisbecker
                         ` (3 more replies)
  1 sibling, 4 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-06-01 19:07 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

Make the issues vs. SIG_IGN understandable and remove the 15 years old
promise that a proper solution is already on the horizon.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2: Clarify comments and use TICK_NSECS - Frederic
---
 kernel/time/posix-timers.c |   62 +++++++++++++++++++++++----------------------
 1 file changed, 32 insertions(+), 30 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -326,11 +326,11 @@ int posix_timer_event(struct k_itimer *t
 }
 
 /*
- * This function gets called when a POSIX.1b interval timer expires.  It
- * is used as a callback from the kernel internal timer.  The
- * run_timer_list code ALWAYS calls with interrupts on.
-
- * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
+ * This function gets called when a POSIX.1b interval timer expires from
+ * the HRTIMER interrupt (soft interrupt on RT kernels).
+ *
+ * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
+ * based timers.
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
@@ -348,9 +348,10 @@ static enum hrtimer_restart posix_timer_
 
 	if (posix_timer_event(timr, si_private)) {
 		/*
-		 * signal was not sent because of sig_ignor
-		 * we will not get a call back to restart it AND
-		 * it should be restarted.
+		 * The signal was not queued due to SIG_IGN. As a
+		 * consequence the timer is not going to be rearmed from
+		 * the signal delivery path. But as a real signal handler
+		 * can be installed later the timer must be rearmed here.
 		 */
 		if (timr->it_interval != 0) {
 			ktime_t now = hrtimer_cb_get_time(timer);
@@ -359,34 +360,35 @@ static enum hrtimer_restart posix_timer_
 			 * FIXME: What we really want, is to stop this
 			 * timer completely and restart it in case the
 			 * SIG_IGN is removed. This is a non trivial
-			 * change which involves sighand locking
-			 * (sigh !), which we don't want to do late in
-			 * the release cycle.
+			 * change to the signal handling code.
+			 *
+			 * For now let timers with an interval less than a
+			 * jiffie expire every jiffie and recheck for a
+			 * valid signal handler.
+			 *
+			 * This avoids interrupt starvation in case of a
+			 * very small interval, which would expire the
+			 * timer immediately again.
 			 *
-			 * For now we just let timers with an interval
-			 * less than a jiffie expire every jiffie to
-			 * avoid softirq starvation in case of SIG_IGN
-			 * and a very small interval, which would put
-			 * the timer right back on the softirq pending
-			 * list. By moving now ahead of time we trick
-			 * hrtimer_forward() to expire the timer
-			 * later, while we still maintain the overrun
-			 * accuracy, but have some inconsistency in
-			 * the timer_gettime() case. This is at least
-			 * better than a starved softirq. A more
-			 * complex fix which solves also another related
-			 * inconsistency is already in the pipeline.
+			 * Moving now ahead of time by one jiffie tricks
+			 * hrtimer_forward() to expire the timer later,
+			 * while it still maintains the overrun accuracy
+			 * for the price of a slight inconsistency in the
+			 * timer_gettime() case. This is at least better
+			 * than a starved softirq.
+			 *
+			 * Only required when high resolution timers are
+			 * enabled as the periodic tick based timers are
+			 * automatically aligned to the next tick.
 			 */
-#ifdef CONFIG_HIGH_RES_TIMERS
-			{
-				ktime_t kj = NSEC_PER_SEC / HZ;
+			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
+				ktime_t kj = TICK_NSECS;
 
 				if (timr->it_interval < kj)
 					now = ktime_add(now, kj);
 			}
-#endif
-			timr->it_overrun += hrtimer_forward(timer, now,
-							    timr->it_interval);
+
+			timr->it_overrun += hrtimer_forward(timer, now, timr->it_interval);
 			ret = HRTIMER_RESTART;
 			++timr->it_requeue_pending;
 			timr->it_active = 1;

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

* [patch v2a 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-06-01 19:00         ` [patch v2 " Thomas Gleixner
@ 2023-06-01 20:16           ` Thomas Gleixner
  2023-06-05 10:59             ` Frederic Weisbecker
                               ` (2 more replies)
  0 siblings, 3 replies; 122+ messages in thread
From: Thomas Gleixner @ 2023-06-01 20:16 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

itimer_delete() has a retry loop when the timer is concurrently expired. On
non-RT kernels this just spin-waits until the timer callback has completed.
On RT kernels this is a potential livelock when the exiting task preempted
the hrtimer soft interrupt.

Replace spin_unlock() with an invocation of timer_wait_running() to handle
it the same way as the other retry loops in the posix timer code.

Fixes: ec8f954a40da ("posix-timers: Use a callback for cancel synchronization on PREEMPT_RT")
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
---
V2:  Remove bogus comments vs. posix CPU timers - Frederic
V2a: Send the real fixed up version
---
 kernel/time/posix-timers.c |   43 +++++++++++++++++++++++++++++++++++--------
 1 file changed, 35 insertions(+), 8 deletions(-)

--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1037,27 +1037,52 @@ SYSCALL_DEFINE1(timer_delete, timer_t, t
 }
 
 /*
- * return timer owned by the process, used by exit_itimers
+ * Delete a timer if it is armed, remove it from the hash and schedule it
+ * for RCU freeing.
  */
 static void itimer_delete(struct k_itimer *timer)
 {
-retry_delete:
-	spin_lock_irq(&timer->it_lock);
+	unsigned long flags;
 
+retry_delete:
+	/*
+	 * irqsave is required to make timer_wait_running() work.
+	 */
+	spin_lock_irqsave(&timer->it_lock, flags);
+
+	/*
+	 * Even if the timer is not longer accessible from other tasks
+	 * it still might be armed and queued in the underlying timer
+	 * mechanism. Worse, that timer mechanism might run the expiry
+	 * function concurrently.
+	 */
 	if (timer_delete_hook(timer) == TIMER_RETRY) {
-		spin_unlock_irq(&timer->it_lock);
+		/*
+		 * Timer is expired concurrently, prevent livelocks
+		 * and pointless spinning on RT.
+		 *
+		 * timer_wait_running() drops timer::it_lock, which opens
+		 * the possibility for another task to delete the timer.
+		 *
+		 * That's not possible here because this is invoked from
+		 * do_exit() only for the last thread of the thread group.
+		 * So no other task can access and delete that timer.
+		 */
+		if (WARN_ON_ONCE(timer_wait_running(timer, &flags) != timer))
+			return;
+
 		goto retry_delete;
 	}
 	list_del(&timer->list);
 
-	spin_unlock_irq(&timer->it_lock);
+	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }
 
 /*
- * This is called by do_exit or de_thread, only when nobody else can
- * modify the signal->posix_timers list. Yet we need sighand->siglock
- * to prevent the race with /proc/pid/timers.
+ * Invoked from do_exit() when the last thread of a thread group exits.
+ * At that point no other task can access the timers of the dying
+ * task anymore.
  */
 void exit_itimers(struct task_struct *tsk)
 {
@@ -1067,10 +1092,12 @@ void exit_itimers(struct task_struct *ts
 	if (list_empty(&tsk->signal->posix_timers))
 		return;
 
+	/* Protect against concurrent read via /proc/$PID/timers */
 	spin_lock_irq(&tsk->sighand->siglock);
 	list_replace_init(&tsk->signal->posix_timers, &timers);
 	spin_unlock_irq(&tsk->sighand->siglock);
 
+	/* The timers are not longer accessible via tsk::signal */
 	while (!list_empty(&timers)) {
 		tmr = list_first_entry(&timers, struct k_itimer, list);
 		itimer_delete(tmr);

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

* Re: [patch v2a 01/20] posix-timers: Prevent RT livelock in itimer_delete()
  2023-06-01 20:16           ` [patch v2a " Thomas Gleixner
@ 2023-06-05 10:59             ` Frederic Weisbecker
  2023-06-05 15:08             ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50             ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-05 10:59 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

Le Thu, Jun 01, 2023 at 10:16:34PM +0200, Thomas Gleixner a écrit :
> itimer_delete() has a retry loop when the timer is concurrently expired. On
> non-RT kernels this just spin-waits until the timer callback has completed.
> On RT kernels this is a potential livelock when the exiting task preempted
> the hrtimer soft interrupt.

It's not just RT but also archs supporting HAVE_POSIX_CPU_TIMERS_TASK_WORK

> 
> Replace spin_unlock() with an invocation of timer_wait_running() to handle
> it the same way as the other retry loops in the posix timer code.
> 
> Fixes: ec8f954a40da ("posix-timers: Use a callback for cancel synchronization on PREEMPT_RT")
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
> ---
> V2:  Remove bogus comments vs. posix CPU timers - Frederic
> V2a: Send the real fixed up version
> ---
>  kernel/time/posix-timers.c |   43 +++++++++++++++++++++++++++++++++++--------
>  1 file changed, 35 insertions(+), 8 deletions(-)
> 
> --- a/kernel/time/posix-timers.c
> +++ b/kernel/time/posix-timers.c
> @@ -1037,27 +1037,52 @@ SYSCALL_DEFINE1(timer_delete, timer_t, t
>  }
>  
>  /*
> - * return timer owned by the process, used by exit_itimers
> + * Delete a timer if it is armed, remove it from the hash and schedule it
> + * for RCU freeing.
>   */
>  static void itimer_delete(struct k_itimer *timer)
>  {
> -retry_delete:
> -	spin_lock_irq(&timer->it_lock);
> +	unsigned long flags;
>  
> +retry_delete:
> +	/*
> +	 * irqsave is required to make timer_wait_running() work.
> +	 */
> +	spin_lock_irqsave(&timer->it_lock, flags);
> +
> +	/*
> +	 * Even if the timer is not longer accessible from other tasks
> +	 * it still might be armed and queued in the underlying timer
> +	 * mechanism. Worse, that timer mechanism might run the expiry
> +	 * function concurrently.
> +	 */
>  	if (timer_delete_hook(timer) == TIMER_RETRY) {
> -		spin_unlock_irq(&timer->it_lock);
> +		/*
> +		 * Timer is expired concurrently, prevent livelocks
> +		 * and pointless spinning on RT.

Ditto.

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

Thanks.

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

* Re: [patch v2 02/20] posix-timers: Ensure timer ID search-loop limit is valid
  2023-06-01 18:58               ` [patch v2 " Thomas Gleixner
@ 2023-06-05 14:17                 ` Frederic Weisbecker
  2023-06-05 15:08                 ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
  2023-06-18 20:50                 ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-05 14:17 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Sebastian Siewior,
	Michael Kerrisk

Le Thu, Jun 01, 2023 at 08:58:47PM +0200, Thomas Gleixner a écrit :
> posix_timer_add() tries to allocate a posix timer ID by starting from the
> cached ID which was stored by the last successful allocation.
> 
> This is done in a loop searching the ID space for a free slot one by
> one. The loop has to terminate when the search wrapped around to the
> starting point.
> 
> But that's racy vs. establishing the starting point. That is read out
> lockless, which leads to the following problem:
> 
> CPU0	  	      	     	   CPU1
> posix_timer_add()
>   start = sig->posix_timer_id;
>   lock(hash_lock);
>   ...				   posix_timer_add()
>   if (++sig->posix_timer_id < 0)
>       			             start = sig->posix_timer_id;
>      sig->posix_timer_id = 0;
> 
> So CPU1 can observe a negative start value, i.e. -1, and the loop break
> never happens because the condition can never be true:
> 
>   if (sig->posix_timer_id == start)
>      break;
> 
> While this is unlikely to ever turn into an endless loop as the ID space is
> huge (INT_MAX), the racy read of the start value caught the attention of
> KCSAN and Dmitry unearthed that incorrectness.
> 
> Rewrite it so that all id operations are under the hash lock.
> 
> Reported-by: syzbot+5c54bd3eb218bb595aa9@syzkaller.appspotmail.com
> Reported-by: Dmitry Vyukov <dvyukov@google.com>
> Signed-off-by: Thomas Gleixner <tglx@linutronix.de>

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 18/20] posix-timers: Clarify posix_timer_fn() comments
  2023-06-01 19:07     ` Thomas Gleixner
@ 2023-06-05 14:26       ` Frederic Weisbecker
  2023-06-05 15:08       ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
                         ` (2 subsequent siblings)
  3 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-05 14:26 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

Le Thu, Jun 01, 2023 at 09:07:37PM +0200, Thomas Gleixner a écrit :
> @@ -359,34 +360,35 @@ static enum hrtimer_restart posix_timer_
>  			 * FIXME: What we really want, is to stop this
>  			 * timer completely and restart it in case the
>  			 * SIG_IGN is removed. This is a non trivial
> -			 * change which involves sighand locking
> -			 * (sigh !), which we don't want to do late in
> -			 * the release cycle.
> +			 * change to the signal handling code.
> +			 *
> +			 * For now let timers with an interval less than a
> +			 * jiffie expire every jiffie and recheck for a
> +			 * valid signal handler.
> +			 *
> +			 * This avoids interrupt starvation in case of a
> +			 * very small interval, which would expire the
> +			 * timer immediately again.
>  			 *
> -			 * For now we just let timers with an interval
> -			 * less than a jiffie expire every jiffie to
> -			 * avoid softirq starvation in case of SIG_IGN
> -			 * and a very small interval, which would put
> -			 * the timer right back on the softirq pending
> -			 * list. By moving now ahead of time we trick
> -			 * hrtimer_forward() to expire the timer
> -			 * later, while we still maintain the overrun
> -			 * accuracy, but have some inconsistency in
> -			 * the timer_gettime() case. This is at least
> -			 * better than a starved softirq. A more
> -			 * complex fix which solves also another related
> -			 * inconsistency is already in the pipeline.
> +			 * Moving now ahead of time by one jiffie tricks
> +			 * hrtimer_forward() to expire the timer later,
> +			 * while it still maintains the overrun accuracy
> +			 * for the price of a slight inconsistency in the
> +			 * timer_gettime() case. This is at least better
> +			 * than a starved softirq.

Could be hardirq. How about:

"This is at least better than a timer storm."

Reviewed-by: Frederic Weisbecker <frederic@kernel.org>

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

* Re: [patch 00/20] posix-timers: Fixes and cleanups
  2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
                   ` (19 preceding siblings ...)
  2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
@ 2023-06-05 14:32 ` Frederic Weisbecker
  20 siblings, 0 replies; 122+ messages in thread
From: Frederic Weisbecker @ 2023-06-05 14:32 UTC (permalink / raw)
  To: Thomas Gleixner
  Cc: LKML, Anna-Maria Behnsen, Peter Zijlstra, Sebastian Siewior,
	syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Michael Kerrisk

Le Tue, Apr 25, 2023 at 08:48:55PM +0200, Thomas Gleixner a écrit :
> Hi!
> 
> A recent syzcaller/KCSAN report about a data race and an actually broken
> loop termination due to that race made me look deeper into the posix timer
> code.
> 
> Aside of the loop termination issue this unearthed another issue in the
> exit() path where timer deletion might livelock on RT enabled kernels.
> 
> While going through the code with a fine comb, I stumbled over another
> place which needs READ/WRITE_ONCE() annotations, tons of outdated and/or
> uncomprehensible comments and a bunch of silly code.
> 
> The series has therefore a larger set of cleanups on top of the two fixes.
> 
> It is based on
> 
>    git://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git timers/core
> 
> and also availble from git:
> 
>    git://git.kernel.org/pub/scm/linux/kernel/git/tglx/devel.git timers/posix
> 
> Thanks,
> 
> 	tglx

Thanks for this extremely useful series!

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

* [tip: timers/core] posix-timers: Polish coding style in a few places
  2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
  2023-06-01 13:50   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     1263a2a9d71bac5ffabf9603c36e36cb6edbcdcf
Gitweb:        https://git.kernel.org/tip/1263a2a9d71bac5ffabf9603c36e36cb6edbcdcf
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:27 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:39 +02:00

posix-timers: Polish coding style in a few places

Make it consistent with the TIP tree documentation.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.888493625@linutronix.de

---
 kernel/time/posix-timers.c | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index fde1ca9..276f231 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -309,10 +309,10 @@ int posix_timer_event(struct k_itimer *timr, int si_private)
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
+	enum hrtimer_restart ret = HRTIMER_NORESTART;
 	struct k_itimer *timr;
 	unsigned long flags;
 	int si_private = 0;
-	enum hrtimer_restart ret = HRTIMER_NORESTART;
 
 	timr = container_of(timer, struct k_itimer, it.real.timer);
 	spin_lock_irqsave(&timr->it_lock, flags);
@@ -400,8 +400,8 @@ static struct pid *good_sigevent(sigevent_t * event)
 
 static struct k_itimer * alloc_posix_timer(void)
 {
-	struct k_itimer *tmr;
-	tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+	struct k_itimer *tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+
 	if (!tmr)
 		return tmr;
 	if (unlikely(!(tmr->sigq = sigqueue_alloc()))) {
@@ -695,8 +695,8 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
-	struct k_itimer *timr;
 	const struct k_clock *kc;
+	struct k_itimer *timr;
 	unsigned long flags;
 	int ret = 0;
 
@@ -767,8 +767,8 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t, timer_id,
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {
 	struct k_itimer *timr;
-	int overrun;
 	unsigned long flags;
+	int overrun;
 
 	timr = lock_timer(timer_id, &flags);
 	if (!timr)
@@ -941,8 +941,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t, timer_id, int, flags,
 		const struct __kernel_itimerspec __user *, new_setting,
 		struct __kernel_itimerspec __user *, old_setting)
 {
-	struct itimerspec64 new_spec, old_spec;
-	struct itimerspec64 *rtn = old_setting ? &old_spec : NULL;
+	struct itimerspec64 new_spec, old_spec, *rtn;
 	int error = 0;
 
 	if (!new_setting)
@@ -951,6 +950,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t, timer_id, int, flags,
 	if (get_itimerspec64(&new_spec, new_setting))
 		return -EFAULT;
 
+	rtn = old_setting ? &old_spec : NULL;
 	error = do_timer_settime(timer_id, flags, &new_spec, rtn);
 	if (!error && old_setting) {
 		if (put_itimerspec64(&old_spec, old_setting))

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

* [tip: timers/core] posix-timers: Remove pointless comments
  2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
  2023-06-01 13:48   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     478a93ac9ed7da5e7cd98e00502b142e22dab4d7
Gitweb:        https://git.kernel.org/tip/478a93ac9ed7da5e7cd98e00502b142e22dab4d7
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:25 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:39 +02:00

posix-timers: Remove pointless comments

Documenting the obvious is just consuming space for no value.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.832240451@linutronix.de

---
 kernel/time/posix-timers.c | 25 -------------------------
 1 file changed, 25 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index a22c183..fde1ca9 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -59,19 +59,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 
-/*
- * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
- *	    to implement others.  This structure defines the various
- *	    clocks.
- *
- * FUNCTIONS: The CLOCKs structure defines possible functions to
- *	    handle various clock functions.
- *
- *	    The standard POSIX timer management code assumes the
- *	    following: 1.) The k_itimer struct (sched.h) is used for
- *	    the timer.  2.) The list, it_lock, it_clock, it_id and
- *	    it_pid fields are not modified by timer code.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
 #define lock_timer(tid, flags)						   \
@@ -141,7 +128,6 @@ static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)
 	spin_unlock_irqrestore(&timr->it_lock, flags);
 }
 
-/* Get clock_realtime */
 static int posix_get_realtime_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_real_ts64(tp);
@@ -153,7 +139,6 @@ static ktime_t posix_get_realtime_ktime(clockid_t which_clock)
 	return ktime_get_real();
 }
 
-/* Set clock_realtime */
 static int posix_clock_realtime_set(const clockid_t which_clock,
 				    const struct timespec64 *tp)
 {
@@ -166,9 +151,6 @@ static int posix_clock_realtime_adj(const clockid_t which_clock,
 	return do_adjtimex(t);
 }
 
-/*
- * Get monotonic time for posix timers
- */
 static int posix_get_monotonic_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_ts64(tp);
@@ -181,9 +163,6 @@ static ktime_t posix_get_monotonic_ktime(clockid_t which_clock)
 	return ktime_get();
 }
 
-/*
- * Get monotonic-raw time for posix timers
- */
 static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_raw_ts64(tp);
@@ -191,7 +170,6 @@ static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 	return 0;
 }
 
-
 static int posix_get_realtime_coarse(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_coarse_real_ts64(tp);
@@ -242,9 +220,6 @@ static int posix_get_hrtimer_res(clockid_t which_clock, struct timespec64 *tp)
 	return 0;
 }
 
-/*
- * Initialize everything, well, just everything in Posix clocks/timers ;)
- */
 static __init int init_posix_timers(void)
 {
 	posix_timers_cache = kmem_cache_create("posix_timers_cache",

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

* [tip: timers/core] posix-timers: Clarify posix_timer_fn() comments
  2023-06-01 19:07     ` Thomas Gleixner
  2023-06-05 14:26       ` Frederic Weisbecker
@ 2023-06-05 15:08       ` tip-bot2 for Thomas Gleixner
  2023-06-05 22:17       ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49       ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     63dede13d09850a8ace210f8e4227ac5a6b309ae
Gitweb:        https://git.kernel.org/tip/63dede13d09850a8ace210f8e4227ac5a6b309ae
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 21:07:37 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Clarify posix_timer_fn() comments

Make the issues vs. SIG_IGN understandable and remove the 15 years old
promise that a proper solution is already on the horizon.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Link: https://lore.kernel.org/r/874jnrdmrq.ffs@tglx

---
 kernel/time/posix-timers.c | 62 +++++++++++++++++++------------------
 1 file changed, 32 insertions(+), 30 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index f1a7c62..a22c183 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -326,11 +326,11 @@ int posix_timer_event(struct k_itimer *timr, int si_private)
 }
 
 /*
- * This function gets called when a POSIX.1b interval timer expires.  It
- * is used as a callback from the kernel internal timer.  The
- * run_timer_list code ALWAYS calls with interrupts on.
-
- * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
+ * This function gets called when a POSIX.1b interval timer expires from
+ * the HRTIMER interrupt (soft interrupt on RT kernels).
+ *
+ * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
+ * based timers.
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
@@ -348,9 +348,10 @@ static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 
 	if (posix_timer_event(timr, si_private)) {
 		/*
-		 * signal was not sent because of sig_ignor
-		 * we will not get a call back to restart it AND
-		 * it should be restarted.
+		 * The signal was not queued due to SIG_IGN. As a
+		 * consequence the timer is not going to be rearmed from
+		 * the signal delivery path. But as a real signal handler
+		 * can be installed later the timer must be rearmed here.
 		 */
 		if (timr->it_interval != 0) {
 			ktime_t now = hrtimer_cb_get_time(timer);
@@ -359,34 +360,35 @@ static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 			 * FIXME: What we really want, is to stop this
 			 * timer completely and restart it in case the
 			 * SIG_IGN is removed. This is a non trivial
-			 * change which involves sighand locking
-			 * (sigh !), which we don't want to do late in
-			 * the release cycle.
+			 * change to the signal handling code.
+			 *
+			 * For now let timers with an interval less than a
+			 * jiffie expire every jiffie and recheck for a
+			 * valid signal handler.
+			 *
+			 * This avoids interrupt starvation in case of a
+			 * very small interval, which would expire the
+			 * timer immediately again.
 			 *
-			 * For now we just let timers with an interval
-			 * less than a jiffie expire every jiffie to
-			 * avoid softirq starvation in case of SIG_IGN
-			 * and a very small interval, which would put
-			 * the timer right back on the softirq pending
-			 * list. By moving now ahead of time we trick
-			 * hrtimer_forward() to expire the timer
-			 * later, while we still maintain the overrun
-			 * accuracy, but have some inconsistency in
-			 * the timer_gettime() case. This is at least
-			 * better than a starved softirq. A more
-			 * complex fix which solves also another related
-			 * inconsistency is already in the pipeline.
+			 * Moving now ahead of time by one jiffie tricks
+			 * hrtimer_forward() to expire the timer later,
+			 * while it still maintains the overrun accuracy
+			 * for the price of a slight inconsistency in the
+			 * timer_gettime() case. This is at least better
+			 * than a timer storm.
+			 *
+			 * Only required when high resolution timers are
+			 * enabled as the periodic tick based timers are
+			 * automatically aligned to the next tick.
 			 */
-#ifdef CONFIG_HIGH_RES_TIMERS
-			{
-				ktime_t kj = NSEC_PER_SEC / HZ;
+			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
+				ktime_t kj = TICK_NSECS;
 
 				if (timr->it_interval < kj)
 					now = ktime_add(now, kj);
 			}
-#endif
-			timr->it_overrun += hrtimer_forward(timer, now,
-							    timr->it_interval);
+
+			timr->it_overrun += hrtimer_forward(timer, now, timr->it_interval);
 			ret = HRTIMER_RESTART;
 			++timr->it_requeue_pending;
 			timr->it_active = 1;

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

* [tip: timers/core] posix-timers: Clarify posix_timer_rearm() comment
  2023-04-25 18:49 ` [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment Thomas Gleixner
  2023-06-01 12:52   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     93a62291070d2836b7cbaf20febadae6ae33934a
Gitweb:        https://git.kernel.org/tip/93a62291070d2836b7cbaf20febadae6ae33934a
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:22 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Clarify posix_timer_rearm() comment

Yet another incomprehensible piece of art.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.724863461@linutronix.de

---
 kernel/time/posix-timers.c | 12 +++---------
 1 file changed, 3 insertions(+), 9 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index fb92036..f1a7c62 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -275,15 +275,9 @@ static void common_hrtimer_rearm(struct k_itimer *timr)
 }
 
 /*
- * This function is exported for use by the signal deliver code.  It is
- * called just prior to the info block being released and passes that
- * block to us.  It's function is to update the overrun entry AND to
- * restart the timer.  It should only be called if the timer is to be
- * restarted (i.e. we have flagged this in the sys_private entry of the
- * info block).
- *
- * To protect against the timer going away while the interrupt is queued,
- * we require that the it_requeue_pending flag be set.
+ * This function is called from the signal delivery code if
+ * info->si_sys_private is not zero, which indicates that the timer has to
+ * be rearmed. Restart the timer and update info::si_overrun.
  */
 void posixtimer_rearm(struct kernel_siginfo *info)
 {

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

* [tip: timers/core] posix-timers: Comment SIGEV_THREAD_ID properly
  2023-04-25 18:49 ` [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly Thomas Gleixner
  2023-06-01 12:47   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     b73b3beb88c26d3d6c644920fa83460ec5f6a559
Gitweb:        https://git.kernel.org/tip/b73b3beb88c26d3d6c644920fa83460ec5f6a559
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:20 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Comment SIGEV_THREAD_ID properly

Replace the word salad.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.672220780@linutronix.de

---
 kernel/time/posix-timers.c | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 5cfd09c..fb92036 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -53,12 +53,9 @@ static const struct k_clock * const posix_clocks[];
 static const struct k_clock *clockid_to_kclock(const clockid_t id);
 static const struct k_clock clock_realtime, clock_monotonic;
 
-/*
- * we assume that the new SIGEV_THREAD_ID shares no bits with the other
- * SIGEV values.  Here we put out an error if this assumption fails.
- */
+/* SIGEV_THREAD_ID cannot share a bit with the other SIGEV values. */
 #if SIGEV_THREAD_ID != (SIGEV_THREAD_ID & \
-                       ~(SIGEV_SIGNAL | SIGEV_NONE | SIGEV_THREAD))
+			~(SIGEV_SIGNAL | SIGEV_NONE | SIGEV_THREAD))
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 

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

* [tip: timers/core] posix-timers: Add proper comments in do_timer_create()
  2023-04-25 18:49 ` [patch 15/20] posix-timers: Add proper comments in do_timer_create() Thomas Gleixner
  2023-06-01 12:43   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     2ef6cf6d46b462a1ff7fb70ad8050a2b35077e19
Gitweb:        https://git.kernel.org/tip/2ef6cf6d46b462a1ff7fb70ad8050a2b35077e19
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:19 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Add proper comments in do_timer_create()

The comment about timer lifetime at the end of the function is misplaced
and uncomprehensible.

Make it understandable and put it at the right place. Add a new comment
about the visibility of the new timer ID to user space.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.619897296@linutronix.de

---
 kernel/time/posix-timers.c | 20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 9ce13c9..5cfd09c 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -529,12 +529,17 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	new_timer->sigq->info.si_tid   = new_timer->it_id;
 	new_timer->sigq->info.si_code  = SI_TIMER;
 
-	if (copy_to_user(created_timer_id,
-			 &new_timer_id, sizeof (new_timer_id))) {
+	if (copy_to_user(created_timer_id, &new_timer_id, sizeof (new_timer_id))) {
 		error = -EFAULT;
 		goto out;
 	}
-
+	/*
+	 * After succesful copy out, the timer ID is visible to user space
+	 * now but not yet valid because new_timer::signal is still NULL.
+	 *
+	 * Complete the initialization with the clock specific create
+	 * callback.
+	 */
 	error = kc->timer_create(new_timer);
 	if (error)
 		goto out;
@@ -544,14 +549,11 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	WRITE_ONCE(new_timer->it_signal, current->signal);
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
-
-	return 0;
 	/*
-	 * In the case of the timer belonging to another task, after
-	 * the task is unlocked, the timer is owned by the other task
-	 * and may cease to exist at any time.  Don't use or modify
-	 * new_timer after the unlock call.
+	 * After unlocking sighand::siglock @new_timer is subject to
+	 * concurrent removal and cannot be touched anymore
 	 */
+	return 0;
 out:
 	posix_timer_unhash_and_free(new_timer);
 	return error;

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

* [tip: timers/core] posix-timers: Document nanosleep() details
  2023-04-25 18:49 ` [patch 14/20] posix-timers: Document nanosleep() details Thomas Gleixner
  2023-06-01 12:30   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     13da885685d6ae339f2924d3c8a1f4fe16a904cc
Gitweb:        https://git.kernel.org/tip/13da885685d6ae339f2924d3c8a1f4fe16a904cc
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:17 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Document nanosleep() details

The descriptions for common_nsleep() is wrong and common_nsleep_timens()
lacks any form of comment.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.567072835@linutronix.de

---
 kernel/time/posix-timers.c |  9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 9d99d4b..9ce13c9 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1370,7 +1370,7 @@ SYSCALL_DEFINE2(clock_getres_time32, clockid_t, which_clock,
 #endif
 
 /*
- * nanosleep for monotonic and realtime clocks
+ * sys_clock_nanosleep() for CLOCK_REALTIME and CLOCK_TAI
  */
 static int common_nsleep(const clockid_t which_clock, int flags,
 			 const struct timespec64 *rqtp)
@@ -1382,8 +1382,13 @@ static int common_nsleep(const clockid_t which_clock, int flags,
 				 which_clock);
 }
 
+/*
+ * sys_clock_nanosleep() for CLOCK_MONOTONIC and CLOCK_BOOTTIME
+ *
+ * Absolute nanosleeps for these clocks are time-namespace adjusted.
+ */
 static int common_nsleep_timens(const clockid_t which_clock, int flags,
-			 const struct timespec64 *rqtp)
+				const struct timespec64 *rqtp)
 {
 	ktime_t texp = timespec64_to_ktime(*rqtp);
 

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

* [tip: timers/core] posix-timers: Document sys_clock_getoverrun()
  2023-04-25 18:49 ` [patch 12/20] posix-timers: Document sys_clock_getoverrun() Thomas Gleixner
  2023-06-01 11:06   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     1adc8de86737c3d134c2da5ed8e56c2de4729923
Gitweb:        https://git.kernel.org/tip/1adc8de86737c3d134c2da5ed8e56c2de4729923
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:14 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Document sys_clock_getoverrun()

Document the syscall in detail and with coherent sentences.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.462051641@linutronix.de

---
 kernel/time/posix-timers.c | 25 +++++++++++++++++--------
 1 file changed, 17 insertions(+), 8 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 663d3c6..67c1d4d 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -783,14 +783,23 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t, timer_id,
 
 #endif
 
-/*
- * Get the number of overruns of a POSIX.1b interval timer.  This is to
- * be the overrun of the timer last delivered.  At the same time we are
- * accumulating overruns on the next timer.  The overrun is frozen when
- * the signal is delivered, either at the notify time (if the info block
- * is not queued) or at the actual delivery time (as we are informed by
- * the call back to posixtimer_rearm().  So all we need to do is
- * to pick up the frozen overrun.
+/**
+ * sys_timer_getoverrun - Get the number of overruns of a POSIX.1b interval timer
+ * @timer_id:	The timer ID which identifies the timer
+ *
+ * The "overrun count" of a timer is one plus the number of expiration
+ * intervals which have elapsed between the first expiry, which queues the
+ * signal and the actual signal delivery. On signal delivery the "overrun
+ * count" is calculated and cached, so it can be returned directly here.
+ *
+ * As this is relative to the last queued signal the returned overrun count
+ * is meaningless outside of the signal delivery path and even there it
+ * does not accurately reflect the current state when user space evaluates
+ * it.
+ *
+ * Returns:
+ *	-EINVAL		@timer_id is invalid
+ *	1..INT_MAX	The number of overruns related to the last delivered signal
  */
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {

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

* [tip: timers/core] posix-timers: Document sys_clock_settime() permissions in place
  2023-04-25 18:49 ` [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place Thomas Gleixner
  2023-06-01 11:22   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     6a0fc851449cdcad42f96e00854af4ac36a4b357
Gitweb:        https://git.kernel.org/tip/6a0fc851449cdcad42f96e00854af4ac36a4b357
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:16 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Document sys_clock_settime() permissions in place

The documentation of sys_clock_settime() permissions is at a random place
and mostly word salad.

Remove it and add a concise comment into sys_clock_settime().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.514700292@linutronix.de

---
 kernel/time/posix-timers.c | 11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 67c1d4d..9d99d4b 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -74,13 +74,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
  *	    following: 1.) The k_itimer struct (sched.h) is used for
  *	    the timer.  2.) The list, it_lock, it_clock, it_id and
  *	    it_pid fields are not modified by timer code.
- *
- * Permissions: It is assumed that the clock_settime() function defined
- *	    for each clock will take care of permission checks.	 Some
- *	    clocks may be set able by any user (i.e. local process
- *	    clocks) others not.	 Currently the only set able clock we
- *	    have is CLOCK_REALTIME and its high res counter part, both of
- *	    which we beg off on and pass to do_sys_settimeofday().
  */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
@@ -1159,6 +1152,10 @@ SYSCALL_DEFINE2(clock_settime, const clockid_t, which_clock,
 	if (get_timespec64(&new_tp, tp))
 		return -EFAULT;
 
+	/*
+	 * Permission checks have to be done inside the clock specific
+	 * setter callback.
+	 */
 	return kc->clock_set(which_clock, &new_tp);
 }
 

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

* [tip: timers/core] posix-timers: Document common_clock_get() correctly
  2023-04-25 18:49 ` [patch 11/20] posix-timers: Document common_clock_get() correctly Thomas Gleixner
  2023-06-01 11:00   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     13a5dd88bb4c8aec6c3f808f75b73b1fe4705244
Gitweb:        https://git.kernel.org/tip/13a5dd88bb4c8aec6c3f808f75b73b1fe4705244
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:12 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:38 +02:00

posix-timers: Document common_clock_get() correctly

Replace another confusing and inaccurate set of comments.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.409169321@linutronix.de

---
 kernel/time/posix-timers.c | 50 ++++++++++++++++++++++---------------
 1 file changed, 30 insertions(+), 20 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index c9896ff..663d3c6 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -660,20 +660,16 @@ static s64 common_hrtimer_forward(struct k_itimer *timr, ktime_t now)
 }
 
 /*
- * Get the time remaining on a POSIX.1b interval timer.  This function
- * is ALWAYS called with spin_lock_irq on the timer, thus it must not
- * mess with irq.
+ * Get the time remaining on a POSIX.1b interval timer.
  *
- * We have a couple of messes to clean up here.  First there is the case
- * of a timer that has a requeue pending.  These timers should appear to
- * be in the timer list with an expiry as if we were to requeue them
- * now.
+ * Two issues to handle here:
  *
- * The second issue is the SIGEV_NONE timer which may be active but is
- * not really ever put in the timer list (to save system resources).
- * This timer may be expired, and if so, we will do it here.  Otherwise
- * it is the same as a requeue pending timer WRT to what we should
- * report.
+ *  1) The timer has a requeue pending. The return value must appear as
+ *     if the timer has been requeued right now.
+ *
+ *  2) The timer is a SIGEV_NONE timer. These timers are never enqueued
+ *     into the hrtimer queue and therefore never expired. Emulate expiry
+ *     here taking #1 into account.
  */
 void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 {
@@ -689,8 +685,12 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 		cur_setting->it_interval = ktime_to_timespec64(iv);
 	} else if (!timr->it_active) {
 		/*
-		 * SIGEV_NONE oneshot timers are never queued. Check them
-		 * below.
+		 * SIGEV_NONE oneshot timers are never queued and therefore
+		 * timr->it_active is always false. The check below
+		 * vs. remaining time will handle this case.
+		 *
+		 * For all other timers there is nothing to update here, so
+		 * return.
 		 */
 		if (!sig_none)
 			return;
@@ -699,18 +699,29 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 	now = kc->clock_get_ktime(timr->it_clock);
 
 	/*
-	 * When a requeue is pending or this is a SIGEV_NONE timer move the
-	 * expiry time forward by intervals, so expiry is > now.
+	 * If this is an interval timer and either has requeue pending or
+	 * is a SIGEV_NONE timer move the expiry time forward by intervals,
+	 * so expiry is > now.
 	 */
 	if (iv && (timr->it_requeue_pending & REQUEUE_PENDING || sig_none))
 		timr->it_overrun += kc->timer_forward(timr, now);
 
 	remaining = kc->timer_remaining(timr, now);
-	/* Return 0 only, when the timer is expired and not pending */
+	/*
+	 * As @now is retrieved before a possible timer_forward() and
+	 * cannot be reevaluated by the compiler @remaining is based on the
+	 * same @now value. Therefore @remaining is consistent vs. @now.
+	 *
+	 * Consequently all interval timers, i.e. @iv > 0, cannot have a
+	 * remaining time <= 0 because timer_forward() guarantees to move
+	 * them forward so that the next timer expiry is > @now.
+	 */
 	if (remaining <= 0) {
 		/*
-		 * A single shot SIGEV_NONE timer must return 0, when
-		 * it is expired !
+		 * A single shot SIGEV_NONE timer must return 0, when it is
+		 * expired! Timers which have a real signal delivery mode
+		 * must return a remaining time greater than 0 because the
+		 * signal has not yet been delivered.
 		 */
 		if (!sig_none)
 			cur_setting->it_value.tv_nsec = 1;
@@ -719,7 +730,6 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 	}
 }
 
-/* Get the time remaining on a POSIX.1b interval timer. */
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
 	struct k_itimer *timr;

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

* [tip: timers/core] posix-timers: Split release_posix_timers()
  2023-04-25 18:49 ` [patch 09/20] posix-timers: Split release_posix_timers() Thomas Gleixner
  2023-06-01 10:25   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     e7441aa344046cbdcb402ac173fb163613471097
Gitweb:        https://git.kernel.org/tip/e7441aa344046cbdcb402ac173fb163613471097
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:09 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Split release_posix_timers()

release_posix_timers() is called for cleaning up both hashed and unhashed
timers. The cases are differentiated by an argument and the usage is
hideous.

Seperate the actual free path out and use it for unhashed timers. Provide a
function for hashed timers.

No functional change.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.301432503@linutronix.de

---
 kernel/time/posix-timers.c | 31 +++++++++++++++----------------
 1 file changed, 15 insertions(+), 16 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index d8d4cdf..8153374 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -466,20 +466,21 @@ static void k_itimer_rcu_free(struct rcu_head *head)
 	kmem_cache_free(posix_timers_cache, tmr);
 }
 
-#define IT_ID_SET	1
-#define IT_ID_NOT_SET	0
-static void release_posix_timer(struct k_itimer *tmr, int it_id_set)
-{
-	if (it_id_set) {
-		spin_lock(&hash_lock, flags);
-		hlist_del_rcu(&tmr->t_hash);
-		spin_unlock(&hash_lock, flags);
-	}
+static void posix_timer_free(struct k_itimer *tmr)
+{
 	put_pid(tmr->it_pid);
 	sigqueue_free(tmr->sigq);
 	call_rcu(&tmr->rcu, k_itimer_rcu_free);
 }
 
+static void posix_timer_unhash_and_free(struct k_itimer *tmr)
+{
+	spin_lock(&hash_lock);
+	hlist_del_rcu(&tmr->t_hash);
+	spin_unlock(&hash_lock);
+	posix_timer_free(tmr);
+}
+
 static int common_timer_create(struct k_itimer *new_timer)
 {
 	hrtimer_init(&new_timer->it.real.timer, new_timer->it_clock, 0);
@@ -493,7 +494,6 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	const struct k_clock *kc = clockid_to_kclock(which_clock);
 	struct k_itimer *new_timer;
 	int error, new_timer_id;
-	int it_id_set = IT_ID_NOT_SET;
 
 	if (!kc)
 		return -EINVAL;
@@ -513,11 +513,10 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	 */
 	new_timer_id = posix_timer_add(new_timer);
 	if (new_timer_id < 0) {
-		error = new_timer_id;
-		goto out;
+		posix_timer_free(new_timer);
+		return new_timer_id;
 	}
 
-	it_id_set = IT_ID_SET;
 	new_timer->it_id = (timer_t) new_timer_id;
 	new_timer->it_clock = which_clock;
 	new_timer->kclock = kc;
@@ -569,7 +568,7 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	 * new_timer after the unlock call.
 	 */
 out:
-	release_posix_timer(new_timer, it_id_set);
+	posix_timer_unhash_and_free(new_timer);
 	return error;
 }
 
@@ -1057,7 +1056,7 @@ retry_delete:
 	WRITE_ONCE(timer->it_signal, NULL);
 
 	unlock_timer(timer, flags);
-	release_posix_timer(timer, IT_ID_SET);
+	posix_timer_unhash_and_free(timer);
 	return 0;
 }
 
@@ -1109,7 +1108,7 @@ retry_delete:
 	WRITE_ONCE(timer->it_signal, NULL);
 
 	spin_unlock_irqrestore(&timer->it_lock, flags);
-	release_posix_timer(timer, IT_ID_SET);
+	posix_timer_unhash_and_free(timer);
 }
 
 /*

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

* [tip: timers/core] posix-timers: Remove pointless irqsafe from hash_lock
  2023-04-25 18:49 ` [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock Thomas Gleixner
  2023-06-01 10:12   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     032c960ac9f3d64487423e0e01a0fb7327509595
Gitweb:        https://git.kernel.org/tip/032c960ac9f3d64487423e0e01a0fb7327509595
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:08 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Remove pointless irqsafe from hash_lock

All usage of hash_lock is in thread context. No point in using
spin_lock_irqsave()/irqrestore() for a single usage site.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.249063953@linutronix.de

---
 kernel/time/posix-timers.c | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index e8d877f..d8d4cdf 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -471,10 +471,9 @@ static void k_itimer_rcu_free(struct rcu_head *head)
 static void release_posix_timer(struct k_itimer *tmr, int it_id_set)
 {
 	if (it_id_set) {
-		unsigned long flags;
-		spin_lock_irqsave(&hash_lock, flags);
+		spin_lock(&hash_lock, flags);
 		hlist_del_rcu(&tmr->t_hash);
-		spin_unlock_irqrestore(&hash_lock, flags);
+		spin_unlock(&hash_lock, flags);
 	}
 	put_pid(tmr->it_pid);
 	sigqueue_free(tmr->sigq);

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

* [tip: timers/core] posix-timers: Document sys_clock_getres() correctly
  2023-04-25 18:49 ` [patch 10/20] posix-timers: Document sys_clock_getres() correctly Thomas Gleixner
  2023-06-01 10:44   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     4898e5d912108c8a865ffb3fc0230559b40ca361
Gitweb:        https://git.kernel.org/tip/4898e5d912108c8a865ffb3fc0230559b40ca361
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:11 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Document sys_clock_getres() correctly

The decades old comment about Posix clock resolution is confusing at best.

Remove it and add a proper explanation to sys_clock_getres().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.356427330@linutronix.de

---
 kernel/time/posix-timers.c | 81 +++++++++++++++++++++++++++++++++----
 1 file changed, 73 insertions(+), 8 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 8153374..c9896ff 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -67,14 +67,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
  *	    to implement others.  This structure defines the various
  *	    clocks.
  *
- * RESOLUTION: Clock resolution is used to round up timer and interval
- *	    times, NOT to report clock times, which are reported with as
- *	    much resolution as the system can muster.  In some cases this
- *	    resolution may depend on the underlying clock hardware and
- *	    may not be quantifiable until run time, and only then is the
- *	    necessary code is written.	The standard says we should say
- *	    something about this issue in the documentation...
- *
  * FUNCTIONS: The CLOCKs structure defines possible functions to
  *	    handle various clock functions.
  *
@@ -1198,6 +1190,79 @@ SYSCALL_DEFINE2(clock_adjtime, const clockid_t, which_clock,
 	return err;
 }
 
+/**
+ * sys_clock_getres - Get the resolution of a clock
+ * @which_clock:	The clock to get the resolution for
+ * @tp:			Pointer to a a user space timespec64 for storage
+ *
+ * POSIX defines:
+ *
+ * "The clock_getres() function shall return the resolution of any
+ * clock. Clock resolutions are implementation-defined and cannot be set by
+ * a process. If the argument res is not NULL, the resolution of the
+ * specified clock shall be stored in the location pointed to by res. If
+ * res is NULL, the clock resolution is not returned. If the time argument
+ * of clock_settime() is not a multiple of res, then the value is truncated
+ * to a multiple of res."
+ *
+ * Due to the various hardware constraints the real resolution can vary
+ * wildly and even change during runtime when the underlying devices are
+ * replaced. The kernel also can use hardware devices with different
+ * resolutions for reading the time and for arming timers.
+ *
+ * The kernel therefore deviates from the POSIX spec in various aspects:
+ *
+ * 1) The resolution returned to user space
+ *
+ *    For CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME, CLOCK_TAI,
+ *    CLOCK_REALTIME_ALARM, CLOCK_BOOTTIME_ALAREM and CLOCK_MONOTONIC_RAW
+ *    the kernel differentiates only two cases:
+ *
+ *    I)  Low resolution mode:
+ *
+ *	  When high resolution timers are disabled at compile or runtime
+ *	  the resolution returned is nanoseconds per tick, which represents
+ *	  the precision at which timers expire.
+ *
+ *    II) High resolution mode:
+ *
+ *	  When high resolution timers are enabled the resolution returned
+ *	  is always one nanosecond independent of the actual resolution of
+ *	  the underlying hardware devices.
+ *
+ *	  For CLOCK_*_ALARM the actual resolution depends on system
+ *	  state. When system is running the resolution is the same as the
+ *	  resolution of the other clocks. During suspend the actual
+ *	  resolution is the resolution of the underlying RTC device which
+ *	  might be way less precise than the clockevent device used during
+ *	  running state.
+ *
+ *   For CLOCK_REALTIME_COARSE and CLOCK_MONOTONIC_COARSE the resolution
+ *   returned is always nanoseconds per tick.
+ *
+ *   For CLOCK_PROCESS_CPUTIME and CLOCK_THREAD_CPUTIME the resolution
+ *   returned is always one nanosecond under the assumption that the
+ *   underlying scheduler clock has a better resolution than nanoseconds
+ *   per tick.
+ *
+ *   For dynamic POSIX clocks (PTP devices) the resolution returned is
+ *   always one nanosecond.
+ *
+ * 2) Affect on sys_clock_settime()
+ *
+ *    The kernel does not truncate the time which is handed in to
+ *    sys_clock_settime(). The kernel internal timekeeping is always using
+ *    nanoseconds precision independent of the clocksource device which is
+ *    used to read the time from. The resolution of that device only
+ *    affects the presicion of the time returned by sys_clock_gettime().
+ *
+ * Returns:
+ *	0		Success. @tp contains the resolution
+ *	-EINVAL		@which_clock is not a valid clock ID
+ *	-EFAULT		Copying the resolution to @tp faulted
+ *	-ENODEV		Dynamic POSIX clock is not backed by a device
+ *	-EOPNOTSUPP	Dynamic POSIX clock does not support getres()
+ */
 SYSCALL_DEFINE2(clock_getres, const clockid_t, which_clock,
 		struct __kernel_timespec __user *, tp)
 {

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

* [tip: timers/core] posix-timers: Set k_itimer:: It_signal to NULL on exit()
  2023-04-25 18:49 ` [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit() Thomas Gleixner
  2023-06-01 10:09   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     515eeda177b797d9a7d0c496739f0bdc02c73093
Gitweb:        https://git.kernel.org/tip/515eeda177b797d9a7d0c496739f0bdc02c73093
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:06 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Set k_itimer:: It_signal to NULL on exit()

Technically it's not required to set k_itimer::it_signal to NULL on exit()
because there is no other thread anymore which could lookup the timer
concurrently.

Set it to NULL for consistency sake and add a comment to that effect.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.196462644@linutronix.de

---
 kernel/time/posix-timers.c | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 4333334..e8d877f 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1101,6 +1101,14 @@ retry_delete:
 	}
 	list_del(&timer->list);
 
+	/*
+	 * Setting timer::it_signal to NULL is technically not required
+	 * here as nothing can access the timer anymore legitimately via
+	 * the hash table. Set it to NULL nevertheless so that all deletion
+	 * paths are consistent.
+	 */
+	WRITE_ONCE(timer->it_signal, NULL);
+
 	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }

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

* [tip: timers/core] posix-timers: Annotate concurrent access to k_itimer:: It_signal
  2023-04-25 18:49 ` [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal Thomas Gleixner
  2023-05-09 11:04   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     d69e873ee50459e3b77019263cf95236f063a97b
Gitweb:        https://git.kernel.org/tip/d69e873ee50459e3b77019263cf95236f063a97b
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:05 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Annotate concurrent access to k_itimer:: It_signal

k_itimer::it_signal is read lockless in the RCU protected hash lookup, but
it can be written concurrently in the timer_create() and timer_delete()
path. Annotate these places with READ_ONCE() and WRITE_ONCE()

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.143596887@linutronix.de

---
 kernel/time/posix-timers.c | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index f6650c1..4333334 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -109,9 +109,9 @@ static struct k_itimer *__posix_timers_find(struct hlist_head *head,
 {
 	struct k_itimer *timer;
 
-	hlist_for_each_entry_rcu(timer, head, t_hash,
-				 lockdep_is_held(&hash_lock)) {
-		if ((timer->it_signal == sig) && (timer->it_id == id))
+	hlist_for_each_entry_rcu(timer, head, t_hash, lockdep_is_held(&hash_lock)) {
+		/* timer->it_signal can be set concurrently */
+		if ((READ_ONCE(timer->it_signal) == sig) && (timer->it_id == id))
 			return timer;
 	}
 	return NULL;
@@ -558,7 +558,7 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 
 	spin_lock_irq(&current->sighand->siglock);
 	/* This makes the timer valid in the hash table */
-	new_timer->it_signal = current->signal;
+	WRITE_ONCE(new_timer->it_signal, current->signal);
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
 
@@ -1052,10 +1052,10 @@ retry_delete:
 	list_del(&timer->list);
 	spin_unlock(&current->sighand->siglock);
 	/*
-	 * This keeps any tasks waiting on the spin lock from thinking
-	 * they got something (see the lock code above).
+	 * A concurrent lookup could check timer::it_signal lockless. It
+	 * will reevaluate with timer::it_lock held and observe the NULL.
 	 */
-	timer->it_signal = NULL;
+	WRITE_ONCE(timer->it_signal, NULL);
 
 	unlock_timer(timer, flags);
 	release_posix_timer(timer, IT_ID_SET);

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

* [tip: timers/core] posix-timers: Add comments about timer lookup
  2023-04-25 18:49 ` [patch 05/20] posix-timers: Add comments about timer lookup Thomas Gleixner
  2023-05-09 10:58   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     46c7863e061f64c96745003f699912667c2a1bd6
Gitweb:        https://git.kernel.org/tip/46c7863e061f64c96745003f699912667c2a1bd6
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:03 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Add comments about timer lookup

Document how the timer ID validation in the hash table works.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.091081515@linutronix.de

---
 kernel/time/posix-timers.c | 39 ++++++++++++++++++++++++++++++-------
 1 file changed, 32 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 355c6f4..f6650c1 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -506,6 +506,12 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 		return -EAGAIN;
 
 	spin_lock_init(&new_timer->it_lock);
+
+	/*
+	 * Add the timer to the hash table. The timer is not yet valid
+	 * because new_timer::it_signal is still NULL. The timer id is also
+	 * not yet visible to user space.
+	 */
 	new_timer_id = posix_timer_add(new_timer);
 	if (new_timer_id < 0) {
 		error = new_timer_id;
@@ -551,6 +557,7 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 		goto out;
 
 	spin_lock_irq(&current->sighand->siglock);
+	/* This makes the timer valid in the hash table */
 	new_timer->it_signal = current->signal;
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
@@ -597,13 +604,6 @@ COMPAT_SYSCALL_DEFINE3(timer_create, clockid_t, which_clock,
 }
 #endif
 
-/*
- * Locking issues: We need to protect the result of the id look up until
- * we get the timer locked down so it is not deleted under us.  The
- * removal is done under the idr spinlock so we use that here to bridge
- * the find to the timer lock.  To avoid a dead lock, the timer id MUST
- * be release with out holding the timer lock.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags)
 {
 	struct k_itimer *timr;
@@ -615,10 +615,35 @@ static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags)
 	if ((unsigned long long)timer_id > INT_MAX)
 		return NULL;
 
+	/*
+	 * The hash lookup and the timers are RCU protected.
+	 *
+	 * Timers are added to the hash in invalid state where
+	 * timr::it_signal == NULL. timer::it_signal is only set after the
+	 * rest of the initialization succeeded.
+	 *
+	 * Timer destruction happens in steps:
+	 *  1) Set timr::it_signal to NULL with timr::it_lock held
+	 *  2) Release timr::it_lock
+	 *  3) Remove from the hash under hash_lock
+	 *  4) Call RCU for removal after the grace period
+	 *
+	 * Holding rcu_read_lock() accross the lookup ensures that
+	 * the timer cannot be freed.
+	 *
+	 * The lookup validates locklessly that timr::it_signal ==
+	 * current::it_signal and timr::it_id == @timer_id. timr::it_id
+	 * can't change, but timr::it_signal becomes NULL during
+	 * destruction.
+	 */
 	rcu_read_lock();
 	timr = posix_timer_by_id(timer_id);
 	if (timr) {
 		spin_lock_irqsave(&timr->it_lock, *flags);
+		/*
+		 * Validate under timr::it_lock that timr::it_signal is
+		 * still valid. Pairs with #1 above.
+		 */
 		if (timr->it_signal == current->signal) {
 			rcu_read_unlock();
 			return timr;

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

* [tip: timers/core] posix-timers: Clarify timer_wait_running() comment
  2023-04-25 18:49 ` [patch 03/20] posix-timers: Clarify timer_wait_running() comment Thomas Gleixner
  2023-05-09  9:50   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     07ca14d623d8b0e6c4a0d3de4ca8a8ea35a85470
Gitweb:        https://git.kernel.org/tip/07ca14d623d8b0e6c4a0d3de4ca8a8ea35a85470
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:00 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Clarify timer_wait_running() comment

Explain it better and add the CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y aspect
for completeness.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183312.985681995@linutronix.de

---
 kernel/time/posix-timers.c | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 2c5daeb..4017533 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -836,10 +836,18 @@ static void common_timer_wait_running(struct k_itimer *timer)
 }
 
 /*
- * On PREEMPT_RT this prevent priority inversion against softirq kthread in
- * case it gets preempted while executing a timer callback. See comments in
- * hrtimer_cancel_wait_running. For PREEMPT_RT=n this just results in a
- * cpu_relax().
+ * On PREEMPT_RT this prevents priority inversion and a potential livelock
+ * against the ksoftirqd thread in case that ksoftirqd gets preempted while
+ * executing a hrtimer callback.
+ *
+ * See the comments in hrtimer_cancel_wait_running(). For PREEMPT_RT=n this
+ * just results in a cpu_relax().
+ *
+ * For POSIX CPU timers with CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n this is
+ * just a cpu_relax(). With CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this
+ * prevents spinning on an eventually scheduled out task and a livelock
+ * when the task which tries to delete or disarm the timer has preempted
+ * the task which runs the expiry in task work context.
  */
 static struct k_itimer *timer_wait_running(struct k_itimer *timer,
 					   unsigned long *flags)

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

* [tip: timers/core] posix-timers: Cleanup comments about timer ID tracking
  2023-04-25 18:49 ` [patch 04/20] posix-timers: Cleanup comments about timer ID tracking Thomas Gleixner
  2023-05-09  9:58   ` Frederic Weisbecker
@ 2023-06-05 15:08   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     10338fd15894827fefc2bf28ed75c81ba1dbfceb
Gitweb:        https://git.kernel.org/tip/10338fd15894827fefc2bf28ed75c81ba1dbfceb
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:01 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:37 +02:00

posix-timers: Cleanup comments about timer ID tracking

Describe the hash table properly and remove the IDR leftover comments.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.038444551@linutronix.de

---
 kernel/time/posix-timers.c | 28 ++++++++--------------------
 1 file changed, 8 insertions(+), 20 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 4017533..355c6f4 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -35,20 +35,17 @@
 #include "timekeeping.h"
 #include "posix-timers.h"
 
-/*
- * Management arrays for POSIX timers. Timers are now kept in static hash table
- * with 512 entries.
- * Timer ids are allocated by local routine, which selects proper hash head by
- * key, constructed from current->signal address and per signal struct counter.
- * This keeps timer ids unique per process, but now they can intersect between
- * processes.
- */
+static struct kmem_cache *posix_timers_cache;
 
 /*
- * Lets keep our timers in a slab cache :-)
+ * Timers are managed in a hash table for lockless lookup. The hash key is
+ * constructed from current::signal and the timer ID and the timer is
+ * matched against current::signal and the timer ID when walking the hash
+ * bucket list.
+ *
+ * This allows checkpoint/restore to reconstruct the exact timer IDs for
+ * a process.
  */
-static struct kmem_cache *posix_timers_cache;
-
 static DEFINE_HASHTABLE(posix_timers_hashtable, 9);
 static DEFINE_SPINLOCK(hash_lock);
 
@@ -66,15 +63,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
 #endif
 
 /*
- * The timer ID is turned into a timer address by idr_find().
- * Verifying a valid ID consists of:
- *
- * a) checking that idr_find() returns other than -1.
- * b) checking that the timer id matches the one in the timer itself.
- * c) that the timer owner is in the callers thread group.
- */
-
-/*
  * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
  *	    to implement others.  This structure defines the various
  *	    clocks.

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

* [tip: timers/core] posix-timers: Ensure timer ID search-loop limit is valid
  2023-06-01 18:58               ` [patch v2 " Thomas Gleixner
  2023-06-05 14:17                 ` Frederic Weisbecker
@ 2023-06-05 15:08                 ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50                 ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Thomas Gleixner,
	Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     cec58cad4614f44b76624f8d62ef771c8725c483
Gitweb:        https://git.kernel.org/tip/cec58cad4614f44b76624f8d62ef771c8725c483
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 20:58:47 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:36 +02:00

posix-timers: Ensure timer ID search-loop limit is valid

posix_timer_add() tries to allocate a posix timer ID by starting from the
cached ID which was stored by the last successful allocation.

This is done in a loop searching the ID space for a free slot one by
one. The loop has to terminate when the search wrapped around to the
starting point.

But that's racy vs. establishing the starting point. That is read out
lockless, which leads to the following problem:

CPU0	  	      	     	   CPU1
posix_timer_add()
  start = sig->posix_timer_id;
  lock(hash_lock);
  ...				   posix_timer_add()
  if (++sig->posix_timer_id < 0)
      			             start = sig->posix_timer_id;
     sig->posix_timer_id = 0;

So CPU1 can observe a negative start value, i.e. -1, and the loop break
never happens because the condition can never be true:

  if (sig->posix_timer_id == start)
     break;

While this is unlikely to ever turn into an endless loop as the ID space is
huge (INT_MAX), the racy read of the start value caught the attention of
KCSAN and Dmitry unearthed that incorrectness.

Rewrite it so that all id operations are under the hash lock.

Reported-by: syzbot+5c54bd3eb218bb595aa9@syzkaller.appspotmail.com
Reported-by: Dmitry Vyukov <dvyukov@google.com>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/87bkhzdn6g.ffs@tglx

---
 include/linux/sched/signal.h |  2 +-
 kernel/time/posix-timers.c   | 31 ++++++++++++++++++-------------
 2 files changed, 19 insertions(+), 14 deletions(-)

diff --git a/include/linux/sched/signal.h b/include/linux/sched/signal.h
index 2009926..669e8cf 100644
--- a/include/linux/sched/signal.h
+++ b/include/linux/sched/signal.h
@@ -135,7 +135,7 @@ struct signal_struct {
 #ifdef CONFIG_POSIX_TIMERS
 
 	/* POSIX.1b Interval Timers */
-	int			posix_timer_id;
+	unsigned int		next_posix_timer_id;
 	struct list_head	posix_timers;
 
 	/* ITIMER_REAL timer for the process */
diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 2d835c2..2c5daeb 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -140,25 +140,30 @@ static struct k_itimer *posix_timer_by_id(timer_t id)
 static int posix_timer_add(struct k_itimer *timer)
 {
 	struct signal_struct *sig = current->signal;
-	int first_free_id = sig->posix_timer_id;
 	struct hlist_head *head;
-	int ret = -ENOENT;
+	unsigned int cnt, id;
 
-	do {
+	/*
+	 * FIXME: Replace this by a per signal struct xarray once there is
+	 * a plan to handle the resulting CRIU regression gracefully.
+	 */
+	for (cnt = 0; cnt <= INT_MAX; cnt++) {
 		spin_lock(&hash_lock);
-		head = &posix_timers_hashtable[hash(sig, sig->posix_timer_id)];
-		if (!__posix_timers_find(head, sig, sig->posix_timer_id)) {
+		id = sig->next_posix_timer_id;
+
+		/* Write the next ID back. Clamp it to the positive space */
+		sig->next_posix_timer_id = (id + 1) & INT_MAX;
+
+		head = &posix_timers_hashtable[hash(sig, id)];
+		if (!__posix_timers_find(head, sig, id)) {
 			hlist_add_head_rcu(&timer->t_hash, head);
-			ret = sig->posix_timer_id;
+			spin_unlock(&hash_lock);
+			return id;
 		}
-		if (++sig->posix_timer_id < 0)
-			sig->posix_timer_id = 0;
-		if ((sig->posix_timer_id == first_free_id) && (ret == -ENOENT))
-			/* Loop over all possible ids completed */
-			ret = -EAGAIN;
 		spin_unlock(&hash_lock);
-	} while (ret == -ENOENT);
-	return ret;
+	}
+	/* POSIX return code when no timer ID could be allocated */
+	return -EAGAIN;
 }
 
 static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)

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

* [tip: timers/core] posix-timers: Prevent RT livelock in itimer_delete()
  2023-06-01 20:16           ` [patch v2a " Thomas Gleixner
  2023-06-05 10:59             ` Frederic Weisbecker
@ 2023-06-05 15:08             ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:50             ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 15:08 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     1b59b2577582f9cf3d0f17245675a76859175cc1
Gitweb:        https://git.kernel.org/tip/1b59b2577582f9cf3d0f17245675a76859175cc1
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 22:16:34 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Mon, 05 Jun 2023 17:03:36 +02:00

posix-timers: Prevent RT livelock in itimer_delete()

itimer_delete() has a retry loop when the timer is concurrently expired. On
non-RT kernels this just spin-waits until the timer callback has completed,
except for posix CPU timers which have HAVE_POSIX_CPU_TIMERS_TASK_WORK
enabled.

In that case and on RT kernels the existing task could live lock when
preempting the task which does the timer delivery.

Replace spin_unlock() with an invocation of timer_wait_running() to handle
it the same way as the other retry loops in the posix timer code.

Fixes: ec8f954a40da ("posix-timers: Use a callback for cancel synchronization on PREEMPT_RT")
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/87v8g7c50d.ffs@tglx

---
 kernel/time/posix-timers.c | 41 ++++++++++++++++++++++++++++++-------
 1 file changed, 34 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 808a247..2d835c2 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1037,27 +1037,52 @@ retry_delete:
 }
 
 /*
- * return timer owned by the process, used by exit_itimers
+ * Delete a timer if it is armed, remove it from the hash and schedule it
+ * for RCU freeing.
  */
 static void itimer_delete(struct k_itimer *timer)
 {
+	unsigned long flags;
+
 retry_delete:
-	spin_lock_irq(&timer->it_lock);
+	/*
+	 * irqsave is required to make timer_wait_running() work.
+	 */
+	spin_lock_irqsave(&timer->it_lock, flags);
 
+	/*
+	 * Even if the timer is not longer accessible from other tasks
+	 * it still might be armed and queued in the underlying timer
+	 * mechanism. Worse, that timer mechanism might run the expiry
+	 * function concurrently.
+	 */
 	if (timer_delete_hook(timer) == TIMER_RETRY) {
-		spin_unlock_irq(&timer->it_lock);
+		/*
+		 * Timer is expired concurrently, prevent livelocks
+		 * and pointless spinning on RT.
+		 *
+		 * timer_wait_running() drops timer::it_lock, which opens
+		 * the possibility for another task to delete the timer.
+		 *
+		 * That's not possible here because this is invoked from
+		 * do_exit() only for the last thread of the thread group.
+		 * So no other task can access and delete that timer.
+		 */
+		if (WARN_ON_ONCE(timer_wait_running(timer, &flags) != timer))
+			return;
+
 		goto retry_delete;
 	}
 	list_del(&timer->list);
 
-	spin_unlock_irq(&timer->it_lock);
+	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }
 
 /*
- * This is called by do_exit or de_thread, only when nobody else can
- * modify the signal->posix_timers list. Yet we need sighand->siglock
- * to prevent the race with /proc/pid/timers.
+ * Invoked from do_exit() when the last thread of a thread group exits.
+ * At that point no other task can access the timers of the dying
+ * task anymore.
  */
 void exit_itimers(struct task_struct *tsk)
 {
@@ -1067,10 +1092,12 @@ void exit_itimers(struct task_struct *tsk)
 	if (list_empty(&tsk->signal->posix_timers))
 		return;
 
+	/* Protect against concurrent read via /proc/$PID/timers */
 	spin_lock_irq(&tsk->sighand->siglock);
 	list_replace_init(&tsk->signal->posix_timers, &timers);
 	spin_unlock_irq(&tsk->sighand->siglock);
 
+	/* The timers are not longer accessible via tsk::signal */
 	while (!list_empty(&timers)) {
 		tmr = list_first_entry(&timers, struct k_itimer, list);
 		itimer_delete(tmr);

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

* [tip: timers/core] posix-timers: Polish coding style in a few places
  2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
  2023-06-01 13:50   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 22:17 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     6fbe3a3b4fa1fe5cf05f91193d81d930c6820eda
Gitweb:        https://git.kernel.org/tip/6fbe3a3b4fa1fe5cf05f91193d81d930c6820eda
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:27 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Tue, 06 Jun 2023 00:12:55 +02:00

posix-timers: Polish coding style in a few places

Make it consistent with the TIP tree documentation.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.888493625@linutronix.de

---
 kernel/time/posix-timers.c | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index bdd718c..8bb0dcf 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -309,10 +309,10 @@ int posix_timer_event(struct k_itimer *timr, int si_private)
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
+	enum hrtimer_restart ret = HRTIMER_NORESTART;
 	struct k_itimer *timr;
 	unsigned long flags;
 	int si_private = 0;
-	enum hrtimer_restart ret = HRTIMER_NORESTART;
 
 	timr = container_of(timer, struct k_itimer, it.real.timer);
 	spin_lock_irqsave(&timr->it_lock, flags);
@@ -400,8 +400,8 @@ static struct pid *good_sigevent(sigevent_t * event)
 
 static struct k_itimer * alloc_posix_timer(void)
 {
-	struct k_itimer *tmr;
-	tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+	struct k_itimer *tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+
 	if (!tmr)
 		return tmr;
 	if (unlikely(!(tmr->sigq = sigqueue_alloc()))) {
@@ -695,8 +695,8 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
-	struct k_itimer *timr;
 	const struct k_clock *kc;
+	struct k_itimer *timr;
 	unsigned long flags;
 	int ret = 0;
 
@@ -767,8 +767,8 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t, timer_id,
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {
 	struct k_itimer *timr;
-	int overrun;
 	unsigned long flags;
+	int overrun;
 
 	timr = lock_timer(timer_id, &flags);
 	if (!timr)
@@ -941,8 +941,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t, timer_id, int, flags,
 		const struct __kernel_itimerspec __user *, new_setting,
 		struct __kernel_itimerspec __user *, old_setting)
 {
-	struct itimerspec64 new_spec, old_spec;
-	struct itimerspec64 *rtn = old_setting ? &old_spec : NULL;
+	struct itimerspec64 new_spec, old_spec, *rtn;
 	int error = 0;
 
 	if (!new_setting)
@@ -951,6 +950,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t, timer_id, int, flags,
 	if (get_itimerspec64(&new_spec, new_setting))
 		return -EFAULT;
 
+	rtn = old_setting ? &old_spec : NULL;
 	error = do_timer_settime(timer_id, flags, &new_spec, rtn);
 	if (!error && old_setting) {
 		if (put_itimerspec64(&old_spec, old_setting))

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

* [tip: timers/core] posix-timers: Remove pointless comments
  2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
  2023-06-01 13:48   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 22:17 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     298356a3c78198f44ba5df65547a9085334daa38
Gitweb:        https://git.kernel.org/tip/298356a3c78198f44ba5df65547a9085334daa38
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:25 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Tue, 06 Jun 2023 00:12:55 +02:00

posix-timers: Remove pointless comments

Documenting the obvious is just consuming space for no value.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.832240451@linutronix.de

---
 kernel/time/posix-timers.c | 25 -------------------------
 1 file changed, 25 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index a942020..bdd718c 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -59,19 +59,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 
-/*
- * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
- *	    to implement others.  This structure defines the various
- *	    clocks.
- *
- * FUNCTIONS: The CLOCKs structure defines possible functions to
- *	    handle various clock functions.
- *
- *	    The standard POSIX timer management code assumes the
- *	    following: 1.) The k_itimer struct (sched.h) is used for
- *	    the timer.  2.) The list, it_lock, it_clock, it_id and
- *	    it_pid fields are not modified by timer code.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
 #define lock_timer(tid, flags)						   \
@@ -141,7 +128,6 @@ static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)
 	spin_unlock_irqrestore(&timr->it_lock, flags);
 }
 
-/* Get clock_realtime */
 static int posix_get_realtime_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_real_ts64(tp);
@@ -153,7 +139,6 @@ static ktime_t posix_get_realtime_ktime(clockid_t which_clock)
 	return ktime_get_real();
 }
 
-/* Set clock_realtime */
 static int posix_clock_realtime_set(const clockid_t which_clock,
 				    const struct timespec64 *tp)
 {
@@ -166,9 +151,6 @@ static int posix_clock_realtime_adj(const clockid_t which_clock,
 	return do_adjtimex(t);
 }
 
-/*
- * Get monotonic time for posix timers
- */
 static int posix_get_monotonic_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_ts64(tp);
@@ -181,9 +163,6 @@ static ktime_t posix_get_monotonic_ktime(clockid_t which_clock)
 	return ktime_get();
 }
 
-/*
- * Get monotonic-raw time for posix timers
- */
 static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_raw_ts64(tp);
@@ -191,7 +170,6 @@ static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 	return 0;
 }
 
-
 static int posix_get_realtime_coarse(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_coarse_real_ts64(tp);
@@ -242,9 +220,6 @@ static int posix_get_hrtimer_res(clockid_t which_clock, struct timespec64 *tp)
 	return 0;
 }
 
-/*
- * Initialize everything, well, just everything in Posix clocks/timers ;)
- */
 static __init int init_posix_timers(void)
 {
 	posix_timers_cache = kmem_cache_create("posix_timers_cache",

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

* [tip: timers/core] posix-timers: Clarify posix_timer_fn() comments
  2023-06-01 19:07     ` Thomas Gleixner
  2023-06-05 14:26       ` Frederic Weisbecker
  2023-06-05 15:08       ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-05 22:17       ` tip-bot2 for Thomas Gleixner
  2023-06-18 20:49       ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-05 22:17 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     c78f261e5dcb415b9e35a13876fbf7d5f134c810
Gitweb:        https://git.kernel.org/tip/c78f261e5dcb415b9e35a13876fbf7d5f134c810
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 21:07:37 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Tue, 06 Jun 2023 00:12:55 +02:00

posix-timers: Clarify posix_timer_fn() comments

Make the issues vs. SIG_IGN understandable and remove the 15 years old
promise that a proper solution is already on the horizon.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Link: https://lore.kernel.org/r/874jnrdmrq.ffs@tglx

---
 kernel/time/posix-timers.c | 62 +++++++++++++++++++------------------
 1 file changed, 32 insertions(+), 30 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index f1a7c62..a942020 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -326,11 +326,11 @@ int posix_timer_event(struct k_itimer *timr, int si_private)
 }
 
 /*
- * This function gets called when a POSIX.1b interval timer expires.  It
- * is used as a callback from the kernel internal timer.  The
- * run_timer_list code ALWAYS calls with interrupts on.
-
- * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
+ * This function gets called when a POSIX.1b interval timer expires from
+ * the HRTIMER interrupt (soft interrupt on RT kernels).
+ *
+ * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
+ * based timers.
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
@@ -348,9 +348,10 @@ static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 
 	if (posix_timer_event(timr, si_private)) {
 		/*
-		 * signal was not sent because of sig_ignor
-		 * we will not get a call back to restart it AND
-		 * it should be restarted.
+		 * The signal was not queued due to SIG_IGN. As a
+		 * consequence the timer is not going to be rearmed from
+		 * the signal delivery path. But as a real signal handler
+		 * can be installed later the timer must be rearmed here.
 		 */
 		if (timr->it_interval != 0) {
 			ktime_t now = hrtimer_cb_get_time(timer);
@@ -359,34 +360,35 @@ static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 			 * FIXME: What we really want, is to stop this
 			 * timer completely and restart it in case the
 			 * SIG_IGN is removed. This is a non trivial
-			 * change which involves sighand locking
-			 * (sigh !), which we don't want to do late in
-			 * the release cycle.
+			 * change to the signal handling code.
+			 *
+			 * For now let timers with an interval less than a
+			 * jiffie expire every jiffie and recheck for a
+			 * valid signal handler.
+			 *
+			 * This avoids interrupt starvation in case of a
+			 * very small interval, which would expire the
+			 * timer immediately again.
 			 *
-			 * For now we just let timers with an interval
-			 * less than a jiffie expire every jiffie to
-			 * avoid softirq starvation in case of SIG_IGN
-			 * and a very small interval, which would put
-			 * the timer right back on the softirq pending
-			 * list. By moving now ahead of time we trick
-			 * hrtimer_forward() to expire the timer
-			 * later, while we still maintain the overrun
-			 * accuracy, but have some inconsistency in
-			 * the timer_gettime() case. This is at least
-			 * better than a starved softirq. A more
-			 * complex fix which solves also another related
-			 * inconsistency is already in the pipeline.
+			 * Moving now ahead of time by one jiffie tricks
+			 * hrtimer_forward() to expire the timer later,
+			 * while it still maintains the overrun accuracy
+			 * for the price of a slight inconsistency in the
+			 * timer_gettime() case. This is at least better
+			 * than a timer storm.
+			 *
+			 * Only required when high resolution timers are
+			 * enabled as the periodic tick based timers are
+			 * automatically aligned to the next tick.
 			 */
-#ifdef CONFIG_HIGH_RES_TIMERS
-			{
-				ktime_t kj = NSEC_PER_SEC / HZ;
+			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
+				ktime_t kj = TICK_NSEC;
 
 				if (timr->it_interval < kj)
 					now = ktime_add(now, kj);
 			}
-#endif
-			timr->it_overrun += hrtimer_forward(timer, now,
-							    timr->it_interval);
+
+			timr->it_overrun += hrtimer_forward(timer, now, timr->it_interval);
 			ret = HRTIMER_RESTART;
 			++timr->it_requeue_pending;
 			timr->it_active = 1;

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

* [tip: timers/core] posix-timers: Polish coding style in a few places
  2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
                     ` (2 preceding siblings ...)
  2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     b96ce4931fcd13d73e32c62c2df3fa8f9f467e33
Gitweb:        https://git.kernel.org/tip/b96ce4931fcd13d73e32c62c2df3fa8f9f467e33
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:27 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:53 +02:00

posix-timers: Polish coding style in a few places

Make it consistent with the TIP tree documentation.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.888493625@linutronix.de

---
 kernel/time/posix-timers.c | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index d357728..e3cddd5 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -309,10 +309,10 @@ int posix_timer_event(struct k_itimer *timr, int si_private)
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
+	enum hrtimer_restart ret = HRTIMER_NORESTART;
 	struct k_itimer *timr;
 	unsigned long flags;
 	int si_private = 0;
-	enum hrtimer_restart ret = HRTIMER_NORESTART;
 
 	timr = container_of(timer, struct k_itimer, it.real.timer);
 	spin_lock_irqsave(&timr->it_lock, flags);
@@ -400,8 +400,8 @@ static struct pid *good_sigevent(sigevent_t * event)
 
 static struct k_itimer * alloc_posix_timer(void)
 {
-	struct k_itimer *tmr;
-	tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+	struct k_itimer *tmr = kmem_cache_zalloc(posix_timers_cache, GFP_KERNEL);
+
 	if (!tmr)
 		return tmr;
 	if (unlikely(!(tmr->sigq = sigqueue_alloc()))) {
@@ -695,8 +695,8 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
-	struct k_itimer *timr;
 	const struct k_clock *kc;
+	struct k_itimer *timr;
 	unsigned long flags;
 	int ret = 0;
 
@@ -767,8 +767,8 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t, timer_id,
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {
 	struct k_itimer *timr;
-	int overrun;
 	unsigned long flags;
+	int overrun;
 
 	timr = lock_timer(timer_id, &flags);
 	if (!timr)
@@ -941,8 +941,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t, timer_id, int, flags,
 		const struct __kernel_itimerspec __user *, new_setting,
 		struct __kernel_itimerspec __user *, old_setting)
 {
-	struct itimerspec64 new_spec, old_spec;
-	struct itimerspec64 *rtn = old_setting ? &old_spec : NULL;
+	struct itimerspec64 new_spec, old_spec, *rtn;
 	int error = 0;
 
 	if (!new_setting)
@@ -951,6 +950,7 @@ SYSCALL_DEFINE4(timer_settime, timer_t, timer_id, int, flags,
 	if (get_itimerspec64(&new_spec, new_setting))
 		return -EFAULT;
 
+	rtn = old_setting ? &old_spec : NULL;
 	error = do_timer_settime(timer_id, flags, &new_spec, rtn);
 	if (!error && old_setting) {
 		if (put_itimerspec64(&old_spec, old_setting))

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

* [tip: timers/core] posix-timers: Remove pointless comments
  2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
                     ` (2 preceding siblings ...)
  2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     200dbd6d14e6a3b22162d78b4f7a253426dba7ee
Gitweb:        https://git.kernel.org/tip/200dbd6d14e6a3b22162d78b4f7a253426dba7ee
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:25 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:52 +02:00

posix-timers: Remove pointless comments

Documenting the obvious is just consuming space for no value.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.832240451@linutronix.de

---
 kernel/time/posix-timers.c | 25 -------------------------
 1 file changed, 25 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index ae8799e..d357728 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -59,19 +59,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 
-/*
- * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
- *	    to implement others.  This structure defines the various
- *	    clocks.
- *
- * FUNCTIONS: The CLOCKs structure defines possible functions to
- *	    handle various clock functions.
- *
- *	    The standard POSIX timer management code assumes the
- *	    following: 1.) The k_itimer struct (sched.h) is used for
- *	    the timer.  2.) The list, it_lock, it_clock, it_id and
- *	    it_pid fields are not modified by timer code.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
 #define lock_timer(tid, flags)						   \
@@ -141,7 +128,6 @@ static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)
 	spin_unlock_irqrestore(&timr->it_lock, flags);
 }
 
-/* Get clock_realtime */
 static int posix_get_realtime_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_real_ts64(tp);
@@ -153,7 +139,6 @@ static ktime_t posix_get_realtime_ktime(clockid_t which_clock)
 	return ktime_get_real();
 }
 
-/* Set clock_realtime */
 static int posix_clock_realtime_set(const clockid_t which_clock,
 				    const struct timespec64 *tp)
 {
@@ -166,9 +151,6 @@ static int posix_clock_realtime_adj(const clockid_t which_clock,
 	return do_adjtimex(t);
 }
 
-/*
- * Get monotonic time for posix timers
- */
 static int posix_get_monotonic_timespec(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_ts64(tp);
@@ -181,9 +163,6 @@ static ktime_t posix_get_monotonic_ktime(clockid_t which_clock)
 	return ktime_get();
 }
 
-/*
- * Get monotonic-raw time for posix timers
- */
 static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_raw_ts64(tp);
@@ -191,7 +170,6 @@ static int posix_get_monotonic_raw(clockid_t which_clock, struct timespec64 *tp)
 	return 0;
 }
 
-
 static int posix_get_realtime_coarse(clockid_t which_clock, struct timespec64 *tp)
 {
 	ktime_get_coarse_real_ts64(tp);
@@ -242,9 +220,6 @@ static int posix_get_hrtimer_res(clockid_t which_clock, struct timespec64 *tp)
 	return 0;
 }
 
-/*
- * Initialize everything, well, just everything in Posix clocks/timers ;)
- */
 static __init int init_posix_timers(void)
 {
 	posix_timers_cache = kmem_cache_create("posix_timers_cache",

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

* [tip: timers/core] posix-timers: Clarify posix_timer_rearm() comment
  2023-04-25 18:49 ` [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment Thomas Gleixner
  2023-06-01 12:52   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     02972d7955341b711d4c392f14faa9f9cd69d551
Gitweb:        https://git.kernel.org/tip/02972d7955341b711d4c392f14faa9f9cd69d551
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:22 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:52 +02:00

posix-timers: Clarify posix_timer_rearm() comment

Yet another incomprehensible piece of art.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.724863461@linutronix.de

---
 kernel/time/posix-timers.c | 12 +++---------
 1 file changed, 3 insertions(+), 9 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index c8b0f52..d8d2169 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -275,15 +275,9 @@ static void common_hrtimer_rearm(struct k_itimer *timr)
 }
 
 /*
- * This function is exported for use by the signal deliver code.  It is
- * called just prior to the info block being released and passes that
- * block to us.  It's function is to update the overrun entry AND to
- * restart the timer.  It should only be called if the timer is to be
- * restarted (i.e. we have flagged this in the sys_private entry of the
- * info block).
- *
- * To protect against the timer going away while the interrupt is queued,
- * we require that the it_requeue_pending flag be set.
+ * This function is called from the signal delivery code if
+ * info->si_sys_private is not zero, which indicates that the timer has to
+ * be rearmed. Restart the timer and update info::si_overrun.
  */
 void posixtimer_rearm(struct kernel_siginfo *info)
 {

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

* [tip: timers/core] posix-timers: Clarify posix_timer_fn() comments
  2023-06-01 19:07     ` Thomas Gleixner
                         ` (2 preceding siblings ...)
  2023-06-05 22:17       ` tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49       ` tip-bot2 for Thomas Gleixner
  3 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     84999b8bdb4969816c7bb7c14c3a55ed42aa4b94
Gitweb:        https://git.kernel.org/tip/84999b8bdb4969816c7bb7c14c3a55ed42aa4b94
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 21:07:37 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:52 +02:00

posix-timers: Clarify posix_timer_fn() comments

Make the issues vs. SIG_IGN understandable and remove the 15 years old
promise that a proper solution is already on the horizon.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Link: https://lore.kernel.org/r/874jnrdmrq.ffs@tglx

---
 kernel/time/posix-timers.c | 62 +++++++++++++++++++------------------
 1 file changed, 32 insertions(+), 30 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index d8d2169..ae8799e 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -326,11 +326,11 @@ int posix_timer_event(struct k_itimer *timr, int si_private)
 }
 
 /*
- * This function gets called when a POSIX.1b interval timer expires.  It
- * is used as a callback from the kernel internal timer.  The
- * run_timer_list code ALWAYS calls with interrupts on.
-
- * This code is for CLOCK_REALTIME* and CLOCK_MONOTONIC* timers.
+ * This function gets called when a POSIX.1b interval timer expires from
+ * the HRTIMER interrupt (soft interrupt on RT kernels).
+ *
+ * Handles CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME and CLOCK_TAI
+ * based timers.
  */
 static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 {
@@ -348,9 +348,10 @@ static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 
 	if (posix_timer_event(timr, si_private)) {
 		/*
-		 * signal was not sent because of sig_ignor
-		 * we will not get a call back to restart it AND
-		 * it should be restarted.
+		 * The signal was not queued due to SIG_IGN. As a
+		 * consequence the timer is not going to be rearmed from
+		 * the signal delivery path. But as a real signal handler
+		 * can be installed later the timer must be rearmed here.
 		 */
 		if (timr->it_interval != 0) {
 			ktime_t now = hrtimer_cb_get_time(timer);
@@ -359,34 +360,35 @@ static enum hrtimer_restart posix_timer_fn(struct hrtimer *timer)
 			 * FIXME: What we really want, is to stop this
 			 * timer completely and restart it in case the
 			 * SIG_IGN is removed. This is a non trivial
-			 * change which involves sighand locking
-			 * (sigh !), which we don't want to do late in
-			 * the release cycle.
+			 * change to the signal handling code.
+			 *
+			 * For now let timers with an interval less than a
+			 * jiffie expire every jiffie and recheck for a
+			 * valid signal handler.
+			 *
+			 * This avoids interrupt starvation in case of a
+			 * very small interval, which would expire the
+			 * timer immediately again.
 			 *
-			 * For now we just let timers with an interval
-			 * less than a jiffie expire every jiffie to
-			 * avoid softirq starvation in case of SIG_IGN
-			 * and a very small interval, which would put
-			 * the timer right back on the softirq pending
-			 * list. By moving now ahead of time we trick
-			 * hrtimer_forward() to expire the timer
-			 * later, while we still maintain the overrun
-			 * accuracy, but have some inconsistency in
-			 * the timer_gettime() case. This is at least
-			 * better than a starved softirq. A more
-			 * complex fix which solves also another related
-			 * inconsistency is already in the pipeline.
+			 * Moving now ahead of time by one jiffie tricks
+			 * hrtimer_forward() to expire the timer later,
+			 * while it still maintains the overrun accuracy
+			 * for the price of a slight inconsistency in the
+			 * timer_gettime() case. This is at least better
+			 * than a timer storm.
+			 *
+			 * Only required when high resolution timers are
+			 * enabled as the periodic tick based timers are
+			 * automatically aligned to the next tick.
 			 */
-#ifdef CONFIG_HIGH_RES_TIMERS
-			{
-				ktime_t kj = NSEC_PER_SEC / HZ;
+			if (IS_ENABLED(CONFIG_HIGHRES_TIMERS)) {
+				ktime_t kj = TICK_NSEC;
 
 				if (timr->it_interval < kj)
 					now = ktime_add(now, kj);
 			}
-#endif
-			timr->it_overrun += hrtimer_forward(timer, now,
-							    timr->it_interval);
+
+			timr->it_overrun += hrtimer_forward(timer, now, timr->it_interval);
 			ret = HRTIMER_RESTART;
 			++timr->it_requeue_pending;
 			timr->it_active = 1;

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

* [tip: timers/core] posix-timers: Comment SIGEV_THREAD_ID properly
  2023-04-25 18:49 ` [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly Thomas Gleixner
  2023-06-01 12:47   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     c575689d66378611bb04137b96e67ebb3ac8482b
Gitweb:        https://git.kernel.org/tip/c575689d66378611bb04137b96e67ebb3ac8482b
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:20 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:51 +02:00

posix-timers: Comment SIGEV_THREAD_ID properly

Replace the word salad.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.672220780@linutronix.de

---
 kernel/time/posix-timers.c | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 20d3b99..c8b0f52 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -53,12 +53,9 @@ static const struct k_clock * const posix_clocks[];
 static const struct k_clock *clockid_to_kclock(const clockid_t id);
 static const struct k_clock clock_realtime, clock_monotonic;
 
-/*
- * we assume that the new SIGEV_THREAD_ID shares no bits with the other
- * SIGEV values.  Here we put out an error if this assumption fails.
- */
+/* SIGEV_THREAD_ID cannot share a bit with the other SIGEV values. */
 #if SIGEV_THREAD_ID != (SIGEV_THREAD_ID & \
-                       ~(SIGEV_SIGNAL | SIGEV_NONE | SIGEV_THREAD))
+			~(SIGEV_SIGNAL | SIGEV_NONE | SIGEV_THREAD))
 #error "SIGEV_THREAD_ID must not share bit with other SIGEV values!"
 #endif
 

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

* [tip: timers/core] posix-timers: Add proper comments in do_timer_create()
  2023-04-25 18:49 ` [patch 15/20] posix-timers: Add proper comments in do_timer_create() Thomas Gleixner
  2023-06-01 12:43   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     52f090b164b59c06a4da87c0599424809a6bba16
Gitweb:        https://git.kernel.org/tip/52f090b164b59c06a4da87c0599424809a6bba16
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:19 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:51 +02:00

posix-timers: Add proper comments in do_timer_create()

The comment about timer lifetime at the end of the function is misplaced
and uncomprehensible.

Make it understandable and put it at the right place. Add a new comment
about the visibility of the new timer ID to user space.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.619897296@linutronix.de

---
 kernel/time/posix-timers.c | 20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 54adb4c..20d3b99 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -529,12 +529,17 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	new_timer->sigq->info.si_tid   = new_timer->it_id;
 	new_timer->sigq->info.si_code  = SI_TIMER;
 
-	if (copy_to_user(created_timer_id,
-			 &new_timer_id, sizeof (new_timer_id))) {
+	if (copy_to_user(created_timer_id, &new_timer_id, sizeof (new_timer_id))) {
 		error = -EFAULT;
 		goto out;
 	}
-
+	/*
+	 * After succesful copy out, the timer ID is visible to user space
+	 * now but not yet valid because new_timer::signal is still NULL.
+	 *
+	 * Complete the initialization with the clock specific create
+	 * callback.
+	 */
 	error = kc->timer_create(new_timer);
 	if (error)
 		goto out;
@@ -544,14 +549,11 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	WRITE_ONCE(new_timer->it_signal, current->signal);
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
-
-	return 0;
 	/*
-	 * In the case of the timer belonging to another task, after
-	 * the task is unlocked, the timer is owned by the other task
-	 * and may cease to exist at any time.  Don't use or modify
-	 * new_timer after the unlock call.
+	 * After unlocking sighand::siglock @new_timer is subject to
+	 * concurrent removal and cannot be touched anymore
 	 */
+	return 0;
 out:
 	posix_timer_unhash_and_free(new_timer);
 	return error;

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

* [tip: timers/core] posix-timers: Document nanosleep() details
  2023-04-25 18:49 ` [patch 14/20] posix-timers: Document nanosleep() details Thomas Gleixner
  2023-06-01 12:30   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:49 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     640fe745d7d4b6e47f3b455cb5de99a08c6b6d23
Gitweb:        https://git.kernel.org/tip/640fe745d7d4b6e47f3b455cb5de99a08c6b6d23
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:17 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:51 +02:00

posix-timers: Document nanosleep() details

The descriptions for common_nsleep() is wrong and common_nsleep_timens()
lacks any form of comment.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.567072835@linutronix.de

---
 kernel/time/posix-timers.c |  9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 03ef6af..54adb4c 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1370,7 +1370,7 @@ SYSCALL_DEFINE2(clock_getres_time32, clockid_t, which_clock,
 #endif
 
 /*
- * nanosleep for monotonic and realtime clocks
+ * sys_clock_nanosleep() for CLOCK_REALTIME and CLOCK_TAI
  */
 static int common_nsleep(const clockid_t which_clock, int flags,
 			 const struct timespec64 *rqtp)
@@ -1382,8 +1382,13 @@ static int common_nsleep(const clockid_t which_clock, int flags,
 				 which_clock);
 }
 
+/*
+ * sys_clock_nanosleep() for CLOCK_MONOTONIC and CLOCK_BOOTTIME
+ *
+ * Absolute nanosleeps for these clocks are time-namespace adjusted.
+ */
 static int common_nsleep_timens(const clockid_t which_clock, int flags,
-			 const struct timespec64 *rqtp)
+				const struct timespec64 *rqtp)
 {
 	ktime_t texp = timespec64_to_ktime(*rqtp);
 

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

* [tip: timers/core] posix-timers: Document sys_clock_settime() permissions in place
  2023-04-25 18:49 ` [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place Thomas Gleixner
  2023-06-01 11:22   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     3561fcb402b7ab7fdb4c1746dae4995889506605
Gitweb:        https://git.kernel.org/tip/3561fcb402b7ab7fdb4c1746dae4995889506605
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:16 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:51 +02:00

posix-timers: Document sys_clock_settime() permissions in place

The documentation of sys_clock_settime() permissions is at a random place
and mostly word salad.

Remove it and add a concise comment into sys_clock_settime().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.514700292@linutronix.de

---
 kernel/time/posix-timers.c | 11 ++++-------
 1 file changed, 4 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 191ecf5..03ef6af 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -74,13 +74,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
  *	    following: 1.) The k_itimer struct (sched.h) is used for
  *	    the timer.  2.) The list, it_lock, it_clock, it_id and
  *	    it_pid fields are not modified by timer code.
- *
- * Permissions: It is assumed that the clock_settime() function defined
- *	    for each clock will take care of permission checks.	 Some
- *	    clocks may be set able by any user (i.e. local process
- *	    clocks) others not.	 Currently the only set able clock we
- *	    have is CLOCK_REALTIME and its high res counter part, both of
- *	    which we beg off on and pass to do_sys_settimeofday().
  */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags);
 
@@ -1159,6 +1152,10 @@ SYSCALL_DEFINE2(clock_settime, const clockid_t, which_clock,
 	if (get_timespec64(&new_tp, tp))
 		return -EFAULT;
 
+	/*
+	 * Permission checks have to be done inside the clock specific
+	 * setter callback.
+	 */
 	return kc->clock_set(which_clock, &new_tp);
 }
 

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

* [tip: timers/core] posix-timers: Document sys_clock_getoverrun()
  2023-04-25 18:49 ` [patch 12/20] posix-timers: Document sys_clock_getoverrun() Thomas Gleixner
  2023-06-01 11:06   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     65cade468dee537885b51897ba41ba05a4dcf6ca
Gitweb:        https://git.kernel.org/tip/65cade468dee537885b51897ba41ba05a4dcf6ca
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:14 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:50 +02:00

posix-timers: Document sys_clock_getoverrun()

Document the syscall in detail and with coherent sentences.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.462051641@linutronix.de

---
 kernel/time/posix-timers.c | 25 +++++++++++++++++--------
 1 file changed, 17 insertions(+), 8 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index e1af74c..191ecf5 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -783,14 +783,23 @@ SYSCALL_DEFINE2(timer_gettime32, timer_t, timer_id,
 
 #endif
 
-/*
- * Get the number of overruns of a POSIX.1b interval timer.  This is to
- * be the overrun of the timer last delivered.  At the same time we are
- * accumulating overruns on the next timer.  The overrun is frozen when
- * the signal is delivered, either at the notify time (if the info block
- * is not queued) or at the actual delivery time (as we are informed by
- * the call back to posixtimer_rearm().  So all we need to do is
- * to pick up the frozen overrun.
+/**
+ * sys_timer_getoverrun - Get the number of overruns of a POSIX.1b interval timer
+ * @timer_id:	The timer ID which identifies the timer
+ *
+ * The "overrun count" of a timer is one plus the number of expiration
+ * intervals which have elapsed between the first expiry, which queues the
+ * signal and the actual signal delivery. On signal delivery the "overrun
+ * count" is calculated and cached, so it can be returned directly here.
+ *
+ * As this is relative to the last queued signal the returned overrun count
+ * is meaningless outside of the signal delivery path and even there it
+ * does not accurately reflect the current state when user space evaluates
+ * it.
+ *
+ * Returns:
+ *	-EINVAL		@timer_id is invalid
+ *	1..INT_MAX	The number of overruns related to the last delivered signal
  */
 SYSCALL_DEFINE1(timer_getoverrun, timer_t, timer_id)
 {

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

* [tip: timers/core] posix-timers: Document sys_clock_getres() correctly
  2023-04-25 18:49 ` [patch 10/20] posix-timers: Document sys_clock_getres() correctly Thomas Gleixner
  2023-06-01 10:44   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     01679b5db7172b898be325ff272e10aebd412911
Gitweb:        https://git.kernel.org/tip/01679b5db7172b898be325ff272e10aebd412911
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:11 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:50 +02:00

posix-timers: Document sys_clock_getres() correctly

The decades old comment about Posix clock resolution is confusing at best.

Remove it and add a proper explanation to sys_clock_getres().

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.356427330@linutronix.de

---
 kernel/time/posix-timers.c | 81 +++++++++++++++++++++++++++++++++----
 1 file changed, 73 insertions(+), 8 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 6ac6933..1acdd04 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -67,14 +67,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
  *	    to implement others.  This structure defines the various
  *	    clocks.
  *
- * RESOLUTION: Clock resolution is used to round up timer and interval
- *	    times, NOT to report clock times, which are reported with as
- *	    much resolution as the system can muster.  In some cases this
- *	    resolution may depend on the underlying clock hardware and
- *	    may not be quantifiable until run time, and only then is the
- *	    necessary code is written.	The standard says we should say
- *	    something about this issue in the documentation...
- *
  * FUNCTIONS: The CLOCKs structure defines possible functions to
  *	    handle various clock functions.
  *
@@ -1198,6 +1190,79 @@ SYSCALL_DEFINE2(clock_adjtime, const clockid_t, which_clock,
 	return err;
 }
 
+/**
+ * sys_clock_getres - Get the resolution of a clock
+ * @which_clock:	The clock to get the resolution for
+ * @tp:			Pointer to a a user space timespec64 for storage
+ *
+ * POSIX defines:
+ *
+ * "The clock_getres() function shall return the resolution of any
+ * clock. Clock resolutions are implementation-defined and cannot be set by
+ * a process. If the argument res is not NULL, the resolution of the
+ * specified clock shall be stored in the location pointed to by res. If
+ * res is NULL, the clock resolution is not returned. If the time argument
+ * of clock_settime() is not a multiple of res, then the value is truncated
+ * to a multiple of res."
+ *
+ * Due to the various hardware constraints the real resolution can vary
+ * wildly and even change during runtime when the underlying devices are
+ * replaced. The kernel also can use hardware devices with different
+ * resolutions for reading the time and for arming timers.
+ *
+ * The kernel therefore deviates from the POSIX spec in various aspects:
+ *
+ * 1) The resolution returned to user space
+ *
+ *    For CLOCK_REALTIME, CLOCK_MONOTONIC, CLOCK_BOOTTIME, CLOCK_TAI,
+ *    CLOCK_REALTIME_ALARM, CLOCK_BOOTTIME_ALAREM and CLOCK_MONOTONIC_RAW
+ *    the kernel differentiates only two cases:
+ *
+ *    I)  Low resolution mode:
+ *
+ *	  When high resolution timers are disabled at compile or runtime
+ *	  the resolution returned is nanoseconds per tick, which represents
+ *	  the precision at which timers expire.
+ *
+ *    II) High resolution mode:
+ *
+ *	  When high resolution timers are enabled the resolution returned
+ *	  is always one nanosecond independent of the actual resolution of
+ *	  the underlying hardware devices.
+ *
+ *	  For CLOCK_*_ALARM the actual resolution depends on system
+ *	  state. When system is running the resolution is the same as the
+ *	  resolution of the other clocks. During suspend the actual
+ *	  resolution is the resolution of the underlying RTC device which
+ *	  might be way less precise than the clockevent device used during
+ *	  running state.
+ *
+ *   For CLOCK_REALTIME_COARSE and CLOCK_MONOTONIC_COARSE the resolution
+ *   returned is always nanoseconds per tick.
+ *
+ *   For CLOCK_PROCESS_CPUTIME and CLOCK_THREAD_CPUTIME the resolution
+ *   returned is always one nanosecond under the assumption that the
+ *   underlying scheduler clock has a better resolution than nanoseconds
+ *   per tick.
+ *
+ *   For dynamic POSIX clocks (PTP devices) the resolution returned is
+ *   always one nanosecond.
+ *
+ * 2) Affect on sys_clock_settime()
+ *
+ *    The kernel does not truncate the time which is handed in to
+ *    sys_clock_settime(). The kernel internal timekeeping is always using
+ *    nanoseconds precision independent of the clocksource device which is
+ *    used to read the time from. The resolution of that device only
+ *    affects the presicion of the time returned by sys_clock_gettime().
+ *
+ * Returns:
+ *	0		Success. @tp contains the resolution
+ *	-EINVAL		@which_clock is not a valid clock ID
+ *	-EFAULT		Copying the resolution to @tp faulted
+ *	-ENODEV		Dynamic POSIX clock is not backed by a device
+ *	-EOPNOTSUPP	Dynamic POSIX clock does not support getres()
+ */
 SYSCALL_DEFINE2(clock_getres, const clockid_t, which_clock,
 		struct __kernel_timespec __user *, tp)
 {

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

* [tip: timers/core] posix-timers: Document common_clock_get() correctly
  2023-04-25 18:49 ` [patch 11/20] posix-timers: Document common_clock_get() correctly Thomas Gleixner
  2023-06-01 11:00   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     a86e9284336729a8f79a65371eacd0c1c7fae142
Gitweb:        https://git.kernel.org/tip/a86e9284336729a8f79a65371eacd0c1c7fae142
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:12 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:50 +02:00

posix-timers: Document common_clock_get() correctly

Replace another confusing and inaccurate set of comments.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.409169321@linutronix.de

---
 kernel/time/posix-timers.c | 50 ++++++++++++++++++++++---------------
 1 file changed, 30 insertions(+), 20 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 1acdd04..e1af74c 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -660,20 +660,16 @@ static s64 common_hrtimer_forward(struct k_itimer *timr, ktime_t now)
 }
 
 /*
- * Get the time remaining on a POSIX.1b interval timer.  This function
- * is ALWAYS called with spin_lock_irq on the timer, thus it must not
- * mess with irq.
+ * Get the time remaining on a POSIX.1b interval timer.
  *
- * We have a couple of messes to clean up here.  First there is the case
- * of a timer that has a requeue pending.  These timers should appear to
- * be in the timer list with an expiry as if we were to requeue them
- * now.
+ * Two issues to handle here:
  *
- * The second issue is the SIGEV_NONE timer which may be active but is
- * not really ever put in the timer list (to save system resources).
- * This timer may be expired, and if so, we will do it here.  Otherwise
- * it is the same as a requeue pending timer WRT to what we should
- * report.
+ *  1) The timer has a requeue pending. The return value must appear as
+ *     if the timer has been requeued right now.
+ *
+ *  2) The timer is a SIGEV_NONE timer. These timers are never enqueued
+ *     into the hrtimer queue and therefore never expired. Emulate expiry
+ *     here taking #1 into account.
  */
 void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 {
@@ -689,8 +685,12 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 		cur_setting->it_interval = ktime_to_timespec64(iv);
 	} else if (!timr->it_active) {
 		/*
-		 * SIGEV_NONE oneshot timers are never queued. Check them
-		 * below.
+		 * SIGEV_NONE oneshot timers are never queued and therefore
+		 * timr->it_active is always false. The check below
+		 * vs. remaining time will handle this case.
+		 *
+		 * For all other timers there is nothing to update here, so
+		 * return.
 		 */
 		if (!sig_none)
 			return;
@@ -699,18 +699,29 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 	now = kc->clock_get_ktime(timr->it_clock);
 
 	/*
-	 * When a requeue is pending or this is a SIGEV_NONE timer move the
-	 * expiry time forward by intervals, so expiry is > now.
+	 * If this is an interval timer and either has requeue pending or
+	 * is a SIGEV_NONE timer move the expiry time forward by intervals,
+	 * so expiry is > now.
 	 */
 	if (iv && (timr->it_requeue_pending & REQUEUE_PENDING || sig_none))
 		timr->it_overrun += kc->timer_forward(timr, now);
 
 	remaining = kc->timer_remaining(timr, now);
-	/* Return 0 only, when the timer is expired and not pending */
+	/*
+	 * As @now is retrieved before a possible timer_forward() and
+	 * cannot be reevaluated by the compiler @remaining is based on the
+	 * same @now value. Therefore @remaining is consistent vs. @now.
+	 *
+	 * Consequently all interval timers, i.e. @iv > 0, cannot have a
+	 * remaining time <= 0 because timer_forward() guarantees to move
+	 * them forward so that the next timer expiry is > @now.
+	 */
 	if (remaining <= 0) {
 		/*
-		 * A single shot SIGEV_NONE timer must return 0, when
-		 * it is expired !
+		 * A single shot SIGEV_NONE timer must return 0, when it is
+		 * expired! Timers which have a real signal delivery mode
+		 * must return a remaining time greater than 0 because the
+		 * signal has not yet been delivered.
 		 */
 		if (!sig_none)
 			cur_setting->it_value.tv_nsec = 1;
@@ -719,7 +730,6 @@ void common_timer_get(struct k_itimer *timr, struct itimerspec64 *cur_setting)
 	}
 }
 
-/* Get the time remaining on a POSIX.1b interval timer. */
 static int do_timer_gettime(timer_t timer_id,  struct itimerspec64 *setting)
 {
 	struct k_itimer *timr;

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

* [tip: timers/core] posix-timers: Split release_posix_timers()
  2023-04-25 18:49 ` [patch 09/20] posix-timers: Split release_posix_timers() Thomas Gleixner
  2023-06-01 10:25   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     8cc96ca2c75f6da59de41321797c87562703c9e1
Gitweb:        https://git.kernel.org/tip/8cc96ca2c75f6da59de41321797c87562703c9e1
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:09 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:50 +02:00

posix-timers: Split release_posix_timers()

release_posix_timers() is called for cleaning up both hashed and unhashed
timers. The cases are differentiated by an argument and the usage is
hideous.

Seperate the actual free path out and use it for unhashed timers. Provide a
function for hashed timers.

No functional change.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.301432503@linutronix.de

---
 kernel/time/posix-timers.c | 31 +++++++++++++++----------------
 1 file changed, 15 insertions(+), 16 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index ed7d260..6ac6933 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -466,20 +466,21 @@ static void k_itimer_rcu_free(struct rcu_head *head)
 	kmem_cache_free(posix_timers_cache, tmr);
 }
 
-#define IT_ID_SET	1
-#define IT_ID_NOT_SET	0
-static void release_posix_timer(struct k_itimer *tmr, int it_id_set)
-{
-	if (it_id_set) {
-		spin_lock(&hash_lock, flags);
-		hlist_del_rcu(&tmr->t_hash);
-		spin_unlock(&hash_lock, flags);
-	}
+static void posix_timer_free(struct k_itimer *tmr)
+{
 	put_pid(tmr->it_pid);
 	sigqueue_free(tmr->sigq);
 	call_rcu(&tmr->rcu, k_itimer_rcu_free);
 }
 
+static void posix_timer_unhash_and_free(struct k_itimer *tmr)
+{
+	spin_lock(&hash_lock);
+	hlist_del_rcu(&tmr->t_hash);
+	spin_unlock(&hash_lock);
+	posix_timer_free(tmr);
+}
+
 static int common_timer_create(struct k_itimer *new_timer)
 {
 	hrtimer_init(&new_timer->it.real.timer, new_timer->it_clock, 0);
@@ -493,7 +494,6 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	const struct k_clock *kc = clockid_to_kclock(which_clock);
 	struct k_itimer *new_timer;
 	int error, new_timer_id;
-	int it_id_set = IT_ID_NOT_SET;
 
 	if (!kc)
 		return -EINVAL;
@@ -513,11 +513,10 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	 */
 	new_timer_id = posix_timer_add(new_timer);
 	if (new_timer_id < 0) {
-		error = new_timer_id;
-		goto out;
+		posix_timer_free(new_timer);
+		return new_timer_id;
 	}
 
-	it_id_set = IT_ID_SET;
 	new_timer->it_id = (timer_t) new_timer_id;
 	new_timer->it_clock = which_clock;
 	new_timer->kclock = kc;
@@ -569,7 +568,7 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 	 * new_timer after the unlock call.
 	 */
 out:
-	release_posix_timer(new_timer, it_id_set);
+	posix_timer_unhash_and_free(new_timer);
 	return error;
 }
 
@@ -1057,7 +1056,7 @@ retry_delete:
 	WRITE_ONCE(timer->it_signal, NULL);
 
 	unlock_timer(timer, flags);
-	release_posix_timer(timer, IT_ID_SET);
+	posix_timer_unhash_and_free(timer);
 	return 0;
 }
 
@@ -1109,7 +1108,7 @@ retry_delete:
 	WRITE_ONCE(timer->it_signal, NULL);
 
 	spin_unlock_irqrestore(&timer->it_lock, flags);
-	release_posix_timer(timer, IT_ID_SET);
+	posix_timer_unhash_and_free(timer);
 }
 
 /*

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

* [tip: timers/core] posix-timers: Remove pointless irqsafe from hash_lock
  2023-04-25 18:49 ` [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock Thomas Gleixner
  2023-06-01 10:12   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     11fbe6cd41210c7b5173257158a22e11e225622d
Gitweb:        https://git.kernel.org/tip/11fbe6cd41210c7b5173257158a22e11e225622d
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:08 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:49 +02:00

posix-timers: Remove pointless irqsafe from hash_lock

All usage of hash_lock is in thread context. No point in using
spin_lock_irqsave()/irqrestore() for a single usage site.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.249063953@linutronix.de

---
 kernel/time/posix-timers.c | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index c1b77c5..ed7d260 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -471,10 +471,9 @@ static void k_itimer_rcu_free(struct rcu_head *head)
 static void release_posix_timer(struct k_itimer *tmr, int it_id_set)
 {
 	if (it_id_set) {
-		unsigned long flags;
-		spin_lock_irqsave(&hash_lock, flags);
+		spin_lock(&hash_lock, flags);
 		hlist_del_rcu(&tmr->t_hash);
-		spin_unlock_irqrestore(&hash_lock, flags);
+		spin_unlock(&hash_lock, flags);
 	}
 	put_pid(tmr->it_pid);
 	sigqueue_free(tmr->sigq);

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

* [tip: timers/core] posix-timers: Annotate concurrent access to k_itimer:: It_signal
  2023-04-25 18:49 ` [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal Thomas Gleixner
  2023-05-09 11:04   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] posix-timers: Annotate concurrent access to k_itimer:: It_signal tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     028cf5eaa12846c4e32104132ff70ca1cd6f5943
Gitweb:        https://git.kernel.org/tip/028cf5eaa12846c4e32104132ff70ca1cd6f5943
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:05 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:49 +02:00

posix-timers: Annotate concurrent access to k_itimer:: It_signal

k_itimer::it_signal is read lockless in the RCU protected hash lookup, but
it can be written concurrently in the timer_create() and timer_delete()
path. Annotate these places with READ_ONCE() and WRITE_ONCE()

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.143596887@linutronix.de

---
 kernel/time/posix-timers.c | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index d7890ac..de3fca8 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -109,9 +109,9 @@ static struct k_itimer *__posix_timers_find(struct hlist_head *head,
 {
 	struct k_itimer *timer;
 
-	hlist_for_each_entry_rcu(timer, head, t_hash,
-				 lockdep_is_held(&hash_lock)) {
-		if ((timer->it_signal == sig) && (timer->it_id == id))
+	hlist_for_each_entry_rcu(timer, head, t_hash, lockdep_is_held(&hash_lock)) {
+		/* timer->it_signal can be set concurrently */
+		if ((READ_ONCE(timer->it_signal) == sig) && (timer->it_id == id))
 			return timer;
 	}
 	return NULL;
@@ -558,7 +558,7 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 
 	spin_lock_irq(&current->sighand->siglock);
 	/* This makes the timer valid in the hash table */
-	new_timer->it_signal = current->signal;
+	WRITE_ONCE(new_timer->it_signal, current->signal);
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
 
@@ -1052,10 +1052,10 @@ retry_delete:
 	list_del(&timer->list);
 	spin_unlock(&current->sighand->siglock);
 	/*
-	 * This keeps any tasks waiting on the spin lock from thinking
-	 * they got something (see the lock code above).
+	 * A concurrent lookup could check timer::it_signal lockless. It
+	 * will reevaluate with timer::it_lock held and observe the NULL.
 	 */
-	timer->it_signal = NULL;
+	WRITE_ONCE(timer->it_signal, NULL);
 
 	unlock_timer(timer, flags);
 	release_posix_timer(timer, IT_ID_SET);

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

* [tip: timers/core] posix-timers: Set k_itimer:: It_signal to NULL on exit()
  2023-04-25 18:49 ` [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit() Thomas Gleixner
  2023-06-01 10:09   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] posix-timers: Set k_itimer:: It_signal " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     72786ff23d5acb7bf3e2535831b2f1dc55c7f44e
Gitweb:        https://git.kernel.org/tip/72786ff23d5acb7bf3e2535831b2f1dc55c7f44e
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:06 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:49 +02:00

posix-timers: Set k_itimer:: It_signal to NULL on exit()

Technically it's not required to set k_itimer::it_signal to NULL on exit()
because there is no other thread anymore which could lookup the timer
concurrently.

Set it to NULL for consistency sake and add a comment to that effect.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.196462644@linutronix.de

---
 kernel/time/posix-timers.c | 8 ++++++++
 1 file changed, 8 insertions(+)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index de3fca8..c1b77c5 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1101,6 +1101,14 @@ retry_delete:
 	}
 	list_del(&timer->list);
 
+	/*
+	 * Setting timer::it_signal to NULL is technically not required
+	 * here as nothing can access the timer anymore legitimately via
+	 * the hash table. Set it to NULL nevertheless so that all deletion
+	 * paths are consistent.
+	 */
+	WRITE_ONCE(timer->it_signal, NULL);
+
 	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }

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

* [tip: timers/core] posix-timers: Add comments about timer lookup
  2023-04-25 18:49 ` [patch 05/20] posix-timers: Add comments about timer lookup Thomas Gleixner
  2023-05-09 10:58   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     ae88967d71f1b4ffb6e48043993d37a106da8109
Gitweb:        https://git.kernel.org/tip/ae88967d71f1b4ffb6e48043993d37a106da8109
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:03 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:49 +02:00

posix-timers: Add comments about timer lookup

Document how the timer ID validation in the hash table works.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.091081515@linutronix.de

---
 kernel/time/posix-timers.c | 39 ++++++++++++++++++++++++++++++-------
 1 file changed, 32 insertions(+), 7 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 79909a2..d7890ac 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -506,6 +506,12 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 		return -EAGAIN;
 
 	spin_lock_init(&new_timer->it_lock);
+
+	/*
+	 * Add the timer to the hash table. The timer is not yet valid
+	 * because new_timer::it_signal is still NULL. The timer id is also
+	 * not yet visible to user space.
+	 */
 	new_timer_id = posix_timer_add(new_timer);
 	if (new_timer_id < 0) {
 		error = new_timer_id;
@@ -551,6 +557,7 @@ static int do_timer_create(clockid_t which_clock, struct sigevent *event,
 		goto out;
 
 	spin_lock_irq(&current->sighand->siglock);
+	/* This makes the timer valid in the hash table */
 	new_timer->it_signal = current->signal;
 	list_add(&new_timer->list, &current->signal->posix_timers);
 	spin_unlock_irq(&current->sighand->siglock);
@@ -597,13 +604,6 @@ COMPAT_SYSCALL_DEFINE3(timer_create, clockid_t, which_clock,
 }
 #endif
 
-/*
- * Locking issues: We need to protect the result of the id look up until
- * we get the timer locked down so it is not deleted under us.  The
- * removal is done under the idr spinlock so we use that here to bridge
- * the find to the timer lock.  To avoid a dead lock, the timer id MUST
- * be release with out holding the timer lock.
- */
 static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags)
 {
 	struct k_itimer *timr;
@@ -615,10 +615,35 @@ static struct k_itimer *__lock_timer(timer_t timer_id, unsigned long *flags)
 	if ((unsigned long long)timer_id > INT_MAX)
 		return NULL;
 
+	/*
+	 * The hash lookup and the timers are RCU protected.
+	 *
+	 * Timers are added to the hash in invalid state where
+	 * timr::it_signal == NULL. timer::it_signal is only set after the
+	 * rest of the initialization succeeded.
+	 *
+	 * Timer destruction happens in steps:
+	 *  1) Set timr::it_signal to NULL with timr::it_lock held
+	 *  2) Release timr::it_lock
+	 *  3) Remove from the hash under hash_lock
+	 *  4) Call RCU for removal after the grace period
+	 *
+	 * Holding rcu_read_lock() accross the lookup ensures that
+	 * the timer cannot be freed.
+	 *
+	 * The lookup validates locklessly that timr::it_signal ==
+	 * current::it_signal and timr::it_id == @timer_id. timr::it_id
+	 * can't change, but timr::it_signal becomes NULL during
+	 * destruction.
+	 */
 	rcu_read_lock();
 	timr = posix_timer_by_id(timer_id);
 	if (timr) {
 		spin_lock_irqsave(&timr->it_lock, *flags);
+		/*
+		 * Validate under timr::it_lock that timr::it_signal is
+		 * still valid. Pairs with #1 above.
+		 */
 		if (timr->it_signal == current->signal) {
 			rcu_read_unlock();
 			return timr;

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

* [tip: timers/core] posix-timers: Clarify timer_wait_running() comment
  2023-04-25 18:49 ` [patch 03/20] posix-timers: Clarify timer_wait_running() comment Thomas Gleixner
  2023-05-09  9:50   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     7d9909026645064cd31b20cee5939a0f72282261
Gitweb:        https://git.kernel.org/tip/7d9909026645064cd31b20cee5939a0f72282261
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:00 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:48 +02:00

posix-timers: Clarify timer_wait_running() comment

Explain it better and add the CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y aspect
for completeness.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183312.985681995@linutronix.de

---
 kernel/time/posix-timers.c | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 2d6cf93..0c61f29 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -836,10 +836,18 @@ static void common_timer_wait_running(struct k_itimer *timer)
 }
 
 /*
- * On PREEMPT_RT this prevent priority inversion against softirq kthread in
- * case it gets preempted while executing a timer callback. See comments in
- * hrtimer_cancel_wait_running. For PREEMPT_RT=n this just results in a
- * cpu_relax().
+ * On PREEMPT_RT this prevents priority inversion and a potential livelock
+ * against the ksoftirqd thread in case that ksoftirqd gets preempted while
+ * executing a hrtimer callback.
+ *
+ * See the comments in hrtimer_cancel_wait_running(). For PREEMPT_RT=n this
+ * just results in a cpu_relax().
+ *
+ * For POSIX CPU timers with CONFIG_POSIX_CPU_TIMERS_TASK_WORK=n this is
+ * just a cpu_relax(). With CONFIG_POSIX_CPU_TIMERS_TASK_WORK=y this
+ * prevents spinning on an eventually scheduled out task and a livelock
+ * when the task which tries to delete or disarm the timer has preempted
+ * the task which runs the expiry in task work context.
  */
 static struct k_itimer *timer_wait_running(struct k_itimer *timer,
 					   unsigned long *flags)

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

* [tip: timers/core] posix-timers: Cleanup comments about timer ID tracking
  2023-04-25 18:49 ` [patch 04/20] posix-timers: Cleanup comments about timer ID tracking Thomas Gleixner
  2023-05-09  9:58   ` Frederic Weisbecker
  2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     8d44b958a1a088195524c884f0437660e0522380
Gitweb:        https://git.kernel.org/tip/8d44b958a1a088195524c884f0437660e0522380
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Tue, 25 Apr 2023 20:49:01 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:48 +02:00

posix-timers: Cleanup comments about timer ID tracking

Describe the hash table properly and remove the IDR leftover comments.

Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/20230425183313.038444551@linutronix.de

---
 kernel/time/posix-timers.c | 28 ++++++++--------------------
 1 file changed, 8 insertions(+), 20 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 0c61f29..79909a2 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -35,20 +35,17 @@
 #include "timekeeping.h"
 #include "posix-timers.h"
 
-/*
- * Management arrays for POSIX timers. Timers are now kept in static hash table
- * with 512 entries.
- * Timer ids are allocated by local routine, which selects proper hash head by
- * key, constructed from current->signal address and per signal struct counter.
- * This keeps timer ids unique per process, but now they can intersect between
- * processes.
- */
+static struct kmem_cache *posix_timers_cache;
 
 /*
- * Lets keep our timers in a slab cache :-)
+ * Timers are managed in a hash table for lockless lookup. The hash key is
+ * constructed from current::signal and the timer ID and the timer is
+ * matched against current::signal and the timer ID when walking the hash
+ * bucket list.
+ *
+ * This allows checkpoint/restore to reconstruct the exact timer IDs for
+ * a process.
  */
-static struct kmem_cache *posix_timers_cache;
-
 static DEFINE_HASHTABLE(posix_timers_hashtable, 9);
 static DEFINE_SPINLOCK(hash_lock);
 
@@ -66,15 +63,6 @@ static const struct k_clock clock_realtime, clock_monotonic;
 #endif
 
 /*
- * The timer ID is turned into a timer address by idr_find().
- * Verifying a valid ID consists of:
- *
- * a) checking that idr_find() returns other than -1.
- * b) checking that the timer id matches the one in the timer itself.
- * c) that the timer owner is in the callers thread group.
- */
-
-/*
  * CLOCKs: The POSIX standard calls for a couple of clocks and allows us
  *	    to implement others.  This structure defines the various
  *	    clocks.

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

* [tip: timers/core] posix-timers: Ensure timer ID search-loop limit is valid
  2023-06-01 18:58               ` [patch v2 " Thomas Gleixner
  2023-06-05 14:17                 ` Frederic Weisbecker
  2023-06-05 15:08                 ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50                 ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits
  Cc: syzbot+5c54bd3eb218bb595aa9, Dmitry Vyukov, Thomas Gleixner,
	Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     8ce8849dd1e78dadcee0ec9acbd259d239b7069f
Gitweb:        https://git.kernel.org/tip/8ce8849dd1e78dadcee0ec9acbd259d239b7069f
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 20:58:47 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:41:48 +02:00

posix-timers: Ensure timer ID search-loop limit is valid

posix_timer_add() tries to allocate a posix timer ID by starting from the
cached ID which was stored by the last successful allocation.

This is done in a loop searching the ID space for a free slot one by
one. The loop has to terminate when the search wrapped around to the
starting point.

But that's racy vs. establishing the starting point. That is read out
lockless, which leads to the following problem:

CPU0	  	      	     	   CPU1
posix_timer_add()
  start = sig->posix_timer_id;
  lock(hash_lock);
  ...				   posix_timer_add()
  if (++sig->posix_timer_id < 0)
      			             start = sig->posix_timer_id;
     sig->posix_timer_id = 0;

So CPU1 can observe a negative start value, i.e. -1, and the loop break
never happens because the condition can never be true:

  if (sig->posix_timer_id == start)
     break;

While this is unlikely to ever turn into an endless loop as the ID space is
huge (INT_MAX), the racy read of the start value caught the attention of
KCSAN and Dmitry unearthed that incorrectness.

Rewrite it so that all id operations are under the hash lock.

Reported-by: syzbot+5c54bd3eb218bb595aa9@syzkaller.appspotmail.com
Reported-by: Dmitry Vyukov <dvyukov@google.com>
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/87bkhzdn6g.ffs@tglx

---
 include/linux/sched/signal.h |  2 +-
 kernel/time/posix-timers.c   | 31 ++++++++++++++++++-------------
 2 files changed, 19 insertions(+), 14 deletions(-)

diff --git a/include/linux/sched/signal.h b/include/linux/sched/signal.h
index 2009926..669e8cf 100644
--- a/include/linux/sched/signal.h
+++ b/include/linux/sched/signal.h
@@ -135,7 +135,7 @@ struct signal_struct {
 #ifdef CONFIG_POSIX_TIMERS
 
 	/* POSIX.1b Interval Timers */
-	int			posix_timer_id;
+	unsigned int		next_posix_timer_id;
 	struct list_head	posix_timers;
 
 	/* ITIMER_REAL timer for the process */
diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index ed3c4a9..2d6cf93 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -140,25 +140,30 @@ static struct k_itimer *posix_timer_by_id(timer_t id)
 static int posix_timer_add(struct k_itimer *timer)
 {
 	struct signal_struct *sig = current->signal;
-	int first_free_id = sig->posix_timer_id;
 	struct hlist_head *head;
-	int ret = -ENOENT;
+	unsigned int cnt, id;
 
-	do {
+	/*
+	 * FIXME: Replace this by a per signal struct xarray once there is
+	 * a plan to handle the resulting CRIU regression gracefully.
+	 */
+	for (cnt = 0; cnt <= INT_MAX; cnt++) {
 		spin_lock(&hash_lock);
-		head = &posix_timers_hashtable[hash(sig, sig->posix_timer_id)];
-		if (!__posix_timers_find(head, sig, sig->posix_timer_id)) {
+		id = sig->next_posix_timer_id;
+
+		/* Write the next ID back. Clamp it to the positive space */
+		sig->next_posix_timer_id = (id + 1) & INT_MAX;
+
+		head = &posix_timers_hashtable[hash(sig, id)];
+		if (!__posix_timers_find(head, sig, id)) {
 			hlist_add_head_rcu(&timer->t_hash, head);
-			ret = sig->posix_timer_id;
+			spin_unlock(&hash_lock);
+			return id;
 		}
-		if (++sig->posix_timer_id < 0)
-			sig->posix_timer_id = 0;
-		if ((sig->posix_timer_id == first_free_id) && (ret == -ENOENT))
-			/* Loop over all possible ids completed */
-			ret = -EAGAIN;
 		spin_unlock(&hash_lock);
-	} while (ret == -ENOENT);
-	return ret;
+	}
+	/* POSIX return code when no timer ID could be allocated */
+	return -EAGAIN;
 }
 
 static inline void unlock_timer(struct k_itimer *timr, unsigned long flags)

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

* [tip: timers/core] posix-timers: Prevent RT livelock in itimer_delete()
  2023-06-01 20:16           ` [patch v2a " Thomas Gleixner
  2023-06-05 10:59             ` Frederic Weisbecker
  2023-06-05 15:08             ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
@ 2023-06-18 20:50             ` tip-bot2 for Thomas Gleixner
  2 siblings, 0 replies; 122+ messages in thread
From: tip-bot2 for Thomas Gleixner @ 2023-06-18 20:50 UTC (permalink / raw)
  To: linux-tip-commits; +Cc: Thomas Gleixner, Frederic Weisbecker, x86, linux-kernel

The following commit has been merged into the timers/core branch of tip:

Commit-ID:     9d9e522010eb5685d8b53e8a24320653d9d4cbbf
Gitweb:        https://git.kernel.org/tip/9d9e522010eb5685d8b53e8a24320653d9d4cbbf
Author:        Thomas Gleixner <tglx@linutronix.de>
AuthorDate:    Thu, 01 Jun 2023 22:16:34 +02:00
Committer:     Thomas Gleixner <tglx@linutronix.de>
CommitterDate: Sun, 18 Jun 2023 22:40:42 +02:00

posix-timers: Prevent RT livelock in itimer_delete()

itimer_delete() has a retry loop when the timer is concurrently expired. On
non-RT kernels this just spin-waits until the timer callback has completed,
except for posix CPU timers which have HAVE_POSIX_CPU_TIMERS_TASK_WORK
enabled.

In that case and on RT kernels the existing task could live lock when
preempting the task which does the timer delivery.

Replace spin_unlock() with an invocation of timer_wait_running() to handle
it the same way as the other retry loops in the posix timer code.

Fixes: ec8f954a40da ("posix-timers: Use a callback for cancel synchronization on PREEMPT_RT")
Signed-off-by: Thomas Gleixner <tglx@linutronix.de>
Reviewed-by: Frederic Weisbecker <frederic@kernel.org>
Link: https://lore.kernel.org/r/87v8g7c50d.ffs@tglx
---
 kernel/time/posix-timers.c | 43 ++++++++++++++++++++++++++++++-------
 1 file changed, 35 insertions(+), 8 deletions(-)

diff --git a/kernel/time/posix-timers.c b/kernel/time/posix-timers.c
index 808a247..ed3c4a9 100644
--- a/kernel/time/posix-timers.c
+++ b/kernel/time/posix-timers.c
@@ -1037,27 +1037,52 @@ retry_delete:
 }
 
 /*
- * return timer owned by the process, used by exit_itimers
+ * Delete a timer if it is armed, remove it from the hash and schedule it
+ * for RCU freeing.
  */
 static void itimer_delete(struct k_itimer *timer)
 {
-retry_delete:
-	spin_lock_irq(&timer->it_lock);
+	unsigned long flags;
+
+	/*
+	 * irqsave is required to make timer_wait_running() work.
+	 */
+	spin_lock_irqsave(&timer->it_lock, flags);
 
+retry_delete:
+	/*
+	 * Even if the timer is not longer accessible from other tasks
+	 * it still might be armed and queued in the underlying timer
+	 * mechanism. Worse, that timer mechanism might run the expiry
+	 * function concurrently.
+	 */
 	if (timer_delete_hook(timer) == TIMER_RETRY) {
-		spin_unlock_irq(&timer->it_lock);
+		/*
+		 * Timer is expired concurrently, prevent livelocks
+		 * and pointless spinning on RT.
+		 *
+		 * timer_wait_running() drops timer::it_lock, which opens
+		 * the possibility for another task to delete the timer.
+		 *
+		 * That's not possible here because this is invoked from
+		 * do_exit() only for the last thread of the thread group.
+		 * So no other task can access and delete that timer.
+		 */
+		if (WARN_ON_ONCE(timer_wait_running(timer, &flags) != timer))
+			return;
+
 		goto retry_delete;
 	}
 	list_del(&timer->list);
 
-	spin_unlock_irq(&timer->it_lock);
+	spin_unlock_irqrestore(&timer->it_lock, flags);
 	release_posix_timer(timer, IT_ID_SET);
 }
 
 /*
- * This is called by do_exit or de_thread, only when nobody else can
- * modify the signal->posix_timers list. Yet we need sighand->siglock
- * to prevent the race with /proc/pid/timers.
+ * Invoked from do_exit() when the last thread of a thread group exits.
+ * At that point no other task can access the timers of the dying
+ * task anymore.
  */
 void exit_itimers(struct task_struct *tsk)
 {
@@ -1067,10 +1092,12 @@ void exit_itimers(struct task_struct *tsk)
 	if (list_empty(&tsk->signal->posix_timers))
 		return;
 
+	/* Protect against concurrent read via /proc/$PID/timers */
 	spin_lock_irq(&tsk->sighand->siglock);
 	list_replace_init(&tsk->signal->posix_timers, &timers);
 	spin_unlock_irq(&tsk->sighand->siglock);
 
+	/* The timers are not longer accessible via tsk::signal */
 	while (!list_empty(&timers)) {
 		tmr = list_first_entry(&timers, struct k_itimer, list);
 		itimer_delete(tmr);

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

end of thread, other threads:[~2023-06-18 20:51 UTC | newest]

Thread overview: 122+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2023-04-25 18:48 [patch 00/20] posix-timers: Fixes and cleanups Thomas Gleixner
2023-04-25 18:48 ` [patch 01/20] posix-timers: Prevent RT livelock in itimer_delete() Thomas Gleixner
2023-05-04 17:06   ` Frederic Weisbecker
2023-05-04 18:20     ` Thomas Gleixner
2023-05-05  7:57       ` Thomas Gleixner
2023-06-01 19:00         ` [patch v2 " Thomas Gleixner
2023-06-01 20:16           ` [patch v2a " Thomas Gleixner
2023-06-05 10:59             ` Frederic Weisbecker
2023-06-05 15:08             ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50             ` tip-bot2 for Thomas Gleixner
2023-04-25 18:48 ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Thomas Gleixner
2023-05-05 14:50   ` Frederic Weisbecker
2023-05-05 22:58     ` Thomas Gleixner
2023-05-05 23:36       ` Thomas Gleixner
2023-05-08 21:57         ` Thomas Gleixner
2023-05-09  9:30           ` Thomas Gleixner
2023-05-09 12:50             ` Thomas Gleixner
2023-05-09 21:42               ` [RFD] posix-timers: CRIU woes Thomas Gleixner
2023-05-10  4:36                 ` Pavel Tikhomirov
2023-05-10  8:30                   ` Thomas Gleixner
2023-05-11  4:12                     ` Pavel Tikhomirov
2023-05-11  7:56                       ` Peter Zijlstra
2023-05-11  9:32                       ` Thomas Gleixner
2023-05-11 10:13                   ` David Laight
2023-05-10  8:16                 ` Andrey Vagin
2023-05-11  3:17                   ` Pavel Tikhomirov
2023-05-11  9:36                     ` Thomas Gleixner
2023-05-11  9:52                       ` Pavel Tikhomirov
2023-05-11 13:42                         ` Thomas Gleixner
2023-05-11 14:54                           ` Pavel Tikhomirov
2023-05-11 15:25                           ` Pavel Tikhomirov
2023-05-12  1:21                       ` Andrey Vagin
2023-05-31 17:38                         ` Thomas Gleixner
2023-05-11  7:49                   ` Cyrill Gorcunov
2023-05-10  0:42               ` [patch 02/20] posix-timers: Ensure timer ID search-loop limit is valid Andrey Vagin
2023-05-09  9:42         ` Frederic Weisbecker
2023-05-09 12:04           ` Thomas Gleixner
2023-05-09 12:38           ` Thomas Gleixner
2023-05-09 14:18             ` Frederic Weisbecker
2023-06-01 18:58               ` [patch v2 " Thomas Gleixner
2023-06-05 14:17                 ` Frederic Weisbecker
2023-06-05 15:08                 ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50                 ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 03/20] posix-timers: Clarify timer_wait_running() comment Thomas Gleixner
2023-05-09  9:50   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 04/20] posix-timers: Cleanup comments about timer ID tracking Thomas Gleixner
2023-05-09  9:58   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 05/20] posix-timers: Add comments about timer lookup Thomas Gleixner
2023-05-09 10:58   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 06/20] posix-timers: Annotate concurrent access to k_itimer::it_signal Thomas Gleixner
2023-05-09 11:04   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] posix-timers: Annotate concurrent access to k_itimer:: It_signal tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 07/20] posix-timers: Set k_itimer::it_signal to NULL on exit() Thomas Gleixner
2023-06-01 10:09   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] posix-timers: Set k_itimer:: It_signal " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 08/20] posix-timers: Remove pointless irqsafe from hash_lock Thomas Gleixner
2023-06-01 10:12   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 09/20] posix-timers: Split release_posix_timers() Thomas Gleixner
2023-06-01 10:25   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 10/20] posix-timers: Document sys_clock_getres() correctly Thomas Gleixner
2023-06-01 10:44   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 11/20] posix-timers: Document common_clock_get() correctly Thomas Gleixner
2023-06-01 11:00   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 12/20] posix-timers: Document sys_clock_getoverrun() Thomas Gleixner
2023-06-01 11:06   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 13/20] posix-timers: Document sys_clock_settime() permissions in place Thomas Gleixner
2023-06-01 11:22   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:50   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 14/20] posix-timers: Document nanosleep() details Thomas Gleixner
2023-06-01 12:30   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 15/20] posix-timers: Add proper comments in do_timer_create() Thomas Gleixner
2023-06-01 12:43   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 16/20] posix-timers: Comment SIGEV_THREAD_ID properly Thomas Gleixner
2023-06-01 12:47   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 17/20] posix-timers: Clarify posix_timer_rearm() comment Thomas Gleixner
2023-06-01 12:52   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 18/20] posix-timers: Clarify posix_timer_fn() comments Thomas Gleixner
2023-06-01 13:21   ` Frederic Weisbecker
2023-06-01 18:43     ` Thomas Gleixner
2023-06-01 19:07     ` Thomas Gleixner
2023-06-05 14:26       ` Frederic Weisbecker
2023-06-05 15:08       ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-05 22:17       ` tip-bot2 for Thomas Gleixner
2023-06-18 20:49       ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 19/20] posix-timers: Remove pointless comments Thomas Gleixner
2023-06-01 13:48   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
2023-04-25 18:49 ` [patch 20/20] posix-timers: Polish coding style in a few places Thomas Gleixner
2023-06-01 13:50   ` Frederic Weisbecker
2023-06-05 15:08   ` [tip: timers/core] " tip-bot2 for Thomas Gleixner
2023-06-05 22:17   ` tip-bot2 for Thomas Gleixner
2023-06-18 20:49   ` tip-bot2 for Thomas Gleixner
2023-06-05 14:32 ` [patch 00/20] posix-timers: Fixes and cleanups Frederic Weisbecker

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.