LKML Archive mirror
 help / color / mirror / Atom feed
* [patch 1/2] epoll fix own poll()
@ 2009-01-27 20:54 Davide Libenzi
  2009-01-27 20:59 ` Davide Libenzi
  2009-01-29  8:01 ` Andrew Morton
  0 siblings, 2 replies; 13+ messages in thread
From: Davide Libenzi @ 2009-01-27 20:54 UTC (permalink / raw
  To: Linux Kernel Mailing List; +Cc: Andrew Morton, Pavel Pisa


The follwing patch fixes a bug inside the epoll's f_op->poll() code, that
returns POLLIN even though there are no actual ready monitored fds.
The bug shows up if you add an epoll fd inside another fd container (poll,
select, epoll).
The problem is that callback-based wake ups used by epoll, does not carry
(patches will follow, to fix this) any information about the events
that actually happened. So the callback code, since it can't call the file*
->poll() inside the callback, chains the file* into a ready-list. So, suppose
you added an fd with EPOLLOUT only, and some data shows up on the fd, the
file* mapped by the fd will be added into the ready-list (via wakeup
callback). During normal epoll_wait() use, this condition is sorted out
at the time we're actually able to call the file*'s f_op->poll().
Inside the old epoll's f_op->poll() though, only a quick check
!list_empty(ready-list) was performed, and this could have led to reporting
POLLIN even though no ready fds would show up at a following epoll_wait().
In order to correctly report the ready status for an epoll fd, the ready-list
must be checked to see if any really available fd+event would be ready in
a following epoll_wait().
Operation (calling f_op->poll() from inside f_op->poll()) that, like wake ups,
must be handled with care because of the fact that epoll fds can be added
to other epoll fds.



Signed-off-by: Davide Libenzi <davidel@xmailserver.org>


- Davide


---
 fs/eventpoll.c |  510 +++++++++++++++++++++++++++++++++------------------------
 1 file changed, 304 insertions(+), 206 deletions(-)

Index: linux-2.6.mod/fs/eventpoll.c
===================================================================
--- linux-2.6.mod.orig/fs/eventpoll.c	2009-01-26 23:13:23.000000000 -0800
+++ linux-2.6.mod/fs/eventpoll.c	2009-01-26 23:13:28.000000000 -0800
@@ -1,6 +1,6 @@
 /*
- *  fs/eventpoll.c (Efficent event polling implementation)
- *  Copyright (C) 2001,...,2007	 Davide Libenzi
+ *  fs/eventpoll.c (Efficient event retrieval implementation)
+ *  Copyright (C) 2001,...,2009	 Davide Libenzi
  *
  *  This program is free software; you can redistribute it and/or modify
  *  it under the terms of the GNU General Public License as published by
@@ -92,8 +92,8 @@
 /* Epoll private bits inside the event mask */
 #define EP_PRIVATE_BITS (EPOLLONESHOT | EPOLLET)
 
-/* Maximum number of poll wake up nests we are allowing */
-#define EP_MAX_POLLWAKE_NESTS 4
+/* Maximum number of nesting allowed inside epoll sets */
+#define EP_MAX_NESTS 4
 
 /* Maximum msec timeout value storeable in a long int */
 #define EP_MAX_MSTIMEO min(1000ULL * MAX_SCHEDULE_TIMEOUT / HZ, (LONG_MAX - 999ULL) / HZ)
@@ -110,24 +110,21 @@
 };
 
 /*
- * Node that is linked into the "wake_task_list" member of the "struct poll_safewake".
- * It is used to keep track on all tasks that are currently inside the wake_up() code
- * to 1) short-circuit the one coming from the same task and same wait queue head
- * (loop) 2) allow a maximum number of epoll descriptors inclusion nesting
- * 3) let go the ones coming from other tasks.
+ * Structure used to track possible nested calls, for too deep recursions
+ * and loop cycles.
  */
-struct wake_task_node {
+struct nested_call_node {
 	struct list_head llink;
 	struct task_struct *task;
-	wait_queue_head_t *wq;
+	void *cookie;
 };
 
 /*
- * This is used to implement the safe poll wake up avoiding to reenter
- * the poll callback from inside wake_up().
+ * This structure is used as collector for nested calls, to check for
+ * maximum recursion dept and loop cycles.
  */
-struct poll_safewake {
-	struct list_head wake_task_list;
+struct nested_calls {
+	struct list_head tasks_call_list;
 	spinlock_t lock;
 };
 
@@ -231,6 +228,12 @@
 	struct epitem *epi;
 };
 
+/* Used by the ep_send_events() function as callback private data */
+struct ep_send_events_data {
+	int maxevents;
+	struct epoll_event __user *events;
+};
+
 /*
  * Configuration options available inside /proc/sys/fs/epoll/
  */
@@ -244,8 +247,11 @@
  */
 static DEFINE_MUTEX(epmutex);
 
-/* Safe wake up implementation */
-static struct poll_safewake psw;
+/* Used for safe wake up implementation */
+static struct nested_calls poll_safewake_ncalls;
+
+/* Used to call file's f_op->poll() under the nested calls boundaries */
+static struct nested_calls poll_readywalk_ncalls;
 
 /* Slab cache used to allocate "struct epitem" */
 static struct kmem_cache *epi_cache __read_mostly;
@@ -322,64 +328,96 @@
 }
 
 /* Initialize the poll safe wake up structure */
-static void ep_poll_safewake_init(struct poll_safewake *psw)
+static void ep_nested_calls_init(struct nested_calls *ncalls)
 {
-
-	INIT_LIST_HEAD(&psw->wake_task_list);
-	spin_lock_init(&psw->lock);
+	INIT_LIST_HEAD(&ncalls->tasks_call_list);
+	spin_lock_init(&ncalls->lock);
 }
 
-/*
- * Perform a safe wake up of the poll wait list. The problem is that
- * with the new callback'd wake up system, it is possible that the
- * poll callback is reentered from inside the call to wake_up() done
- * on the poll wait queue head. The rule is that we cannot reenter the
- * wake up code from the same task more than EP_MAX_POLLWAKE_NESTS times,
- * and we cannot reenter the same wait queue head at all. This will
- * enable to have a hierarchy of epoll file descriptor of no more than
- * EP_MAX_POLLWAKE_NESTS deep. We need the irq version of the spin lock
- * because this one gets called by the poll callback, that in turn is called
- * from inside a wake_up(), that might be called from irq context.
+/**
+ * ep_call_nested - Perform a bound (possibly) nested call, by checking
+ *                  that the recursion limit is not exceeded, and that
+ *                  the same nested call (by the meaning of same cookie) is
+ *                  no re-entered.
+ *
+ * @ncalls: Pointer to the nested_calls structure to be used for this call.
+ * @max_nests: Maximum number of allowed nesting calls.
+ * @nproc: Nested call core function pointer.
+ * @priv: Opaque data to be passed to the @nproc callback.
+ * @cookie: Cookie to be used to identify this nested call.
+ *
+ * Returns: Returns the code returned by the @nproc callback, or -1 if
+ *          the maximum recursion limit has been exceeded.
  */
