Print this page
OS-5464 signalfd deadlock on pollwakeup
Reviewed by: Jerry Jelinek <jerry.jelinek@joyent.com>
Reviewed by: Robert Mustacchi <rm@joyent.com>
OS-5370 panic in signalfd
Reviewed by: Jerry Jelinek <jerry.jelinek@joyent.com>
Reviewed by: Robert Mustacchi <rm@joyent.com>
OS-3742 lxbrand add support for signalfd
OS-4382 remove obsolete brand hooks added during lx development
        
@@ -8,110 +8,94 @@
  * source.  A copy of the CDDL is also available via the Internet at
  * http://www.illumos.org/license/CDDL.
  */
 
 /*
- * Copyright 2015 Joyent, Inc.
+ * Copyright 2016 Joyent, Inc.
  */
 
 /*
  * Support for the signalfd facility, a Linux-borne facility for
  * file descriptor-based synchronous signal consumption.
  *
  * As described on the signalfd(3C) man page, the general idea behind these
  * file descriptors is that they can be used to synchronously consume signals
- * via the read(2) syscall. That capability already exists with the
- * sigwaitinfo(3C) function but the key advantage of signalfd is that, because
- * it is file descriptor based, poll(2) can be used to determine when signals
- * are available to be consumed.
+ * via the read(2) syscall.  While that capability already exists with the
+ * sigwaitinfo(3C) function, signalfd holds an advantage since it is file
+ * descriptor based: It is able use the event facilities (poll(2), /dev/poll,
+ * event ports) to notify interested parties when consumable signals arrive.
  *
- * The general implementation uses signalfd_state to hold both the signal set
- * and poll head for an open file descriptor. Because a process can be using
- * different sigfds with different signal sets, each signalfd_state poll head
- * can be thought of as an independent signal stream and the thread(s) waiting
- * on that stream will get poll notification when any signal in the
- * corresponding set is received.
+ * The signalfd lifecycle begins When a process opens /dev/signalfd.  A minor
+ * will be allocated for them along with an associated signalfd_state_t struct.
+ * It is there where the mask of desired signals resides.
  *
- * The sigfd_proc_state_t struct lives on the proc_t and maintains per-proc
- * state for function callbacks and data when the proc needs to do work during
- * signal delivery for pollwakeup.
+ * Reading from the signalfd is straightforward and mimics the kernel behavior
+ * for sigtimedwait().  Signals continue to live on either the proc's p_sig, or
+ * thread's t_sig, member.  During a read operation, those which match the mask
+ * are consumed so they are no longer pending.
  *
- * The read side of the implementation is straightforward and mimics the
- * kernel behavior for sigtimedwait(). Signals continue to live on either
- * the proc's p_sig, or thread's t_sig, member. Read consumes the signal so
- * that it is no longer pending.
+ * The poll side is more complex.  Every time a signal is delivered, all of the
+ * signalfds on the process need to be examined in order to pollwake threads
+ * waiting for signal arrival.
  *
- * The poll side is more complex since all of the sigfds on the process need
- * to be examined every time a signal is delivered to the process in order to
- * pollwake any thread waiting in poll for that signal.
+ * When a thread polling on a signalfd requires a pollhead, several steps must
+ * be taken to safely ensure the proper result.  A sigfd_proc_state_t is
+ * created for the calling process if it does not yet exist.  It is there where
+ * a list of sigfd_poll_waiter_t structures reside which associate pollheads to
+ * signalfd_state_t entries.  The sigfd_proc_state_t list is walked to find a
+ * sigfd_poll_waiter_t matching the signalfd_state_t which corresponds to the
+ * polled resource.  If one is found, it is reused.  Otherwise a new one is
+ * created, incrementing the refcount on the signalfd_state_t, and it is added
+ * to the sigfd_poll_waiter_t list.
  *
- * Because it is likely that a process will only be using one, or a few, sigfds,
- * but many total file descriptors, we maintain a list of sigfds which need
- * pollwakeup. The list lives on the proc's p_sigfd struct. In this way only
- * zero, or a few, of the state structs will need to be examined every time a
- * signal is delivered to the process, instead of having to examine all of the
- * file descriptors to find the state structs. When a state struct with a
- * matching signal set is found then pollwakeup is called.
+ * The complications imposed by fork(2) are why the pollhead is stored in the
+ * associated sigfd_poll_waiter_t instead of directly in the signalfd_state_t.
+ * More than one process can hold a reference to the signalfd at a time but
+ * arriving signals should wake only process-local pollers.  Additionally,
+ * signalfd_close is called only when the last referencing fd is closed, hiding
+ * occurrences of preceeding threads which released their references.  This
+ * necessitates reference counting on the signalfd_state_t so it is able to
+ * persist after close until all poll references have been cleansed.  Doing so
+ * ensures that blocked pollers which hold references to the signalfd_state_t
+ * will be able to do clean-up after the descriptor itself has been closed.
  *
- * The sigfd_list is self-cleaning; as signalfd_pollwake_cb is called, the list
- * will clear out on its own. There is an exit helper (signalfd_exit_helper)
- * which cleans up any remaining per-proc state when the process exits.
+ * When a signal arrives in a process polling on signalfd, signalfd_pollwake_cb
+ * is called via the pointer in sigfd_proc_state_t.  It will walk over the
+ * sigfd_poll_waiter_t entries present in the list, searching for any
+ * associated with a signalfd_state_t with a matching signal mask.  The
+ * approach of keeping the poller list in p_sigfd was chosen because a process
+ * is likely to use few signalfds relative to its total file descriptors.  It
+ * reduces the work required for each received signal.
  *
- * The main complexity with signalfd is the interaction of forking and polling.
- * This interaction is complex because now two processes have a fd that
- * references the same dev_t (and its associated signalfd_state), but signals
- * go to only one of those processes. Also, we don't know when one of the
- * processes closes its fd because our 'close' entry point is only called when
- * the last fd is closed (which could be by either process).
+ * When matching sigfd_poll_waiter_t entries are encountered in the poller list
+ * during signalfd_pollwake_cb, they are dispatched into signalfd_wakeq to
+ * perform the pollwake.  This is due to a lock ordering conflict between
+ * signalfd_poll and signalfd_pollwake_cb.  The former acquires
+ * pollcache_t`pc_lock before proc_t`p_lock.  The latter (via sigtoproc)
+ * reverses the order.  Defering the pollwake into a taskq means it can be
+ * performed without proc_t`p_lock held, avoiding the deadlock.
  *
- * Because the state struct is referenced by both file descriptors, and the
- * state struct represents a signal stream needing a pollwakeup, if both
- * processes were polling then both processes would get a pollwakeup when a
- * signal arrives for either process (that is, the pollhead is associated with
- * our dev_t so when a signal arrives the pollwakeup wakes up all waiters).
+ * The sigfd_list is self-cleaning; as signalfd_pollwake_cb is called, the list
+ * will clear out on its own.  Any remaining per-process state which remains
+ * will be cleaned up by the exit helper (signalfd_exit_helper).
  *
- * Fortunately this is not a common problem in practice, but the implementation
- * attempts to mitigate unexpected behavior. The typical behavior is that the
- * parent has been polling the signalfd (which is why it was open in the first
- * place) and the parent might have a pending signalfd_state (with the
- * pollhead) on its per-process sigfd_list. After the fork the child will
- * simply close that fd (among others) as part of the typical fork/close/exec
- * pattern. Because the child will never poll that fd, it will never get any
- * state onto its own sigfd_list (the child starts with a null list). The
- * intention is that the child sees no pollwakeup activity for signals unless
- * it explicitly reinvokes poll on the sigfd.
+ * The structures associated with signalfd state are designed to operate
+ * correctly across fork, but there is one caveat that applies.  Using
+ * fork-shared signalfd descriptors in conjuction with fork-shared caching poll
+ * descriptors (such as /dev/poll or event ports) will result in missed poll
+ * wake-ups.  This is caused by the pollhead identity of signalfd descriptors
+ * being dependent on the process they are polled from.  Because it has a
+ * thread-local cache, poll(2) is unaffected by this limitation.
  *
- * As background, there are two primary polling cases to consider when the
- * parent process forks:
- * 1) If any thread is blocked in poll(2) then both the parent and child will
- *    return from the poll syscall with EINTR. This means that if either
- *    process wants to re-poll on a sigfd then it needs to re-run poll and
- *    would come back in to the signalfd_poll entry point. The parent would
- *    already have the dev_t's state on its sigfd_list and the child would not
- *    have anything there unless it called poll again on its fd.
- * 2) If the process is using /dev/poll(7D) then the polling info is being
- *    cached by the poll device and the process might not currently be blocked
- *    on anything polling related. A subsequent DP_POLL ioctl will not invoke
- *    our signalfd_poll entry point again. Because the parent still has its
- *    sigfd_list setup, an incoming signal will hit our signalfd_pollwake_cb
- *    entry point, which in turn calls pollwake, and /dev/poll will do the
- *    right thing on DP_POLL. The child will not have a sigfd_list yet so the
- *    signal will not cause a pollwakeup. The dp code does its own handling for
- *    cleaning up its cache.
+ * Lock ordering:
  *
- * This leaves only one odd corner case. If the parent and child both use
- * the dup-ed sigfd to poll then when a signal is delivered to either process
- * there is no way to determine which one should get the pollwakeup (since
- * both processes will be queued on the same signal stream poll head). What
- * happens in this case is that both processes will return from poll, but only
- * one of them will actually have a signal to read. The other will return
- * from read with EAGAIN, or block. This case is actually similar to the
- * situation within a single process which got two different sigfd's with the
- * same mask (or poll on two fd's that are dup-ed). Both would return from poll
- * when a signal arrives but only one read would consume the signal and the
- * other read would fail or block. Applications which poll on shared fd's
- * cannot assume that a subsequent read will actually obtain data.
+ * 1. signalfd_lock
+ * 2. signalfd_state_t`sfd_lock
+ *
+ * 1. proc_t`p_lock (to walk p_sigfd)
+ * 2. signalfd_state_t`sfd_lock
+ * 2a. signalfd_lock (after sfd_lock is dropped, when sfd_count falls to 0)
  */
 
 #include <sys/ddi.h>
 #include <sys/sunddi.h>
 #include <sys/signalfd.h>
