linux-fsdevel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v3] eventpoll: Fix priority inversion problem
@ 2025-05-27  9:08 Nam Cao
  2025-05-30  5:08 ` Christian Brauner
                   ` (4 more replies)
  0 siblings, 5 replies; 38+ messages in thread
From: Nam Cao @ 2025-05-27  9:08 UTC (permalink / raw)
  To: Alexander Viro, Christian Brauner, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe
  Cc: Nam Cao, Frederic Weisbecker, Valentin Schneider

The ready event list of an epoll object is protected by read-write
semaphore:

  - The consumer (waiter) acquires the write lock and takes items.
  - the producer (waker) takes the read lock and adds items.

The point of this design is enabling epoll to scale well with large number
of producers, as multiple producers can hold the read lock at the same
time.

Unfortunately, this implementation may cause scheduling priority inversion
problem. Suppose the consumer has higher scheduling priority than the
producer. The consumer needs to acquire the write lock, but may be blocked
by the producer holding the read lock. Since read-write semaphore does not
support priority-boosting for the readers (even with CONFIG_PREEMPT_RT=y),
we have a case of priority inversion: a higher priority consumer is blocked
by a lower priority producer. This problem was reported in [1].

Furthermore, this could also cause stall problem, as described in [2].

To fix this problem, make the event list half-lockless:

  - The consumer acquires a mutex (ep->mtx) and takes items.
  - The producer locklessly adds items to the list.

Performance is not the main goal of this patch, but as the producer now can
add items without waiting for consumer to release the lock, performance
improvement is observed using the stress test from
https://github.com/rouming/test-tools/blob/master/stress-epoll.c. This is
the same test that justified using read-write semaphore in the past.

Testing using 12 x86_64 CPUs:

          Before     After        Diff
threads  events/ms  events/ms
      8       6932      19753    +185%
     16       7820      27923    +257%
     32       7648      35164    +360%
     64       9677      37780    +290%
    128      11166      38174    +242%

Testing using 1 riscv64 CPU (averaged over 10 runs, as the numbers are
noisy):

          Before     After        Diff
threads  events/ms  events/ms
      1         73        129     +77%
      2        151        216     +43%
      4        216        364     +69%
      8        234        382     +63%
     16        251        392     +56%

Reported-by: Frederic Weisbecker <frederic@kernel.org>
Closes: https://lore.kernel.org/linux-rt-users/20210825132754.GA895675@lothringen/ [1]
Reported-by: Valentin Schneider <vschneid@redhat.com>
Closes: https://lore.kernel.org/linux-rt-users/xhsmhttqvnall.mognet@vschneid.remote.csb/ [2]
Signed-off-by: Nam Cao <namcao@linutronix.de>
---
v3:
  - get rid of the "link_used" and "ready" flags. They are hard to
    understand and unnecessary
  - get rid of the obsolete lockdep_assert_irqs_enabled()
  - Add lockdep_assert_held(&ep->mtx)
  - rewrite some comments
v2:
  - rename link_locked -> link_used
  - replace xchg() with smp_store_release() when applicable
  - make sure llist_node is in clean state when not on a list
  - remove now-unused list_add_tail_lockless()
---
 fs/eventpoll.c | 458 +++++++++++++++----------------------------------
 1 file changed, 134 insertions(+), 324 deletions(-)

diff --git a/fs/eventpoll.c b/fs/eventpoll.c
index d4dbffdedd08e..a97a771a459c9 100644
--- a/fs/eventpoll.c
+++ b/fs/eventpoll.c
@@ -137,13 +137,7 @@ struct epitem {
 	};
 
 	/* List header used to link this structure to the eventpoll ready list */
-	struct list_head rdllink;
-
-	/*
-	 * Works together "struct eventpoll"->ovflist in keeping the
-	 * single linked chain of items.
-	 */
-	struct epitem *next;
+	struct llist_node rdllink;
 
 	/* The file descriptor information this item refers to */
 	struct epoll_filefd ffd;
@@ -191,22 +185,15 @@ struct eventpoll {
 	/* Wait queue used by file->poll() */
 	wait_queue_head_t poll_wait;
 
-	/* List of ready file descriptors */
-	struct list_head rdllist;
-
-	/* Lock which protects rdllist and ovflist */
-	rwlock_t lock;
+	/*
+	 * List of ready file descriptors. Adding to this list is lockless. Items can be removed
+	 * only with eventpoll::mtx
+	 */
+	struct llist_head rdllist;
 
 	/* RB tree root used to store monitored fd structs */
 	struct rb_root_cached rbr;
 
-	/*
-	 * This is a single linked list that chains all the "struct epitem" that
-	 * happened while transferring ready events to userspace w/out
-	 * holding ->lock.
-	 */
-	struct epitem *ovflist;
-
 	/* wakeup_source used when ep_send_events or __ep_eventpoll_poll is running */
 	struct wakeup_source *ws;
 
@@ -361,10 +348,14 @@ static inline int ep_cmp_ffd(struct epoll_filefd *p1,
 	        (p1->file < p2->file ? -1 : p1->fd - p2->fd));
 }
 
-/* Tells us if the item is currently linked */
-static inline int ep_is_linked(struct epitem *epi)
+/*
+ * Add the item to its container eventpoll's rdllist; do nothing if the item is already on rdllist.
+ */
+static void epitem_ready(struct epitem *epi)
 {
-	return !list_empty(&epi->rdllink);
+	if (&epi->rdllink == cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))
+		llist_add(&epi->rdllink, &epi->ep->rdllist);
+
 }
 
 static inline struct eppoll_entry *ep_pwq_from_wait(wait_queue_entry_t *p)
@@ -383,13 +374,26 @@ static inline struct epitem *ep_item_from_wait(wait_queue_entry_t *p)
  *
  * @ep: Pointer to the eventpoll context.
  *
- * Return: a value different than %zero if ready events are available,
- *          or %zero otherwise.
+ * Return: true if ready events might be available, false otherwise.
  */
-static inline int ep_events_available(struct eventpoll *ep)
+static inline bool ep_events_available(struct eventpoll *ep)
 {
-	return !list_empty_careful(&ep->rdllist) ||
-		READ_ONCE(ep->ovflist) != EP_UNACTIVE_PTR;
+	bool available;
+	int locked;
+
+	locked = mutex_trylock(&ep->mtx);
+	if (!locked) {
+		/*
+		 * The lock held and someone might have removed all items while inspecting it. The
+		 * llist_empty() check in this case is futile. Assume that something is enqueued and
+		 * let ep_try_send_events() figure it out.
+		 */
+		return true;
+	}
+
+	available = !llist_empty(&ep->rdllist);
+	mutex_unlock(&ep->mtx);
+	return available;
 }
 
 #ifdef CONFIG_NET_RX_BUSY_POLL
@@ -724,77 +728,6 @@ static inline void ep_pm_stay_awake_rcu(struct epitem *epi)
 	rcu_read_unlock();
 }
 
-
-/*
- * ep->mutex needs to be held because we could be hit by
- * eventpoll_release_file() and epoll_ctl().
- */
-static void ep_start_scan(struct eventpoll *ep, struct list_head *txlist)
-{
-	/*
-	 * Steal the ready list, and re-init the original one to the
-	 * empty list. Also, set ep->ovflist to NULL so that events
-	 * happening while looping w/out locks, are not lost. We cannot
-	 * have the poll callback to queue directly on ep->rdllist,
-	 * because we want the "sproc" callback to be able to do it
-	 * in a lockless way.
-	 */
-	lockdep_assert_irqs_enabled();
-	write_lock_irq(&ep->lock);
-	list_splice_init(&ep->rdllist, txlist);
-	WRITE_ONCE(ep->ovflist, NULL);
-	write_unlock_irq(&ep->lock);
-}
-
-static void ep_done_scan(struct eventpoll *ep,
-			 struct list_head *txlist)
-{
-	struct epitem *epi, *nepi;
-
-	write_lock_irq(&ep->lock);
-	/*
-	 * During the time we spent inside the "sproc" callback, some
-	 * other events might have been queued by the poll callback.
-	 * We re-insert them inside the main ready-list here.
-	 */
-	for (nepi = READ_ONCE(ep->ovflist); (epi = nepi) != NULL;
-	     nepi = epi->next, epi->next = EP_UNACTIVE_PTR) {
-		/*
-		 * We need to check if the item is already in the list.
-		 * During the "sproc" callback execution time, items are
-		 * queued into ->ovflist but the "txlist" might already
-		 * contain them, and the list_splice() below takes care of them.
-		 */
-		if (!ep_is_linked(epi)) {
-			/*
-			 * ->ovflist is LIFO, so we have to reverse it in order
-			 * to keep in FIFO.
-			 */
-			list_add(&epi->rdllink, &ep->rdllist);
-			ep_pm_stay_awake(epi);
-		}
-	}
-	/*
-	 * We need to set back ep->ovflist to EP_UNACTIVE_PTR, so that after
-	 * releasing the lock, events will be queued in the normal way inside
-	 * ep->rdllist.
-	 */
-	WRITE_ONCE(ep->ovflist, EP_UNACTIVE_PTR);
-
-	/*
-	 * Quickly re-inject items left on "txlist".
-	 */
-	list_splice(txlist, &ep->rdllist);
-	__pm_relax(ep->ws);
-
-	if (!list_empty(&ep->rdllist)) {
-		if (waitqueue_active(&ep->wq))
-			wake_up(&ep->wq);
-	}
-
-	write_unlock_irq(&ep->lock);
-}
-
 static void ep_get(struct eventpoll *ep)
 {
 	refcount_inc(&ep->refcount);
@@ -832,10 +765,12 @@ static void ep_free(struct eventpoll *ep)
 static bool __ep_remove(struct eventpoll *ep, struct epitem *epi, bool force)
 {
 	struct file *file = epi->ffd.file;
+	struct llist_node *put_back_last;
 	struct epitems_head *to_free;
 	struct hlist_head *head;
+	LLIST_HEAD(put_back);
 
-	lockdep_assert_irqs_enabled();
+	lockdep_assert_held(&ep->mtx);
 
 	/*
 	 * Removes poll wait queue hooks.
@@ -867,10 +802,20 @@ static bool __ep_remove(struct eventpoll *ep, struct epitem *epi, bool force)
 
 	rb_erase_cached(&epi->rbn, &ep->rbr);
 
-	write_lock_irq(&ep->lock);
-	if (ep_is_linked(epi))
-		list_del_init(&epi->rdllink);
-	write_unlock_irq(&ep->lock);
+	if (llist_on_list(&epi->rdllink)) {
+		put_back_last = NULL;
+		while (true) {
+			struct llist_node *n = llist_del_first(&ep->rdllist);
+
+			if (&epi->rdllink == n || WARN_ON(!n))
+				break;
+			if (!put_back_last)
+				put_back_last = n;
+			__llist_add(n, &put_back);
+		}
+		if (put_back_last)
+			llist_add_batch(put_back.first, put_back_last, &ep->rdllist);
+	}
 
 	wakeup_source_unregister(ep_wakeup_source(epi));
 	/*
@@ -974,8 +919,9 @@ static __poll_t ep_item_poll(const struct epitem *epi, poll_table *pt, int depth
 static __poll_t __ep_eventpoll_poll(struct file *file, poll_table *wait, int depth)
 {
 	struct eventpoll *ep = file->private_data;
-	LIST_HEAD(txlist);
-	struct epitem *epi, *tmp;
+	struct wakeup_source *ws;
+	struct llist_node *n;
+	struct epitem *epi;
 	poll_table pt;
 	__poll_t res = 0;
 
@@ -989,22 +935,39 @@ static __poll_t __ep_eventpoll_poll(struct file *file, poll_table *wait, int dep
 	 * the ready list.
 	 */
 	mutex_lock_nested(&ep->mtx, depth);
-	ep_start_scan(ep, &txlist);
-	list_for_each_entry_safe(epi, tmp, &txlist, rdllink) {
+	while (true) {
+		n = llist_del_first_init(&ep->rdllist);
+		if (!n)
+			break;
+
+		epi = llist_entry(n, struct epitem, rdllink);
+
 		if (ep_item_poll(epi, &pt, depth + 1)) {
 			res = EPOLLIN | EPOLLRDNORM;
+			epitem_ready(epi);
 			break;
 		} else {
 			/*
-			 * Item has been dropped into the ready list by the poll
-			 * callback, but it's not actually ready, as far as
-			 * caller requested events goes. We can remove it here.
+			 * We need to activate ep before deactivating epi, to prevent autosuspend
+			 * just in case epi becomes active after ep_item_poll() above.
+			 *
+			 * This is similar to ep_send_events().
 			 */
+			ws = ep_wakeup_source(epi);
+			if (ws) {
+				if (ws->active)
+					__pm_stay_awake(ep->ws);
+				__pm_relax(ws);
+			}
 			__pm_relax(ep_wakeup_source(epi));
-			list_del_init(&epi->rdllink);
+
+			/* Just in case epi becomes active right before __pm_relax() */
+			if (unlikely(ep_item_poll(epi, &pt, depth + 1)))
+				ep_pm_stay_awake(epi);
+
+			__pm_relax(ep->ws);
 		}
 	}
-	ep_done_scan(ep, &txlist);
 	mutex_unlock(&ep->mtx);
 	return res;
 }
@@ -1153,12 +1116,10 @@ static int ep_alloc(struct eventpoll **pep)
 		return -ENOMEM;
 
 	mutex_init(&ep->mtx);
-	rwlock_init(&ep->lock);
 	init_waitqueue_head(&ep->wq);
 	init_waitqueue_head(&ep->poll_wait);
-	INIT_LIST_HEAD(&ep->rdllist);
+	init_llist_head(&ep->rdllist);
 	ep->rbr = RB_ROOT_CACHED;
-	ep->ovflist = EP_UNACTIVE_PTR;
 	ep->user = get_current_user();
 	refcount_set(&ep->refcount, 1);
 
@@ -1240,94 +1201,11 @@ struct file *get_epoll_tfile_raw_ptr(struct file *file, int tfd,
 }
 #endif /* CONFIG_KCMP */
 
-/*
- * Adds a new entry to the tail of the list in a lockless way, i.e.
- * multiple CPUs are allowed to call this function concurrently.
- *
- * Beware: it is necessary to prevent any other modifications of the
- *         existing list until all changes are completed, in other words
- *         concurrent list_add_tail_lockless() calls should be protected
- *         with a read lock, where write lock acts as a barrier which
- *         makes sure all list_add_tail_lockless() calls are fully
- *         completed.
- *
- *        Also an element can be locklessly added to the list only in one
- *        direction i.e. either to the tail or to the head, otherwise
- *        concurrent access will corrupt the list.
- *
- * Return: %false if element has been already added to the list, %true
- * otherwise.
- */
-static inline bool list_add_tail_lockless(struct list_head *new,
-					  struct list_head *head)
-{
-	struct list_head *prev;
-
-	/*
-	 * This is simple 'new->next = head' operation, but cmpxchg()
-	 * is used in order to detect that same element has been just
-	 * added to the list from another CPU: the winner observes
-	 * new->next == new.
-	 */
-	if (!try_cmpxchg(&new->next, &new, head))
-		return false;
-
-	/*
-	 * Initially ->next of a new element must be updated with the head
-	 * (we are inserting to the tail) and only then pointers are atomically
-	 * exchanged.  XCHG guarantees memory ordering, thus ->next should be
-	 * updated before pointers are actually swapped and pointers are
-	 * swapped before prev->next is updated.
-	 */
-
-	prev = xchg(&head->prev, new);
-
-	/*
-	 * It is safe to modify prev->next and new->prev, because a new element
-	 * is added only to the tail and new->next is updated before XCHG.
-	 */
-
-	prev->next = new;
-	new->prev = prev;
-
-	return true;
-}
-
-/*
- * Chains a new epi entry to the tail of the ep->ovflist in a lockless way,
- * i.e. multiple CPUs are allowed to call this function concurrently.
- *
- * Return: %false if epi element has been already chained, %true otherwise.
- */
-static inline bool chain_epi_lockless(struct epitem *epi)
-{
-	struct eventpoll *ep = epi->ep;
-
-	/* Fast preliminary check */
-	if (epi->next != EP_UNACTIVE_PTR)
-		return false;
-
-	/* Check that the same epi has not been just chained from another CPU */
-	if (cmpxchg(&epi->next, EP_UNACTIVE_PTR, NULL) != EP_UNACTIVE_PTR)
-		return false;
-
-	/* Atomically exchange tail */
-	epi->next = xchg(&ep->ovflist, epi);
-
-	return true;
-}
-
 /*
  * This is the callback that is passed to the wait queue wakeup
  * mechanism. It is called by the stored file descriptors when they
  * have events to report.
  *
- * This callback takes a read lock in order not to contend with concurrent
- * events from another file descriptor, thus all modifications to ->rdllist
- * or ->ovflist are lockless.  Read lock is paired with the write lock from
- * ep_start/done_scan(), which stops all list modifications and guarantees
- * that lists state is seen correctly.
- *
  * Another thing worth to mention is that ep_poll_callback() can be called
  * concurrently for the same @epi from different CPUs if poll table was inited
  * with several wait queues entries.  Plural wakeup from different CPUs of a
@@ -1337,15 +1215,11 @@ static inline bool chain_epi_lockless(struct epitem *epi)
  */
 static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, void *key)
 {
-	int pwake = 0;
 	struct epitem *epi = ep_item_from_wait(wait);
 	struct eventpoll *ep = epi->ep;
 	__poll_t pollflags = key_to_poll(key);
-	unsigned long flags;
 	int ewake = 0;
 
-	read_lock_irqsave(&ep->lock, flags);
-
 	ep_set_busy_poll_napi_id(epi);
 
 	/*
@@ -1355,7 +1229,7 @@ static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, v
 	 * until the next EPOLL_CTL_MOD will be issued.
 	 */
 	if (!(epi->event.events & ~EP_PRIVATE_BITS))
-		goto out_unlock;
+		goto out;
 
 	/*
 	 * Check the events coming with the callback. At this stage, not
@@ -1364,22 +1238,10 @@ static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, v
 	 * test for "key" != NULL before the event match test.
 	 */
 	if (pollflags && !(pollflags & epi->event.events))
-		goto out_unlock;
+		goto out;
 
-	/*
-	 * If we are transferring events to userspace, we can hold no locks
-	 * (because we're accessing user memory, and because of linux f_op->poll()
-	 * semantics). All the events that happen during that period of time are
-	 * chained in ep->ovflist and requeued later on.
-	 */
-	if (READ_ONCE(ep->ovflist) != EP_UNACTIVE_PTR) {
-		if (chain_epi_lockless(epi))
-			ep_pm_stay_awake_rcu(epi);
-	} else if (!ep_is_linked(epi)) {
-		/* In the usual case, add event to ready list. */
-		if (list_add_tail_lockless(&epi->rdllink, &ep->rdllist))
-			ep_pm_stay_awake_rcu(epi);
-	}
+	ep_pm_stay_awake_rcu(epi);
+	epitem_ready(epi);
 
 	/*
 	 * Wake up ( if active ) both the eventpoll wait list and the ->poll()
@@ -1408,15 +1270,9 @@ static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, v
 			wake_up(&ep->wq);
 	}
 	if (waitqueue_active(&ep->poll_wait))
-		pwake++;
-
-out_unlock:
-	read_unlock_irqrestore(&ep->lock, flags);
-
-	/* We have to call this outside the lock */
-	if (pwake)
 		ep_poll_safewake(ep, epi, pollflags & EPOLL_URING_WAKE);
 
+out:
 	if (!(epi->event.events & EPOLLEXCLUSIVE))
 		ewake = 1;
 
@@ -1661,8 +1517,6 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
 	if (is_file_epoll(tfile))
 		tep = tfile->private_data;
 
-	lockdep_assert_irqs_enabled();
-
 	if (unlikely(percpu_counter_compare(&ep->user->epoll_watches,
 					    max_user_watches) >= 0))
 		return -ENOSPC;
@@ -1674,11 +1528,10 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
 	}
 
 	/* Item initialization follow here ... */
-	INIT_LIST_HEAD(&epi->rdllink);
+	init_llist_node(&epi->rdllink);
 	epi->ep = ep;
 	ep_set_ffd(&epi->ffd, tfile, fd);
 	epi->event = *event;
-	epi->next = EP_UNACTIVE_PTR;
 
 	if (tep)
 		mutex_lock_nested(&tep->mtx, 1);
@@ -1745,16 +1598,13 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
 		return -ENOMEM;
 	}
 