-static void ep_poll_safewake(struct poll_safewake *psw, wait_queue_head_t *wq)
+static int ep_call_nested(struct nested_calls *ncalls, int max_nests,
+			  int (*nproc)(void *, void *, int), void *priv,
+			  void *cookie)
 {
-	int wake_nests = 0;
+	int error, call_nests = 0;
 	unsigned long flags;
 	struct task_struct *this_task = current;
-	struct list_head *lsthead = &psw->wake_task_list;
-	struct wake_task_node *tncur;
-	struct wake_task_node tnode;
+	struct list_head *lsthead = &ncalls->tasks_call_list;
+	struct nested_call_node *tncur;
+	struct nested_call_node tnode;
 
-	spin_lock_irqsave(&psw->lock, flags);
+	spin_lock_irqsave(&ncalls->lock, flags);
 
-	/* Try to see if the current task is already inside this wakeup call */
+	/*
+	 * Try to see if the current task is already inside this wakeup call.
+	 * We use a list here, since the population inside this set is always
+	 * very much limited.
+	 */
 	list_for_each_entry(tncur, lsthead, llink) {
-
-		if (tncur->wq == wq ||
-		    (tncur->task == this_task && ++wake_nests > EP_MAX_POLLWAKE_NESTS)) {
+		if (tncur->task == this_task &&
+		    (tncur->cookie == cookie || ++call_nests > max_nests)) {
 			/*
 			 * Ops ... loop detected or maximum nest level reached.
 			 * We abort this wake by breaking the cycle itself.
 			 */
-			spin_unlock_irqrestore(&psw->lock, flags);
-			return;
+			spin_unlock_irqrestore(&ncalls->lock, flags);
+
+			return -1;
 		}
 	}
 
-	/* Add the current task to the list */
+	/* Add the current task and cookie to the list */
 	tnode.task = this_task;
-	tnode.wq = wq;
+	tnode.cookie = cookie;
 	list_add(&tnode.llink, lsthead);
 
-	spin_unlock_irqrestore(&psw->lock, flags);
+	spin_unlock_irqrestore(&ncalls->lock, flags);
 
-	/* Do really wake up now */
-	wake_up_nested(wq, 1 + wake_nests);
+	/* Call the nested function */
+	error = (*nproc)(priv, cookie, call_nests);
 
 	/* Remove the current task from the list */
-	spin_lock_irqsave(&psw->lock, flags);
+	spin_lock_irqsave(&ncalls->lock, flags);
 	list_del(&tnode.llink);
-	spin_unlock_irqrestore(&psw->lock, flags);
+	spin_unlock_irqrestore(&ncalls->lock, flags);
+
+	return error;
+}
+
+static int ep_poll_wakeup_proc(void *priv, void *cookie, int call_nests)
+{
+	wake_up_nested((wait_queue_head_t *) cookie, 1 + call_nests);
+	return 0;
+}
+
+/*
+ * Perform a safe wake up of the poll wait list. The problem is that
+ * with the new callback'd wake up system, it is possible that the
+ * poll callback is reentered from inside the call to wake_up() done
+ * on the poll wait queue head. The rule is that we cannot reenter the
+ * wake up code from the same task more than EP_MAX_NESTS times,
+ * and we cannot reenter the same wait queue head at all. This will
+ * enable to have a hierarchy of epoll file descriptor of no more than
+ * EP_MAX_NESTS deep.
+ */
+static void ep_poll_safewake(wait_queue_head_t *wq)
+{
+	ep_call_nested(&poll_safewake_ncalls, EP_MAX_NESTS,
+		       ep_poll_wakeup_proc, NULL, wq);
 }
 
 /*
@@ -407,6 +445,104 @@
 	}
 }
 
+/**
+ * ep_scan_ready_list - Scans the ready list in a way that makes possible for
+ *                      the scan code, to call f_op->poll(). Also allows for
+ *                      O(NumReady) performance.
+ *
+ * @ep: Pointer to the epoll private data structure.
+ * @sproc: Pointer to the scan callback.
+ * @priv: Private opaque data passed to the @sproc callback.
+ *
+ * Returns: The same integer error code returned by the @sproc callback.
+ */
+static int ep_scan_ready_list(struct eventpoll *ep,
+			      int (*sproc)(struct eventpoll *,
+					   struct list_head *, void *),
+			      void *priv)
+{
+	int error, pwake = 0;
+	unsigned long flags;
+	struct epitem *epi, *nepi;
+	struct list_head txlist;
+
+	INIT_LIST_HEAD(&txlist);
+
+	/*
+	 * We need to lock this because we could be hit by
+	 * eventpoll_release_file() and epoll_ctl(EPOLL_CTL_DEL).
+	 */
+	mutex_lock(&ep->mtx);
+
+	/*
+	 * 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.
+	 */
+	spin_lock_irqsave(&ep->lock, flags);
+	list_splice(&ep->rdllist, &txlist);
+	INIT_LIST_HEAD(&ep->rdllist);
+	ep->ovflist = NULL;
+	spin_unlock_irqrestore(&ep->lock, flags);
+
+	/*
+	 * Now call the callback function.
+	 */
+	error = (*sproc)(ep, &txlist, priv);
+
+	spin_lock_irqsave(&ep->lock, flags);
+	/*
+	 * 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 = 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->rdllink))
+			list_add_tail(&epi->rdllink, &ep->rdllist);
+	}
+	/*
+	 * 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.
+	 */
+	ep->ovflist = EP_UNACTIVE_PTR;
+
+	/*
+	 * Quickly re-inject items left on "txlist".
+	 */
+	list_splice(&txlist, &ep->rdllist);
+
+	if (!list_empty(&ep->rdllist)) {
+		/*
+		 * Wake up (if active) both the eventpoll wait list and the ->poll()
+		 * wait list (delayed after we release the lock).
+		 */
+		if (waitqueue_active(&ep->wq))
+			wake_up_locked(&ep->wq);
+		if (waitqueue_active(&ep->poll_wait))
+			pwake++;
+	}
+	spin_unlock_irqrestore(&ep->lock, flags);
+
+	mutex_unlock(&ep->mtx);
+
+	/* We have to call this outside the lock */
+	if (pwake)
+		ep_poll_safewake(&ep->poll_wait);
+
+	return error;
+}
+
 /*
  * Removes a "struct epitem" from the eventpoll RB tree and deallocates
  * all the associated resources. Must be called with "mtx" held.
@@ -457,7 +593,7 @@
 
 	/* We need to release all tasks waiting for these file */
 	if (waitqueue_active(&ep->poll_wait))
-		ep_poll_safewake(&psw, &ep->poll_wait);
+		ep_poll_safewake(&ep->poll_wait);
 
 	/*
 	 * We need to lock this because we could be hit by
@@ -507,22 +643,49 @@
 	return 0;
 }
 
+static int ep_read_events_proc(struct eventpoll *ep, struct list_head *head, void *priv)
+{
+	struct epitem *epi, *tmp;
+
+	list_for_each_entry_safe(epi, tmp, head, rdllink) {
+		if (epi->ffd.file->f_op->poll(epi->ffd.file, NULL) &
+		    epi->event.events)
+			return POLLIN | POLLRDNORM;
+		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.
+			 */
+			list_del_init(&epi->rdllink);
+	}
+
+	return 0;
+}
+
+static int ep_poll_readyevents_proc(void *priv, void *cookie, int call_nests)
+{
+	return ep_scan_ready_list(priv, ep_read_events_proc, NULL);
+}
+
 static unsigned int ep_eventpoll_poll(struct file *file, poll_table *wait)
 {
-	unsigned int pollflags = 0;
-	unsigned long flags;
+	int pollflags;
 	struct eventpoll *ep = file->private_data;
 
 	/* Insert inside our poll wait queue */
 	poll_wait(file, &ep->poll_wait, wait);
 
-	/* Check our condition */
-	spin_lock_irqsave(&ep->lock, flags);
-	if (!list_empty(&ep->rdllist))
-		pollflags = POLLIN | POLLRDNORM;
-	spin_unlock_irqrestore(&ep->lock, flags);
+	/*
+	 * Proceed to find out if wanted events are really available inside
+	 * the ready list. This need to be done under ep_call_nested()
+	 * supervision, since the call to f_op->poll() done on listed files
+	 * could re-enter here.
+	 */
+	pollflags = ep_call_nested(&poll_readywalk_ncalls, EP_MAX_NESTS,
+				   ep_poll_readyevents_proc, ep, ep);
 
-	return pollflags;
+	return pollflags != -1 ? pollflags: 0;
 }
 
 /* File callbacks that implement the eventpoll file behaviour */
@@ -552,7 +715,7 @@
 	 * We don't want to get "file->f_ep_lock" because it is not
 	 * necessary. It is not necessary because we're in the "struct file"
 	 * cleanup path, and this means that noone is using this file anymore.
-	 * So, for example, epoll_ctl() cannot hit here sicne if we reach this
+	 * So, for example, epoll_ctl() cannot hit here since if we reach this
 	 * point, the file counter already went to zero and fget() would fail.
 	 * The only hit might come from ep_free() but by holding the mutex
 	 * will correctly serialize the operation. We do need to acquire
@@ -683,12 +846,9 @@
 	}
 
 	/* If this file is already in the ready list we exit soon */
-	if (ep_is_linked(&epi->rdllink))
-		goto is_linked;
-
-	list_add_tail(&epi->rdllink, &ep->rdllist);
+	if (!ep_is_linked(&epi->rdllink))
+		list_add_tail(&epi->rdllink, &ep->rdllist);
 
-is_linked:
 	/*
 	 * Wake up ( if active ) both the eventpoll wait list and the ->poll()
 	 * wait list.
@@ -703,7 +863,7 @@
 
 	/* We have to call this outside the lock */
 	if (pwake)
-		ep_poll_safewake(&psw, &ep->poll_wait);
+		ep_poll_safewake(&ep->poll_wait);
 
 	return 1;
 }
@@ -725,10 +885,9 @@
 		add_wait_queue(whead, &pwq->wait);
 		list_add_tail(&pwq->llink, &epi->pwqlist);
 		epi->nwait++;
-	} else {
+	} else
 		/* We have to signal that an error occurred */
 		epi->nwait = -1;
-	}
 }
 
 static void ep_rbtree_insert(struct eventpoll *ep, struct epitem *epi)
@@ -830,7 +989,7 @@
 
 	/* We have to call this outside the lock */
 	if (pwake)
-		ep_poll_safewake(&psw, &ep->poll_wait);
+		ep_poll_safewake(&ep->poll_wait);
 
 	DNPRINTK(3, (KERN_INFO "[%p] eventpoll: ep_insert(%p, %p, %d)\n",
 		     current, ep, tfile, fd));
@@ -904,137 +1063,74 @@
 
 	/* We have to call this outside the lock */
 	if (pwake)