@@ -121,126 +105,188 @@
 #include <sys/stat.h>
 #include <sys/file.h>
 #include <sys/schedctl.h>
 #include <sys/id_space.h>
 #include <sys/sdt.h>
+#include <sys/brand.h>
+#include <sys/disp.h>
+#include <sys/taskq_impl.h>
 
 typedef struct signalfd_state signalfd_state_t;
 
 struct signalfd_state {
-        kmutex_t sfd_lock;                      /* lock protecting state */
-        pollhead_t sfd_pollhd;                  /* poll head */
+        list_node_t     sfd_list;               /* node in global list */
+        kmutex_t        sfd_lock;               /* protects fields below */
+        uint_t          sfd_count;              /* ref count */
+        boolean_t       sfd_valid;              /* valid while open */
         k_sigset_t sfd_set;                     /* signals for this fd */
-        signalfd_state_t *sfd_next;             /* next state on global list */
 };
 
+typedef struct sigfd_poll_waiter {
+        list_node_t             spw_list;
+        signalfd_state_t        *spw_state;
+        pollhead_t              spw_pollhd;
+        taskq_ent_t             spw_taskent;
+        short                   spw_pollev;
+} sigfd_poll_waiter_t;
+
 /*
- * Internal global variables.
+ * Protects global state in signalfd_devi, signalfd_minor, signalfd_softstate,
+ * and signalfd_state (including sfd_list field of members)
  */