-	/* We have to drop the new item inside our item list to keep track of it */
-	write_lock_irq(&ep->lock);
-
 	/* record NAPI ID of new item if present */
 	ep_set_busy_poll_napi_id(epi);
 
 	/* If the file is already "ready" we drop it inside the ready list */
-	if (revents && !ep_is_linked(epi)) {
-		list_add_tail(&epi->rdllink, &ep->rdllist);
+	if (revents) {
 		ep_pm_stay_awake(epi);
+		epitem_ready(epi);
 
 		/* Notify waiting tasks that events are available */
 		if (waitqueue_active(&ep->wq))
@@ -1763,8 +1613,6 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
 			pwake++;
 	}
 
-	write_unlock_irq(&ep->lock);
-
 	/* We have to call this outside the lock */
 	if (pwake)
 		ep_poll_safewake(ep, NULL, 0);
@@ -1779,11 +1627,8 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
 static int ep_modify(struct eventpoll *ep, struct epitem *epi,
 		     const struct epoll_event *event)
 {
-	int pwake = 0;
 	poll_table pt;
 
-	lockdep_assert_irqs_enabled();
-
 	init_poll_funcptr(&pt, NULL);
 
 	/*
@@ -1827,24 +1672,16 @@ static int ep_modify(struct eventpoll *ep, struct epitem *epi,
 	 * list, push it inside.
 	 */
 	if (ep_item_poll(epi, &pt, 1)) {
-		write_lock_irq(&ep->lock);
-		if (!ep_is_linked(epi)) {
-			list_add_tail(&epi->rdllink, &ep->rdllist);
-			ep_pm_stay_awake(epi);
+		ep_pm_stay_awake(epi);
+		epitem_ready(epi);
 
-			/* Notify waiting tasks that events are available */
-			if (waitqueue_active(&ep->wq))
-				wake_up(&ep->wq);
-			if (waitqueue_active(&ep->poll_wait))
-				pwake++;
-		}
-		write_unlock_irq(&ep->lock);
+		/* Notify waiting tasks that events are available */
+		if (waitqueue_active(&ep->wq))
+			wake_up(&ep->wq);
+		if (waitqueue_active(&ep->poll_wait))
+			ep_poll_safewake(ep, NULL, 0);
 	}
 
-	/* We have to call this outside the lock */
-	if (pwake)
-		ep_poll_safewake(ep, NULL, 0);
-
 	return 0;
 }
 
@@ -1852,7 +1689,7 @@ static int ep_send_events(struct eventpoll *ep,
 			  struct epoll_event __user *events, int maxevents)
 {
 	struct epitem *epi, *tmp;
-	LIST_HEAD(txlist);
+	LLIST_HEAD(txlist);
 	poll_table pt;
 	int res = 0;
 
@@ -1867,19 +1704,18 @@ static int ep_send_events(struct eventpoll *ep,
 	init_poll_funcptr(&pt, NULL);
 
 	mutex_lock(&ep->mtx);
-	ep_start_scan(ep, &txlist);
 
-	/*
-	 * We can loop without lock because we are passed a task private list.
-	 * Items cannot vanish during the loop we are holding ep->mtx.
-	 */
-	list_for_each_entry_safe(epi, tmp, &txlist, rdllink) {
+	while (res < maxevents) {
 		struct wakeup_source *ws;
+		struct llist_node *n;
 		__poll_t revents;
 
-		if (res >= maxevents)
+		n = llist_del_first(&ep->rdllist);
+		if (!n)
 			break;
 
+		epi = llist_entry(n, struct epitem, rdllink);
+
 		/*
 		 * Activate ep->ws before deactivating epi->ws to prevent
 		 * triggering auto-suspend here (in case we reactive epi->ws
@@ -1896,21 +1732,30 @@ static int ep_send_events(struct eventpoll *ep,
 			__pm_relax(ws);
 		}
 
-		list_del_init(&epi->rdllink);
-
 		/*
 		 * If the event mask intersect the caller-requested one,
 		 * deliver the event to userspace. Again, we are holding ep->mtx,
 		 * so no operations coming from userspace can change the item.
 		 */
 		revents = ep_item_poll(epi, &pt, 1);
-		if (!revents)
+		if (!revents) {
+			init_llist_node(n);
+
+			/*
+			 * Just in case epi becomes ready after ep_item_poll() above, but before
+			 * init_llist_node(). Make sure to add it to the ready list, otherwise an
+			 * event may be lost.
+			 */
+			if (unlikely(ep_item_poll(epi, &pt, 1))) {
+				ep_pm_stay_awake(epi);
+				epitem_ready(epi);
+			}
 			continue;
+		}
 
 		events = epoll_put_uevent(revents, epi->event.data, events);
 		if (!events) {
-			list_add(&epi->rdllink, &txlist);
-			ep_pm_stay_awake(epi);
+			llist_add(&epi->rdllink, &ep->rdllist);
 			if (!res)
 				res = -EFAULT;
 			break;
@@ -1918,25 +1763,31 @@ static int ep_send_events(struct eventpoll *ep,
 		res++;
 		if (epi->event.events & EPOLLONESHOT)
 			epi->event.events &= EP_PRIVATE_BITS;
-		else if (!(epi->event.events & EPOLLET)) {
+		__llist_add(n, &txlist);
+	}
+
+	llist_for_each_entry_safe(epi, tmp, txlist.first, rdllink) {
+		init_llist_node(&epi->rdllink);
+
+		if (!(epi->event.events & EPOLLET)) {
 			/*
-			 * If this file has been added with Level
-			 * Trigger mode, we need to insert back inside
-			 * the ready list, so that the next call to
-			 * epoll_wait() will check again the events
-			 * availability. At this point, no one can insert
-			 * into ep->rdllist besides us. The epoll_ctl()
-			 * callers are locked out by
-			 * ep_send_events() holding "mtx" and the
-			 * poll callback will queue them in ep->ovflist.
+			 * If this file has been added with Level Trigger mode, we need to insert
+			 * back inside the ready list, so that the next call to epoll_wait() will
+			 * check again the events availability.
 			 */
-			list_add_tail(&epi->rdllink, &ep->rdllist);
 			ep_pm_stay_awake(epi);
+			epitem_ready(epi);
 		}
 	}
-	ep_done_scan(ep, &txlist);
+
+	__pm_relax(ep->ws);
 	mutex_unlock(&ep->mtx);
 
+	if (!llist_empty(&ep->rdllist)) {
+		if (waitqueue_active(&ep->wq))
+			wake_up(&ep->wq);
+	}
+
 	return res;
 }
 
@@ -2029,8 +1880,6 @@ static int ep_poll(struct eventpoll *ep, struct epoll_event __user *events,
 	wait_queue_entry_t wait;
 	ktime_t expires, *to = NULL;
 
-	lockdep_assert_irqs_enabled();
-
 	if (timeout && (timeout->tv_sec | timeout->tv_nsec)) {
 		slack = select_estimate_accuracy(timeout);
 		to = &expires;
@@ -2090,54 +1939,15 @@ static int ep_poll(struct eventpoll *ep, struct epoll_event __user *events,
 		init_wait(&wait);
 		wait.func = ep_autoremove_wake_function;
 
-		write_lock_irq(&ep->lock);
-		/*
-		 * Barrierless variant, waitqueue_active() is called under
-		 * the same lock on wakeup ep_poll_callback() side, so it
-		 * is safe to avoid an explicit barrier.
-		 */
-		__set_current_state(TASK_INTERRUPTIBLE);
+		prepare_to_wait_exclusive(&ep->wq, &wait, TASK_INTERRUPTIBLE);
 
-		/*
-		 * Do the final check under the lock. ep_start/done_scan()
-		 * plays with two lists (->rdllist and ->ovflist) and there
-		 * is always a race when both lists are empty for short
-		 * period of time although events are pending, so lock is
-		 * important.
-		 */
-		eavail = ep_events_available(ep);
-		if (!eavail)
-			__add_wait_queue_exclusive(&ep->wq, &wait);
-
-		write_unlock_irq(&ep->lock);
-
-		if (!eavail)
+		if (!ep_events_available(ep))
 			timed_out = !ep_schedule_timeout(to) ||
 				!schedule_hrtimeout_range(to, slack,
 							  HRTIMER_MODE_ABS);
-		__set_current_state(TASK_RUNNING);
-
-		/*
-		 * We were woken up, thus go and try to harvest some events.
-		 * If timed out and still on the wait queue, recheck eavail
-		 * carefully under lock, below.
-		 */
-		eavail = 1;
 
-		if (!list_empty_careful(&wait.entry)) {
-			write_lock_irq(&ep->lock);
-			/*
-			 * If the thread timed out and is not on the wait queue,
-			 * it means that the thread was woken up after its
-			 * timeout expired before it could reacquire the lock.
-			 * Thus, when wait.entry is empty, it needs to harvest
-			 * events.
-			 */
-			if (timed_out)
-				eavail = list_empty(&wait.entry);
-			__remove_wait_queue(&ep->wq, &wait);
-			write_unlock_irq(&ep->lock);
-		}
+		finish_wait(&ep->wq, &wait);
+		eavail = ep_events_available(ep);
 	}
 }
 
-- 
2.39.5


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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-05-27  9:08 [PATCH v3] eventpoll: Fix priority inversion problem Nam Cao
@ 2025-05-30  5:08 ` Christian Brauner
  2025-06-25 15:35   ` Sebastian Andrzej Siewior
  2025-06-25 14:50 ` Sebastian Andrzej Siewior
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 38+ messages in thread
From: Christian Brauner @ 2025-05-30  5:08 UTC (permalink / raw)
  To: Nam Cao, Frederic Weisbecker
  Cc: Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Valentin Schneider

On Tue, May 27, 2025 at 11:08:36AM +0200, Nam Cao wrote:
> The ready event list of an epoll object is protected by read-write
> semaphore:
> 
>   - The consumer (waiter) acquires the write lock and takes items.
>   - the producer (waker) takes the read lock and adds items.
> 
> The point of this design is enabling epoll to scale well with large number
> of producers, as multiple producers can hold the read lock at the same
> time.
> 
> Unfortunately, this implementation may cause scheduling priority inversion
> problem. Suppose the consumer has higher scheduling priority than the
> producer. The consumer needs to acquire the write lock, but may be blocked
> by the producer holding the read lock. Since read-write semaphore does not
> support priority-boosting for the readers (even with CONFIG_PREEMPT_RT=y),
> we have a case of priority inversion: a higher priority consumer is blocked
> by a lower priority producer. This problem was reported in [1].
> 
> Furthermore, this could also cause stall problem, as described in [2].
> 
> To fix this problem, make the event list half-lockless:
> 
>   - The consumer acquires a mutex (ep->mtx) and takes items.
>   - The producer locklessly adds items to the list.
> 
> Performance is not the main goal of this patch, but as the producer now can
> add items without waiting for consumer to release the lock, performance
> improvement is observed using the stress test from
> https://github.com/rouming/test-tools/blob/master/stress-epoll.c. This is
> the same test that justified using read-write semaphore in the past.
> 
> Testing using 12 x86_64 CPUs:
> 
>           Before     After        Diff
> threads  events/ms  events/ms
>       8       6932      19753    +185%
>      16       7820      27923    +257%
>      32       7648      35164    +360%
>      64       9677      37780    +290%
>     128      11166      38174    +242%
> 
> Testing using 1 riscv64 CPU (averaged over 10 runs, as the numbers are
> noisy):
> 
>           Before     After        Diff
> threads  events/ms  events/ms
>       1         73        129     +77%
>       2        151        216     +43%
>       4        216        364     +69%
>       8        234        382     +63%
>      16        251        392     +56%
> 
> Reported-by: Frederic Weisbecker <frederic@kernel.org>
> Closes: https://lore.kernel.org/linux-rt-users/20210825132754.GA895675@lothringen/ [1]
> Reported-by: Valentin Schneider <vschneid@redhat.com>
> Closes: https://lore.kernel.org/linux-rt-users/xhsmhttqvnall.mognet@vschneid.remote.csb/ [2]
> Signed-off-by: Nam Cao <namcao@linutronix.de>
> ---
> v3:
>   - get rid of the "link_used" and "ready" flags. They are hard to
>     understand and unnecessary
>   - get rid of the obsolete lockdep_assert_irqs_enabled()
>   - Add lockdep_assert_held(&ep->mtx)
>   - rewrite some comments
> v2:
>   - rename link_locked -> link_used
>   - replace xchg() with smp_store_release() when applicable
>   - make sure llist_node is in clean state when not on a list
>   - remove now-unused list_add_tail_lockless()
> ---

Care to review this, Frederic?

>  fs/eventpoll.c | 458 +++++++++++++++----------------------------------
>  1 file changed, 134 insertions(+), 324 deletions(-)
> 
> diff --git a/fs/eventpoll.c b/fs/eventpoll.c
> index d4dbffdedd08e..a97a771a459c9 100644
> --- a/fs/eventpoll.c
> +++ b/fs/eventpoll.c
> @@ -137,13 +137,7 @@ struct epitem {
>  	};
>  
>  	/* List header used to link this structure to the eventpoll ready list */
> -	struct list_head rdllink;
> -
> -	/*
> -	 * Works together "struct eventpoll"->ovflist in keeping the
> -	 * single linked chain of items.
> -	 */
> -	struct epitem *next;
> +	struct llist_node rdllink;
>  
>  	/* The file descriptor information this item refers to */
>  	struct epoll_filefd ffd;
> @@ -191,22 +185,15 @@ struct eventpoll {
>  	/* Wait queue used by file->poll() */
>  	wait_queue_head_t poll_wait;
>  
> -	/* List of ready file descriptors */
> -	struct list_head rdllist;
> -
> -	/* Lock which protects rdllist and ovflist */
> -	rwlock_t lock;
> +	/*
> +	 * List of ready file descriptors. Adding to this list is lockless. Items can be removed
> +	 * only with eventpoll::mtx
> +	 */
> +	struct llist_head rdllist;
>  
>  	/* RB tree root used to store monitored fd structs */
>  	struct rb_root_cached rbr;
>  
> -	/*
> -	 * This is a single linked list that chains all the "struct epitem" that
> -	 * happened while transferring ready events to userspace w/out
> -	 * holding ->lock.
> -	 */
> -	struct epitem *ovflist;
> -
>  	/* wakeup_source used when ep_send_events or __ep_eventpoll_poll is running */
>  	struct wakeup_source *ws;
>  
> @@ -361,10 +348,14 @@ static inline int ep_cmp_ffd(struct epoll_filefd *p1,
>  	        (p1->file < p2->file ? -1 : p1->fd - p2->fd));
>  }
>  
> -/* Tells us if the item is currently linked */
> -static inline int ep_is_linked(struct epitem *epi)
> +/*
> + * Add the item to its container eventpoll's rdllist; do nothing if the item is already on rdllist.
> + */
> +static void epitem_ready(struct epitem *epi)
>  {
> -	return !list_empty(&epi->rdllink);
> +	if (&epi->rdllink == cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))
> +		llist_add(&epi->rdllink, &epi->ep->rdllist);
> +
>  }
>  
>  static inline struct eppoll_entry *ep_pwq_from_wait(wait_queue_entry_t *p)
> @@ -383,13 +374,26 @@ static inline struct epitem *ep_item_from_wait(wait_queue_entry_t *p)
>   *
>   * @ep: Pointer to the eventpoll context.
>   *
> - * Return: a value different than %zero if ready events are available,
> - *          or %zero otherwise.
> + * Return: true if ready events might be available, false otherwise.
>   */
> -static inline int ep_events_available(struct eventpoll *ep)
> +static inline bool ep_events_available(struct eventpoll *ep)
>  {
> -	return !list_empty_careful(&ep->rdllist) ||
> -		READ_ONCE(ep->ovflist) != EP_UNACTIVE_PTR;
> +	bool available;
> +	int locked;
> +
> +	locked = mutex_trylock(&ep->mtx);
> +	if (!locked) {
> +		/*
> +		 * The lock held and someone might have removed all items while inspecting it. The
> +		 * llist_empty() check in this case is futile. Assume that something is enqueued and
> +		 * let ep_try_send_events() figure it out.
> +		 */
> +		return true;
> +	}
> +
> +	available = !llist_empty(&ep->rdllist);
> +	mutex_unlock(&ep->mtx);
> +	return available;
>  }
>  
>  #ifdef CONFIG_NET_RX_BUSY_POLL
> @@ -724,77 +728,6 @@ static inline void ep_pm_stay_awake_rcu(struct epitem *epi)
>  	rcu_read_unlock();
>  }
>  
> -
> -/*
> - * ep->mutex needs to be held because we could be hit by
> - * eventpoll_release_file() and epoll_ctl().
> - */
> -static void ep_start_scan(struct eventpoll *ep, struct list_head *txlist)
> -{
> -	/*
> -	 * Steal the ready list, and re-init the original one to the
> -	 * empty list. Also, set ep->ovflist to NULL so that events
> -	 * happening while looping w/out locks, are not lost. We cannot
> -	 * have the poll callback to queue directly on ep->rdllist,
> -	 * because we want the "sproc" callback to be able to do it
> -	 * in a lockless way.
> -	 */
> -	lockdep_assert_irqs_enabled();
> -	write_lock_irq(&ep->lock);
> -	list_splice_init(&ep->rdllist, txlist);
> -	WRITE_ONCE(ep->ovflist, NULL);
> -	write_unlock_irq(&ep->lock);
> -}
> -
> -static void ep_done_scan(struct eventpoll *ep,
> -			 struct list_head *txlist)
> -{
> -	struct epitem *epi, *nepi;
> -
> -	write_lock_irq(&ep->lock);
> -	/*
> -	 * During the time we spent inside the "sproc" callback, some
> -	 * other events might have been queued by the poll callback.
> -	 * We re-insert them inside the main ready-list here.
> -	 */
> -	for (nepi = READ_ONCE(ep->ovflist); (epi = nepi) != NULL;
> -	     nepi = epi->next, epi->next = EP_UNACTIVE_PTR) {
> -		/*
> -		 * We need to check if the item is already in the list.
> -		 * During the "sproc" callback execution time, items are
> -		 * queued into ->ovflist but the "txlist" might already
> -		 * contain them, and the list_splice() below takes care of them.
> -		 */
> -		if (!ep_is_linked(epi)) {
> -			/*
> -			 * ->ovflist is LIFO, so we have to reverse it in order
> -			 * to keep in FIFO.
> -			 */
> -			list_add(&epi->rdllink, &ep->rdllist);
> -			ep_pm_stay_awake(epi);
> -		}
> -	}
> -	/*
> -	 * We need to set back ep->ovflist to EP_UNACTIVE_PTR, so that after
> -	 * releasing the lock, events will be queued in the normal way inside
> -	 * ep->rdllist.
> -	 */
> -	WRITE_ONCE(ep->ovflist, EP_UNACTIVE_PTR);
> -
> -	/*
> -	 * Quickly re-inject items left on "txlist".
> -	 */
> -	list_splice(txlist, &ep->rdllist);
> -	__pm_relax(ep->ws);
> -
> -	if (!list_empty(&ep->rdllist)) {
> -		if (waitqueue_active(&ep->wq))
> -			wake_up(&ep->wq);
> -	}
> -
> -	write_unlock_irq(&ep->lock);
> -}
> -
>  static void ep_get(struct eventpoll *ep)
>  {
>  	refcount_inc(&ep->refcount);
> @@ -832,10 +765,12 @@ static void ep_free(struct eventpoll *ep)
>  static bool __ep_remove(struct eventpoll *ep, struct epitem *epi, bool force)
>  {
>  	struct file *file = epi->ffd.file;
> +	struct llist_node *put_back_last;
>  	struct epitems_head *to_free;
>  	struct hlist_head *head;
> +	LLIST_HEAD(put_back);
>  
> -	lockdep_assert_irqs_enabled();
> +	lockdep_assert_held(&ep->mtx);
>  
>  	/*
>  	 * Removes poll wait queue hooks.
> @@ -867,10 +802,20 @@ static bool __ep_remove(struct eventpoll *ep, struct epitem *epi, bool force)
>  
>  	rb_erase_cached(&epi->rbn, &ep->rbr);
>  
> -	write_lock_irq(&ep->lock);
> -	if (ep_is_linked(epi))
> -		list_del_init(&epi->rdllink);
> -	write_unlock_irq(&ep->lock);
> +	if (llist_on_list(&epi->rdllink)) {
> +		put_back_last = NULL;
> +		while (true) {
> +			struct llist_node *n = llist_del_first(&ep->rdllist);
> +
> +			if (&epi->rdllink == n || WARN_ON(!n))
> +				break;
> +			if (!put_back_last)
> +				put_back_last = n;
> +			__llist_add(n, &put_back);
> +		}
> +		if (put_back_last)
> +			llist_add_batch(put_back.first, put_back_last, &ep->rdllist);
> +	}
>  
>  	wakeup_source_unregister(ep_wakeup_source(epi));
>  	/*
> @@ -974,8 +919,9 @@ static __poll_t ep_item_poll(const struct epitem *epi, poll_table *pt, int depth
>  static __poll_t __ep_eventpoll_poll(struct file *file, poll_table *wait, int depth)
>  {
>  	struct eventpoll *ep = file->private_data;
> -	LIST_HEAD(txlist);
> -	struct epitem *epi, *tmp;
> +	struct wakeup_source *ws;
> +	struct llist_node *n;
> +	struct epitem *epi;
>  	poll_table pt;
>  	__poll_t res = 0;
>  
> @@ -989,22 +935,39 @@ static __poll_t __ep_eventpoll_poll(struct file *file, poll_table *wait, int dep
>  	 * the ready list.
>  	 */
>  	mutex_lock_nested(&ep->mtx, depth);
> -	ep_start_scan(ep, &txlist);
> -	list_for_each_entry_safe(epi, tmp, &txlist, rdllink) {
> +	while (true) {
> +		n = llist_del_first_init(&ep->rdllist);
> +		if (!n)
> +			break;
> +
> +		epi = llist_entry(n, struct epitem, rdllink);
> +
>  		if (ep_item_poll(epi, &pt, depth + 1)) {
>  			res = EPOLLIN | EPOLLRDNORM;
> +			epitem_ready(epi);
>  			break;
>  		} else {
>  			/*
> -			 * Item has been dropped into the ready list by the poll
> -			 * callback, but it's not actually ready, as far as
> -			 * caller requested events goes. We can remove it here.
> +			 * We need to activate ep before deactivating epi, to prevent autosuspend
> +			 * just in case epi becomes active after ep_item_poll() above.
> +			 *
> +			 * This is similar to ep_send_events().
>  			 */
> +			ws = ep_wakeup_source(epi);
> +			if (ws) {
> +				if (ws->active)
> +					__pm_stay_awake(ep->ws);
> +				__pm_relax(ws);
> +			}
>  			__pm_relax(ep_wakeup_source(epi));
> -			list_del_init(&epi->rdllink);
> +
> +			/* Just in case epi becomes active right before __pm_relax() */
> +			if (unlikely(ep_item_poll(epi, &pt, depth + 1)))
> +				ep_pm_stay_awake(epi);
> +
> +			__pm_relax(ep->ws);
>  		}
>  	}
> -	ep_done_scan(ep, &txlist);
>  	mutex_unlock(&ep->mtx);
>  	return res;
>  }
> @@ -1153,12 +1116,10 @@ static int ep_alloc(struct eventpoll **pep)
>  		return -ENOMEM;
>  
>  	mutex_init(&ep->mtx);
> -	rwlock_init(&ep->lock);
>  	init_waitqueue_head(&ep->wq);
>  	init_waitqueue_head(&ep->poll_wait);
> -	INIT_LIST_HEAD(&ep->rdllist);
> +	init_llist_head(&ep->rdllist);
>  	ep->rbr = RB_ROOT_CACHED;
> -	ep->ovflist = EP_UNACTIVE_PTR;
>  	ep->user = get_current_user();
>  	refcount_set(&ep->refcount, 1);
>  
> @@ -1240,94 +1201,11 @@ struct file *get_epoll_tfile_raw_ptr(struct file *file, int tfd,
>  }
>  #endif /* CONFIG_KCMP */
>  
> -/*
> - * Adds a new entry to the tail of the list in a lockless way, i.e.
> - * multiple CPUs are allowed to call this function concurrently.
> - *
> - * Beware: it is necessary to prevent any other modifications of the
> - *         existing list until all changes are completed, in other words
> - *         concurrent list_add_tail_lockless() calls should be protected
> - *         with a read lock, where write lock acts as a barrier which
> - *         makes sure all list_add_tail_lockless() calls are fully
> - *         completed.
> - *
> - *        Also an element can be locklessly added to the list only in one
> - *        direction i.e. either to the tail or to the head, otherwise
> - *        concurrent access will corrupt the list.
> - *
> - * Return: %false if element has been already added to the list, %true
> - * otherwise.
> - */
> -static inline bool list_add_tail_lockless(struct list_head *new,
> -					  struct list_head *head)
> -{
> -	struct list_head *prev;
> -
> -	/*
> -	 * This is simple 'new->next = head' operation, but cmpxchg()
> -	 * is used in order to detect that same element has been just
> -	 * added to the list from another CPU: the winner observes
> -	 * new->next == new.
> -	 */
> -	if (!try_cmpxchg(&new->next, &new, head))
> -		return false;
> -
> -	/*
> -	 * Initially ->next of a new element must be updated with the head
> -	 * (we are inserting to the tail) and only then pointers are atomically
> -	 * exchanged.  XCHG guarantees memory ordering, thus ->next should be
> -	 * updated before pointers are actually swapped and pointers are
> -	 * swapped before prev->next is updated.
> -	 */
> -
> -	prev = xchg(&head->prev, new);
> -
> -	/*
> -	 * It is safe to modify prev->next and new->prev, because a new element
> -	 * is added only to the tail and new->next is updated before XCHG.
> -	 */
> -
> -	prev->next = new;
> -	new->prev = prev;
> -
> -	return true;
> -}
> -
> -/*
> - * Chains a new epi entry to the tail of the ep->ovflist in a lockless way,
> - * i.e. multiple CPUs are allowed to call this function concurrently.
> - *
> - * Return: %false if epi element has been already chained, %true otherwise.
> - */
> -static inline bool chain_epi_lockless(struct epitem *epi)
> -{
> -	struct eventpoll *ep = epi->ep;
> -
> -	/* Fast preliminary check */
> -	if (epi->next != EP_UNACTIVE_PTR)
> -		return false;
> -
> -	/* Check that the same epi has not been just chained from another CPU */
> -	if (cmpxchg(&epi->next, EP_UNACTIVE_PTR, NULL) != EP_UNACTIVE_PTR)
> -		return false;
> -
> -	/* Atomically exchange tail */
> -	epi->next = xchg(&ep->ovflist, epi);
> -
> -	return true;
> -}
> -
>  /*
>   * This is the callback that is passed to the wait queue wakeup
>   * mechanism. It is called by the stored file descriptors when they
>   * have events to report.
>   *
> - * This callback takes a read lock in order not to contend with concurrent
> - * events from another file descriptor, thus all modifications to ->rdllist
> - * or ->ovflist are lockless.  Read lock is paired with the write lock from
> - * ep_start/done_scan(), which stops all list modifications and guarantees
> - * that lists state is seen correctly.
> - *
>   * Another thing worth to mention is that ep_poll_callback() can be called
>   * concurrently for the same @epi from different CPUs if poll table was inited
>   * with several wait queues entries.  Plural wakeup from different CPUs of a
> @@ -1337,15 +1215,11 @@ static inline bool chain_epi_lockless(struct epitem *epi)
>   */
>  static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, void *key)
>  {
> -	int pwake = 0;
>  	struct epitem *epi = ep_item_from_wait(wait);
>  	struct eventpoll *ep = epi->ep;
>  	__poll_t pollflags = key_to_poll(key);
> -	unsigned long flags;
>  	int ewake = 0;
>  
> -	read_lock_irqsave(&ep->lock, flags);
> -
>  	ep_set_busy_poll_napi_id(epi);
>  
>  	/*
> @@ -1355,7 +1229,7 @@ static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, v
>  	 * until the next EPOLL_CTL_MOD will be issued.
>  	 */
>  	if (!(epi->event.events & ~EP_PRIVATE_BITS))
> -		goto out_unlock;
> +		goto out;
>  
>  	/*
>  	 * Check the events coming with the callback. At this stage, not
> @@ -1364,22 +1238,10 @@ static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, v
>  	 * test for "key" != NULL before the event match test.
>  	 */
>  	if (pollflags && !(pollflags & epi->event.events))
> -		goto out_unlock;
> +		goto out;
>  
> -	/*
> -	 * If we are transferring events to userspace, we can hold no locks
> -	 * (because we're accessing user memory, and because of linux f_op->poll()
> -	 * semantics). All the events that happen during that period of time are
> -	 * chained in ep->ovflist and requeued later on.
> -	 */
> -	if (READ_ONCE(ep->ovflist) != EP_UNACTIVE_PTR) {
> -		if (chain_epi_lockless(epi))
> -			ep_pm_stay_awake_rcu(epi);
> -	} else if (!ep_is_linked(epi)) {
> -		/* In the usual case, add event to ready list. */
> -		if (list_add_tail_lockless(&epi->rdllink, &ep->rdllist))
> -			ep_pm_stay_awake_rcu(epi);
> -	}
> +	ep_pm_stay_awake_rcu(epi);
> +	epitem_ready(epi);
>  
>  	/*
>  	 * Wake up ( if active ) both the eventpoll wait list and the ->poll()
> @@ -1408,15 +1270,9 @@ static int ep_poll_callback(wait_queue_entry_t *wait, unsigned mode, int sync, v
>  			wake_up(&ep->wq);
>  	}
>  	if (waitqueue_active(&ep->poll_wait))
> -		pwake++;
> -
> -out_unlock:
> -	read_unlock_irqrestore(&ep->lock, flags);
> -
> -	/* We have to call this outside the lock */
> -	if (pwake)
>  		ep_poll_safewake(ep, epi, pollflags & EPOLL_URING_WAKE);
>  
> +out:
>  	if (!(epi->event.events & EPOLLEXCLUSIVE))
>  		ewake = 1;
>  
> @@ -1661,8 +1517,6 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
>  	if (is_file_epoll(tfile))
>  		tep = tfile->private_data;
>  
> -	lockdep_assert_irqs_enabled();
> -
>  	if (unlikely(percpu_counter_compare(&ep->user->epoll_watches,
>  					    max_user_watches) >= 0))
>  		return -ENOSPC;
> @@ -1674,11 +1528,10 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
>  	}
>  
>  	/* Item initialization follow here ... */
> -	INIT_LIST_HEAD(&epi->rdllink);
> +	init_llist_node(&epi->rdllink);
>  	epi->ep = ep;
>  	ep_set_ffd(&epi->ffd, tfile, fd);
>  	epi->event = *event;
> -	epi->next = EP_UNACTIVE_PTR;
>  
>  	if (tep)
>  		mutex_lock_nested(&tep->mtx, 1);
> @@ -1745,16 +1598,13 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
>  		return -ENOMEM;
>  	}
>  
> -	/* We have to drop the new item inside our item list to keep track of it */
> -	write_lock_irq(&ep->lock);
> -
>  	/* record NAPI ID of new item if present */
>  	ep_set_busy_poll_napi_id(epi);
>  
>  	/* If the file is already "ready" we drop it inside the ready list */
> -	if (revents && !ep_is_linked(epi)) {
> -		list_add_tail(&epi->rdllink, &ep->rdllist);
> +	if (revents) {
>  		ep_pm_stay_awake(epi);
> +		epitem_ready(epi);
>  
>  		/* Notify waiting tasks that events are available */
>  		if (waitqueue_active(&ep->wq))
> @@ -1763,8 +1613,6 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
>  			pwake++;
>  	}
>  
> -	write_unlock_irq(&ep->lock);
> -
>  	/* We have to call this outside the lock */
>  	if (pwake)
>  		ep_poll_safewake(ep, NULL, 0);
> @@ -1779,11 +1627,8 @@ static int ep_insert(struct eventpoll *ep, const struct epoll_event *event,
>  static int ep_modify(struct eventpoll *ep, struct epitem *epi,
>  		     const struct epoll_event *event)
>  {
> -	int pwake = 0;
>  	poll_table pt;
>  
> -	lockdep_assert_irqs_enabled();
> -
>  	init_poll_funcptr(&pt, NULL);
>  
>  	/*
> @@ -1827,24 +1672,16 @@ static int ep_modify(struct eventpoll *ep, struct epitem *epi,
>  	 * list, push it inside.
>  	 */
>  	if (ep_item_poll(epi, &pt, 1)) {
> -		write_lock_irq(&ep->lock);
> -		if (!ep_is_linked(epi)) {
> -			list_add_tail(&epi->rdllink, &ep->rdllist);
> -			ep_pm_stay_awake(epi);
> +		ep_pm_stay_awake(epi);
> +		epitem_ready(epi);
>  
> -			/* Notify waiting tasks that events are available */
> -			if (waitqueue_active(&ep->wq))
> -				wake_up(&ep->wq);
> -			if (waitqueue_active(&ep->poll_wait))
> -				pwake++;
> -		}
> -		write_unlock_irq(&ep->lock);
> +		/* Notify waiting tasks that events are available */
> +		if (waitqueue_active(&ep->wq))
> +			wake_up(&ep->wq);
> +		if (waitqueue_active(&ep->poll_wait))
> +			ep_poll_safewake(ep, NULL, 0);
>  	}
>  
> -	/* We have to call this outside the lock */
> -	if (pwake)
> -		ep_poll_safewake(ep, NULL, 0);
> -
>  	return 0;
>  }
>  
> @@ -1852,7 +1689,7 @@ static int ep_send_events(struct eventpoll *ep,
>  			  struct epoll_event __user *events, int maxevents)
>  {
>  	struct epitem *epi, *tmp;
> -	LIST_HEAD(txlist);
> +	LLIST_HEAD(txlist);
>  	poll_table pt;
>  	int res = 0;
>  
> @@ -1867,19 +1704,18 @@ static int ep_send_events(struct eventpoll *ep,
>  	init_poll_funcptr(&pt, NULL);
>  
>  	mutex_lock(&ep->mtx);
> -	ep_start_scan(ep, &txlist);
>  
> -	/*
> -	 * We can loop without lock because we are passed a task private list.
> -	 * Items cannot vanish during the loop we are holding ep->mtx.
> -	 */
> -	list_for_each_entry_safe(epi, tmp, &txlist, rdllink) {
> +	while (res < maxevents) {
>  		struct wakeup_source *ws;
> +		struct llist_node *n;
>  		__poll_t revents;
>  
> -		if (res >= maxevents)
> +		n = llist_del_first(&ep->rdllist);
> +		if (!n)
>  			break;
>  
> +		epi = llist_entry(n, struct epitem, rdllink);
> +
>  		/*
>  		 * Activate ep->ws before deactivating epi->ws to prevent
>  		 * triggering auto-suspend here (in case we reactive epi->ws
> @@ -1896,21 +1732,30 @@ static int ep_send_events(struct eventpoll *ep,
>  			__pm_relax(ws);
>  		}
>  
> -		list_del_init(&epi->rdllink);
> -
>  		/*
>  		 * If the event mask intersect the caller-requested one,
>  		 * deliver the event to userspace. Again, we are holding ep->mtx,
>  		 * so no operations coming from userspace can change the item.
>  		 */
>  		revents = ep_item_poll(epi, &pt, 1);
> -		if (!revents)
> +		if (!revents) {
> +			init_llist_node(n);
> +
> +			/*
> +			 * Just in case epi becomes ready after ep_item_poll() above, but before
> +			 * init_llist_node(). Make sure to add it to the ready list, otherwise an
> +			 * event may be lost.
> +			 */
> +			if (unlikely(ep_item_poll(epi, &pt, 1))) {
> +				ep_pm_stay_awake(epi);
> +				epitem_ready(epi);
> +			}
>  			continue;
> +		}
>  
>  		events = epoll_put_uevent(revents, epi->event.data, events);
>  		if (!events) {
> -			list_add(&epi->rdllink, &txlist);
> -			ep_pm_stay_awake(epi);
> +			llist_add(&epi->rdllink, &ep->rdllist);
>  			if (!res)
>  				res = -EFAULT;
>  			break;
> @@ -1918,25 +1763,31 @@ static int ep_send_events(struct eventpoll *ep,
>  		res++;
>  		if (epi->event.events & EPOLLONESHOT)
>  			epi->event.events &= EP_PRIVATE_BITS;
> -		else if (!(epi->event.events & EPOLLET)) {
> +		__llist_add(n, &txlist);
> +	}
> +
> +	llist_for_each_entry_safe(epi, tmp, txlist.first, rdllink) {
> +		init_llist_node(&epi->rdllink);
> +
> +		if (!(epi->event.events & EPOLLET)) {
>  			/*
> -			 * If this file has been added with Level
> -			 * Trigger mode, we need to insert back inside
> -			 * the ready list, so that the next call to
> -			 * epoll_wait() will check again the events
> -			 * availability. At this point, no one can insert
> -			 * into ep->rdllist besides us. The epoll_ctl()
> -			 * callers are locked out by
> -			 * ep_send_events() holding "mtx" and the
> -			 * poll callback will queue them in ep->ovflist.
> +			 * If this file has been added with Level Trigger mode, we need to insert
> +			 * back inside the ready list, so that the next call to epoll_wait() will
> +			 * check again the events availability.
>  			 */
> -			list_add_tail(&epi->rdllink, &ep->rdllist);
>  			ep_pm_stay_awake(epi);
> +			epitem_ready(epi);
>  		}
>  	}
> -	ep_done_scan(ep, &txlist);
> +
> +	__pm_relax(ep->ws);
>  	mutex_unlock(&ep->mtx);
>  
> +	if (!llist_empty(&ep->rdllist)) {
> +		if (waitqueue_active(&ep->wq))
> +			wake_up(&ep->wq);
> +	}
> +
>  	return res;
>  }
>  
> @@ -2029,8 +1880,6 @@ static int ep_poll(struct eventpoll *ep, struct epoll_event __user *events,
>  	wait_queue_entry_t wait;
>  	ktime_t expires, *to = NULL;
>  
> -	lockdep_assert_irqs_enabled();
> -
>  	if (timeout && (timeout->tv_sec | timeout->tv_nsec)) {
>  		slack = select_estimate_accuracy(timeout);
>  		to = &expires;
> @@ -2090,54 +1939,15 @@ static int ep_poll(struct eventpoll *ep, struct epoll_event __user *events,
>  		init_wait(&wait);
>  		wait.func = ep_autoremove_wake_function;
>  
> -		write_lock_irq(&ep->lock);
> -		/*
> -		 * Barrierless variant, waitqueue_active() is called under
> -		 * the same lock on wakeup ep_poll_callback() side, so it
> -		 * is safe to avoid an explicit barrier.
> -		 */
> -		__set_current_state(TASK_INTERRUPTIBLE);
> +		prepare_to_wait_exclusive(&ep->wq, &wait, TASK_INTERRUPTIBLE);
>  
> -		/*
> -		 * Do the final check under the lock. ep_start/done_scan()
> -		 * plays with two lists (->rdllist and ->ovflist) and there
> -		 * is always a race when both lists are empty for short
> -		 * period of time although events are pending, so lock is
> -		 * important.
> -		 */
> -		eavail = ep_events_available(ep);
> -		if (!eavail)
> -			__add_wait_queue_exclusive(&ep->wq, &wait);
> -
> -		write_unlock_irq(&ep->lock);
> -
> -		if (!eavail)
> +		if (!ep_events_available(ep))
>  			timed_out = !ep_schedule_timeout(to) ||
>  				!schedule_hrtimeout_range(to, slack,
>  							  HRTIMER_MODE_ABS);
> -		__set_current_state(TASK_RUNNING);
> -
> -		/*
> -		 * We were woken up, thus go and try to harvest some events.
> -		 * If timed out and still on the wait queue, recheck eavail
> -		 * carefully under lock, below.
> -		 */
> -		eavail = 1;
>  
> -		if (!list_empty_careful(&wait.entry)) {
> -			write_lock_irq(&ep->lock);
> -			/*
> -			 * If the thread timed out and is not on the wait queue,
> -			 * it means that the thread was woken up after its
> -			 * timeout expired before it could reacquire the lock.
> -			 * Thus, when wait.entry is empty, it needs to harvest
> -			 * events.
> -			 */
> -			if (timed_out)
> -				eavail = list_empty(&wait.entry);
> -			__remove_wait_queue(&ep->wq, &wait);
> -			write_unlock_irq(&ep->lock);
> -		}
> +		finish_wait(&ep->wq, &wait);
> +		eavail = ep_events_available(ep);
>  	}
>  }
>  
> -- 
> 2.39.5
> 

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-05-27  9:08 [PATCH v3] eventpoll: Fix priority inversion problem Nam Cao
  2025-05-30  5:08 ` Christian Brauner
@ 2025-06-25 14:50 ` Sebastian Andrzej Siewior
  2025-06-25 15:27   ` Nam Cao
  2025-06-26 15:23 ` John Ogness
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 38+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-06-25 14:50 UTC (permalink / raw)
  To: Nam Cao
  Cc: Alexander Viro, Christian Brauner, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On 2025-05-27 11:08:36 [+0200], Nam Cao wrote:
> --- a/fs/eventpoll.c
> +++ b/fs/eventpoll.c
> @@ -1867,19 +1704,18 @@ static int ep_send_events(struct eventpoll *ep,
>  	init_poll_funcptr(&pt, NULL);
>  
>  	mutex_lock(&ep->mtx);
> -	ep_start_scan(ep, &txlist);
>  
> -	/*
> -	 * We can loop without lock because we are passed a task private list.
> -	 * Items cannot vanish during the loop we are holding ep->mtx.
> -	 */
> -	list_for_each_entry_safe(epi, tmp, &txlist, rdllink) {
> +	while (res < maxevents) {
>  		struct wakeup_source *ws;
> +		struct llist_node *n;
>  		__poll_t revents;
>  
> -		if (res >= maxevents)
> +		n = llist_del_first(&ep->rdllist);
> +		if (!n)
>  			break;
>  
> +		epi = llist_entry(n, struct epitem, rdllink);
> +
>  		/*
>  		 * Activate ep->ws before deactivating epi->ws to prevent
>  		 * triggering auto-suspend here (in case we reactive epi->ws
> @@ -1896,21 +1732,30 @@ static int ep_send_events(struct eventpoll *ep,
>  			__pm_relax(ws);
>  		}
>  
> -		list_del_init(&epi->rdllink);
> -
>  		/*
>  		 * If the event mask intersect the caller-requested one,
>  		 * deliver the event to userspace. Again, we are holding ep->mtx,
>  		 * so no operations coming from userspace can change the item.
>  		 */
>  		revents = ep_item_poll(epi, &pt, 1);
> -		if (!revents)
> +		if (!revents) {
> +			init_llist_node(n);
> +
> +			/*
> +			 * Just in case epi becomes ready after ep_item_poll() above, but before
> +			 * init_llist_node(). Make sure to add it to the ready list, otherwise an
> +			 * event may be lost.
> +			 */

So why not llist_del_first_init() at the top? Wouldn't this avoid the
add below? 

> +			if (unlikely(ep_item_poll(epi, &pt, 1))) {
> +				ep_pm_stay_awake(epi);
> +				epitem_ready(epi);
> +			}
>  			continue;
> +		}
>  
>  		events = epoll_put_uevent(revents, epi->event.data, events);
>  		if (!events) {
> -			list_add(&epi->rdllink, &txlist);
> -			ep_pm_stay_awake(epi);
> +			llist_add(&epi->rdllink, &ep->rdllist);

That epitem_ready() above and this llist_add() add epi back where it was
retrieved from. Wouldn't it loop in this case?

I think you can avoid the add above and here adding it to txlist would
avoid the loop. (It returns NULL if the copy-to-user failed so I am not
sure why another retry will change something but the old code did it,
too so).

>  			if (!res)
>  				res = -EFAULT;
>  			break;

One note: The old code did "list_add() + ep_pm_stay_awake()". Now you do
"ep_pm_stay_awake() + epitem_ready()". epitem_ready() adds the item
conditionally to the list so you may do ep_pm_stay_awake() without
adding it to the list because it already is. Looking through
ep_pm_stay_awake() it shouldn't do any harm except incrementing a
counter again.

Sebastian

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-25 14:50 ` Sebastian Andrzej Siewior
@ 2025-06-25 15:27   ` Nam Cao
  2025-06-25 15:33     ` Sebastian Andrzej Siewior
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-06-25 15:27 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Alexander Viro, Christian Brauner, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On Wed, Jun 25, 2025 at 04:50:31PM +0200, Sebastian Andrzej Siewior wrote:
> On 2025-05-27 11:08:36 [+0200], Nam Cao wrote:
> > --- a/fs/eventpoll.c
> > +++ b/fs/eventpoll.c
> > @@ -1867,19 +1704,18 @@ static int ep_send_events(struct eventpoll *ep,
> >  	init_poll_funcptr(&pt, NULL);
> >  
> >  	mutex_lock(&ep->mtx);
> > -	ep_start_scan(ep, &txlist);
> >  
> > -	/*
> > -	 * We can loop without lock because we are passed a task private list.
> > -	 * Items cannot vanish during the loop we are holding ep->mtx.
> > -	 */
> > -	list_for_each_entry_safe(epi, tmp, &txlist, rdllink) {
> > +	while (res < maxevents) {
> >  		struct wakeup_source *ws;
> > +		struct llist_node *n;
> >  		__poll_t revents;
> >  
> > -		if (res >= maxevents)
> > +		n = llist_del_first(&ep->rdllist);
> > +		if (!n)
> >  			break;
> >  
> > +		epi = llist_entry(n, struct epitem, rdllink);
> > +
> >  		/*
> >  		 * Activate ep->ws before deactivating epi->ws to prevent
> >  		 * triggering auto-suspend here (in case we reactive epi->ws
> > @@ -1896,21 +1732,30 @@ static int ep_send_events(struct eventpoll *ep,
> >  			__pm_relax(ws);
> >  		}
> >  
> > -		list_del_init(&epi->rdllink);
> > -
> >  		/*
> >  		 * If the event mask intersect the caller-requested one,
> >  		 * deliver the event to userspace. Again, we are holding ep->mtx,
> >  		 * so no operations coming from userspace can change the item.
> >  		 */
> >  		revents = ep_item_poll(epi, &pt, 1);
> > -		if (!revents)
> > +		if (!revents) {
> > +			init_llist_node(n);
> > +
> > +			/*
> > +			 * Just in case epi becomes ready after ep_item_poll() above, but before
> > +			 * init_llist_node(). Make sure to add it to the ready list, otherwise an
> > +			 * event may be lost.
> > +			 */
> 
> So why not llist_del_first_init() at the top? Wouldn't this avoid the
> add below? 

Look at that function:
	static inline struct llist_node *llist_del_first_init(struct llist_head *head)
	{
		struct llist_node *n = llist_del_first(head);

		// BROKEN: another task does llist_add() here for the same node

		if (n)
			init_llist_node(n);
		return n;
	}

It is not atomic to another task doing llist_add() to the same node.
init_llist_node() would then put the list in an inconsistent state.

To be sure, I tried your suggestion. Systemd sometimes failed to boot, and
my stress test crashed instantly.

> 
> > +			if (unlikely(ep_item_poll(epi, &pt, 1))) {
> > +				ep_pm_stay_awake(epi);
> > +				epitem_ready(epi);
> > +			}
> >  			continue;
> > +		}
> >  
> >  		events = epoll_put_uevent(revents, epi->event.data, events);
> >  		if (!events) {
> > -			list_add(&epi->rdllink, &txlist);
> > -			ep_pm_stay_awake(epi);
> > +			llist_add(&epi->rdllink, &ep->rdllist);
> 
> That epitem_ready() above and this llist_add() add epi back where it was
> retrieved from. Wouldn't it loop in this case?

This is the EFAULT case, we are giving up, therefore we put the item back
and bail out. Therefore no loop.

If we have already done at least one item, then we report that to user. If
none, then we report -EFAULT. Regardless, this current item is not
"successfully consumed", so we put it back for the others to take it. We
are done here.

> I think you can avoid the add above and here adding it to txlist would
> avoid the loop. (It returns NULL if the copy-to-user failed so I am not
> sure why another retry will change something but the old code did it,
> too so).
> 
> >  			if (!res)
> >  				res = -EFAULT;
> >  			break;
> 
> One note: The old code did "list_add() + ep_pm_stay_awake()". Now you do
> "ep_pm_stay_awake() + epitem_ready()". epitem_ready() adds the item
> conditionally to the list so you may do ep_pm_stay_awake() without
> adding it to the list because it already is. Looking through
> ep_pm_stay_awake() it shouldn't do any harm except incrementing a
> counter again.

Yes, it shouldn't do any harm.

Thanks for reviewing, I know this lockless thing is annoying to look at.

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-25 15:27   ` Nam Cao
@ 2025-06-25 15:33     ` Sebastian Andrzej Siewior
  2025-06-25 15:57       ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-06-25 15:33 UTC (permalink / raw)
  To: Nam Cao
  Cc: Alexander Viro, Christian Brauner, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On 2025-06-25 17:27:02 [+0200], Nam Cao wrote:
> > > @@ -1896,21 +1732,30 @@ static int ep_send_events(struct eventpoll *ep,
> > >  			__pm_relax(ws);
> > >  		}
> > >  
> > > -		list_del_init(&epi->rdllink);
> > > -
> > >  		/*
> > >  		 * If the event mask intersect the caller-requested one,
> > >  		 * deliver the event to userspace. Again, we are holding ep->mtx,
> > >  		 * so no operations coming from userspace can change the item.
> > >  		 */
> > >  		revents = ep_item_poll(epi, &pt, 1);
> > > -		if (!revents)
> > > +		if (!revents) {
> > > +			init_llist_node(n);
> > > +
> > > +			/*
> > > +			 * Just in case epi becomes ready after ep_item_poll() above, but before
> > > +			 * init_llist_node(). Make sure to add it to the ready list, otherwise an
> > > +			 * event may be lost.
> > > +			 */
> > 
> > So why not llist_del_first_init() at the top? Wouldn't this avoid the
> > add below? 
> 
> Look at that function:
> 	static inline struct llist_node *llist_del_first_init(struct llist_head *head)
> 	{
> 		struct llist_node *n = llist_del_first(head);
> 
> 		// BROKEN: another task does llist_add() here for the same node
> 
> 		if (n)
> 			init_llist_node(n);
> 		return n;
> 	}
> 
> It is not atomic to another task doing llist_add() to the same node.
> init_llist_node() would then put the list in an inconsistent state.

Okay, I wasn't expecting another llist_add() from somewhere else. Makes
sense.

> To be sure, I tried your suggestion. Systemd sometimes failed to boot, and
> my stress test crashed instantly.

I had a trace_printk() there while testing and it never triggered.

> > 
> > > +			if (unlikely(ep_item_poll(epi, &pt, 1))) {
> > > +				ep_pm_stay_awake(epi);
> > > +				epitem_ready(epi);
> > > +			}
> > >  			continue;
> > > +		}
> > >  
> > >  		events = epoll_put_uevent(revents, epi->event.data, events);
> > >  		if (!events) {
> > > -			list_add(&epi->rdllink, &txlist);
> > > -			ep_pm_stay_awake(epi);
> > > +			llist_add(&epi->rdllink, &ep->rdllist);
> > 
> > That epitem_ready() above and this llist_add() add epi back where it was
> > retrieved from. Wouldn't it loop in this case?
> 
> This is the EFAULT case, we are giving up, therefore we put the item back
> and bail out. Therefore no loop.

Right.

> If we have already done at least one item, then we report that to user. If
> none, then we report -EFAULT. Regardless, this current item is not
> "successfully consumed", so we put it back for the others to take it. We
> are done here.
> 
> > I think you can avoid the add above and here adding it to txlist would
> > avoid the loop. (It returns NULL if the copy-to-user failed so I am not
> > sure why another retry will change something but the old code did it,
> > too so).
> > 
> > >  			if (!res)
> > >  				res = -EFAULT;
> > >  			break;
> > 
> > One note: The old code did "list_add() + ep_pm_stay_awake()". Now you do
> > "ep_pm_stay_awake() + epitem_ready()". epitem_ready() adds the item
> > conditionally to the list so you may do ep_pm_stay_awake() without
> > adding it to the list because it already is. Looking through
> > ep_pm_stay_awake() it shouldn't do any harm except incrementing a
> > counter again.
> 
> Yes, it shouldn't do any harm.
> 
> Thanks for reviewing, I know this lockless thing is annoying to look at.

but it looks now a bit smaller :)

> Nam

Sebastian

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-05-30  5:08 ` Christian Brauner
@ 2025-06-25 15:35   ` Sebastian Andrzej Siewior
  2025-06-26 13:35     ` Frederic Weisbecker
  0 siblings, 1 reply; 38+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-06-25 15:35 UTC (permalink / raw)
  To: Christian Brauner, Frederic Weisbecker
  Cc: Nam Cao, Frederic Weisbecker, Alexander Viro, Jan Kara,
	John Ogness, Clark Williams, Steven Rostedt, linux-fsdevel,
	linux-kernel, linux-rt-devel, linux-rt-users, Joe Damato,
	Martin Karsten, Jens Axboe, Valentin Schneider

On 2025-05-30 07:08:45 [+0200], Christian Brauner wrote:
> Care to review this, Frederic?

Frederic, may I summon you?

Sebastian

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-25 15:33     ` Sebastian Andrzej Siewior
@ 2025-06-25 15:57       ` Nam Cao
  2025-06-25 16:02         ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-06-25 15:57 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Alexander Viro, Christian Brauner, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On Wed, Jun 25, 2025 at 05:33:54PM +0200, Sebastian Andrzej Siewior wrote:
> On 2025-06-25 17:27:02 [+0200], Nam Cao wrote:
> > > > @@ -1896,21 +1732,30 @@ static int ep_send_events(struct eventpoll *ep,
> > > >  			__pm_relax(ws);
> > > >  		}
> > > >  
> > > > -		list_del_init(&epi->rdllink);
> > > > -
> > > >  		/*
> > > >  		 * If the event mask intersect the caller-requested one,
> > > >  		 * deliver the event to userspace. Again, we are holding ep->mtx,
> > > >  		 * so no operations coming from userspace can change the item.
> > > >  		 */
> > > >  		revents = ep_item_poll(epi, &pt, 1);
> > > > -		if (!revents)
> > > > +		if (!revents) {
> > > > +			init_llist_node(n);
> > > > +
> > > > +			/*
> > > > +			 * Just in case epi becomes ready after ep_item_poll() above, but before
> > > > +			 * init_llist_node(). Make sure to add it to the ready list, otherwise an
> > > > +			 * event may be lost.
> > > > +			 */
> > > 
> > > So why not llist_del_first_init() at the top? Wouldn't this avoid the
> > > add below? 
> > 
> > Look at that function:
> > 	static inline struct llist_node *llist_del_first_init(struct llist_head *head)
> > 	{
> > 		struct llist_node *n = llist_del_first(head);
> > 
> > 		// BROKEN: another task does llist_add() here for the same node
> > 
> > 		if (n)
> > 			init_llist_node(n);
> > 		return n;
> > 	}
> > 
> > It is not atomic to another task doing llist_add() to the same node.
> > init_llist_node() would then put the list in an inconsistent state.
> 
> Okay, I wasn't expecting another llist_add() from somewhere else. Makes
> sense.

Sorry, it's been a few weeks and I misremembered. But that wasn't the
reason. epitem_ready() is atomic to llist_del_first_init().

The actual reason is that, llist_del_first_init() would allow another
llist_add() to happen. So in the future loop iterations, we could see the
same item again, and we would incorrectly report more events than actually
available.

Thus, init_llist_node() doesn't happen until we are done looping.

> > To be sure, I tried your suggestion. Systemd sometimes failed to boot, and
> > my stress test crashed instantly.
> 
> I had a trace_printk() there while testing and it never triggered.

This code path is only executed for broken userspace.

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-25 15:57       ` Nam Cao
@ 2025-06-25 16:02         ` Nam Cao
  0 siblings, 0 replies; 38+ messages in thread
From: Nam Cao @ 2025-06-25 16:02 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Alexander Viro, Christian Brauner, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On Wed, Jun 25, 2025 at 05:57:18PM +0200, Nam Cao wrote:
> On Wed, Jun 25, 2025 at 05:33:54PM +0200, Sebastian Andrzej Siewior wrote:
> > On 2025-06-25 17:27:02 [+0200], Nam Cao wrote:
> > > To be sure, I tried your suggestion. Systemd sometimes failed to boot, and
> > > my stress test crashed instantly.
> > 
> > I had a trace_printk() there while testing and it never triggered.
> 
> This code path is only executed for broken userspace.

Forgot to mention, my test crashed because the __llist_add(n, &txlist)
below doesn't care if 'n' is already in a list. By changing to from
llist_del_first() to llist_del_first_init(), it is possible for 'n' to be
in a list, therefore __llist_add() would break.

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-25 15:35   ` Sebastian Andrzej Siewior
@ 2025-06-26 13:35     ` Frederic Weisbecker
  2025-06-26 13:51       ` Sebastian Andrzej Siewior
  0 siblings, 1 reply; 38+ messages in thread
From: Frederic Weisbecker @ 2025-06-26 13:35 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: Christian Brauner, Nam Cao, Alexander Viro, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Valentin Schneider

Le Wed, Jun 25, 2025 at 05:35:19PM +0200, Sebastian Andrzej Siewior a écrit :
> On 2025-05-30 07:08:45 [+0200], Christian Brauner wrote:
> > Care to review this, Frederic?
> 
> Frederic, may I summon you?

I can't review the guts and details but it looks like a sane
approach to me. Also the numbers are nice:

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

-- 
Frederic Weisbecker
SUSE Labs

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-26 13:35     ` Frederic Weisbecker
@ 2025-06-26 13:51       ` Sebastian Andrzej Siewior
  0 siblings, 0 replies; 38+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-06-26 13:51 UTC (permalink / raw)
  To: Frederic Weisbecker
  Cc: Christian Brauner, Nam Cao, Alexander Viro, Jan Kara, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Valentin Schneider

On 2025-06-26 15:35:47 [+0200], Frederic Weisbecker wrote:
> I can't review the guts and details but it looks like a sane
> approach to me. Also the numbers are nice:

Thank you.

Sebastian

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-05-27  9:08 [PATCH v3] eventpoll: Fix priority inversion problem Nam Cao
  2025-05-30  5:08 ` Christian Brauner
  2025-06-25 14:50 ` Sebastian Andrzej Siewior
@ 2025-06-26 15:23 ` John Ogness
  2025-06-26 15:49   ` Sebastian Andrzej Siewior
  2025-06-30 15:08 ` K Prateek Nayak
  2025-07-01 12:03 ` Christian Brauner
  4 siblings, 1 reply; 38+ messages in thread
From: John Ogness @ 2025-06-26 15:23 UTC (permalink / raw)
  To: Nam Cao, Alexander Viro, Christian Brauner, Jan Kara,
	Sebastian Andrzej Siewior, Clark Williams, Steven Rostedt,
	linux-fsdevel, linux-kernel, linux-rt-devel, linux-rt-users,
	Joe Damato, Martin Karsten, Jens Axboe
  Cc: Nam Cao, Frederic Weisbecker, Valentin Schneider

On 2025-05-27, Nam Cao <namcao@linutronix.de> wrote:
> diff --git a/fs/eventpoll.c b/fs/eventpoll.c
> index d4dbffdedd08e..a97a771a459c9 100644
> --- a/fs/eventpoll.c
> +++ b/fs/eventpoll.c
> @@ -137,13 +137,7 @@ struct epitem {
>  	};
>  
>  	/* List header used to link this structure to the eventpoll ready list */
> -	struct list_head rdllink;
> -
> -	/*
> -	 * Works together "struct eventpoll"->ovflist in keeping the
> -	 * single linked chain of items.
> -	 */
> -	struct epitem *next;
> +	struct llist_node rdllink;
>  
>  	/* The file descriptor information this item refers to */
>  	struct epoll_filefd ffd;
> @@ -191,22 +185,15 @@ struct eventpoll {
>  	/* Wait queue used by file->poll() */
>  	wait_queue_head_t poll_wait;
>  
> -	/* List of ready file descriptors */
> -	struct list_head rdllist;
> -
> -	/* Lock which protects rdllist and ovflist */
> -	rwlock_t lock;
> +	/*
> +	 * List of ready file descriptors. Adding to this list is lockless. Items can be removed
> +	 * only with eventpoll::mtx
> +	 */
> +	struct llist_head rdllist;
>  
>  	/* RB tree root used to store monitored fd structs */
>  	struct rb_root_cached rbr;
>  
> -	/*
> -	 * This is a single linked list that chains all the "struct epitem" that
> -	 * happened while transferring ready events to userspace w/out
> -	 * holding ->lock.
> -	 */
> -	struct epitem *ovflist;
> -
>  	/* wakeup_source used when ep_send_events or __ep_eventpoll_poll is running */
>  	struct wakeup_source *ws;
>  
> @@ -361,10 +348,14 @@ static inline int ep_cmp_ffd(struct epoll_filefd *p1,
>  	        (p1->file < p2->file ? -1 : p1->fd - p2->fd));
>  }
>  
> -/* Tells us if the item is currently linked */
> -static inline int ep_is_linked(struct epitem *epi)
> +/*
> + * Add the item to its container eventpoll's rdllist; do nothing if the item is already on rdllist.
> + */
> +static void epitem_ready(struct epitem *epi)
>  {
> -	return !list_empty(&epi->rdllink);
> +	if (&epi->rdllink == cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))

Perhaps:

	if (try_cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))

> +		llist_add(&epi->rdllink, &epi->ep->rdllist);
> +
>  }
>  
>  static inline struct eppoll_entry *ep_pwq_from_wait(wait_queue_entry_t *p)

John Ogness

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-26 15:23 ` John Ogness
@ 2025-06-26 15:49   ` Sebastian Andrzej Siewior
  2025-06-26 15:56     ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Sebastian Andrzej Siewior @ 2025-06-26 15:49 UTC (permalink / raw)
  To: John Ogness
  Cc: Nam Cao, Alexander Viro, Christian Brauner, Jan Kara,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On 2025-06-26 17:29:46 [+0206], John Ogness wrote:
> > @@ -361,10 +348,14 @@ static inline int ep_cmp_ffd(struct epoll_filefd *p1,
> >  	        (p1->file < p2->file ? -1 : p1->fd - p2->fd));
> >  }
> >  
> > -/* Tells us if the item is currently linked */
> > -static inline int ep_is_linked(struct epitem *epi)
> > +/*
> > + * Add the item to its container eventpoll's rdllist; do nothing if the item is already on rdllist.
> > + */
> > +static void epitem_ready(struct epitem *epi)
> >  {
> > -	return !list_empty(&epi->rdllink);
> > +	if (&epi->rdllink == cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))
> 
> Perhaps:
> 
> 	if (try_cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))

Not sure this is the same.
This will write back the current value of epi->rdllink.next to
epi->rdllink if epi->rdllink.next is not &epi->rdllink.

The intention is to check if epi->rdllink.next is set to &epi->rdllink
(pointing to itself) and if so set it NULL just to avoid to ensure
further cmpxchg() will fail here.

> > +		llist_add(&epi->rdllink, &epi->ep->rdllist);
> > +
> >  }
> >  
> >  static inline struct eppoll_entry *ep_pwq_from_wait(wait_queue_entry_t *p)
> 
> John Ogness

Sebastian

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-26 15:49   ` Sebastian Andrzej Siewior
@ 2025-06-26 15:56     ` Nam Cao
  0 siblings, 0 replies; 38+ messages in thread
From: Nam Cao @ 2025-06-26 15:56 UTC (permalink / raw)
  To: Sebastian Andrzej Siewior
  Cc: John Ogness, Alexander Viro, Christian Brauner, Jan Kara,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe, Frederic Weisbecker, Valentin Schneider

On Thu, Jun 26, 2025 at 05:49:54PM +0200, Sebastian Andrzej Siewior wrote:
> On 2025-06-26 17:29:46 [+0206], John Ogness wrote:
> > > @@ -361,10 +348,14 @@ static inline int ep_cmp_ffd(struct epoll_filefd *p1,
> > >  	        (p1->file < p2->file ? -1 : p1->fd - p2->fd));
> > >  }
> > >  
> > > -/* Tells us if the item is currently linked */
> > > -static inline int ep_is_linked(struct epitem *epi)
> > > +/*
> > > + * Add the item to its container eventpoll's rdllist; do nothing if the item is already on rdllist.
> > > + */
> > > +static void epitem_ready(struct epitem *epi)
> > >  {
> > > -	return !list_empty(&epi->rdllink);
> > > +	if (&epi->rdllink == cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))
> > 
> > Perhaps:
> > 
> > 	if (try_cmpxchg(&epi->rdllink.next, &epi->rdllink, NULL))
> 
> Not sure this is the same.
> This will write back the current value of epi->rdllink.next to
> epi->rdllink if epi->rdllink.next is not &epi->rdllink.
> 
> The intention is to check if epi->rdllink.next is set to &epi->rdllink
> (pointing to itself) and if so set it NULL just to avoid to ensure
> further cmpxchg() will fail here.

Exactly, thanks Sebastian.

I tested the suggestion, and systemd blew up.

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-05-27  9:08 [PATCH v3] eventpoll: Fix priority inversion problem Nam Cao
                   ` (2 preceding siblings ...)
  2025-06-26 15:23 ` John Ogness
@ 2025-06-30 15:08 ` K Prateek Nayak
  2025-07-01 20:33   ` Florian Bezdeka
  2025-07-01 12:03 ` Christian Brauner
  4 siblings, 1 reply; 38+ messages in thread
From: K Prateek Nayak @ 2025-06-30 15:08 UTC (permalink / raw)
  To: Nam Cao, Alexander Viro, Christian Brauner, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe
  Cc: Frederic Weisbecker, Valentin Schneider

Hello Nam,

On 5/27/2025 2:38 PM, Nam Cao wrote:
> The ready event list of an epoll object is protected by read-write
> semaphore:
> 
>    - The consumer (waiter) acquires the write lock and takes items.
>    - the producer (waker) takes the read lock and adds items.
> 
> The point of this design is enabling epoll to scale well with large number
> of producers, as multiple producers can hold the read lock at the same
> time.
> 
> Unfortunately, this implementation may cause scheduling priority inversion
> problem. Suppose the consumer has higher scheduling priority than the
> producer. The consumer needs to acquire the write lock, but may be blocked
> by the producer holding the read lock. Since read-write semaphore does not
> support priority-boosting for the readers (even with CONFIG_PREEMPT_RT=y),
> we have a case of priority inversion: a higher priority consumer is blocked
> by a lower priority producer. This problem was reported in [1].
> 
> Furthermore, this could also cause stall problem, as described in [2].
> 
> To fix this problem, make the event list half-lockless:
> 
>    - The consumer acquires a mutex (ep->mtx) and takes items.
>    - The producer locklessly adds items to the list.
> 
> Performance is not the main goal of this patch, but as the producer now can
> add items without waiting for consumer to release the lock, performance
> improvement is observed using the stress test from
> https://github.com/rouming/test-tools/blob/master/stress-epoll.c. This is
> the same test that justified using read-write semaphore in the past.
> 
> Testing using 12 x86_64 CPUs:
> 
>            Before     After        Diff
> threads  events/ms  events/ms
>        8       6932      19753    +185%
>       16       7820      27923    +257%
>       32       7648      35164    +360%
>       64       9677      37780    +290%
>      128      11166      38174    +242%
> 
> Testing using 1 riscv64 CPU (averaged over 10 runs, as the numbers are
> noisy):
> 
>            Before     After        Diff
> threads  events/ms  events/ms
>        1         73        129     +77%
>        2        151        216     +43%
>        4        216        364     +69%
>        8        234        382     +63%
>       16        251        392     +56%
> 

I gave this patch a spin on top of tip:sched/core (PREEMPT_RT) with
Jan's reproducer from
https://lore.kernel.org/all/7483d3ae-5846-4067-b9f7-390a614ba408@siemens.com/.

On tip:sched/core, I see a hang few seconds into the run and rcu-stall
a minute after when I pin the epoll-stall and epoll-stall-writer on the
same CPU as the Bandwidth timer on a 2vCPU VM. (I'm using a printk to
log the CPU where the timer was started in pinned mode)

With this series, I haven't seen any stalls yet over multiple short
runs (~10min) and even a longer run (~3Hrs).

Feel free to include:

Tested-by: K Prateek Nayak <kprateek.nayak@amd.com>

> Reported-by: Frederic Weisbecker <frederic@kernel.org>
> Closes: https://lore.kernel.org/linux-rt-users/20210825132754.GA895675@lothringen/ [1]
> Reported-by: Valentin Schneider <vschneid@redhat.com>
> Closes: https://lore.kernel.org/linux-rt-users/xhsmhttqvnall.mognet@vschneid.remote.csb/ [2]
> Signed-off-by: Nam Cao <namcao@linutronix.de>
> ---
> v3:
>    - get rid of the "link_used" and "ready" flags. They are hard to
>      understand and unnecessary
>    - get rid of the obsolete lockdep_assert_irqs_enabled()
>    - Add lockdep_assert_held(&ep->mtx)
>    - rewrite some comments
> v2:
>    - rename link_locked -> link_used
>    - replace xchg() with smp_store_release() when applicable
>    - make sure llist_node is in clean state when not on a list
>    - remove now-unused list_add_tail_lockless()

-- 
Thanks and Regards,
Prateek


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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-05-27  9:08 [PATCH v3] eventpoll: Fix priority inversion problem Nam Cao
                   ` (3 preceding siblings ...)
  2025-06-30 15:08 ` K Prateek Nayak
@ 2025-07-01 12:03 ` Christian Brauner
  2025-07-10  3:08   ` Xi Ruoyao
  4 siblings, 1 reply; 38+ messages in thread
From: Christian Brauner @ 2025-07-01 12:03 UTC (permalink / raw)
  To: Nam Cao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Tue, 27 May 2025 11:08:36 +0200, Nam Cao wrote:
> The ready event list of an epoll object is protected by read-write
> semaphore:
> 
>   - The consumer (waiter) acquires the write lock and takes items.
>   - the producer (waker) takes the read lock and adds items.
> 
> The point of this design is enabling epoll to scale well with large number
> of producers, as multiple producers can hold the read lock at the same
> time.
> 
> [...]

Applied to the vfs.fixes branch of the vfs/vfs.git tree.
Patches in the vfs.fixes branch should appear in linux-next soon.

Please report any outstanding bugs that were missed during review in a
new review to the original patch series allowing us to drop it.

It's encouraged to provide Acked-bys and Reviewed-bys even though the
patch has now been applied. If possible patch trailers will be updated.

Note that commit hashes shown below are subject to change due to rebase,
trailer updates or similar. If in doubt, please check the listed branch.

tree:   https://git.kernel.org/pub/scm/linux/kernel/git/vfs/vfs.git
branch: vfs.fixes

[1/1] eventpoll: Fix priority inversion problem
      https://git.kernel.org/vfs/vfs/c/e73f3008405b

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-06-30 15:08 ` K Prateek Nayak
@ 2025-07-01 20:33   ` Florian Bezdeka
  0 siblings, 0 replies; 38+ messages in thread
From: Florian Bezdeka @ 2025-07-01 20:33 UTC (permalink / raw)
  To: K Prateek Nayak, Nam Cao, Alexander Viro, Christian Brauner,
	Jan Kara, Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe
  Cc: Frederic Weisbecker, Valentin Schneider

On Mon, 2025-06-30 at 20:38 +0530, K Prateek Nayak wrote:
> Hello Nam,
> 
> On 5/27/2025 2:38 PM, Nam Cao wrote:
> > The ready event list of an epoll object is protected by read-write
> > semaphore:
> > 
> >    - The consumer (waiter) acquires the write lock and takes items.
> >    - the producer (waker) takes the read lock and adds items.
> > 
> > The point of this design is enabling epoll to scale well with large number
> > of producers, as multiple producers can hold the read lock at the same
> > time.
> > 
> > Unfortunately, this implementation may cause scheduling priority inversion
> > problem. Suppose the consumer has higher scheduling priority than the
> > producer. The consumer needs to acquire the write lock, but may be blocked
> > by the producer holding the read lock. Since read-write semaphore does not
> > support priority-boosting for the readers (even with CONFIG_PREEMPT_RT=y),
> > we have a case of priority inversion: a higher priority consumer is blocked
> > by a lower priority producer. This problem was reported in [1].
> > 
> > Furthermore, this could also cause stall problem, as described in [2].
> > 
> > To fix this problem, make the event list half-lockless:
> > 
> >    - The consumer acquires a mutex (ep->mtx) and takes items.
> >    - The producer locklessly adds items to the list.
> > 
> > Performance is not the main goal of this patch, but as the producer now can
> > add items without waiting for consumer to release the lock, performance
> > improvement is observed using the stress test from
> > https://github.com/rouming/test-tools/blob/master/stress-epoll.c. This is
> > the same test that justified using read-write semaphore in the past.
> > 
> > Testing using 12 x86_64 CPUs:
> > 
> >            Before     After        Diff
> > threads  events/ms  events/ms
> >        8       6932      19753    +185%
> >       16       7820      27923    +257%
> >       32       7648      35164    +360%
> >       64       9677      37780    +290%
> >      128      11166      38174    +242%
> > 
> > Testing using 1 riscv64 CPU (averaged over 10 runs, as the numbers are
> > noisy):
> > 
> >            Before     After        Diff
> > threads  events/ms  events/ms
> >        1         73        129     +77%
> >        2        151        216     +43%
> >        4        216        364     +69%
> >        8        234        382     +63%
> >       16        251        392     +56%
> > 
> 
> I gave this patch a spin on top of tip:sched/core (PREEMPT_RT) with
> Jan's reproducer from
> https://lore.kernel.org/all/7483d3ae-5846-4067-b9f7-390a614ba408@siemens.com/.
> 
> On tip:sched/core, I see a hang few seconds into the run and rcu-stall
> a minute after when I pin the epoll-stall and epoll-stall-writer on the
> same CPU as the Bandwidth timer on a 2vCPU VM. (I'm using a printk to
> log the CPU where the timer was started in pinned mode)
> 
> With this series, I haven't seen any stalls yet over multiple short
> runs (~10min) and even a longer run (~3Hrs).

Many thanks for running those tests and posting the results as comments
to this series. Highly appreciated!

Florian



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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-01 12:03 ` Christian Brauner
@ 2025-07-10  3:08   ` Xi Ruoyao
  2025-07-10  3:48     ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-10  3:08 UTC (permalink / raw)
  To: Christian Brauner, Nam Cao
  Cc: Frederic Weisbecker, Valentin Schneider, Alexander Viro, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe

On Tue, 2025-07-01 at 14:03 +0200, Christian Brauner wrote:
> On Tue, 27 May 2025 11:08:36 +0200, Nam Cao wrote:
> > The ready event list of an epoll object is protected by read-write
> > semaphore:
> > 
> >   - The consumer (waiter) acquires the write lock and takes items.
> >   - the producer (waker) takes the read lock and adds items.
> > 
> > The point of this design is enabling epoll to scale well with large number
> > of producers, as multiple producers can hold the read lock at the same
> > time.
> > 
> > [...]
> 
> Applied to the vfs.fixes branch of the vfs/vfs.git tree.
> Patches in the vfs.fixes branch should appear in linux-next soon.

> Please report any outstanding bugs that were missed during review in a
> new review to the original patch series allowing us to drop it.

Hi,

After upgrading my kernel to the recent mainline I've encountered some
stability issue, like:

- When GDM started gnome-shell, the screen often froze and the only
thing I could do was to switch into a VT and reboot.
- Sometimes gnome-shell started "fine" but then starting an application
(like gnome-console) needed to wait for about a minute.
- Sometimes the system shutdown process hangs waiting for a service to
stop.
- Rarely the system boot process hangs for no obvious reason.

Most strangely in all the cases there are nothing alarming in dmesg or
system journal.

I'm unsure if this is the culprit but I'm almost sure it's the trigger.
Maybe there's some race condition in my userspace that the priority
inversion had happened to hide...  but anyway reverting this patch
seemed to "fix" the issue.

Any thoughts or pointers to diagnose further?

-- 
Xi Ruoyao <xry111@xry111.site>

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  3:08   ` Xi Ruoyao
@ 2025-07-10  3:48     ` Nam Cao
  2025-07-10  4:06       ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-10  3:48 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Thu, Jul 10, 2025 at 11:08:18AM +0800, Xi Ruoyao wrote:
> After upgrading my kernel to the recent mainline I've encountered some
> stability issue, like:
> 
> - When GDM started gnome-shell, the screen often froze and the only
> thing I could do was to switch into a VT and reboot.
> - Sometimes gnome-shell started "fine" but then starting an application
> (like gnome-console) needed to wait for about a minute.
> - Sometimes the system shutdown process hangs waiting for a service to
> stop.
> - Rarely the system boot process hangs for no obvious reason.
> 
> Most strangely in all the cases there are nothing alarming in dmesg or
> system journal.
> 
> I'm unsure if this is the culprit but I'm almost sure it's the trigger.
> Maybe there's some race condition in my userspace that the priority
> inversion had happened to hide...  but anyway reverting this patch
> seemed to "fix" the issue.
> 
> Any thoughts or pointers to diagnose further?

No immediate idea, sorry.

May I know your exact setup (distro version etc.)? I will attempt to
reproduce the issue.

Best regards,
Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  3:48     ` Nam Cao
@ 2025-07-10  4:06       ` Nam Cao
  2025-07-10  4:10         ` Xi Ruoyao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-10  4:06 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Thu, Jul 10, 2025 at 05:48:08AM +0200, Nam Cao wrote:
> On Thu, Jul 10, 2025 at 11:08:18AM +0800, Xi Ruoyao wrote:
> > After upgrading my kernel to the recent mainline I've encountered some
> > stability issue, like:
> > 
> > - When GDM started gnome-shell, the screen often froze and the only
> > thing I could do was to switch into a VT and reboot.
> > - Sometimes gnome-shell started "fine" but then starting an application
> > (like gnome-console) needed to wait for about a minute.
> > - Sometimes the system shutdown process hangs waiting for a service to
> > stop.
> > - Rarely the system boot process hangs for no obvious reason.
> > 
> > Most strangely in all the cases there are nothing alarming in dmesg or
> > system journal.
> > 
> > I'm unsure if this is the culprit but I'm almost sure it's the trigger.
> > Maybe there's some race condition in my userspace that the priority
> > inversion had happened to hide...  but anyway reverting this patch
> > seemed to "fix" the issue.
> > 
> > Any thoughts or pointers to diagnose further?

I fetched Linus's tree, and noticed that the latest commit in Linus tree
fixes an use-after-free issue in eventpoll:
https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=8c2e52ebbe885c7eeaabd3b7ddcdc1246fc400d2

Any chance it resolves your issue?

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  4:06       ` Nam Cao
@ 2025-07-10  4:10         ` Xi Ruoyao
  2025-07-10  6:21           ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-10  4:10 UTC (permalink / raw)
  To: Nam Cao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

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

On Thu, 2025-07-10 at 06:06 +0200, Nam Cao wrote:
> On Thu, Jul 10, 2025 at 05:48:08AM +0200, Nam Cao wrote:
> > On Thu, Jul 10, 2025 at 11:08:18AM +0800, Xi Ruoyao wrote:
> > > After upgrading my kernel to the recent mainline I've encountered some
> > > stability issue, like:
> > > 
> > > - When GDM started gnome-shell, the screen often froze and the only
> > > thing I could do was to switch into a VT and reboot.
> > > - Sometimes gnome-shell started "fine" but then starting an application
> > > (like gnome-console) needed to wait for about a minute.
> > > - Sometimes the system shutdown process hangs waiting for a service to
> > > stop.
> > > - Rarely the system boot process hangs for no obvious reason.
> > > 
> > > Most strangely in all the cases there are nothing alarming in dmesg or
> > > system journal.
> > > 
> > > I'm unsure if this is the culprit but I'm almost sure it's the trigger.
> > > Maybe there's some race condition in my userspace that the priority
> > > inversion had happened to hide...  but anyway reverting this patch
> > > seemed to "fix" the issue.
> > > 
> > > Any thoughts or pointers to diagnose further?
> 
> I fetched Linus's tree, and noticed that the latest commit in Linus tree
> fixes an use-after-free issue in eventpoll:
> https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=8c2e52ebbe885c7eeaabd3b7ddcdc1246fc400d2
> 
> Any chance it resolves your issue?

Already tried but no luck.

The problem is my distro is a custom build from source and it may be
hard to reproduce the exactly same environment :(.

I'll attach my .config anyway.


-- 
Xi Ruoyao <xry111@xry111.site>

[-- Attachment #2: config.gz --]
[-- Type: application/gzip, Size: 36290 bytes --]

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  4:10         ` Xi Ruoyao
@ 2025-07-10  6:21           ` Nam Cao
  2025-07-10  6:54             ` Xi Ruoyao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-10  6:21 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Thu, Jul 10, 2025 at 11:08:18AM +0800, Xi Ruoyao wrote:
> After upgrading my kernel to the recent mainline I've encountered some
> stability issue, like:
> 
> - When GDM started gnome-shell, the screen often froze and the only
> thing I could do was to switch into a VT and reboot.
> - Sometimes gnome-shell started "fine" but then starting an application
> (like gnome-console) needed to wait for about a minute.
> - Sometimes the system shutdown process hangs waiting for a service to
> stop.
> - Rarely the system boot process hangs for no obvious reason.
> 
> Most strangely in all the cases there are nothing alarming in dmesg or
> system journal.
> 
> I'm unsure if this is the culprit but I'm almost sure it's the trigger.
> Maybe there's some race condition in my userspace that the priority
> inversion had happened to hide...  but anyway reverting this patch
> seemed to "fix" the issue.
> 
> Any thoughts or pointers to diagnose further?

I have been running this new epoll on my work machine for weeks by now
without issue, while you seem to reproduce it reliably. I'm guessing that
the problem is on some code path which is dead on my system, but executed
on yours.

I am curious if Gnome is using some epoll options which are unused on my
system.

I presume you can still access dmesg despite the freeze. Do you mind
running the below patch, let me know what's in your dmesg? It may help
identifying that code path.

Best regards,
Nam

diff --git a/fs/eventpoll.c b/fs/eventpoll.c
index 895256cd2786..e3dafc48a59a 100644
--- a/fs/eventpoll.c
+++ b/fs/eventpoll.c
@@ -532,6 +532,9 @@ static long ep_eventpoll_bp_ioctl(struct file *file, unsigned int cmd,
 		WRITE_ONCE(ep->busy_poll_usecs, epoll_params.busy_poll_usecs);
 		WRITE_ONCE(ep->busy_poll_budget, epoll_params.busy_poll_budget);
 		WRITE_ONCE(ep->prefer_busy_poll, epoll_params.prefer_busy_poll);
+		printk("%s busy_poll_usecs=%d busy_poll_budget=%d prefer_busy_poll=%d\n",
+			__func__, epoll_params.busy_poll_usecs, epoll_params.busy_poll_budget,
+			epoll_params.prefer_busy_poll);
 		return 0;
 	case EPIOCGPARAMS:
 		memset(&epoll_params, 0, sizeof(epoll_params));
@@ -2120,6 +2123,9 @@ int do_epoll_ctl(int epfd, int op, int fd, struct epoll_event *epds,
 	struct epitem *epi;
 	struct eventpoll *tep = NULL;
 
+	printk("%s: epfd=%d op=%d fd=%d events=0x%x data=0x%llx nonblock=%d\n",
+		__func__, epfd, op, fd, epds->events, epds->data, nonblock);
+
 	CLASS(fd, f)(epfd);
 	if (fd_empty(f))
 		return -EBADF;
diff --git a/io_uring/epoll.c b/io_uring/epoll.c
index 8d4610246ba0..e9c33c0c8cc5 100644
--- a/io_uring/epoll.c
+++ b/io_uring/epoll.c
@@ -54,6 +54,8 @@ int io_epoll_ctl(struct io_kiocb *req, unsigned int issue_flags)
 	int ret;
 	bool force_nonblock = issue_flags & IO_URING_F_NONBLOCK;
 
+	printk("%s flags=0x%x\n", __func__, issue_flags);
+
 	ret = do_epoll_ctl(ie->epfd, ie->op, ie->fd, &ie->event, force_nonblock);
 	if (force_nonblock && ret == -EAGAIN)
 		return -EAGAIN;

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  6:21           ` Nam Cao
@ 2025-07-10  6:54             ` Xi Ruoyao
  2025-07-10  8:32               ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-10  6:54 UTC (permalink / raw)
  To: Nam Cao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

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

On Thu, 2025-07-10 at 08:21 +0200, Nam Cao wrote:
> On Thu, Jul 10, 2025 at 11:08:18AM +0800, Xi Ruoyao wrote:
> > After upgrading my kernel to the recent mainline I've encountered some
> > stability issue, like:
> > 
> > - When GDM started gnome-shell, the screen often froze and the only
> > thing I could do was to switch into a VT and reboot.
> > - Sometimes gnome-shell started "fine" but then starting an application
> > (like gnome-console) needed to wait for about a minute.
> > - Sometimes the system shutdown process hangs waiting for a service to
> > stop.
> > - Rarely the system boot process hangs for no obvious reason.
> > 
> > Most strangely in all the cases there are nothing alarming in dmesg or
> > system journal.
> > 
> > I'm unsure if this is the culprit but I'm almost sure it's the trigger.
> > Maybe there's some race condition in my userspace that the priority
> > inversion had happened to hide...  but anyway reverting this patch
> > seemed to "fix" the issue.
> > 
> > Any thoughts or pointers to diagnose further?
> 
> I have been running this new epoll on my work machine for weeks by now
> without issue, while you seem to reproduce it reliably. I'm guessing that
> the problem is on some code path which is dead on my system, but executed
> on yours.

I also failed to reproduce it in a VM running the latest Fedora Rawhide
(in 3 attempts).

> I am curious if Gnome is using some epoll options which are unused on my
> system.

> I presume you can still access dmesg despite the freeze. Do you mind
> running the below patch, let me know what's in your dmesg? It may help
> identifying that code path.

Attached the system journal (dmesg was truncated due to too many lines).
I guess the relevant part should be between line 6947 ("New session 2 of
user xry111") and line 8022 ("start operation timed out. Terminating").

-- 
Xi Ruoyao <xry111@xry111.site>

[-- Attachment #2: log.gz --]
[-- Type: application/gzip, Size: 66431 bytes --]

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  6:54             ` Xi Ruoyao
@ 2025-07-10  8:32               ` Nam Cao
  2025-07-10  9:47                 ` Xi Ruoyao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-10  8:32 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Thu, Jul 10, 2025 at 02:54:06PM +0800, Xi Ruoyao wrote:
> On Thu, 2025-07-10 at 08:21 +0200, Nam Cao wrote:
> > I am curious if Gnome is using some epoll options which are unused on my
> > system.
> 
> > I presume you can still access dmesg despite the freeze. Do you mind
> > running the below patch, let me know what's in your dmesg? It may help
> > identifying that code path.
> 
> Attached the system journal (dmesg was truncated due to too many lines).
> I guess the relevant part should be between line 6947 ("New session 2 of
> user xry111") and line 8022 ("start operation timed out. Terminating").

Thanks! I have an idea..

Looking at the boot log you sent, I noticed some time gap immediately after
EPOLL_CTL_DEL.

So I looked at EPOLL_CTL_DEL again, and noticed something that could
explain your timed out issue:

  1. EPOLL_CTL_DEL may need to temporarily remove the entire event list.

  2. While the above is happening, another task may do epoll_wait(). It sees
     nothing in the event list, and goes to sleep.

  3. EPOLL_CTL_DEL is now finished and puts the items back into the event
     list. However, the task from (2.) is not woken up, therefore it keep
     sleeping despite there are events available.

If this is really what causing you problem, the below patch should fix it:


diff --git a/fs/eventpoll.c b/fs/eventpoll.c
index 895256cd2786..a8fb8ec51751 100644
--- a/fs/eventpoll.c
+++ b/fs/eventpoll.c
@@ -813,8 +813,13 @@ static bool __ep_remove(struct eventpoll *ep, struct epitem *epi, bool force)
 				put_back_last = n;
 			__llist_add(n, &put_back);
 		}
-		if (put_back_last)
+		if (put_back_last) {
 			llist_add_batch(put_back.first, put_back_last, &ep->rdllist);
+
+			/* borrow the memory barrier from llist_add_batch() */
+			if (waitqueue_active(&ep->wq))
+				wake_up(&ep->wq);
+		}
 	}
 
 	wakeup_source_unregister(ep_wakeup_source(epi));

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  8:32               ` Nam Cao
@ 2025-07-10  9:47                 ` Xi Ruoyao
  2025-07-11  5:02                   ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-10  9:47 UTC (permalink / raw)
  To: Nam Cao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

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

On Thu, 2025-07-10 at 10:32 +0200, Nam Cao wrote:
> On Thu, Jul 10, 2025 at 02:54:06PM +0800, Xi Ruoyao wrote:
> > On Thu, 2025-07-10 at 08:21 +0200, Nam Cao wrote:
> > > I am curious if Gnome is using some epoll options which are unused on my
> > > system.
> > 
> > > I presume you can still access dmesg despite the freeze. Do you mind
> > > running the below patch, let me know what's in your dmesg? It may help
> > > identifying that code path.
> > 
> > Attached the system journal (dmesg was truncated due to too many lines).
> > I guess the relevant part should be between line 6947 ("New session 2 of
> > user xry111") and line 8022 ("start operation timed out. Terminating").
> 
> Thanks! I have an idea..
> 
> Looking at the boot log you sent, I noticed some time gap immediately after
> EPOLL_CTL_DEL.
> 
> So I looked at EPOLL_CTL_DEL again, and noticed something that could
> explain your timed out issue:
> 
>   1. EPOLL_CTL_DEL may need to temporarily remove the entire event list.
> 
>   2. While the above is happening, another task may do epoll_wait(). It sees
>      nothing in the event list, and goes to sleep.
> 
>   3. EPOLL_CTL_DEL is now finished and puts the items back into the event
>      list. However, the task from (2.) is not woken up, therefore it keep
>      sleeping despite there are events available.
> 
> If this is really what causing you problem, the below patch should fix it:
> 
> 
> diff --git a/fs/eventpoll.c b/fs/eventpoll.c
> index 895256cd2786..a8fb8ec51751 100644
> --- a/fs/eventpoll.c
> +++ b/fs/eventpoll.c
> @@ -813,8 +813,13 @@ static bool __ep_remove(struct eventpoll *ep, struct epitem *epi, bool force)
>  				put_back_last = n;
>  			__llist_add(n, &put_back);
>  		}
> -		if (put_back_last)
> +		if (put_back_last) {
>  			llist_add_batch(put_back.first, put_back_last, &ep->rdllist);
> +
> +			/* borrow the memory barrier from llist_add_batch() */
> +			if (waitqueue_active(&ep->wq))
> +				wake_up(&ep->wq);
> +		}
>  	}
>  
>  	wakeup_source_unregister(ep_wakeup_source(epi));

It didn't work :(.

I've attached a new log.  This time to reduce noise I didn't make any
keyboard/mouse input until "start operation timed out."

-- 
Xi Ruoyao <xry111@xry111.site>

[-- Attachment #2: log.gz --]
[-- Type: application/gzip, Size: 59246 bytes --]

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-10  9:47                 ` Xi Ruoyao
@ 2025-07-11  5:02                   ` Nam Cao
  2025-07-11  9:44                     ` Christian Brauner
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-11  5:02 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Thu, Jul 10, 2025 at 05:47:57PM +0800, Xi Ruoyao wrote:
> It didn't work :(.

Argh :(

Another possibility is that you are running into event starvation problem.

Can you give the below patch a try? It is not the real fix, the patch hurts
performance badly. But if starvation is really your problem, it should
ameliorate the situation:

diff --git a/fs/eventpoll.c b/fs/eventpoll.c
index 895256cd2786..0dcf8e18de0d 100644
--- a/fs/eventpoll.c
+++ b/fs/eventpoll.c
@@ -1764,6 +1764,8 @@ static int ep_send_events(struct eventpoll *ep,
 		__llist_add(n, &txlist);
 	}
 
+	struct llist_node *shuffle = llist_del_all(&ep->rdllist);
+
 	llist_for_each_entry_safe(epi, tmp, txlist.first, rdllink) {
 		init_llist_node(&epi->rdllink);
 
@@ -1778,6 +1780,13 @@ static int ep_send_events(struct eventpoll *ep,
 		}
 	}
 
+	if (shuffle) {
+		struct llist_node *last = shuffle;
+		while (last->next)
+			last = last->next;
+		llist_add_batch(shuffle, last, &ep->rdllist);
+	}
+
 	__pm_relax(ep->ws);
 	mutex_unlock(&ep->mtx);
 

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11  5:02                   ` Nam Cao
@ 2025-07-11  9:44                     ` Christian Brauner
  2025-07-11  9:48                       ` Xi Ruoyao
  2025-07-11  9:50                       ` Nam Cao
  0 siblings, 2 replies; 38+ messages in thread
From: Christian Brauner @ 2025-07-11  9:44 UTC (permalink / raw)
  To: Nam Cao
  Cc: Xi Ruoyao, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, Jul 11, 2025 at 07:02:17AM +0200, Nam Cao wrote:
> On Thu, Jul 10, 2025 at 05:47:57PM +0800, Xi Ruoyao wrote:
> > It didn't work :(.
> 
> Argh :(
> 
> Another possibility is that you are running into event starvation problem.
> 
> Can you give the below patch a try? It is not the real fix, the patch hurts
> performance badly. But if starvation is really your problem, it should
> ameliorate the situation:
> 
> diff --git a/fs/eventpoll.c b/fs/eventpoll.c
> index 895256cd2786..0dcf8e18de0d 100644
> --- a/fs/eventpoll.c
> +++ b/fs/eventpoll.c
> @@ -1764,6 +1764,8 @@ static int ep_send_events(struct eventpoll *ep,
>  		__llist_add(n, &txlist);
>  	}
>  
> +	struct llist_node *shuffle = llist_del_all(&ep->rdllist);
> +
>  	llist_for_each_entry_safe(epi, tmp, txlist.first, rdllink) {
>  		init_llist_node(&epi->rdllink);
>  
> @@ -1778,6 +1780,13 @@ static int ep_send_events(struct eventpoll *ep,
>  		}
>  	}
>  
> +	if (shuffle) {
> +		struct llist_node *last = shuffle;
> +		while (last->next)
> +			last = last->next;
> +		llist_add_batch(shuffle, last, &ep->rdllist);
> +	}
> +
>  	__pm_relax(ep->ws);
>  	mutex_unlock(&ep->mtx);
>  

I think we should revert the fix so we have time to fix it properly
during v6.17+. This patch was a bit too adventurous for a fix in the
first place tbh.

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11  9:44                     ` Christian Brauner
@ 2025-07-11  9:48                       ` Xi Ruoyao
  2025-07-11  9:58                         ` Nam Cao
  2025-07-11  9:50                       ` Nam Cao
  1 sibling, 1 reply; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-11  9:48 UTC (permalink / raw)
  To: Christian Brauner, Nam Cao
  Cc: Frederic Weisbecker, Valentin Schneider, Alexander Viro, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe

On Fri, 2025-07-11 at 11:44 +0200, Christian Brauner wrote:
> On Fri, Jul 11, 2025 at 07:02:17AM +0200, Nam Cao wrote:
> > On Thu, Jul 10, 2025 at 05:47:57PM +0800, Xi Ruoyao wrote:
> > > It didn't work :(.
> > 
> > Argh :(
> > 
> > Another possibility is that you are running into event starvation problem.
> > 
> > Can you give the below patch a try? It is not the real fix, the patch hurts
> > performance badly. But if starvation is really your problem, it should
> > ameliorate the situation:
> > 
> > diff --git a/fs/eventpoll.c b/fs/eventpoll.c
> > index 895256cd2786..0dcf8e18de0d 100644
> > --- a/fs/eventpoll.c
> > +++ b/fs/eventpoll.c
> > @@ -1764,6 +1764,8 @@ static int ep_send_events(struct eventpoll *ep,
> >  		__llist_add(n, &txlist);
> >  	}
> >  
> > +	struct llist_node *shuffle = llist_del_all(&ep->rdllist);
> > +
> >  	llist_for_each_entry_safe(epi, tmp, txlist.first, rdllink) {
> >  		init_llist_node(&epi->rdllink);
> >  
> > @@ -1778,6 +1780,13 @@ static int ep_send_events(struct eventpoll *ep,
> >  		}
> >  	}
> >  
> > +	if (shuffle) {
> > +		struct llist_node *last = shuffle;
> > +		while (last->next)
> > +			last = last->next;
> > +		llist_add_batch(shuffle, last, &ep->rdllist);
> > +	}
> > +

Sadly, still no luck.

> >  	__pm_relax(ep->ws);
> >  	mutex_unlock(&ep->mtx);
> >  
> 
> I think we should revert the fix so we have time to fix it properly
> during v6.17+. This patch was a bit too adventurous for a fix in the
> first place tbh.

I tried to understand the code but all the comments seem outdated (they
still mention the removed rwlock).  IMO we'd at least clean them up...
and maybe we can notice something erratic.

-- 
Xi Ruoyao <xry111@xry111.site>

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11  9:44                     ` Christian Brauner
  2025-07-11  9:48                       ` Xi Ruoyao
@ 2025-07-11  9:50                       ` Nam Cao
  2025-07-14  8:59                         ` Christian Brauner
  1 sibling, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-11  9:50 UTC (permalink / raw)
  To: Christian Brauner
  Cc: Xi Ruoyao, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, Jul 11, 2025 at 11:44:28AM +0200, Christian Brauner wrote:
> I think we should revert the fix so we have time to fix it properly
> during v6.17+. This patch was a bit too adventurous for a fix in the
> first place tbh.

Agreed. I did feel a bit uneasy when I saw the patch being applied to
vfs.fixes.

Let me know if you want me to send a patch, otherwise I assume you will do
the revert yourself.

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11  9:48                       ` Xi Ruoyao
@ 2025-07-11  9:58                         ` Nam Cao
  2025-07-11 12:09                           ` Xi Ruoyao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-11  9:58 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, Jul 11, 2025 at 05:48:56PM +0800, Xi Ruoyao wrote:
> Sadly, still no luck.

That's unfortunate.

I'm still unable to reproduce the issue, so all I can do is staring at the
code and guessing. But I'm out of idea for now.

This one is going to be hard to figure out..

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11  9:58                         ` Nam Cao
@ 2025-07-11 12:09                           ` Xi Ruoyao
  2025-07-11 12:21                             ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-11 12:09 UTC (permalink / raw)
  To: Nam Cao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, 2025-07-11 at 11:58 +0200, Nam Cao wrote:
> On Fri, Jul 11, 2025 at 05:48:56PM +0800, Xi Ruoyao wrote:
> > Sadly, still no luck.
> 
> That's unfortunate.
> 
> I'm still unable to reproduce the issue, so all I can do is staring at the
> code and guessing. But I'm out of idea for now.

Same as I.  I tried to reproduce in a VM running Fedora Rawhide but
failed.

> This one is going to be hard to figure out..

And I'm afraid this may be a bug in my userspace... Then I'd feel guilty
if this is reverted because of an invalid bug report from I :(.

-- 
Xi Ruoyao <xry111@xry111.site>

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11 12:09                           ` Xi Ruoyao
@ 2025-07-11 12:21                             ` Nam Cao
  2025-07-12  0:09                               ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-11 12:21 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, Jul 11, 2025 at 08:09:12PM +0800, Xi Ruoyao wrote:
> On Fri, 2025-07-11 at 11:58 +0200, Nam Cao wrote:
> > On Fri, Jul 11, 2025 at 05:48:56PM +0800, Xi Ruoyao wrote:
> > > Sadly, still no luck.
> > 
> > That's unfortunate.
> > 
> > I'm still unable to reproduce the issue, so all I can do is staring at the
> > code and guessing. But I'm out of idea for now.
> 
> Same as I.  I tried to reproduce in a VM running Fedora Rawhide but
> failed.
> 
> > This one is going to be hard to figure out..
> 
> And I'm afraid this may be a bug in my userspace...

Yeah I expect the same thing. In the log you sent me, there are some
strange-looking event masks that look like a userspace bug.

I tried those strange looking event masks, but epoll_ctl() correctly
rejects them with EINVAL. So those masks probably aren't the reason, but
they still suggest that userspace may also be broken somewhere else and
triggers hard-to-reproduce problems in kernel's epoll.

> Then I'd feel guilty if this is reverted because of an invalid bug report
> from I :(.

But even when userspace is broken, if the kernel patch makes userspace
non-functional, I would still call it a kernel bug. So it's a valid bug
report alright, don't worry about it.

Until we know exactly the root cause, I think revert is the right thing to
do.

Best regards,
Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11 12:21                             ` Nam Cao
@ 2025-07-12  0:09                               ` Nam Cao
  2025-07-12  8:54                                 ` Xi Ruoyao
  0 siblings, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-12  0:09 UTC (permalink / raw)
  To: Xi Ruoyao
  Cc: Christian Brauner, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, Jul 11, 2025 at 08:09:12PM +0800, Xi Ruoyao wrote:
> And I'm afraid this may be a bug in my userspace... Then I'd feel guilty
> if this is reverted because of an invalid bug report from I :(.

FYI I just got a separate bug report. So something is definitely wrong with
this patch.

Best regards,
Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-12  0:09                               ` Nam Cao
@ 2025-07-12  8:54                                 ` Xi Ruoyao
  0 siblings, 0 replies; 38+ messages in thread
From: Xi Ruoyao @ 2025-07-12  8:54 UTC (permalink / raw)
  To: linux-fsdevel
  Cc: Nam Cao, Christian Brauner, Frederic Weisbecker,
	Valentin Schneider, Alexander Viro, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-kernel, linux-rt-devel, linux-rt-users,
	Joe Damato, Martin Karsten, Jens Axboe

On Sat, 2025-07-12 at 02:09 +0200, Nam Cao wrote:
> On Fri, Jul 11, 2025 at 08:09:12PM +0800, Xi Ruoyao wrote:
> > And I'm afraid this may be a bug in my userspace... Then I'd feel guilty
> > if this is reverted because of an invalid bug report from I :(.
> 
> FYI I just got a separate bug report. So something is definitely wrong with
> this patch.

FTR Linus has reverted this patch after encountering the same issue at
https://git.kernel.org/torvalds/c/5f02b80c21e1.

-- 
Xi Ruoyao <xry111@xry111.site>

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-11  9:50                       ` Nam Cao
@ 2025-07-14  8:59                         ` Christian Brauner
  2025-07-14 10:14                           ` Nam Cao
  2025-07-14 16:16                           ` Linus Torvalds
  0 siblings, 2 replies; 38+ messages in thread
From: Christian Brauner @ 2025-07-14  8:59 UTC (permalink / raw)
  To: Nam Cao, Linus Torvalds
  Cc: Xi Ruoyao, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Fri, Jul 11, 2025 at 11:50:08AM +0200, Nam Cao wrote:
> On Fri, Jul 11, 2025 at 11:44:28AM +0200, Christian Brauner wrote:
> > I think we should revert the fix so we have time to fix it properly
> > during v6.17+. This patch was a bit too adventurous for a fix in the
> > first place tbh.
> 
> Agreed. I did feel a bit uneasy when I saw the patch being applied to
> vfs.fixes.

I was on the fence myself and I juggled the commit between vfs.fixes and
vfs-6.17.misc because I wasn't sure whether we should consider such
priority inversion fix something that's urgent or not.

I tried to communicate to Linus that I was uncertain about this
particular patch at the beginning of

https://lore.kernel.org/20250704-vfs-fixes-fa31f5ff8c05@brauner

But I should've called it earlier.

Linus, thanks for reverting this! I had a revert ready as announced on
Friday for today but it's good that this got kicked out already.

My lesson from this is that touching epoll without a really really good
and urgent reason always end up a complete mess. So going forward we'll
just be very careful here.

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-14  8:59                         ` Christian Brauner
@ 2025-07-14 10:14                           ` Nam Cao
  2025-07-15  9:37                             ` Yann Ylavic
  2025-07-14 16:16                           ` Linus Torvalds
  1 sibling, 1 reply; 38+ messages in thread
From: Nam Cao @ 2025-07-14 10:14 UTC (permalink / raw)
  To: Christian Brauner
  Cc: Linus Torvalds, Xi Ruoyao, Frederic Weisbecker,
	Valentin Schneider, Alexander Viro, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe

On Mon, Jul 14, 2025 at 10:59:58AM +0200, Christian Brauner wrote:
> My lesson from this is that touching epoll without a really really good
> and urgent reason always end up a complete mess. So going forward we'll
> just be very careful here.

And my lesson is that lockless is hard. I still have no clue what is the
bug in this patch.

I am implementing a new solution now, completely ditching this old
approach. It survives some basic testing, and the numbers look promising. I
may post it later this week.

Nam

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-14  8:59                         ` Christian Brauner
  2025-07-14 10:14                           ` Nam Cao
@ 2025-07-14 16:16                           ` Linus Torvalds
  1 sibling, 0 replies; 38+ messages in thread
From: Linus Torvalds @ 2025-07-14 16:16 UTC (permalink / raw)
  To: Christian Brauner
  Cc: Nam Cao, Xi Ruoyao, Frederic Weisbecker, Valentin Schneider,
	Alexander Viro, Jan Kara, Sebastian Andrzej Siewior, John Ogness,
	Clark Williams, Steven Rostedt, linux-fsdevel, linux-kernel,
	linux-rt-devel, linux-rt-users, Joe Damato, Martin Karsten,
	Jens Axboe

On Mon, 14 Jul 2025 at 02:00, Christian Brauner <brauner@kernel.org> wrote:
>
> I was on the fence myself and I juggled the commit between vfs.fixes and
> vfs-6.17.misc because I wasn't sure whether we should consider such
> priority inversion fix something that's urgent or not.

Well, this time it actually helped that it didn't come in through the
merge window, because it made the bisection much shorter.

But in general, I do think that eventpoll should be considered to be
something that needs to die, rather than something that needs to be
improved upon. It's horrendous.

The indirections it does have been huge problems, even if they are
"powerful", because we've had lots of issues with recursion and loops,
which are all bad for reference counting - and not using reference
counting for lifetimes is just fundamentally a design bug.

For example, the vfs file close thing has a special
"eventpoll_release()" thing just because epoll can't use file
references for the references it holds (because that would just cause
recursive refs), and dammit, that's just the result of a fundamental
mis-design. And this is all after all the years of fixing outright
bugs (with hidden ones still lurking - unusually we had *another*
long-standing epoll bug fixed last week)

(Don't get me wrong: unix domain fd passing has caused all these
problems and more, so it's not like epoll is the *only* thing that
causes these kinds of horrendous issues, but unix domain fd passing
was something we did due to external reasons, not some self-inflicted
pain)

So this is just a heads-up that I will *NOT* be taking any epoll
patches AT ALL unless they are

 (a) obvious bug fixes

 (b) clearly separated into well-explained series with each individual
patch simple and obvious.

Because it was really a mistake to take that big epoll patch. That was
not a "small and obvious" fix to a big bug. That was literally a
"makes things worse" thing.

I didn't react very much to that patch because epoll has been fairly
calm for the last decade, and I had forgotten how much of a pain it
could be. So I was "whatever".

But this all re-awakened my "epoll is horrendous" memories.

Nam - please disregard performance as a primary thing in epoll. The
*only* thing that matters is "make it simpler, fix bugs".

Because long-term, epoll needs to die, or at least be seen as a legacy
interface that should be cut down, not something to be improved upon.

And yes, I hate epoll. It has caused *so* many problems over the
years. And it causes problems *outside* of epoll, ie we have that
horrendous pipe hackery:

         * Epoll nonsensically wants a wakeup whether the pipe
         * was already empty or not.

and the pipe code has that "poll_usage" flag just to deal with the
fallout of bad epoll fallout.

THAT was fun too.

Not.

              Linus

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-14 10:14                           ` Nam Cao
@ 2025-07-15  9:37                             ` Yann Ylavic
  2025-07-15 10:08                               ` Nam Cao
  0 siblings, 1 reply; 38+ messages in thread
From: Yann Ylavic @ 2025-07-15  9:37 UTC (permalink / raw)
  To: Nam Cao
  Cc: Christian Brauner, Linus Torvalds, Xi Ruoyao, Frederic Weisbecker,
	Valentin Schneider, Alexander Viro, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe

On Mon, Jul 14, 2025 at 9:48 PM Nam Cao <namcao@linutronix.de> wrote:
>
> And my lesson is that lockless is hard. I still have no clue what is the
> bug in this patch.

Maybe this is related:
https://lore.kernel.org/all/20250704180804.3598503-1-shakeel.butt@linux.dev/
?


Regards;
Yann.

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

* Re: [PATCH v3] eventpoll: Fix priority inversion problem
  2025-07-15  9:37                             ` Yann Ylavic
@ 2025-07-15 10:08                               ` Nam Cao
  0 siblings, 0 replies; 38+ messages in thread
From: Nam Cao @ 2025-07-15 10:08 UTC (permalink / raw)
  To: Yann Ylavic
  Cc: Christian Brauner, Linus Torvalds, Xi Ruoyao, Frederic Weisbecker,
	Valentin Schneider, Alexander Viro, Jan Kara,
	Sebastian Andrzej Siewior, John Ogness, Clark Williams,
	Steven Rostedt, linux-fsdevel, linux-kernel, linux-rt-devel,
	linux-rt-users, Joe Damato, Martin Karsten, Jens Axboe

On Tue, Jul 15, 2025 at 11:37:05AM +0200, Yann Ylavic wrote:
> On Mon, Jul 14, 2025 at 9:48 PM Nam Cao <namcao@linutronix.de> wrote:
> >
> > And my lesson is that lockless is hard. I still have no clue what is the
> > bug in this patch.
> 
> Maybe this is related:
> https://lore.kernel.org/all/20250704180804.3598503-1-shakeel.butt@linux.dev/
> ?

Maybe, you would need to ask Xi to validate it, because I cannot reproduce
the issue.

But I have abandoned this patch, even if we figure out the problem, sorry.
The patch was a mistake, it makes the code much more complicated. Even
myself got confused when I looked at it again after a few weeks.

The performance numbers were surely impressive, but it is probably not that
important. I don't think the benchmark reflects real workload anyway.

I'm going to simply switch the rwlock to spinlock instead. The numbers are
not as nice as before, but the code is simpler. The new patch is obvious to
look at.

I'm running tests right now. If no issue appears then I'm going to post it.

Best regards,
Nam

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

end of thread, other threads:[~2025-07-15 10:08 UTC | newest]

Thread overview: 38+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2025-05-27  9:08 [PATCH v3] eventpoll: Fix priority inversion problem Nam Cao
2025-05-30  5:08 ` Christian Brauner
2025-06-25 15:35   ` Sebastian Andrzej Siewior
2025-06-26 13:35     ` Frederic Weisbecker
2025-06-26 13:51       ` Sebastian Andrzej Siewior
2025-06-25 14:50 ` Sebastian Andrzej Siewior
2025-06-25 15:27   ` Nam Cao
2025-06-25 15:33     ` Sebastian Andrzej Siewior
2025-06-25 15:57       ` Nam Cao
2025-06-25 16:02         ` Nam Cao
2025-06-26 15:23 ` John Ogness
2025-06-26 15:49   ` Sebastian Andrzej Siewior
2025-06-26 15:56     ` Nam Cao
2025-06-30 15:08 ` K Prateek Nayak
2025-07-01 20:33   ` Florian Bezdeka
2025-07-01 12:03 ` Christian Brauner
2025-07-10  3:08   ` Xi Ruoyao
2025-07-10  3:48     ` Nam Cao
2025-07-10  4:06       ` Nam Cao
2025-07-10  4:10         ` Xi Ruoyao
2025-07-10  6:21           ` Nam Cao
2025-07-10  6:54             ` Xi Ruoyao
2025-07-10  8:32               ` Nam Cao
2025-07-10  9:47                 ` Xi Ruoyao
2025-07-11  5:02                   ` Nam Cao
2025-07-11  9:44                     ` Christian Brauner
2025-07-11  9:48                       ` Xi Ruoyao
2025-07-11  9:58                         ` Nam Cao
2025-07-11 12:09                           ` Xi Ruoyao
2025-07-11 12:21                             ` Nam Cao
2025-07-12  0:09                               ` Nam Cao
2025-07-12  8:54                                 ` Xi Ruoyao
2025-07-11  9:50                       ` Nam Cao
2025-07-14  8:59                         ` Christian Brauner
2025-07-14 10:14                           ` Nam Cao
2025-07-15  9:37                             ` Yann Ylavic
2025-07-15 10:08                               ` Nam Cao
2025-07-14 16:16                           ` Linus Torvalds

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).