-		ep_poll_safewake(&psw, &ep->poll_wait);
+		ep_poll_safewake(&ep->poll_wait);
 
 	return 0;
 }
 
-static int ep_send_events(struct eventpoll *ep, struct epoll_event __user *events,
-			  int maxevents)
+static int ep_send_events_proc(struct eventpoll *ep, struct list_head *head, void *priv)
 {
-	int eventcnt, error = -EFAULT, pwake = 0;
-	unsigned int revents;
-	unsigned long flags;
-	struct epitem *epi, *nepi;
-	struct list_head txlist;
-
-	INIT_LIST_HEAD(&txlist);
-
-	/*
-	 * We need to lock this because we could be hit by
-	 * eventpoll_release_file() and epoll_ctl(EPOLL_CTL_DEL).
-	 */
-	mutex_lock(&ep->mtx);
-
-	/*
-	 * 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 are doing it in the loop below, in a lockless way.
-	 */
-	spin_lock_irqsave(&ep->lock, flags);
-	list_splice(&ep->rdllist, &txlist);
-	INIT_LIST_HEAD(&ep->rdllist);
-	ep->ovflist = NULL;
-	spin_unlock_irqrestore(&ep->lock, flags);
+	struct ep_send_events_data *esed = priv;
+	int eventcnt;
+  	unsigned int revents;
+	struct epitem *epi;
+	struct epoll_event __user *uevent;
 
-	/*
-	 * We can loop without lock because this is a task private list.
-	 * We just splice'd out the ep->rdllist in ep_collect_ready_items().
-	 * Items cannot vanish during the loop because we are holding "mtx".
-	 */
-	for (eventcnt = 0; !list_empty(&txlist) && eventcnt < maxevents;) {
-		epi = list_first_entry(&txlist, struct epitem, rdllink);
+  	/*
+	 * We can loop without lock because we are passed a task private list.
+	 * Items cannot vanish during the loop because ep_scan_ready_list() is
+	 * holding "mtx" during this call.
+  	 */
+	for (eventcnt = 0, uevent = esed->events;
+	     !list_empty(head) && eventcnt < esed->maxevents;) {
+		epi = list_first_entry(head, struct epitem, rdllink);
 
 		list_del_init(&epi->rdllink);
 
-		/*
-		 * Get the ready file event set. We can safely use the file
-		 * because we are holding the "mtx" and this will guarantee
-		 * that both the file and the item will not vanish.
-		 */
-		revents = epi->ffd.file->f_op->poll(epi->ffd.file, NULL);
-		revents &= epi->event.events;
+  		revents = epi->ffd.file->f_op->poll(epi->ffd.file, NULL) &
+  			epi->event.events;
 
-		/*
-		 * Is the event mask intersect the caller-requested one,
-		 * deliver the event to userspace. Again, we are holding
-		 * "mtx", so no operations coming from userspace can change
-		 * the item.
-		 */
-		if (revents) {
-			if (__put_user(revents,
-				       &events[eventcnt].events) ||
-			    __put_user(epi->event.data,
-				       &events[eventcnt].data))
-				goto errxit;
-			if (epi->event.events & EPOLLONESHOT)
-				epi->event.events &= EP_PRIVATE_BITS;
-			eventcnt++;
-		}
-		/*
-		 * At this point, noone can insert into ep->rdllist besides
-		 * us. The epoll_ctl() callers are locked out by us holding
-		 * "mtx" and the poll callback will queue them in ep->ovflist.
-		 */
-		if (!(epi->event.events & EPOLLET) &&
-		    (revents & epi->event.events))
-			list_add_tail(&epi->rdllink, &ep->rdllist);
-	}
-	error = 0;
-
-errxit:
+  		/*
+		 * If the event mask intersect the caller-requested one,
+		 * deliver the event to userspace. Again, ep_scan_ready_list()
+		 * is holding "mtx", so no operations coming from userspace
+		 * can change the item.
+  		 */
+  		if (revents) {
+			if (__put_user(revents, &uevent->events) ||
+			    __put_user(epi->event.data, &uevent->data))
+				return eventcnt ? eventcnt: -EFAULT;
+  			eventcnt++;
+			uevent++;
+  			if (epi->event.events & EPOLLONESHOT)
+  				epi->event.events &= EP_PRIVATE_BITS;
+  			else 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, noone can insert into ep->rdllist
+  				 * besides us. The epoll_ctl() callers are locked
+				 * out by ep_scan_ready_list() holding "mtx" and
+				 * the poll callback will queue them in ep->ovflist.
+  				 */
+  				list_add_tail(&epi->rdllink, &ep->rdllist);
+  		}
+  	}
 
-	spin_lock_irqsave(&ep->lock, flags);
-	/*
-	 * During the time we spent in the loop above, some other events
-	 * might have been queued by the poll callback. We re-insert them
-	 * inside the main ready-list here.
-	 */
-	for (nepi = ep->ovflist; (epi = nepi) != NULL;
-	     nepi = epi->next, epi->next = EP_UNACTIVE_PTR) {
-		/*
-		 * If the above loop quit with errors, the epoll item might still
-		 * be linked to "txlist", and the list_splice() done below will
-		 * take care of those cases.
-		 */
-		if (!ep_is_linked(&epi->rdllink))
-			list_add_tail(&epi->rdllink, &ep->rdllist);
-	}
-	/*
-	 * 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.
-	 */
-	ep->ovflist = EP_UNACTIVE_PTR;
-
-	/*
-	 * In case of error in the event-send loop, or in case the number of
-	 * ready events exceeds the userspace limit, we need to splice the
-	 * "txlist" back inside ep->rdllist.
-	 */
-	list_splice(&txlist, &ep->rdllist);
-
-	if (!list_empty(&ep->rdllist)) {
-		/*
-		 * Wake up (if active) both the eventpoll wait list and the ->poll()
-		 * wait list (delayed after we release the lock).
-		 */
-		if (waitqueue_active(&ep->wq))
-			wake_up_locked(&ep->wq);
-		if (waitqueue_active(&ep->poll_wait))
-			pwake++;
-	}
-	spin_unlock_irqrestore(&ep->lock, flags);
+	return eventcnt;
+}
 
-	mutex_unlock(&ep->mtx);
+static int ep_send_events(struct eventpoll *ep, struct epoll_event __user *events,
+			  int maxevents)
+{
+	struct ep_send_events_data esed;
 
-	/* We have to call this outside the lock */
-	if (pwake)
-		ep_poll_safewake(&psw, &ep->poll_wait);
+	esed.maxevents = maxevents;
+	esed.events = events;
 
-	return eventcnt == 0 ? error: eventcnt;
+	return ep_scan_ready_list(ep, ep_send_events_proc, &esed);
 }
 
 static int ep_poll(struct eventpoll *ep, struct epoll_event __user *events,
@@ -1046,7 +1142,7 @@
 	wait_queue_t wait;
 
 	/*
-	 * Calculate the timeout by checking for the "infinite" value ( -1 )
+	 * Calculate the timeout by checking for the "infinite" value (-1)
 	 * and the overflow condition. The passed timeout is in milliseconds,
 	 * that why (t * HZ) / 1000.
 	 */
@@ -1089,9 +1185,8 @@
 
 		set_current_state(TASK_RUNNING);
 	}
-
 	/* Is it worth to try to dig for events ? */
-	eavail = !list_empty(&ep->rdllist);
+	eavail = !list_empty(&ep->rdllist) || ep->ovflist != EP_UNACTIVE_PTR;
 
 	spin_unlock_irqrestore(&ep->lock, flags);
 
@@ -1112,42 +1207,42 @@
  */
 SYSCALL_DEFINE1(epoll_create1, int, flags)
 {
-	int error, fd = -1;
-	struct eventpoll *ep;
+	int error;
+	struct eventpoll *ep = NULL;
 
 	/* Check the EPOLL_* constant for consistency.  */
 	BUILD_BUG_ON(EPOLL_CLOEXEC != O_CLOEXEC);
 
-	if (flags & ~EPOLL_CLOEXEC)
-		return -EINVAL;
-
 	DNPRINTK(3, (KERN_INFO "[%p] eventpoll: sys_epoll_create(%d)\n",
 		     current, flags));
 
+	error = -EINVAL;
+	if (flags & ~EPOLL_CLOEXEC)
+		goto error_return;
+
 	/*
-	 * Create the internal data structure ( "struct eventpoll" ).
+	 * Create the internal data structure ("struct eventpoll").
 	 */
 	error = ep_alloc(&ep);
-	if (error < 0) {
-		fd = error;
+	if (error < 0)
 		goto error_return;
-	}
 
 	/*
 	 * Creates all the items needed to setup an eventpoll file. That is,
 	 * a file structure and a free file descriptor.
 	 */
-	fd = anon_inode_getfd("[eventpoll]", &eventpoll_fops, ep,
-			      flags & O_CLOEXEC);
-	if (fd < 0)
+	error = anon_inode_getfd("[eventpoll]", &eventpoll_fops, ep,
+				 flags & O_CLOEXEC);
+	if (error < 0)
 		ep_free(ep);
-	atomic_inc(&ep->user->epoll_devs);
+	else
+		atomic_inc(&ep->user->epoll_devs);
 
 error_return:
 	DNPRINTK(3, (KERN_INFO "[%p] eventpoll: sys_epoll_create(%d) = %d\n",
-		     current, flags, fd));
+		     current, flags, error));
 