-static kmutex_t         signalfd_lock;          /* lock protecting state */
+static kmutex_t         signalfd_lock;
 static dev_info_t       *signalfd_devi;         /* device info */
 static id_space_t       *signalfd_minor;        /* minor number arena */
 static void             *signalfd_softstate;    /* softstate pointer */
-static signalfd_state_t *signalfd_state;        /* global list of state */
+static list_t           signalfd_state;         /* global list of state */
+static taskq_t          *signalfd_wakeq;        /* pollwake event taskq */
 
-/*
- * If we don't already have an entry in the proc's list for this state, add one.
- */
+
 static void
-signalfd_wake_list_add(signalfd_state_t *state)
+signalfd_state_enter_locked(signalfd_state_t *state)
 {
-        proc_t *p = curproc;
-        list_t *lst;
-        sigfd_wake_list_t *wlp;
+        ASSERT(MUTEX_HELD(&state->sfd_lock));
+        ASSERT(state->sfd_count > 0);
+        VERIFY(state->sfd_valid == B_TRUE);
 
-        ASSERT(MUTEX_HELD(&p->p_lock));
-        ASSERT(p->p_sigfd != NULL);
+        state->sfd_count++;
+}
 
-        lst = &((sigfd_proc_state_t *)p->p_sigfd)->sigfd_list;
-        for (wlp = list_head(lst); wlp != NULL; wlp = list_next(lst, wlp)) {
-                if (wlp->sigfd_wl_state == state)
-                        break;
+static void
+signalfd_state_release(signalfd_state_t *state, boolean_t force_invalidate)
+{
+        mutex_enter(&state->sfd_lock);
+
+        if (force_invalidate) {
+                state->sfd_valid = B_FALSE;
         }
 
-        if (wlp == NULL) {
-                wlp = kmem_zalloc(sizeof (sigfd_wake_list_t), KM_SLEEP);
-                wlp->sigfd_wl_state = state;
-                list_insert_head(lst, wlp);
+        ASSERT(state->sfd_count > 0);
+        if (state->sfd_count == 1) {
+                VERIFY(state->sfd_valid == B_FALSE);
+                mutex_exit(&state->sfd_lock);
+                if (force_invalidate) {
+                        /*
+                         * The invalidation performed in signalfd_close is done
+                         * while signalfd_lock is held.
+                         */
+                        ASSERT(MUTEX_HELD(&signalfd_lock));
+                        list_remove(&signalfd_state, state);
+                } else {
+                        ASSERT(MUTEX_NOT_HELD(&signalfd_lock));
+                        mutex_enter(&signalfd_lock);
+                        list_remove(&signalfd_state, state);
+                        mutex_exit(&signalfd_lock);
         }
+                kmem_free(state, sizeof (*state));
+                return;
+        }
+        state->sfd_count--;
+        mutex_exit(&state->sfd_lock);
 }
 
-static void
-signalfd_wake_rm(list_t *lst, sigfd_wake_list_t *wlp)
+static sigfd_poll_waiter_t *
+signalfd_wake_list_add(sigfd_proc_state_t *pstate, signalfd_state_t *state)
 {
-        list_remove(lst, wlp);
-        kmem_free(wlp, sizeof (sigfd_wake_list_t));
-}
+        list_t *lst = &pstate->sigfd_list;
+        sigfd_poll_waiter_t *pw;
 
-static void
-signalfd_wake_list_rm(proc_t *p, signalfd_state_t *state)
-{
-        sigfd_wake_list_t *wlp;
-        list_t *lst;
+        for (pw = list_head(lst); pw != NULL; pw = list_next(lst, pw)) {
+                if (pw->spw_state == state)
+                        break;
+        }
 
-        ASSERT(MUTEX_HELD(&p->p_lock));
+        if (pw == NULL) {
+                pw = kmem_zalloc(sizeof (*pw), KM_SLEEP);
 
-        if (p->p_sigfd == NULL)
-                return;
+                mutex_enter(&state->sfd_lock);
+                signalfd_state_enter_locked(state);
+                pw->spw_state = state;
+                mutex_exit(&state->sfd_lock);
+                list_insert_head(lst, pw);
+        }
+        return (pw);
+}
 
-        lst = &((sigfd_proc_state_t *)p->p_sigfd)->sigfd_list;
-        for (wlp = list_head(lst); wlp != NULL; wlp = list_next(lst, wlp)) {
-                if (wlp->sigfd_wl_state == state) {
-                        signalfd_wake_rm(lst, wlp);
+static sigfd_poll_waiter_t *
+signalfd_wake_list_rm(sigfd_proc_state_t *pstate, signalfd_state_t *state)
+{
+        list_t *lst = &pstate->sigfd_list;
+        sigfd_poll_waiter_t *pw;
+
+        for (pw = list_head(lst); pw != NULL; pw = list_next(lst, pw)) {
+                if (pw->spw_state == state) {
                         break;
                 }
         }
 
-        if (list_is_empty(lst)) {
-                ((sigfd_proc_state_t *)p->p_sigfd)->sigfd_pollwake_cb = NULL;
-                list_destroy(lst);
-                kmem_free(p->p_sigfd, sizeof (sigfd_proc_state_t));
-                p->p_sigfd = NULL;
+        if (pw != NULL) {
+                list_remove(lst, pw);
+                pw->spw_state = NULL;
+                signalfd_state_release(state, B_FALSE);
         }
+
+        return (pw);
 }
 
 static void
 signalfd_wake_list_cleanup(proc_t *p)
 {
-        sigfd_wake_list_t *wlp;
+        sigfd_proc_state_t *pstate = p->p_sigfd;
+        sigfd_poll_waiter_t *pw;
         list_t *lst;
 
         ASSERT(MUTEX_HELD(&p->p_lock));
+        ASSERT(pstate != NULL);
 
-        ((sigfd_proc_state_t *)p->p_sigfd)->sigfd_pollwake_cb = NULL;
+        lst = &pstate->sigfd_list;
+        while ((pw = list_remove_head(lst)) != NULL) {
+                signalfd_state_t *state = pw->spw_state;
 
-        lst = &((sigfd_proc_state_t *)p->p_sigfd)->sigfd_list;
-        while (!list_is_empty(lst)) {
-                wlp = (sigfd_wake_list_t *)list_remove_head(lst);
-                kmem_free(wlp, sizeof (sigfd_wake_list_t));
+                pw->spw_state = NULL;
+                signalfd_state_release(state, B_FALSE);
+
+                pollwakeup(&pw->spw_pollhd, POLLERR);
+                pollhead_clean(&pw->spw_pollhd);
+                kmem_free(pw, sizeof (*pw));
         }
+        list_destroy(lst);
+
+        p->p_sigfd = NULL;
+        kmem_free(pstate, sizeof (*pstate));
 }
 
 static void
 signalfd_exit_helper(void)
 {
         proc_t *p = curproc;
-        list_t *lst;
 
-        /* This being non-null is the only way we can get here */
-        ASSERT(p->p_sigfd != NULL);
-
         mutex_enter(&p->p_lock);
-        lst = &((sigfd_proc_state_t *)p->p_sigfd)->sigfd_list;
-
         signalfd_wake_list_cleanup(p);
-        list_destroy(lst);
-        kmem_free(p->p_sigfd, sizeof (sigfd_proc_state_t));
-        p->p_sigfd = NULL;
         mutex_exit(&p->p_lock);
 }
 
 /*
+ * Perform pollwake for a sigfd_poll_waiter_t entry.
+ * Thanks to the strict and conflicting lock orders required for signalfd_poll
+ * (pc_lock before p_lock) and signalfd_pollwake_cb (p_lock before pc_lock),
+ * this is relegated to a taskq to avoid deadlock.
+ */
+static void
+signalfd_wake_task(void *arg)
+{
+        sigfd_poll_waiter_t *pw = arg;
+        signalfd_state_t *state = pw->spw_state;
+
+        pw->spw_state = NULL;
+        signalfd_state_release(state, B_FALSE);
+        pollwakeup(&pw->spw_pollhd, pw->spw_pollev);
+        pollhead_clean(&pw->spw_pollhd);
+        kmem_free(pw, sizeof (*pw));
+}
+
+/*
  * Called every time a signal is delivered to the process so that we can
  * see if any signal stream needs a pollwakeup. We maintain a list of
  * signal state elements so that we don't have to look at every file descriptor
  * on the process. If necessary, a further optimization would be to maintain a
  * signal set mask that is a union of all of the sets in the list so that
@@ -252,69 +298,78 @@
  */
 static void
 signalfd_pollwake_cb(void *arg0, int sig)
 {
         proc_t *p = (proc_t *)arg0;
+        sigfd_proc_state_t *pstate = (sigfd_proc_state_t *)p->p_sigfd;
         list_t *lst;
-        sigfd_wake_list_t *wlp;
+        sigfd_poll_waiter_t *pw;
 
         ASSERT(MUTEX_HELD(&p->p_lock));
+        ASSERT(pstate != NULL);
 
-        if (p->p_sigfd == NULL)
-                return;
+        lst = &pstate->sigfd_list;
+        pw = list_head(lst);
+        while (pw != NULL) {
+                signalfd_state_t *state = pw->spw_state;
+                sigfd_poll_waiter_t *next;
 
-        lst = &((sigfd_proc_state_t *)p->p_sigfd)->sigfd_list;
-        wlp = list_head(lst);
-        while (wlp != NULL) {
-                signalfd_state_t *state = wlp->sigfd_wl_state;
-
                 mutex_enter(&state->sfd_lock);
-
-                if (sigismember(&state->sfd_set, sig) &&
-                    state->sfd_pollhd.ph_list != NULL) {
-                        sigfd_wake_list_t *tmp = wlp;
-
-                        /* remove it from the list */
-                        wlp = list_next(lst, wlp);
-                        signalfd_wake_rm(lst, tmp);
-
-                        mutex_exit(&state->sfd_lock);
-                        pollwakeup(&state->sfd_pollhd, POLLRDNORM | POLLIN);
+                if (!state->sfd_valid) {
+                        pw->spw_pollev = POLLERR;
+                } else if (sigismember(&state->sfd_set, sig)) {
+                        pw->spw_pollev = POLLRDNORM | POLLIN;
                 } else {
                         mutex_exit(&state->sfd_lock);
-                        wlp = list_next(lst, wlp);
+                        pw = list_next(lst, pw);
+                        continue;
                 }
+                mutex_exit(&state->sfd_lock);
+
+                /*
+                 * Pull the sigfd_poll_waiter_t out of the list and dispatch it
+                 * to perform a pollwake.  This cannot be done synchronously
+                 * since signalfd_poll and signalfd_pollwake_cb have
+                 * conflicting lock orders which can deadlock.
+                 */
+                next = list_next(lst, pw);
+                list_remove(lst, pw);
+                taskq_dispatch_ent(signalfd_wakeq, signalfd_wake_task, pw, 0,
+                    &pw->spw_taskent);
+                pw = next;
         }
 }
 
 _NOTE(ARGSUSED(1))
 static int
 signalfd_open(dev_t *devp, int flag, int otyp, cred_t *cred_p)
 {
-        signalfd_state_t *state;
+        signalfd_state_t *state, **sstate;
         major_t major = getemajor(*devp);
         minor_t minor = getminor(*devp);
 
         if (minor != SIGNALFDMNRN_SIGNALFD)
                 return (ENXIO);
 
         mutex_enter(&signalfd_lock);
 
         minor = (minor_t)id_allocff(signalfd_minor);
-
         if (ddi_soft_state_zalloc(signalfd_softstate, minor) != DDI_SUCCESS) {
                 id_free(signalfd_minor, minor);
                 mutex_exit(&signalfd_lock);
                 return (ENODEV);
         }
 
-        state = ddi_get_soft_state(signalfd_softstate, minor);
+        state = kmem_zalloc(sizeof (*state), KM_SLEEP);
+        state->sfd_valid = B_TRUE;
+        state->sfd_count = 1;
+        list_insert_head(&signalfd_state, (void *)state);
+
+        sstate = ddi_get_soft_state(signalfd_softstate, minor);
+        *sstate = state;
         *devp = makedevice(major, minor);
 
-        state->sfd_next = signalfd_state;
-        signalfd_state = state;
-
         mutex_exit(&signalfd_lock);
 
         return (0);
 }
 
@@ -403,10 +458,13 @@
         DTRACE_PROC2(signal__clear, int, ret, ksiginfo_t *, infop);
         lwp->lwp_cursig = 0;
         lwp->lwp_extsig = 0;
         mutex_exit(&p->p_lock);
 
+        if (PROC_IS_BRANDED(p) && BROP(p)->b_sigfd_translate)
+                BROP(p)->b_sigfd_translate(infop);
+
         /* Convert k_siginfo into external, datamodel independent, struct. */
         bzero(ssp, sizeof (*ssp));
         ssp->ssi_signo = infop->si_signo;
         ssp->ssi_errno = infop->si_errno;
         ssp->ssi_code = infop->si_code;
@@ -437,21 +495,22 @@
  */
 _NOTE(ARGSUSED(2))
 static int
 signalfd_read(dev_t dev, uio_t *uio, cred_t *cr)
 {
-        signalfd_state_t *state;
+        signalfd_state_t *state, **sstate;
         minor_t minor = getminor(dev);
         boolean_t block = B_TRUE;
         k_sigset_t set;
         boolean_t got_one = B_FALSE;
         int res;
 
         if (uio->uio_resid < sizeof (signalfd_siginfo_t))
                 return (EINVAL);
 
-        state = ddi_get_soft_state(signalfd_softstate, minor);
+        sstate = ddi_get_soft_state(signalfd_softstate, minor);
+        state = *sstate;
 
         if (uio->uio_fmode & (FNDELAY|FNONBLOCK))
                 block = B_FALSE;
 
         mutex_enter(&state->sfd_lock);
@@ -460,19 +519,30 @@
 
         if (sigisempty(&set))
                 return (set_errno(EINVAL));
 
         do  {
-                res = consume_signal(state->sfd_set, uio, block);
-                if (res == 0)
-                        got_one = B_TRUE;
+                res = consume_signal(set, uio, block);
 
+                if (res == 0) {
                 /*
-                 * After consuming one signal we won't block trying to consume
-                 * further signals.
+                         * After consuming one signal, do not block while
+                         * trying to consume more.
                  */
+                        got_one = B_TRUE;
                 block = B_FALSE;
+
+                        /*
+                         * Refresh the matching signal set in case it was
+                         * updated during the wait.
+                         */
+                        mutex_enter(&state->sfd_lock);
+                        set = state->sfd_set;
+                        mutex_exit(&state->sfd_lock);
+                        if (sigisempty(&set))
+                                break;
+                }
         } while (res == 0 && uio->uio_resid >= sizeof (signalfd_siginfo_t));
 
         if (got_one)
                 res = 0;
 
@@ -497,59 +567,57 @@
 _NOTE(ARGSUSED(4))
 static int
 signalfd_poll(dev_t dev, short events, int anyyet, short *reventsp,
     struct pollhead **phpp)
 {
-        signalfd_state_t *state;
+        signalfd_state_t *state, **sstate;
         minor_t minor = getminor(dev);
         kthread_t *t = curthread;
         proc_t *p = ttoproc(t);
         short revents = 0;
 
-        state = ddi_get_soft_state(signalfd_softstate, minor);
+        sstate = ddi_get_soft_state(signalfd_softstate, minor);
+        state = *sstate;
 
         mutex_enter(&state->sfd_lock);
 
         if (signalfd_sig_pending(p, t, state->sfd_set) != 0)
                 revents |= POLLRDNORM | POLLIN;
 
         mutex_exit(&state->sfd_lock);
 
         if (!(*reventsp = revents & events) && !anyyet) {
-                *phpp = &state->sfd_pollhd;
+                sigfd_proc_state_t *pstate;
+                sigfd_poll_waiter_t *pw;
 
                 /*
                  * Enable pollwakeup handling.
                  */
-                if (p->p_sigfd == NULL) {
-                        sigfd_proc_state_t *pstate;
+                mutex_enter(&p->p_lock);
+                if ((pstate = (sigfd_proc_state_t *)p->p_sigfd) == NULL) {
 
-                        pstate = kmem_zalloc(sizeof (sigfd_proc_state_t),
-                            KM_SLEEP);
+                        mutex_exit(&p->p_lock);
+                        pstate = kmem_zalloc(sizeof (*pstate), KM_SLEEP);
                         list_create(&pstate->sigfd_list,
-                            sizeof (sigfd_wake_list_t),
-                            offsetof(sigfd_wake_list_t, sigfd_wl_lst));
+                            sizeof (sigfd_poll_waiter_t),
+                            offsetof(sigfd_poll_waiter_t, spw_list));
+                        pstate->sigfd_pollwake_cb = signalfd_pollwake_cb;
 
+                        /* Check again, after blocking for the alloc. */
                         mutex_enter(&p->p_lock);
-                        /* check again now that we're locked */
                         if (p->p_sigfd == NULL) {
                                 p->p_sigfd = pstate;
                         } else {
                                 /* someone beat us to it */
                                 list_destroy(&pstate->sigfd_list);
-                                kmem_free(pstate, sizeof (sigfd_proc_state_t));
+                                kmem_free(pstate, sizeof (*pstate));
+                                pstate = p->p_sigfd;
                         }
-                        mutex_exit(&p->p_lock);
                 }
 
-                mutex_enter(&p->p_lock);
-                if (((sigfd_proc_state_t *)p->p_sigfd)->sigfd_pollwake_cb ==
-                    NULL) {
-                        ((sigfd_proc_state_t *)p->p_sigfd)->sigfd_pollwake_cb =
-                            signalfd_pollwake_cb;
-                }
-                signalfd_wake_list_add(state);
+                pw = signalfd_wake_list_add(pstate, state);
+                *phpp = &pw->spw_pollhd;
                 mutex_exit(&p->p_lock);
         }
 
         return (0);
 }
@@ -556,15 +624,16 @@
 
 _NOTE(ARGSUSED(4))
 static int
 signalfd_ioctl(dev_t dev, int cmd, intptr_t arg, int md, cred_t *cr, int *rv)
 {
-        signalfd_state_t *state;
+        signalfd_state_t *state, **sstate;
         minor_t minor = getminor(dev);
         sigset_t mask;
 
-        state = ddi_get_soft_state(signalfd_softstate, minor);
+        sstate = ddi_get_soft_state(signalfd_softstate, minor);
+        state = *sstate;
 
         switch (cmd) {
         case SIGNALFDIOC_MASK:
                 if (ddi_copyin((caddr_t)arg, (caddr_t)&mask, sizeof (sigset_t),
                     md) != 0)
@@ -585,37 +654,44 @@
 
 _NOTE(ARGSUSED(1))
 static int
 signalfd_close(dev_t dev, int flag, int otyp, cred_t *cred_p)
 {
-        signalfd_state_t *state, **sp;
+        signalfd_state_t *state, **sstate;
+        sigfd_poll_waiter_t *pw = NULL;
         minor_t minor = getminor(dev);
         proc_t *p = curproc;
 
-        state = ddi_get_soft_state(signalfd_softstate, minor);
+        sstate = ddi_get_soft_state(signalfd_softstate, minor);
+        state = *sstate;
 
-        if (state->sfd_pollhd.ph_list != NULL) {
-                pollwakeup(&state->sfd_pollhd, POLLERR);
-                pollhead_clean(&state->sfd_pollhd);
-        }
-
-        /* Make sure our state is removed from our proc's pollwake list. */
+        /* Make sure state is removed from this proc's pollwake list. */
         mutex_enter(&p->p_lock);
-        signalfd_wake_list_rm(p, state);
+        if (p->p_sigfd != NULL) {
+                sigfd_proc_state_t *pstate = p->p_sigfd;
+
+                pw = signalfd_wake_list_rm(pstate, state);
+                if (list_is_empty(&pstate->sigfd_list)) {
+                        signalfd_wake_list_cleanup(p);
+                }
+        }
         mutex_exit(&p->p_lock);
 
+        if (pw != NULL) {
+                pollwakeup(&pw->spw_pollhd, POLLERR);
+                pollhead_clean(&pw->spw_pollhd);
+                kmem_free(pw, sizeof (*pw));
+        }
+
         mutex_enter(&signalfd_lock);
 
-        /* Remove our state from our global list. */
-        for (sp = &signalfd_state; *sp != state; sp = &((*sp)->sfd_next))
-                VERIFY(*sp != NULL);
-
-        *sp = (*sp)->sfd_next;
-
+        *sstate = NULL;
         ddi_soft_state_free(signalfd_softstate, minor);
         id_free(signalfd_minor, minor);
 
+        signalfd_state_release(state, B_TRUE);
+
         mutex_exit(&signalfd_lock);
 
         return (0);
 }
 
@@ -633,11 +709,11 @@
                 mutex_exit(&signalfd_lock);
                 return (DDI_FAILURE);
         }
 
         if (ddi_soft_state_init(&signalfd_softstate,
-            sizeof (signalfd_state_t), 0) != 0) {
+            sizeof (signalfd_state_t *), 0) != 0) {
                 cmn_err(CE_WARN, "signalfd failed to create soft state");
                 id_space_destroy(signalfd_minor);
                 mutex_exit(&signalfd_lock);
                 return (DDI_FAILURE);
         }
@@ -654,10 +730,16 @@
         ddi_report_dev(devi);
         signalfd_devi = devi;
 
         sigfd_exit_helper = signalfd_exit_helper;
 
+        list_create(&signalfd_state, sizeof (signalfd_state_t),
+            offsetof(signalfd_state_t, sfd_list));
+
+        signalfd_wakeq = taskq_create("signalfd_wake", 1, minclsyspri,
+            0, INT_MAX, TASKQ_PREPOPULATE);
+
         mutex_exit(&signalfd_lock);
 
         return (DDI_SUCCESS);
 }
 
@@ -671,14 +753,29 @@
 
         default:
                 return (DDI_FAILURE);
         }
 
-        /* list should be empty */
-        VERIFY(signalfd_state == NULL);
-
         mutex_enter(&signalfd_lock);
+
+        if (!list_is_empty(&signalfd_state)) {
+                /*
+                 * There are dangling poll waiters holding signalfd_state_t
+                 * entries on the global list.  Detach is not possible until
+                 * they purge themselves.
+                 */
+                mutex_exit(&signalfd_lock);
+                return (DDI_FAILURE);
+        }
+        list_destroy(&signalfd_state);
+
+        /*
+         * With no remaining entries in the signalfd_state list, the wake taskq
+         * should be empty with no possibility for new entries.
+         */
+        taskq_destroy(signalfd_wakeq);
+
         id_space_destroy(signalfd_minor);
 
         ddi_remove_minor_node(signalfd_devi, NULL);
         signalfd_devi = NULL;
         sigfd_exit_helper = NULL;