-	return fd;
+	return error;
 }
 
 SYSCALL_DEFINE1(epoll_create, int, size)
@@ -1371,7 +1466,10 @@
 		EP_ITEM_COST;
 
 	/* Initialize the structure used to perform safe poll wait head wake ups */
-	ep_poll_safewake_init(&psw);
+	ep_nested_calls_init(&poll_safewake_ncalls);
+
+	/* Initialize the structure used to perform file's f_op->poll() calls */
+	ep_nested_calls_init(&poll_readywalk_ncalls);
 
 	/* Allocates slab cache used to allocate "struct epitem" items */
 	epi_cache = kmem_cache_create("eventpoll_epi", sizeof(struct epitem),


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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-27 20:54 [patch 1/2] epoll fix own poll() Davide Libenzi
@ 2009-01-27 20:59 ` Davide Libenzi
  2009-01-29  8:01 ` Andrew Morton
  1 sibling, 0 replies; 13+ messages in thread
From: Davide Libenzi @ 2009-01-27 20:59 UTC (permalink / raw
  To: Linux Kernel Mailing List; +Cc: Andrew Morton, Pavel Pisa

[-- Attachment #1: Type: TEXT/PLAIN, Size: 1605 bytes --]

On Tue, 27 Jan 2009, Davide Libenzi wrote:

> 
> The follwing patch fixes a bug inside the epoll's f_op->poll() code, that
> returns POLLIN even though there are no actual ready monitored fds.
> The bug shows up if you add an epoll fd inside another fd container (poll,
> select, epoll).
> The problem is that callback-based wake ups used by epoll, does not carry
> (patches will follow, to fix this) any information about the events
> that actually happened. So the callback code, since it can't call the file*
> ->poll() inside the callback, chains the file* into a ready-list. So, suppose
> you added an fd with EPOLLOUT only, and some data shows up on the fd, the
> file* mapped by the fd will be added into the ready-list (via wakeup
> callback). During normal epoll_wait() use, this condition is sorted out
> at the time we're actually able to call the file*'s f_op->poll().
> Inside the old epoll's f_op->poll() though, only a quick check
> !list_empty(ready-list) was performed, and this could have led to reporting
> POLLIN even though no ready fds would show up at a following epoll_wait().
> In order to correctly report the ready status for an epoll fd, the ready-list
> must be checked to see if any really available fd+event would be ready in
> a following epoll_wait().
> Operation (calling f_op->poll() from inside f_op->poll()) that, like wake ups,
> must be handled with care because of the fact that epoll fds can be added
> to other epoll fds.

Forgot to include the test program that can be used to check for correct 
event reporting, and long-chains/loop trimming code.



- Davide


[-- Warning: decoded text below may be mangled, UTF-8 assumed --]
[-- Attachment #2: Type: TEXT/x-csrc; name=epoll_test.c, Size: 8504 bytes --]

/*
 *  epoll_test by Davide Libenzi (Simple code to test epoll internals)
 *  Copyright (C) 2008  Davide Libenzi
 *
 *  This program is free software; you can redistribute it and/or modify
 *  it under the terms of the GNU General Public License as published by
 *  the Free Software Foundation; either version 2 of the License, or
 *  (at your option) any later version.
 *
 *  This program is distributed in the hope that it will be useful,
 *  but WITHOUT ANY WARRANTY; without even the implied warranty of
 *  MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
 *  GNU General Public License for more details.
 *
 *  You should have received a copy of the GNU General Public License
 *  along with this program; if not, write to the Free Software
 *  Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA  02111-1307  USA
 *
 *  Davide Libenzi <davidel@xmailserver.org>
 *
 */

#include <sys/types.h>
#include <unistd.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <errno.h>
#include <signal.h>
#include <limits.h>
#include <poll.h>
#include <sys/epoll.h>
#include <sys/wait.h>

#define EPWAIT_TIMEO	(1 * 1000)
#ifndef POLLRDHUP
#define POLLRDHUP 0x2000
#endif

#define EPOLL_MAX_CHAIN	100L

#define EPOLL_TF_LOOP (1 << 0)

struct epoll_test_cfg {
	long size;
	long flags;
};

static int xepoll_create(int n) {
	int epfd;

	if ((epfd = epoll_create(n)) == -1) {
		perror("epoll_create");
		exit(2);
	}

	return epfd;
}

static void xepoll_ctl(int epfd, int cmd, int fd, struct epoll_event *evt) {
	if (epoll_ctl(epfd, cmd, fd, evt) < 0) {
		perror("epoll_ctl");
		exit(3);
	}
}

static void xpipe(int *fds) {
	if (pipe(fds)) {
		perror("pipe");
		exit(4);
	}
}

static pid_t xfork(void) {
	pid_t pid;

	if ((pid = fork()) == (pid_t) -1) {
		perror("pipe");
		exit(5);
	}

	return pid;
}

static int run_forked_proc(int (*proc)(void *), void *data) {
	int status;
	pid_t pid;

	if ((pid = xfork()) == 0)
		exit((*proc)(data));
	if (waitpid(pid, &status, 0) != pid) {
		perror("waitpid");
		return -1;
	}

	return WIFEXITED(status) ? WEXITSTATUS(status): -2;
}

static int check_events(int fd, int timeo) {
	struct pollfd pfd;

	fprintf(stdout, "Checking events for fd %d\n", fd);
	memset(&pfd, 0, sizeof(pfd));
	pfd.fd = fd;
	pfd.events = POLLIN | POLLOUT;
	if (poll(&pfd, 1, timeo) < 0) {
		perror("poll()");
		return 0;
	}
	if (pfd.revents & POLLIN)
		fprintf(stdout, "\tPOLLIN\n");
	if (pfd.revents & POLLOUT)
		fprintf(stdout, "\tPOLLOUT\n");
	if (pfd.revents & POLLERR)
		fprintf(stdout, "\tPOLLERR\n");
	if (pfd.revents & POLLHUP)
		fprintf(stdout, "\tPOLLHUP\n");
	if (pfd.revents & POLLRDHUP)
		fprintf(stdout, "\tPOLLRDHUP\n");

	return pfd.revents;
}

static int epoll_test_tty(void *data) {
	int epfd, ifd = fileno(stdin), res;
	struct epoll_event evt;

	if (check_events(ifd, 0) != POLLOUT) {
		fprintf(stderr, "Something is cooking on STDIN (%d)\n", ifd);
		return 1;
	}
	epfd = xepoll_create(1);
	fprintf(stdout, "Created epoll fd (%d)\n", epfd);
	memset(&evt, 0, sizeof(evt));
	evt.events = EPOLLIN;
	xepoll_ctl(epfd, EPOLL_CTL_ADD, ifd, &evt);
	if (check_events(epfd, 0) & POLLIN) {
		res = epoll_wait(epfd, &evt, 1, 0);
		if (res == 0) {
			fprintf(stderr, "Epoll fd (%d) is ready when it shouldn't!\n",
				epfd);
			return 2;
		}
	}

	return 0;
}

static int epoll_wakeup_chain(void *data) {
	struct epoll_test_cfg *tcfg = data;
	int i, res, epfd, bfd, nfd, pfds[2];
	pid_t pid;
	struct epoll_event evt;

	memset(&evt, 0, sizeof(evt));
	evt.events = EPOLLIN;

	epfd = bfd = xepoll_create(1);

	for (i = 0; i < tcfg->size; i++) {
		nfd = xepoll_create(1);
		xepoll_ctl(bfd, EPOLL_CTL_ADD, nfd, &evt);
		bfd = nfd;
	}
	xpipe(pfds);
	if (tcfg->flags & EPOLL_TF_LOOP)
	{
		xepoll_ctl(bfd, EPOLL_CTL_ADD, epfd, &evt);
		/*
		 * If we're testing for loop, we want that the wakeup
		 * triggered by the write to the pipe done in the child
		 * process, triggers a fake event. So we add the pipe
		 * read size with EPOLLOUT events. This will trigger
		 * an addition to the ready-list, but no real events
		 * will be there. The the epoll kernel code will proceed
		 * in calling f_op->poll() of the epfd, triggering the
		 * loop we want to test.
		 */
		evt.events = EPOLLOUT;
	}
	xepoll_ctl(bfd, EPOLL_CTL_ADD, pfds[0], &evt);

	/*
	 * The pipe write must come after the poll(2) call inside
	 * check_events(). This tests the nested wakeup code in
	 * fs/eventpoll.c:ep_poll_safewake()
	 * By having the check_events() (hence poll(2)) happens first,
	 * we have poll wait queue filled up, and the write(2) in the
	 * child will trigger the wakeup chain.
	 */
	if ((pid = xfork()) == 0) {
		sleep(1);
		write(pfds[1], "w", 1);
		exit(0);
	}

	res = check_events(epfd, 2000) & POLLIN;

	if (waitpid(pid, NULL, 0) != pid) {
		perror("waitpid");
		return -1;
	}

	return res;
}

static int epoll_poll_chain(void *data) {
	struct epoll_test_cfg *tcfg = data;
	int i, res, epfd, bfd, nfd, pfds[2];
	pid_t pid;
	struct epoll_event evt;

	memset(&evt, 0, sizeof(evt));
	evt.events = EPOLLIN;

	epfd = bfd = xepoll_create(1);

	for (i = 0; i < tcfg->size; i++) {
		nfd = xepoll_create(1);
		xepoll_ctl(bfd, EPOLL_CTL_ADD, nfd, &evt);
		bfd = nfd;
	}
	xpipe(pfds);
	if (tcfg->flags & EPOLL_TF_LOOP)
	{
		xepoll_ctl(bfd, EPOLL_CTL_ADD, epfd, &evt);
		/*
		 * If we're testing for loop, we want that the wakeup
		 * triggered by the write to the pipe done in the child
		 * process, triggers a fake event. So we add the pipe
		 * read size with EPOLLOUT events. This will trigger
		 * an addition to the ready-list, but no real events
		 * will be there. The the epoll kernel code will proceed
		 * in calling f_op->poll() of the epfd, triggering the
		 * loop we want to test.
		 */
		evt.events = EPOLLOUT;
	}
	xepoll_ctl(bfd, EPOLL_CTL_ADD, pfds[0], &evt);

	/*
	 * The pipe write mush come before the poll(2) call inside
	 * check_events(). This tests the nested f_op->poll calls code in
	 * fs/eventpoll.c:ep_eventpoll_poll()
	 * By having the pipe write(2) happen first, we make the kernel
	 * epoll code to load the ready lists, and the following poll(2)
	 * done inside check_events() will test nested poll code in
	 * ep_eventpoll_poll().
	 */
	if ((pid = xfork()) == 0) {
		write(pfds[1], "w", 1);
		exit(0);
	}
	sleep(1);
	res = check_events(epfd, 1000) & POLLIN;

	if (waitpid(pid, NULL, 0) != pid) {
		perror("waitpid");
		return -1;
	}

	return res;
}

int main(int ac, char **av) {
	int error;
	struct epoll_test_cfg tcfg;

	fprintf(stdout, "\n********** Testing TTY events\n");
	error = run_forked_proc(epoll_test_tty, NULL);
	fprintf(stdout, error == 0 ?
		"********** OK\n": "********** FAIL (%d)\n", error);

	tcfg.size = 3;
	tcfg.flags = 0;
	fprintf(stdout, "\n********** Testing short wakeup chain\n");
	error = run_forked_proc(epoll_wakeup_chain, &tcfg);
	fprintf(stdout, error == POLLIN ?
		"********** OK\n": "********** FAIL (%d)\n", error);

	tcfg.size = EPOLL_MAX_CHAIN;
	tcfg.flags = 0;
	fprintf(stdout, "\n********** Testing long wakeup chain (HOLD ON)\n");
	error = run_forked_proc(epoll_wakeup_chain, &tcfg);
	fprintf(stdout, error == 0 ?
		"********** OK\n": "********** FAIL (%d)\n", error);

	tcfg.size = 3;
	tcfg.flags = 0;
	fprintf(stdout, "\n********** Testing short poll chain\n");
	error = run_forked_proc(epoll_poll_chain, &tcfg);
	fprintf(stdout, error == POLLIN ?
		"********** OK\n": "********** FAIL (%d)\n", error);

	tcfg.size = EPOLL_MAX_CHAIN;
	tcfg.flags = 0;
	fprintf(stdout, "\n********** Testing long poll chain (HOLD ON)\n");
	error = run_forked_proc(epoll_poll_chain, &tcfg);
	fprintf(stdout, error == 0 ?
		"********** OK\n": "********** FAIL (%d)\n", error);

	tcfg.size = 3;
	tcfg.flags = EPOLL_TF_LOOP;
	fprintf(stdout, "\n********** Testing loopy wakeup chain (HOLD ON)\n");
	error = run_forked_proc(epoll_wakeup_chain, &tcfg);
	fprintf(stdout, error == 0 ?
		"********** OK\n": "********** FAIL (%d)\n", error);

	tcfg.size = 3;
	tcfg.flags = EPOLL_TF_LOOP;
	fprintf(stdout, "\n********** Testing loopy poll chain (HOLD ON)\n");
	error = run_forked_proc(epoll_poll_chain, &tcfg);
	fprintf(stdout, error == 0 ?
		"********** OK\n": "********** FAIL (%d)\n", error);


	return 0;
}


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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-27 20:54 [patch 1/2] epoll fix own poll() Davide Libenzi
  2009-01-27 20:59 ` Davide Libenzi
@ 2009-01-29  8:01 ` Andrew Morton
  2009-01-29 18:16   ` Davide Libenzi
  1 sibling, 1 reply; 13+ messages in thread
From: Andrew Morton @ 2009-01-29  8:01 UTC (permalink / raw
  To: Davide Libenzi; +Cc: Linux Kernel Mailing List, Pavel Pisa

On Tue, 27 Jan 2009 12:54:38 -0800 (PST) Davide Libenzi <davidel@xmailserver.org> wrote:

> 
> The follwing patch fixes a bug inside the epoll's f_op->poll() code, that
> returns POLLIN even though there are no actual ready monitored fds.
> The bug shows up if you add an epoll fd inside another fd container (poll,
> select, epoll).
> The problem is that callback-based wake ups used by epoll, does not carry
> (patches will follow, to fix this) any information about the events
> that actually happened. So the callback code, since it can't call the file*
> ->poll() inside the callback, chains the file* into a ready-list. So, suppose
> you added an fd with EPOLLOUT only, and some data shows up on the fd, the
> file* mapped by the fd will be added into the ready-list (via wakeup
> callback). During normal epoll_wait() use, this condition is sorted out
> at the time we're actually able to call the file*'s f_op->poll().
> Inside the old epoll's f_op->poll() though, only a quick check
> !list_empty(ready-list) was performed, and this could have led to reporting
> POLLIN even though no ready fds would show up at a following epoll_wait().
> In order to correctly report the ready status for an epoll fd, the ready-list
> must be checked to see if any really available fd+event would be ready in
> a following epoll_wait().
> Operation (calling f_op->poll() from inside f_op->poll()) that, like wake ups,
> must be handled with care because of the fact that epoll fds can be added
> to other epoll fds.

One reason why I break up paragraphs like this one when cleaning up
changelogs is so that I can actually read them :(

>  fs/eventpoll.c |  510 +++++++++++++++++++++++++++++++++------------------------
>  1 file changed, 304 insertions(+), 206 deletions(-)

Holy cow man, this patch is HUGE!  I don't have a clue what it does nor
how it does it.  I'd be somewhat scared to merge it into 2.6.29.  How
serious is this bug?

Please use checkpatch?  The patch attempts to add a large amount of
crap, must notably lots of lines which for some reason start with
space-space-tab-tab-tab.  I doubt if you meant to do that (editor brain
damage), and checkpatch's main purpose is to tell you about things
which you didn't mean to do.

Minor nitpicks follow:

>
> ...
>
> +static int ep_scan_ready_list(struct eventpoll *ep,
> +			      int (*sproc)(struct eventpoll *,
> +					   struct list_head *, void *),
> +			      void *priv)
> +{
> +	int error, pwake = 0;
> +	unsigned long flags;
> +	struct epitem *epi, *nepi;
> +	struct list_head txlist;
> +
> +	INIT_LIST_HEAD(&txlist);

Could use

	LIST_HEAD(tx_list);

here

> +	/*
> +	 * We need to lock this because we could be hit by
> +	 * eventpoll_release_file() and epoll_ctl(EPOLL_CTL_DEL).
> +	 */
> +	mutex_lock(&ep->mtx);
> +
> +	/*
> +	 * 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.
> +	 */
> +	spin_lock_irqsave(&ep->lock, flags);
> +	list_splice(&ep->rdllist, &txlist);
> +	INIT_LIST_HEAD(&ep->rdllist);

list_splice_init()?

> +	ep->ovflist = NULL;
> +	spin_unlock_irqrestore(&ep->lock, flags);
> +
> +	/*
> +	 * Now call the callback function.
> +	 */
> +	error = (*sproc)(ep, &txlist, priv);
> +
> +	spin_lock_irqsave(&ep->lock, flags);
> +	/*
> +	 * 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 = 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->rdllink))
> +			list_add_tail(&epi->rdllink, &ep->rdllist);
> +	}
> +	/*
> +	 * 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.
> +	 */
> +	ep->ovflist = EP_UNACTIVE_PTR;
> +
> +	/*
> +	 * Quickly re-inject items left on "txlist".
> +	 */
> +	list_splice(&txlist, &ep->rdllist);
> +
> +	if (!list_empty(&ep->rdllist)) {

Some places the code uses ep_is_linked(&ep->rdllist), other places it
uses open-coded list_empty().

ep_is_linked() is a fairly poor helper, really - it could be passed any
list_head at all.  I think that it would be better to do

static inline int ep_is_linked(struct epitem *ep)
{
	return !list_empty(&ep->rdllist);
}

and then use this consistently.

> +		/*
> +		 * Wake up (if active) both the eventpoll wait list and the ->poll()
> +		 * wait list (delayed after we release the lock).
> +		 */
> +		if (waitqueue_active(&ep->wq))
> +			wake_up_locked(&ep->wq);
> +		if (waitqueue_active(&ep->poll_wait))
> +			pwake++;
> +	}
> +	spin_unlock_irqrestore(&ep->lock, flags);
> +
> +	mutex_unlock(&ep->mtx);
> +
> +	/* We have to call this outside the lock */
> +	if (pwake)
> +		ep_poll_safewake(&ep->poll_wait);
> +
> +	return error;
> +}
>
> ...
>
> +static int ep_read_events_proc(struct eventpoll *ep, struct list_head *head, void *priv)
> +{
> +	struct epitem *epi, *tmp;
> +
> +	list_for_each_entry_safe(epi, tmp, head, rdllink) {
> +		if (epi->ffd.file->f_op->poll(epi->ffd.file, NULL) &
> +		    epi->event.events)
> +			return POLLIN | POLLRDNORM;
> +		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.
> +			 */
> +			list_del_init(&epi->rdllink);

Please use braces around the comment and code when doing this. 
Otherwise readers can lose track of the fact that we're in a
single-statement leg of an `if' here.

> +	}
> +
> +	return 0;
> +}
> +
>
> ...
>
> -is_linked:
>  	/*
>  	 * Wake up ( if active ) both the eventpoll wait list and the ->poll()
>  	 * wait list.
> @@ -703,7 +863,7 @@
>  
>  	/* We have to call this outside the lock */
>  	if (pwake)
> -		ep_poll_safewake(&psw, &ep->poll_wait);
> +		ep_poll_safewake(&ep->poll_wait);
>  
>  	return 1;
>  }
> @@ -725,10 +885,9 @@
>  		add_wait_queue(whead, &pwq->wait);
>  		list_add_tail(&pwq->llink, &epi->pwqlist);
>  		epi->nwait++;
> -	} else {
> +	} else
>  		/* We have to signal that an error occurred */
>  		epi->nwait = -1;
> -	}

Please put the braces back?  The `if' clause has them, and most people
prefer that the `else' clause have them too.  Plus they nicely enclose
the comment as well, as described above.

>  }
>  
>
> ...
>


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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-29  8:01 ` Andrew Morton
@ 2009-01-29 18:16   ` Davide Libenzi
  2009-01-29 18:27     ` Andrew Morton
  0 siblings, 1 reply; 13+ messages in thread
From: Davide Libenzi @ 2009-01-29 18:16 UTC (permalink / raw
  To: Andrew Morton; +Cc: Linux Kernel Mailing List, Pavel Pisa

On Thu, 29 Jan 2009, Andrew Morton wrote:

> >  fs/eventpoll.c |  510 +++++++++++++++++++++++++++++++++------------------------
> >  1 file changed, 304 insertions(+), 206 deletions(-)
> 
> Holy cow man, this patch is HUGE!  I don't have a clue what it does nor
> how it does it.  I'd be somewhat scared to merge it into 2.6.29.  How
> serious is this bug?

It is a 3 in a scale of 5. The reason the patch is HUGE is because the 
epoll ->poll() code now has to perform an operation similar to what was 
performing in epoll_wait(), and under the same constraints (check out for 
recursions and too long nesting chains) that were checked in the wakeups.
So instead of duplicating the code, I made the two core operations such 
that they get a function pointer for the core operation they have to 
perform. That required some code movement.



> Please use checkpatch?  The patch attempts to add a large amount of
> crap, must notably lots of lines which for some reason start with
> space-space-tab-tab-tab.  I doubt if you meant to do that (editor brain
> damage), and checkpatch's main purpose is to tell you about things
> which you didn't mean to do.

I know why. Don't ask :)


> > +{
> > +	int error, pwake = 0;
> > +	unsigned long flags;
> > +	struct epitem *epi, *nepi;
> > +	struct list_head txlist;
> > +
> > +	INIT_LIST_HEAD(&txlist);
> 
> Could use
> 
> 	LIST_HEAD(tx_list);

ACK


> > +	list_splice(&ep->rdllist, &txlist);
> > +	INIT_LIST_HEAD(&ep->rdllist);
> 
> list_splice_init()?

ACK



> Some places the code uses ep_is_linked(&ep->rdllist), other places it
> uses open-coded list_empty().
> 
> ep_is_linked() is a fairly poor helper, really - it could be passed any
> list_head at all.  I think that it would be better to do
> 
> static inline int ep_is_linked(struct epitem *ep)
> {
> 	return !list_empty(&ep->rdllist);
> }
> 
> and then use this consistently.

Seems they're used the same, but they aren't. There are two different 
entities, even though characterized by the same structure. One are the 
lists head itself (like ep->rdllist), that is a list, checked with list_empty, 
the others are the elements chained to the list (like epi->rdllink), 
checked with the helper. 



> > +		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.
> > +			 */
> > +			list_del_init(&epi->rdllink);
> 
> Please use braces around the comment and code when doing this. 
> Otherwise readers can lose track of the fact that we're in a
> single-statement leg of an `if' here.

ACK


> > -	} else {
> > +	} else
> >  		/* We have to signal that an error occurred */
> >  		epi->nwait = -1;
> > -	}
> 
> Please put the braces back?  The `if' clause has them, and most people
> prefer that the `else' clause have them too.  Plus they nicely enclose
> the comment as well, as described above.

ACK


You always confuse me with your comments. Before you comment, then you 
merge w/out giving me time to change.
Would you like the updated patches?



- Davide



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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-29 18:16   ` Davide Libenzi
@ 2009-01-29 18:27     ` Andrew Morton
  2009-01-29 18:32       ` Davide Libenzi
  0 siblings, 1 reply; 13+ messages in thread
From: Andrew Morton @ 2009-01-29 18:27 UTC (permalink / raw
  To: Davide Libenzi; +Cc: Linux Kernel Mailing List, Pavel Pisa

On Thu, 29 Jan 2009 10:16:31 -0800 (PST) Davide Libenzi <davidel@xmailserver.org> wrote:

> On Thu, 29 Jan 2009, Andrew Morton wrote:
> 
> > >  fs/eventpoll.c |  510 +++++++++++++++++++++++++++++++++------------------------
> > >  1 file changed, 304 insertions(+), 206 deletions(-)
> > 
> > Holy cow man, this patch is HUGE!  I don't have a clue what it does nor
> > how it does it.  I'd be somewhat scared to merge it into 2.6.29.  How
> > serious is this bug?
> 
> It is a 3 in a scale of 5. The reason the patch is HUGE is because the 
> epoll ->poll() code now has to perform an operation similar to what was 
> performing in epoll_wait(), and under the same constraints (check out for 
> recursions and too long nesting chains) that were checked in the wakeups.
> So instead of duplicating the code, I made the two core operations such 
> that they get a function pointer for the core operation they have to 
> perform. That required some code movement.
> 

But which kernel version are you looking to get this merged into?

> 
> You always confuse me with your comments. Before you comment, then you 
> merge w/out giving me time to change.
>

yeah, I often do that.  If I see nothing fatal in the patch I'll merge
it so that it gets a bit of testing and so that it doesn't get
forgotten about.  Especially if it's a bugfix.

Sometimes I'll send a reply and the originator goes to sleep for weeks,
and stuff could even get lost.  I hate losing stuff.  I'll mark the patch
as "needs an update" in the series file so I don't accidentally merge it
upstream.

> Would you like the updated patches?

Sure, when convenient.

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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-29 18:27     ` Andrew Morton
@ 2009-01-29 18:32       ` Davide Libenzi
  2009-01-29 18:37         ` Andrew Morton
  0 siblings, 1 reply; 13+ messages in thread
From: Davide Libenzi @ 2009-01-29 18:32 UTC (permalink / raw
  To: Andrew Morton; +Cc: Linux Kernel Mailing List, Pavel Pisa

On Thu, 29 Jan 2009, Andrew Morton wrote:

> But which kernel version are you looking to get this merged into?

The limit reworking one, ASAP. The epoll poll fix, dunno.
I'd like to see it staged in your tree for a bit. How much timeframe 
difference are we looking at, from .29 to .30?


> > Would you like the updated patches?
> 
> Sure, when convenient.

OK


- Davide



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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-29 18:32       ` Davide Libenzi
@ 2009-01-29 18:37         ` Andrew Morton
  2009-01-29 18:43           ` Davide Libenzi
  2009-01-30  2:29           ` Pavel Pisa
  0 siblings, 2 replies; 13+ messages in thread
From: Andrew Morton @ 2009-01-29 18:37 UTC (permalink / raw
  To: Davide Libenzi; +Cc: Linux Kernel Mailing List, Pavel Pisa

On Thu, 29 Jan 2009 10:32:38 -0800 (PST) Davide Libenzi <davidel@xmailserver.org> wrote:

> On Thu, 29 Jan 2009, Andrew Morton wrote:
> 
> > But which kernel version are you looking to get this merged into?
> 
> The limit reworking one, ASAP. The epoll poll fix, dunno.
> I'd like to see it staged in your tree for a bit. How much timeframe 
> difference are we looking at, from .29 to .30?

The usual - 2.5 months(ish)

What is the impact of the bug which this patch fixes?


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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-29 18:37         ` Andrew Morton
@ 2009-01-29 18:43           ` Davide Libenzi
  2009-01-30  2:29           ` Pavel Pisa
  1 sibling, 0 replies; 13+ messages in thread
From: Davide Libenzi @ 2009-01-29 18:43 UTC (permalink / raw
  To: Andrew Morton; +Cc: Linux Kernel Mailing List, Pavel Pisa

On Thu, 29 Jan 2009, Andrew Morton wrote:

> On Thu, 29 Jan 2009 10:32:38 -0800 (PST) Davide Libenzi <davidel@xmailserver.org> wrote:
> 
> > On Thu, 29 Jan 2009, Andrew Morton wrote:
> > 
> > > But which kernel version are you looking to get this merged into?
> > 
> > The limit reworking one, ASAP. The epoll poll fix, dunno.
> > I'd like to see it staged in your tree for a bit. How much timeframe 
> > difference are we looking at, from .29 to .30?
> 
> The usual - 2.5 months(ish)
> 
> What is the impact of the bug which this patch fixes?

I think we can move the poll-fix (and definitely the debug-nuke-fix) to .30.
The bug happens when you add an epoll fd inside another wait container 
(epoll, poll, select). You may get a wakeup, to only find out later that 
epoll_wait() has no event for you. The bug has been there since day-1, and 
it has only recently been noticed.
Since it is quite some coe change, I'd like it to stage in your toxic tree 
for a while :)


- Davide



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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-29 18:37         ` Andrew Morton
  2009-01-29 18:43           ` Davide Libenzi
@ 2009-01-30  2:29           ` Pavel Pisa
  2009-01-30  2:52             ` Davide Libenzi
  2009-01-30  7:41             ` Davide Libenzi
  1 sibling, 2 replies; 13+ messages in thread
From: Pavel Pisa @ 2009-01-30  2:29 UTC (permalink / raw
  To: Andrew Morton; +Cc: Davide Libenzi, Linux Kernel Mailing List

On Thursday 29 January 2009 19:37:15 Andrew Morton wrote:
> On Thu, 29 Jan 2009 10:32:38 -0800 (PST) Davide Libenzi <davidel@xmailserver.org> wrote:
> > On Thu, 29 Jan 2009, Andrew Morton wrote:
> > > But which kernel version are you looking to get this merged into?
> >
> > The limit reworking one, ASAP. The epoll poll fix, dunno.
> > I'd like to see it staged in your tree for a bit. How much timeframe
> > difference are we looking at, from .29 to .30?
>
> The usual - 2.5 months(ish)
>
> What is the impact of the bug which this patch fixes?

Hello Andrew and Davide

the problem is caused by epoll used in another event waiting mechanism.
In my problem appeared only for epoll cascaded to epoll and slight
changes in code resulted in problem vanishing. It is probably timing
sensitive. Davide analyzed, that it can appear even for cascading of
epoll into other mechanisms - poll, select etc.

The situations which could result in the problem

The idea to cascade epoll into other mechanisms comes to mind
in many cases. When I have been working on my event processing
solution and finding how to live in peace with GLIB (and through it with
Qt) I have found, that others have headed same direction.
The concept to cover more corresponding sources under one FD
is promissing to sole other task too (prioritization found
in Mikhail Zabaluev's GPollSet implementation for GLIB for example).

There is my personal feeling about problem severity:

The severity is increased by fact, that epoll cascading is documented
and in fact offered as feature by epoll manpage. That there exists
more attempts to use it. The problem in typical case appears
as random stuck of userspace code in busyloop without any
error indication from system calls.

  do {
      select/poll/epoll_wait on epfd
      /* above line returns imediately because epoll->poll() indicates ready event*/
      if there is event on epfd {
         epoll_wait ( epfd, timeout = 0) /* to find which event is ready */
         /* but no ready event is reported and cycle starts again */
      }
  } while(1)

So if there exists applications using epoll, they could waste sometimes
most of CPU time without without any visible error indication. 
The most critical problem is, that even epoll_wait() on epoll set, which
reports falsely ready condition, doe not clear that state for outer
loop and event waiting mechanism. So in theory, some patch with
smaller scale and impact which only ensures, that event is not falsely
reported even after epoll_wait() call would be enough to save system
from busyloop load. The false wakeup and wasted call to epoll_wait ()
with no event returned is not so big problem.

On the other hand, most of today applications are based on GLIB (poll),
Qt (for pure Qt select) or libevent (not cascaded poll/epoll), which all are
not prone to this problem. Even for my intended use, my code can
work without cascading and if cascading is required, then standard
poll and  then moving of events triggers into GLIB loop is enough
for 10 to 100 FDs. So actual real severity is not so high.

I am happy that Davide has found fix to the problem, but even if the
fix gets into 2.6.29 there would be older kernels for years there
and use of epoll cascading would be problem. So I see as most
important, that information about problem is known and does not
surprise others. It would be great, if there is found safe way,
how to ensure even without fix to revive from ill situation
by some userspace action on older kernels.
I do not see yet, why call to epoll_wait() does not to clean
"rdllist" on unpatched kernel. 

epoll_wait() unconditionally calls  ep_poll(), the "rdllist" is not empty
(cheked by previous ep_eventpoll_poll() call), so ep_send_events()
has to be called. It moves all events from "rdllist" onto  "txlist"
and reports events confirmed by individual poll() calls to user.
There could be non signaling events in the "rdlist" left there from
previous scan for level triggering events or in case, that condition
is reset by some other intervention before  ep_eventpoll_poll().
All this is not problem yet, it would result in one abundant call to
epoll_wait() which returns 0. No problem. The EPOLLET events
are moved from "rdlist", others (level triggered events) signalling
are queued on "rdlist" again. If userspace does its duty, they would
be removed during next call. This all seems to suggests, that one
abundant epoll_wait() would get things back on the right path.
All critical processing (moving between lists) seems to be under
held of spinlock, so possibility for bugs is low. I have not yet analyzed
overflow list which complicates things.

I am not clever enough, to see, where is the problem, that situation
is not stabilized. Can you teach me Davide, please?

Thanks again much for help,

                                  Pavel













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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-30  2:29           ` Pavel Pisa
@ 2009-01-30  2:52             ` Davide Libenzi
  2009-01-30  7:41             ` Davide Libenzi
  1 sibling, 0 replies; 13+ messages in thread
From: Davide Libenzi @ 2009-01-30  2:52 UTC (permalink / raw
  To: Pavel Pisa; +Cc: Andrew Morton, Linux Kernel Mailing List

On Fri, 30 Jan 2009, Pavel Pisa wrote:

> So if there exists applications using epoll, they could waste sometimes
> most of CPU time without without any visible error indication. 
> The most critical problem is, that even epoll_wait() on epoll set, which
> reports falsely ready condition, doe not clear that state for outer
> loop and event waiting mechanism. So in theory, some patch with
> smaller scale and impact which only ensures, that event is not falsely
> reported even after epoll_wait() call would be enough to save system
> from busyloop load. The false wakeup and wasted call to epoll_wait ()
> with no event returned is not so big problem.
> 
> On the other hand, most of today applications are based on GLIB (poll),
> Qt (for pure Qt select) or libevent (not cascaded poll/epoll), which all are
> not prone to this problem. Even for my intended use, my code can
> work without cascading and if cascading is required, then standard
> poll and  then moving of events triggers into GLIB loop is enough
> for 10 to 100 FDs. So actual real severity is not so high.
> 
> I am happy that Davide has found fix to the problem, but even if the
> fix gets into 2.6.29 there would be older kernels for years there
> and use of epoll cascading would be problem. So I see as most
> important, that information about problem is known and does not
> surprise others. It would be great, if there is found safe way,
> how to ensure even without fix to revive from ill situation
> by some userspace action on older kernels.
> I do not see yet, why call to epoll_wait() does not to clean
> "rdllist" on unpatched kernel. 

Epoll cleans, even on older kernels, the rdllist from spurious events, 
upon epoll_wait(). If later on you get another spurious event, you fall 
into the same condition. Or, can you send a minimal code snippet that 
shows this not being true for older kernels?


- Davide



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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-30  2:29           ` Pavel Pisa
  2009-01-30  2:52             ` Davide Libenzi
@ 2009-01-30  7:41             ` Davide Libenzi
  2009-01-30  9:42               ` Pavel Pisa
  1 sibling, 1 reply; 13+ messages in thread
From: Davide Libenzi @ 2009-01-30  7:41 UTC (permalink / raw
  To: Pavel Pisa; +Cc: Andrew Morton, Linux Kernel Mailing List

On Fri, 30 Jan 2009, Pavel Pisa wrote:

> epoll_wait() unconditionally calls  ep_poll(), the "rdllist" is not empty
> (cheked by previous ep_eventpoll_poll() call), so ep_send_events()
> has to be called. It moves all events from "rdllist" onto  "txlist"
> and reports events confirmed by individual poll() calls to user.
> There could be non signaling events in the "rdlist" left there from
> previous scan for level triggering events or in case, that condition
> is reset by some other intervention before  ep_eventpoll_poll().
> All this is not problem yet, it would result in one abundant call to
> epoll_wait() which returns 0. No problem. The EPOLLET events
> are moved from "rdlist", others (level triggered events) signalling
> are queued on "rdlist" again. If userspace does its duty, they would
> be removed during next call. This all seems to suggests, that one
> abundant epoll_wait() would get things back on the right path.
> All critical processing (moving between lists) seems to be under
> held of spinlock, so possibility for bugs is low. I have not yet analyzed
> overflow list which complicates things.
> 
> I am not clever enough, to see, where is the problem, that situation
> is not stabilized. Can you teach me Davide, please?

Sorry, I missed part of the email. Yes, if revents is zero in the 
send-lood, I don't see how files can be added back to the ready-list.
Please, create a very small test program that exhibit such behaviour ...



- Davide



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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-30  7:41             ` Davide Libenzi
@ 2009-01-30  9:42               ` Pavel Pisa
  2009-01-30 18:40                 ` Davide Libenzi
  0 siblings, 1 reply; 13+ messages in thread
From: Pavel Pisa @ 2009-01-30  9:42 UTC (permalink / raw
  To: Davide Libenzi; +Cc: Andrew Morton, Linux Kernel Mailing List

On Friday 30 January 2009 08:41:53 Davide Libenzi wrote:
> On Fri, 30 Jan 2009, Pavel Pisa wrote:
> > epoll_wait() unconditionally calls  ep_poll(), the "rdllist" is not empty
> > (cheked by previous ep_eventpoll_poll() call), so ep_send_events()
> > has to be called. It moves all events from "rdllist" onto  "txlist"
> > and reports events confirmed by individual poll() calls to user.
> > There could be non signaling events in the "rdlist" left there from
> > previous scan for level triggering events or in case, that condition
> > is reset by some other intervention before  ep_eventpoll_poll().
> > All this is not problem yet, it would result in one abundant call to
> > epoll_wait() which returns 0. No problem. The EPOLLET events
> > are moved from "rdlist", others (level triggered events) signalling
> > are queued on "rdlist" again. If userspace does its duty, they would
> > be removed during next call. This all seems to suggests, that one
> > abundant epoll_wait() would get things back on the right path.
> > All critical processing (moving between lists) seems to be under
> > held of spinlock, so possibility for bugs is low. I have not yet analyzed
> > overflow list which complicates things.
> >
> > I am not clever enough, to see, where is the problem, that situation
> > is not stabilized. Can you teach me Davide, please?
>
> Sorry, I missed part of the email. Yes, if revents is zero in the
> send-lood, I don't see how files can be added back to the ready-list.
> Please, create a very small test program that exhibit such behaviour ...

Hello Davide,

I have seen the reason exactly when I have sent the e-mail
and switched computer off at the night.

The reason is simple and it is caused by debugging print in each
loop in my application and by way how poll concept is implemented
in the kernel.

There is continuation of my analyze. The event is correctly cleaned
in ep_send_events but there is the culprit in my code, I have not
count before

+ add FD #0 into epoll set to be monitored for read event 
  do {
      select/poll/epoll_wait on epfd
      /* above line returns imediately because epoll->poll() indicates ready event*/
      if there is event on epfd {
         epoll_wait ( epfd, timeout = 0) /* to find which event is ready */
         /* but no ready event is reported and cycle starts again */
      }
+      debugging code prints something to FD #0
  } while(1)

The the concept how file_operations_xxx.poll(struct file *file, poll_table *wait)
 functions works, does not allow to distinguish for which conditions
callers waits. So there is only one list for IN/OUT/HUP etc events.
Even if the device/file/socked has three wait queues (one for these
waiting for IN, other for OUT and third third for IN|OUT),
there is no way to check callers intention so IN|OUT has to be
used unconditionally. Back to my code, when debug write is finished,
wait queue callback for FD #0 is called and this inserts epitem
back on "rdlist" and loop can continue for ever.

So the fault in this case is distributed between 
- my application (it should not perform any activity -
   debugging output - if there is no real noncascade event reported
   and no time expired or event event activated by other thread)
- poll concept with common wait queue which results
  in abundant wakeups and recheck need
- epoll implementation, which is not able to hide abundant wakes
  to not be seen from userspace

I have not fully analyzed updated code version, but I expect,
that it moves full "rdlist" checking into ep_eventpoll_poll().
This hides abundant wakes at price of checking groving
list of really signaled events for each call. It has advantage,
that it could remove entries which really stop to signal events
for other reasons, but rechecking again and again has performance
penalty.

So the ideal solution is to change whole kernel poll mechanism
to allow separate IN and OUT queues for all important poll() function
providers. The other solution is to provide mask of intentionally wait
for events as additional parameter in file_operations_xxx.poll
which would to use optionally three queue concept (IN, OUT, IN|OUT)
for high bandwidth devices and significantly reduce number of
unnecessary wakeups. 
 
As for epoll, you still need overflow list there, so the checking overhead
could be minimized, if all newly reported events are not put onto
"rdlist" directly, but onto some "potentionally_ready_list".
The ep_eventpoll_poll() should only call file.poll() for these epitems.
If the poll() reports ready, they should be moved to the final "rdlist".
If the "rdlist" is not empty, the IN event should be reported back.

Even in the actual poll scenario, the epoll overhead can be decreased
and ill cases can be eliminated.
The ep_poll() should call this processing of "potentionally_ready_list"
before check for "rdlist" before wait decision. In the current scenario,
same busy loop can be created even without cascading if there
is invoked unconditionally some operation on some of epoll set
monitored events which causes wakeup call but is then masked
by epitem events mask.

I have feeling, that I have seen problem report about some application
and epoll, which could be caused by this scenario somewhere.
But I cannot find it again.

So thanks for support and reply,

                    Pavel





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

* Re: [patch 1/2] epoll fix own poll()
  2009-01-30  9:42               ` Pavel Pisa
@ 2009-01-30 18:40                 ` Davide Libenzi
  0 siblings, 0 replies; 13+ messages in thread
From: Davide Libenzi @ 2009-01-30 18:40 UTC (permalink / raw
  To: Pavel Pisa; +Cc: Andrew Morton, Linux Kernel Mailing List

On Fri, 30 Jan 2009, Pavel Pisa wrote:

> I have seen the reason exactly when I have sent the e-mail
> and switched computer off at the night.
> 
> The reason is simple and it is caused by debugging print in each
> loop in my application and by way how poll concept is implemented
> in the kernel.

Yes, this is a known thing. Like I already told you yesterday, calling it 
a "spurious" event. So you code *is* generating a spurios event to the 
tty, with a write.
Unfortunately this is how Linux poll subsystem works. But things can be 
improved, w/out any revolution.
I already have patches to send Andrew-ward, that (gradually) uses the 
wakeup "key" to report event that really happened, during the wakeup call. 
This will allow the epoll callback to avoid epoll waiters wakeups, when 
events do not match the interest mask.


- Davide


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

end of thread, other threads:[~2009-01-30 18:41 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2009-01-27 20:54 [patch 1/2] epoll fix own poll() Davide Libenzi
2009-01-27 20:59 ` Davide Libenzi
2009-01-29  8:01 ` Andrew Morton
2009-01-29 18:16   ` Davide Libenzi
2009-01-29 18:27     ` Andrew Morton
2009-01-29 18:32       ` Davide Libenzi
2009-01-29 18:37         ` Andrew Morton
2009-01-29 18:43           ` Davide Libenzi
2009-01-30  2:29           ` Pavel Pisa
2009-01-30  2:52             ` Davide Libenzi
2009-01-30  7:41             ` Davide Libenzi
2009-01-30  9:42               ` Pavel Pisa
2009-01-30 18:40                 ` Davide Libenzi

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 read-only IMAP folder(s) and NNTP newsgroup(s).