Print this page
OS-2366 ddi_periodic_add(9F) is entirely rubbish

@@ -21,812 +21,378 @@
 
 /*
  * Copyright 2009 Sun Microsystems, Inc.  All rights reserved.
  * Use is subject to license terms.
  */
+/*
+ * Copyright (c) 2013, Joyent, Inc. All rights reserved.
+ */
 
-#include <sys/atomic.h>
-#include <sys/callb.h>
-#include <sys/conf.h>
 #include <sys/cmn_err.h>
-#include <sys/taskq.h>
-#include <sys/dditypes.h>
 #include <sys/ddi_timer.h>
-#include <sys/disp.h>
+#include <sys/id_space.h>
 #include <sys/kobj.h>
-#include <sys/note.h>
-#include <sys/param.h>
 #include <sys/sysmacros.h>
 #include <sys/systm.h>
+#include <sys/taskq.h>
+#include <sys/taskq_impl.h>
 #include <sys/time.h>
 #include <sys/types.h>
+#include <sys/sdt.h>
 
 /*
- * global variables for timeout request
- */
-static kmem_cache_t *req_cache;         /* kmem cache for timeout request */
-
-/*
- * taskq parameters for cyclic_timer
+ * The ddi_periodic_add(9F) Implementation
  *
- * timer_taskq_num:
- * timer_taskq_num represents the number of taskq threads.
- * Currently 4 threads are pooled to handle periodic timeout requests.
- * This number is chosen based on the fact that the callout (one-time
- * timeout framework) uses 8 threads with TQ_NOSLEEP; the periodic timeout
- * calls taskq_dispatch() with TQ_SLEEP instead, and in this case, 4 threads
- * should be sufficient to handle periodic timeout requests. (see also
- * timer_taskq_max_num below)
+ * This file contains the implementation of the ddi_periodic_add(9F) interface.
+ * It is a thin wrapper around the cyclic subsystem (see documentation in
+ * uts/common/os/cyclic.c), providing a DDI interface for registering
+ * (and unregistering) callbacks for periodic invocation at arbitrary
+ * interrupt levels, or in kernel context.
  *
- * timer_taskq_min_num:
- * timer_taskq_min_num represents the number of pre-populated taskq_ent
- * structures, and this variable holds the same value as timer_taskq_num does.
+ * Each call to ddi_periodic_add will result in a new opaque handle, as
+ * allocated from an id_space, a new "periodic" object (ddi_periodic_impl_t)
+ * and a registered cyclic.
  *
- * timer_taskq_max_num:
- * Since TQ_SLEEP is set when taskq_dispatch() is called, the framework waits
- * for one second if more taskq_ent structures than timer_taskq_max_num are
- * required. However, from the timeout point of view, one second is much longer
- * than expected, and to prevent this occurrence, timer_taskq_max_num should
- * hold a sufficiently-large value, which is 128 here. Note that since the size
- * of taskq_ent_t is relatively small, this doesn't use up the resource so much.
- * (Currently the size is less than 8k at most)
+ * Operation
  *
- * About the detailed explanation of the taskq function arguments, please see
- * usr/src/uts/common/os/taskq.c.
+ * Whenever the cyclic fires, our cyclic handler checks that the particular
+ * periodic is not dispatched already (we do not support overlapping execution
+ * of the consumer's handler function), and not yet cancelled.  If both of
+ * these conditions hold, we mark the periodic as DPF_DISPATCHED and enqueue it
+ * to either the taskq (for DDI_IPL_0) or to one of the soft interrupt queues
+ * (DDI_IPL_1 to DDI_IPL_10).
+ *
+ * While the taskq (or soft interrupt handler) is handling a particular
+ * periodic, we mark it as DPF_EXECUTING.  When complete, we reset both
+ * DPF_DISPATCHED and DPF_EXECUTING.
+ *
+ * Cancellation
+ *
+ * ddi_periodic_delete(9F) historically had spectacularly loose semantics with
+ * respect to cancellation concurrent with handler execution.  These semantics
+ * are now tighter:
+ *
+ *   1. At most one invocation of ddi_periodic_delete(9F) will actually
+ *      perform the deletion, all others will return immediately.
+ *   2. The invocation that performs the deletion will _block_ until
+ *      the handler is no longer running, and all resources have been
+ *      released.
+ *
+ * We affect this model by removing the cancelling periodic from the
+ * global list and marking it DPF_CANCELLED.  This will prevent further
+ * execution of the handler.  We then wait on a CV until the DPF_EXECUTING
+ * and DPF_DISPATCHED flags are clear, which means the periodic is removed
+ * from all request queues, is no longer executing, and may be freed.  At this
+ * point we return the opaque ID to the id_space and free the memory.
+ *
+ * NOTE:
+ * The ddi_periodic_add(9F) interface is presently limited to a minimum period
+ * of 10ms between firings.
  */
-int timer_taskq_num = 4;                /* taskq thread number */
-int timer_taskq_min_num = 4;            /* min. number of taskq_ent structs */
-int timer_taskq_max_num = 128;          /* max. number of taskq_ent structs */
-static taskq_t *tm_taskq;               /* taskq thread pool */
-static kthread_t *tm_work_thread;       /* work thread invoking taskq */
 
 /*
- * timer variables
+ * Tuneables:
  */
-static cyc_timer_t *ddi_timer;          /* ddi timer based on the cyclic */
-static volatile hrtime_t timer_hrtime;  /* current tick time on the timer */
+int ddi_periodic_max_id = 1024;
+int ddi_periodic_taskq_threadcount = 4;
+hrtime_t ddi_periodic_resolution = 10000000;
 
 /*
- * Variable used for the suspend/resume.
+ * Globals:
  */
-static volatile boolean_t timer_suspended;
+static kmem_cache_t *periodic_cache;
+static id_space_t *periodic_id_space;
+static taskq_t *periodic_taskq;
 
 /*
- * Kernel taskq queue to ddi timer
+ * periodics_lock protects the list of all periodics (periodics), and
+ * each of the soft interrupt request queues (periodic_softint_queue).
+ *
+ * Do not hold an individual periodic's lock while obtaining periodics_lock.
+ * While in the periodic_softint_queue list, the periodic will be marked
+ * DPF_DISPATCHED, and thus safe from frees.  Only the invocation of
+ * i_untimeout() that removes the periodic from the global list is allowed
+ * to free it.
  */
-static list_t kern_queue;       /* kernel thread request queue */
-static kcondvar_t kern_cv;      /* condition variable for taskq queue */
+static kmutex_t periodics_lock;
+static list_t periodics;
+static list_t periodic_softint_queue[10]; /* for IPL1 up to IPL10 */
 
-/*
- * Software interrupt queue dedicated to ddi timer
- */
-static list_t intr_queue;       /* software interrupt request queue */
-static uint_t intr_state;       /* software interrupt state */
+typedef enum periodic_ipl {
+        PERI_IPL_0 = 0,
+        PERI_IPL_1,
+        PERI_IPL_2,
+        PERI_IPL_3,
+        PERI_IPL_4,
+        PERI_IPL_5,
+        PERI_IPL_6,
+        PERI_IPL_7,
+        PERI_IPL_8,
+        PERI_IPL_9,
+        PERI_IPL_10,
+} periodic_ipl_t;
 
 /*
- * This lock is used to protect the intr_queue and kern_queue.
- * It's also used to protect the intr_state which represents the software
- * interrupt state for the timer.
+ * This function may be called either from a soft interrupt handler
+ * (ddi_periodic_softintr), or as a taskq worker function.
  */
-static kmutex_t disp_req_lock;
-
-/*
- * the periodic timer interrupt priority level
- */
-enum {
-        TM_IPL_0 = 0,                   /* kernel context */
-        TM_IPL_1, TM_IPL_2, TM_IPL_3,   /* level 1-3 */
-        TM_IPL_4, TM_IPL_5, TM_IPL_6,   /* level 4-6 */
-        TM_IPL_7, TM_IPL_8, TM_IPL_9,   /* level 7-9 */
-        TM_IPL_10                       /* level 10 */
-};
-
-/*
- * A callback handler used by CPR to stop and resume callouts.
- * Since the taskq uses TASKQ_CPR_SAFE, the function just set the boolean
- * flag to timer_suspended here.
- */
-/*ARGSUSED*/
-static boolean_t
-timer_cpr_callb(void *arg, int code)
+static void
+periodic_execute(void *arg)
 {
-        timer_suspended = (code == CB_CODE_CPR_CHKPT);
-        return (B_TRUE);
-}
+        ddi_periodic_impl_t *dpr = arg;
+        mutex_enter(&dpr->dpr_lock);
 
-/*
- * Return a proposed timeout request id. add_req() determines whether
- * or not the proposed one is used. If it's not suitable, add_req()
- * recalls get_req_cnt(). To reduce the lock contention between the
- * timer and i_untimeout(), the atomic instruction should be used here.
+        /*
+         * We must be DISPATCHED, but not yet EXECUTING:
  */
-static timeout_t
-get_req_cnt(void)
-{
-        static volatile ulong_t timeout_cnt = 0;
-        return ((timeout_t)atomic_inc_ulong_nv(&timeout_cnt));
-}
+        VERIFY((dpr->dpr_flags & (DPF_DISPATCHED | DPF_EXECUTING)) ==
+            DPF_DISPATCHED);
 
-/*
- * Get the system resolution.
- * Note. currently there is a restriction about the system resolution, and
- * the 10ms tick (the default clock resolution) is only supported now.
- */
-static hrtime_t
-i_get_res(void)
-{
-        return ((hrtime_t)10000000); /* 10ms tick only */
-}
-
-/*
- * Return the value for the cog of the timing wheel.
- * TICK_FACTOR is used to gain a finer cog on the clock resolution.
- */
-static hrtime_t
-tw_tick(hrtime_t time)
-{
-        return ((time << TICK_FACTOR) / ddi_timer->res);
-}
-
-/*
- * Calculate the expiration time for the timeout request.
- */
-static hrtime_t
-expire_tick(tm_req_t *req)
-{
-        return (tw_tick(req->exp_time));
-}
-
-/*
- * Register a timeout request to the timer. This function is used
- * in i_timeout().
- */
-static timeout_t
-add_req(tm_req_t *req)
-{
-        timer_tw_t *tid, *tw;
-        tm_req_t *next;
-        timeout_t id;
-
-retry:
+        if (!(dpr->dpr_flags & DPF_CANCELLED)) {
+                int level = dpr->dpr_level;
+                uint64_t count = dpr->dpr_fire_count;
         /*
-         * Retrieve a timeout request id. Since i_timeout() needs to return
-         * a non-zero value, re-try if the zero is gotten.
+                 * If we have not yet been cancelled, then
+                 * mark us executing:
          */
-        if ((id = get_req_cnt()) == 0)
-                id = get_req_cnt();
+                dpr->dpr_flags |= DPF_EXECUTING;
+                mutex_exit(&dpr->dpr_lock);
 
         /*
-         * Check if the id is not used yet. Since the framework now deals
-         * with the periodic timeout requests, we cannot assume the id
-         * allocated (long) before doesn't exist any more when it will
-         * be re-assigned again (especially on 32bit) but need to handle
-         * this case to solve the conflicts. If it's used already, retry
-         * another.
+                 * Execute the handler, without holding locks:
          */
-        tid = &ddi_timer->idhash[TM_HASH((uintptr_t)id)];
-        mutex_enter(&tid->lock);
-        for (next = list_head(&tid->req); next != NULL;
-            next = list_next(&tid->req, next)) {
-                if (next->id == id) {
-                        mutex_exit(&tid->lock);
-                        goto retry;
+                DTRACE_PROBE4(ddi__periodic__execute, void *, dpr->dpr_handler,
+                    void *, dpr->dpr_arg, int, level, uint64_t, count);
+                (*dpr->dpr_handler)(dpr->dpr_arg);
+                DTRACE_PROBE4(ddi__periodic__done, void *, dpr->dpr_handler,
+                    void *, dpr->dpr_arg, int, level, uint64_t, count);
+
+                mutex_enter(&dpr->dpr_lock);
+                dpr->dpr_fire_count++;
                 }
-        }
-        /* Nobody uses this id yet */
-        req->id = id;
 
         /*
-         * Register this request to the timer.
-         * The list operation must be list_insert_head().
-         * Other operations can degrade performance.
+         * We're done with this periodic for now, so release it and
+         * wake anybody that was waiting for us to be finished:
          */
-        list_insert_head(&tid->req, req);
-        mutex_exit(&tid->lock);
-
-        tw = &ddi_timer->exhash[TM_HASH(expire_tick(req))];
-        mutex_enter(&tw->lock);
-        /*
-         * Other operations than list_insert_head() can
-         * degrade performance here.
-         */
-        list_insert_head(&tw->req, req);
-        mutex_exit(&tw->lock);
-
-        return (id);
+        dpr->dpr_flags &= ~(DPF_DISPATCHED | DPF_EXECUTING);
+        cv_broadcast(&dpr->dpr_cv);
+        mutex_exit(&dpr->dpr_lock);
 }
 
-/*
- * Periodic timeout requests cannot be removed until they are canceled
- * explicitly. Until then, they need to be re-registerd after they are
- * fired. transfer_req() re-registers the requests for the next fires.
- * Note. transfer_req() sends the cv_signal to timeout_execute(), which
- * runs in interrupt context. Make sure this function will not be blocked,
- * otherwise the deadlock situation can occur.
- */
-static void
-transfer_req(tm_req_t *req, timer_tw_t *tw)
+void
+ddi_periodic_softintr(int level)
 {
-        timer_tw_t *new_tw;
-        hrtime_t curr_time;
-        ASSERT(tw && MUTEX_HELD(&tw->lock));
+        ddi_periodic_impl_t *dpr;
+        VERIFY(level >= PERI_IPL_1 && level <= PERI_IPL_10);
 
-        /* Calculate the next expiration time by interval */
-        req->exp_time += req->interval;
-        curr_time = gethrtime();
-
+        mutex_enter(&periodics_lock);
         /*
-         * If a long time (more than 1 clock resolution) has already
-         * passed for some reason (e.g. debugger or high interrupt),
-         * round up the next expiration to the appropriate one
-         * since this request is periodic and never catches with it.
+         * Pull the first scheduled periodic off the queue for this priority
+         * level:
          */
-        if (curr_time - req->exp_time >= ddi_timer->res) {
-                req->exp_time = roundup(curr_time + req->interval,
-                    ddi_timer->res);
-        }
-
+        while ((dpr = list_remove_head(&periodic_softint_queue[level - 1]))
+            != NULL) {
+                mutex_exit(&periodics_lock);
         /*
-         * Re-register this request.
-         * Note. since it is guaranteed that the timer is invoked on only
-         * one CPU at any time (by the cyclic subsystem), a deadlock
-         * cannot occur regardless of the lock order here.
+                 * And execute it:
          */
-        new_tw = &ddi_timer->exhash[TM_HASH(expire_tick(req))];
-
-        /*
-         * If it's on the timer cog already, there is nothing
-         * to do. Just return.
-         */
-        if (new_tw == tw)
-                return;
-
-        /* Remove this request from the timer */
-        list_remove(&tw->req, req);
-
-        /* Re-register this request to the timer */
-        mutex_enter(&new_tw->lock);
-
-        /*
-         * Other operations than list_insert_head() can
-         * degrade performance here.
-         */
-        list_insert_head(&new_tw->req, req);
-        mutex_exit(&new_tw->lock);
-
-        /*
-         * Set the TM_TRANSFER flag and notify the request is transfered
-         * completely. This prevents a race in the case that this request
-         * is serviced on another CPU already.
-         */
-        mutex_enter(&req->lock);
-        req->flags |= TM_TRANSFER;
-        cv_signal(&req->cv);
-        mutex_exit(&req->lock);
+                periodic_execute(dpr);
+                mutex_enter(&periodics_lock);
+        }
+        mutex_exit(&periodics_lock);
 }
 
-/*
- * Execute timeout requests.
- * Note. since timeout_execute() can run in interrupt context and block
- * on condition variables, there are restrictions on the timer code that
- * signals these condition variables (see i_untimeout(), transfer_req(),
- * and condvar(9F)). Functions that signal these cvs must ensure that
- * they will not be blocked (for memory allocations or any other reason)
- * since condition variables don't support priority inheritance.
- */
-static void
-timeout_execute(void *arg)
+void
+ddi_periodic_init(void)
 {
-        tm_req_t *req = (tm_req_t *)arg;
-        ASSERT(req->flags & TM_INVOKING && !(req->flags & TM_EXECUTING));
+        int i;
 
-        for (;;) {
                 /*
-                 * Check if this request is canceled. If it's canceled, do not
-                 * execute this request.
+         * Create a kmem_cache for request tracking objects, and a list
+         * to store them in so we can later delete based on opaque handles:
                  */
-                mutex_enter(&req->lock);
-                if (!(req->flags & TM_CANCEL)) {
+        periodic_cache = kmem_cache_create("ddi_periodic",
+            sizeof (ddi_periodic_impl_t), 0, NULL, NULL, NULL, NULL, NULL, 0);
+        list_create(&periodics, sizeof (ddi_periodic_impl_t),
+            offsetof(ddi_periodic_impl_t, dpr_link));
+
                         /*
-                         * Set the current thread to prevent a dead lock
-                         * situation in case that this timeout request is
-                         * canceled in the handler being invoked now.
-                         * (this doesn't violate the spec) Set TM_EXECUTING
-                         * to show this handler is invoked soon.
+         * Initialise the identifier space for ddi_periodic_add(9F):
                          */
-                        req->h_thread = curthread;
-                        req->flags |= TM_EXECUTING;
-                        mutex_exit(&req->lock);
+        periodic_id_space = id_space_create("ddi_periodic", 1,
+            ddi_periodic_max_id);
 
-                        /* The handler is invoked without holding any locks */
-                        (*req->handler)(req->arg);
-
-                        mutex_enter(&req->lock);
-                }
-
                 /*
-                 * Check if this request is canceled or not. If not, prepare
-                 * for the next fire.
+         * Initialise the request queue for each soft interrupt level:
                  */
-                if (req->flags & TM_CANCEL) {
-                        timer_tw_t *tw;
-                        /*
-                         * Wait until the timer finishes all things for
-                         * this request.
-                         */
-                        while (!(req->flags & TM_TRANSFER))
-                                cv_wait(&req->cv, &req->lock);
-                        mutex_exit(&req->lock);
-                        ASSERT(req->flags & TM_TRANSFER);
-
-                        /* Remove this request from the timer */
-                        tw = &ddi_timer->exhash[TM_HASH(expire_tick(req))];
-                        mutex_enter(&tw->lock);
-                        list_remove(&tw->req, req);
-                        mutex_exit(&tw->lock);
-
-                        /* Free this request */
-                        kmem_cache_free(req_cache, req);
-                        return;
+        for (i = PERI_IPL_1; i <= PERI_IPL_10; i++) {
+                list_create(&periodic_softint_queue[i - 1],
+                    sizeof (ddi_periodic_impl_t), offsetof(ddi_periodic_impl_t,
+                    dpr_softint_link));
                 }
-                ASSERT(req->flags & TM_EXECUTING);
 
                 /*
-                 * TM_EXECUTING must be set at this point.
-                 * Unset the flag.
+         * Create the taskq for running PERI_IPL_0 handlers.  This taskq will
+         * _only_ be used with taskq_dispatch_ent(), and a taskq_ent_t
+         * pre-allocated with the ddi_periodic_impl_t.
                  */
-                req->flags &= ~(TM_EXECUTING | TM_TRANSFER);
+        periodic_taskq = taskq_create_instance("ddi_periodic_taskq", -1,
+            ddi_periodic_taskq_threadcount, maxclsyspri, 0, 0, 0);
 
                 /*
-                 * Decrease the request cnt. The reqest cnt shows
-                 * how many times this request is executed now.
-                 * If this counter becomes the zero, drop TM_INVOKING
-                 * to show there is no requests to do now.
+         * Initialize the mutex lock used for the soft interrupt request
+         * queues.
                  */
-                req->cnt--;
-                if (req->cnt == 0) {
-                        req->flags &= ~TM_INVOKING;
-                        mutex_exit(&req->lock);
-                        return;
-                }
-                mutex_exit(&req->lock);
-        }
+        mutex_init(&periodics_lock, NULL, MUTEX_ADAPTIVE, NULL);
 }
 
-/*
- * Timeout worker thread for processing task queue.
- */
 static void
-timeout_taskq_thread(void *arg)
+periodic_cyclic_handler(void *arg)
 {
-        _NOTE(ARGUNUSED(arg));
-        tm_req_t *kern_req;
-        callb_cpr_t cprinfo;
+        extern void sir_on(int);
+        ddi_periodic_impl_t *dpr = arg;
 
-        CALLB_CPR_INIT(&cprinfo, &disp_req_lock, callb_generic_cpr,
-            "timeout_taskq_thread");
-
+        mutex_enter(&dpr->dpr_lock);
         /*
-         * This thread is wakened up when a new request is added to
-         * the queue. Then pick up all requests and dispatch them
-         * via taskq_dispatch().
+         * If we've been cancelled, or we're already dispatched, then exit
+         * immediately:
          */
-        for (;;) {
-                /*
-                 * Check the queue and pick up a request if the queue
-                 * is not NULL.
-                 */
-                mutex_enter(&disp_req_lock);
-                while ((kern_req = list_head(&kern_queue)) == NULL) {
-                        CALLB_CPR_SAFE_BEGIN(&cprinfo);
-                        cv_wait(&kern_cv, &disp_req_lock);
-                        CALLB_CPR_SAFE_END(&cprinfo, &disp_req_lock);
+        if (dpr->dpr_flags & (DPF_CANCELLED | DPF_DISPATCHED)) {
+                mutex_exit(&dpr->dpr_lock);
+                return;
                 }
-                list_remove(&kern_queue, kern_req);
-                mutex_exit(&disp_req_lock);
+        VERIFY(!(dpr->dpr_flags & DPF_EXECUTING));
 
-                /* Execute the timeout request via the taskq thread */
-                (void) taskq_dispatch(tm_taskq, timeout_execute,
-                    (void *)kern_req, TQ_SLEEP);
-        }
-}
-
-/*
- * Dispatch the timeout request based on the level specified.
- * If the level is equal to zero, notify the worker thread to
- * call taskq_dispatch() in kernel context. If the level is bigger
- * than zero, add a software interrupt request to the queue and raise
- * the interrupt level to the specified one.
+        /*
+         * This periodic is not presently dispatched, so dispatch it now:
  */
-static void
-timeout_dispatch(tm_req_t *req)
-{
-        int level = req->level;
-        extern void sir_on(int);
+        dpr->dpr_flags |= DPF_DISPATCHED;
+        mutex_exit(&dpr->dpr_lock);
 
-        if (level == TM_IPL_0) {
-                /* Add a new request to the tail */
-                mutex_enter(&disp_req_lock);
-                list_insert_tail(&kern_queue, req);
-                mutex_exit(&disp_req_lock);
-
+        if (dpr->dpr_level == PERI_IPL_0) {
                 /*
-                 * notify the worker thread that this request
-                 * is newly added to the queue.
-                 * Note. this cv_signal() can be called after the
-                 * mutex_lock.
+                 * DDI_IPL_0 periodics are dispatched onto the taskq:
                  */
-                cv_signal(&kern_cv);
+                taskq_dispatch_ent(periodic_taskq, periodic_execute,
+                    dpr, 0, &dpr->dpr_taskq_ent);
         } else {
-                /* Add a new request to the tail */
-                mutex_enter(&disp_req_lock);
-                list_insert_tail(&intr_queue, req);
-
-                /* Issue the software interrupt */
-                if (intr_state & TM_INTR_START(level)) {
                         /*
-                         * timer_softintr() is already running; no need to
-                         * raise a siron. Due to lock protection of
-                         * the intr_queue and intr_state, we know that
-                         * timer_softintr() will see the new addition to
-                         * the intr_queue.
+                 * Higher priority periodics are handled by a soft
+                 * interrupt handler.  Enqueue us for processing and
+                 * fire the soft interrupt:
                          */
-                        mutex_exit(&disp_req_lock);
-                } else {
-                        intr_state |= TM_INTR_SET(level);
-                        mutex_exit(&disp_req_lock);
+                mutex_enter(&periodics_lock);
+                list_insert_tail(&periodic_softint_queue[dpr->dpr_level - 1],
+                    dpr);
+                mutex_exit(&periodics_lock);
 
-                        /* Raise an interrupt to execute timeout requests */
-                        sir_on(level);
-                }
-        }
-}
-
-/*
- * Check the software interrupt queue and invoke requests at the specified
- * interrupt level.
- * Note that the queue may change during call so that the disp_req_lock
- * and the intr_state are used to protect it.
- * The software interrupts supported here are up to the level 10. Higher
- * than 10 interrupts cannot be supported.
+                /*
+                 * Raise the soft interrupt level for this periodic:
  */
-void
-timer_softintr(int level)
-{
-        tm_req_t *intr_req;
-        ASSERT(level >= TM_IPL_1 && level <= TM_IPL_10);
-
-        /* Check if we are asked to process the softcall list */
-        mutex_enter(&disp_req_lock);
-        if (!(intr_state & TM_INTR_SET(level))) {
-                mutex_exit(&disp_req_lock);
-                return;
+                sir_on(dpr->dpr_level);
         }
-
-        /* Notify this software interrupt request will be executed soon */
-        intr_state |= TM_INTR_START(level);
-        intr_state &= ~TM_INTR_SET(level);
-
-        /* loop the link until there is no requests */
-        for (intr_req = list_head(&intr_queue); intr_req != NULL;
-            /* Nothing */) {
-
-                /* Check the interrupt level */
-                if (intr_req->level != level) {
-                        intr_req = list_next(&intr_queue, intr_req);
-                        continue;
-                }
-                list_remove(&intr_queue, intr_req);
-                mutex_exit(&disp_req_lock);
-
-                /* Execute the software interrupt request */
-                timeout_execute(intr_req);
-
-                mutex_enter(&disp_req_lock);
-                /* Restart the loop since new requests might be added */
-                intr_req = list_head(&intr_queue);
-        }
-
-        /* reset the interrupt state */
-        intr_state &= ~TM_INTR_START(level);
-        mutex_exit(&disp_req_lock);
 }
 
-/*
- *  void
- *  cyclic_timer(void)
- *
- *  Overview
- *   cyclic_timer() is a function invoked periodically by the cyclic
- *   subsystem.
- *
- *   The function calls timeout_invoke() with timeout requests whose
- *   expiration time is already reached.
- *
- *  Arguments
- *   Nothing
- *
- *  Return value
- *   Nothing
- */
-void
-cyclic_timer(void)
+static void
+periodic_destroy(ddi_periodic_impl_t *dpr)
 {
-        tm_req_t *req;
-        timer_tw_t *tw;
-        hrtime_t curr_tick, curr;
-
-        /* If the system is suspended, just return */
-        if (timer_suspended)
+        if (dpr == NULL)
                 return;
 
-        /* Get the current time */
-        timer_hrtime = ddi_timer->tick_time = curr = gethrtime();
-        curr_tick = tw_tick(ddi_timer->tick_time);
-
-restart:
         /*
-         * Check the timer cogs to see if there are timeout requests
-         * who reach the expiration time. Call timeout_invoke() to execute
-         * the requests, then.
+         * By now, we should have a periodic that is not busy, and has been
+         * cancelled:
          */
-        while (curr_tick >= ddi_timer->tick) {
-                tm_req_t *next;
-                tw = &ddi_timer->exhash[TM_HASH(ddi_timer->tick)];
-                mutex_enter(&tw->lock);
-                for (req = list_head(&tw->req); req != NULL; req = next) {
-                        next = list_next(&tw->req, req);
-                        /*
-                         * If this request is already obsolete, free
-                         * it here.
-                         */
-                        if (req->flags & TM_UTMCOMP) {
-                                /*
-                                 * Remove this request from the timer,
-                                 * then free it.
-                                 */
-                                list_remove(&tw->req, req);
-                                kmem_cache_free(req_cache, req);
-                        } else if (curr >= req->exp_time) {
-                                mutex_enter(&req->lock);
-                                /*
-                                 * Check if this request is canceled, but not
-                                 * being executed now.
-                                 */
-                                if (req->flags & TM_CANCEL &&
-                                    !(req->flags & TM_INVOKING)) {
-                                        mutex_exit(&req->lock);
-                                        continue;
-                                }
-                                /*
-                                 * Record how many times timeout_execute()
-                                 * must be invoked.
-                                 */
-                                req->cnt++;
-                                /*
-                                 * Invoke timeout_execute() via taskq or
-                                 * software interrupt.
-                                 */
-                                if (req->flags & TM_INVOKING) {
-                                        /*
-                                         * If it's already invoked,
-                                         * There is nothing to do.
-                                         */
-                                        mutex_exit(&req->lock);
-                                } else {
-                                        req->flags |= TM_INVOKING;
-                                        mutex_exit(&req->lock);
-                                        /*
-                                         * Dispatch this timeout request.
-                                         * timeout_dispatch() chooses either
-                                         * a software interrupt or taskq thread
-                                         * based on the level.
-                                         */
-                                        timeout_dispatch(req);
-                                }
-                                /*
-                                 * Periodic timeout requests must prepare for
-                                 * the next fire.
-                                 */
-                                transfer_req(req, tw);
-                        }
-                }
-                mutex_exit(&tw->lock);
-                ddi_timer->tick++;
-        }
+        VERIFY(dpr->dpr_flags == DPF_CANCELLED);
 
-        /*
-         * Check the current time. If we spend some amount of time,
-         * double-check if some of the requests reaches the expiration
-         * time during the work.
-         */
-        curr = gethrtime();
-        curr_tick = tw_tick(curr);
-        if (curr_tick >= ddi_timer->tick) {
-                ddi_timer->tick -= 1;
-                goto restart;
-        }
-        /* Adjustment for the next rolling */
-        ddi_timer->tick -= 1;
+        id_free(periodic_id_space, dpr->dpr_id);
+        kmem_cache_free(periodic_cache, dpr);
 }
 
-/*
- *  void
- *  timer_init(void)
- *
- *  Overview
- *    timer_init() allocates the internal data structures used by
- *    i_timeout(), i_untimeout() and the timer.
- *
- *  Arguments
- *    Nothing
- *
- *  Return value
- *    Nothing
- *
- *  Caller's context
- *    timer_init() can be called in kernel context only.
- */
-void
-timer_init(void)
+static ddi_periodic_impl_t *
+periodic_create(void)
 {
-        int i;
+        ddi_periodic_impl_t *dpr;
 
-        /* Create kmem_cache for timeout requests */
-        req_cache = kmem_cache_create("timeout_request", sizeof (tm_req_t),
-            0, NULL, NULL, NULL, NULL, NULL, 0);
+        dpr = kmem_cache_alloc(periodic_cache, KM_SLEEP);
+        bzero(dpr, sizeof (*dpr));
+        dpr->dpr_id = id_alloc(periodic_id_space);
+        mutex_init(&dpr->dpr_lock, NULL, MUTEX_ADAPTIVE, NULL);
+        cv_init(&dpr->dpr_cv, NULL, CV_DEFAULT, NULL);
 
-        /* Initialize the timer which is invoked by the cyclic subsystem */
-        ddi_timer = kmem_alloc(sizeof (cyc_timer_t), KM_SLEEP);
-        ddi_timer->res = nsec_per_tick;
-        ddi_timer->tick = tw_tick(gethrtime());
-        ddi_timer->tick_time = 0;
-
-        /* Initialize the timing wheel */
-        bzero((char *)&ddi_timer->idhash[0], TM_HASH_SZ * sizeof (timer_tw_t));
-        bzero((char *)&ddi_timer->exhash[0], TM_HASH_SZ * sizeof (timer_tw_t));
-
-        for (i = 0; i < TM_HASH_SZ; i++) {
-                list_create(&ddi_timer->idhash[i].req, sizeof (tm_req_t),
-                    offsetof(tm_req_t, id_req));
-                mutex_init(&ddi_timer->idhash[i].lock, NULL, MUTEX_ADAPTIVE,
-                    NULL);
-
-                list_create(&ddi_timer->exhash[i].req, sizeof (tm_req_t),
-                    offsetof(tm_req_t, ex_req));
-                mutex_init(&ddi_timer->exhash[i].lock, NULL, MUTEX_ADAPTIVE,
-                    NULL);
-        }
-
-        /* Create a taskq thread pool */
-        tm_taskq = taskq_create_instance("timeout_taskq", 0,
-            timer_taskq_num, MAXCLSYSPRI,
-            timer_taskq_min_num, timer_taskq_max_num,
-            TASKQ_PREPOPULATE | TASKQ_CPR_SAFE);
-
-        /*
-         * Initialize the taskq queue which is dedicated to this timeout
-         * interface/timer.
-         */
-        list_create(&kern_queue, sizeof (tm_req_t),
-            offsetof(tm_req_t, disp_req));
-
-        /* Create a worker thread to dispatch the taskq thread */
-        tm_work_thread = thread_create(NULL, 0, timeout_taskq_thread, NULL,
-            0, &p0, TS_RUN, MAXCLSYSPRI);
-
-        /*
-         * Initialize the software interrupt queue which is dedicated to
-         * this timeout interface/timer.
-         */
-        list_create(&intr_queue, sizeof (tm_req_t),
-            offsetof(tm_req_t, disp_req));
-
-        /*
-         * Initialize the mutex lock used for both of kern_queue and
-         * intr_queue.
-         */
-        mutex_init(&disp_req_lock, NULL, MUTEX_ADAPTIVE, NULL);
-        cv_init(&kern_cv, NULL, CV_DEFAULT, NULL);
-
-        /* Register the callback handler for the system suspend/resume */
-        (void) callb_add(timer_cpr_callb, 0, CB_CL_CPR_CALLOUT, "cyclicTimer");
+        return (dpr);
 }
 
-/*
- *  timeout_t
- *  i_timeout(void (*func)(void *), void *arg,  hrtime_t interval,
- *      int level, int flags)
- *
- *  Overview
- *    i_timeout() is an internal function scheduling the passed function
- *    to be invoked in the interval in nanoseconds. The callback function
- *    keeps invoked until the request is explicitly canceled by i_untimeout().
- *    This function is used for ddi_periodic_add(9F).
- *
- *  Arguments
- *
- *    func: the callback function
- *          the callback function will be invoked in kernel context if
- *          the level passed is the zero. Otherwise be invoked in interrupt
- *          context at the specified level by the argument "level".
- *
- *          Note that It's guaranteed by the cyclic subsystem that the
- *          function is invoked on the only one CPU and is never executed
- *          simultaneously even on MP system.
- *
- *     arg: the argument passed to the callback function
- *
- * interval: interval time in nanoseconds
- *          if the interval is the zero, the timer resolution is used.
- *
- *  level : callback interrupt level
- *          If the value is 0 (the zero), the callback function is invoked
- *          in kernel context. If the value is more than 0 (the zero), but
- *          less than or equal to 10, the callback function is invoked in
- *          interrupt context at the specified interrupt level.
- *          This value must be in range of 0-10.
- *
- *  Return value
- *    returns a non-zero opaque value (timeout_t) on success.
- *
- *  Caller's context
- *    i_timeout() can be called in user or kernel context.
- */
 timeout_t
 i_timeout(void (*func)(void *), void *arg, hrtime_t interval, int level)
 {
-        hrtime_t start_time = gethrtime(), res;
-        tm_req_t *req = NULL;
+        cyc_handler_t cyh;
+        cyc_time_t cyt;
+        ddi_periodic_impl_t *dpr;
 
-        /* Allocate and initialize the timeout request */
-        req = kmem_cache_alloc(req_cache, KM_SLEEP);
-        req->handler = func;
-        req->arg = arg;
-        req->h_thread = NULL;
-        req->level = level;
-        req->flags = 0;
-        req->cnt = 0;
-        mutex_init(&req->lock, NULL, MUTEX_ADAPTIVE, NULL);
-        cv_init(&req->cv, NULL, CV_DEFAULT, NULL);
+        VERIFY(func != NULL);
+        VERIFY(level >= 0 && level <= 10);
 
         /*
+         * Allocate object to track this periodic:
+         */
+        dpr = periodic_create();
+        dpr->dpr_level = level;
+        dpr->dpr_handler = func;
+        dpr->dpr_arg = arg;
+
+        /*
          * The resolution must be finer than or equal to
          * the requested interval. If it's not, set the resolution
          * to the interval.
          * Note. There is a restriction currently. Regardless of the
          * clock resolution used here, 10ms is set as the timer resolution.
          * Even on the 1ms resolution timer, the minimum interval is 10ms.
          */
-        if ((res = i_get_res()) > interval) {
-                uintptr_t pc = (uintptr_t)req->handler;
+        if (ddi_periodic_resolution > interval) {
+                uintptr_t pc = (uintptr_t)dpr->dpr_handler;
                 ulong_t off;
                 cmn_err(CE_WARN,
                     "The periodic timeout (handler=%s, interval=%lld) "
                     "requests a finer interval than the supported resolution. "
                     "It rounds up to %lld\n", kobj_getsymname(pc, &off),
-                    interval, res);
-                interval = res;
+                    interval, ddi_periodic_resolution);
+                interval = ddi_periodic_resolution;
         }
 
         /*
          * If the specified interval is already multiples of
          * the resolution, use it as is. Otherwise, it rounds
          * up to multiples of the timer resolution.
          */
-        req->interval = roundup(interval, i_get_res());
+        dpr->dpr_interval = roundup(interval, ddi_periodic_resolution);
 
         /*
-         * For the periodic timeout requests, the first expiration time will
-         * be adjusted to the timer tick edge to take advantage of the cyclic
-         * subsystem. In that case, the first fire is likely not an expected
-         * one, but the fires later can be more accurate due to this.
+         * Create the underlying cyclic:
          */
-        req->exp_time = roundup(start_time + req->interval, i_get_res());
+        cyh.cyh_func = periodic_cyclic_handler;
+        cyh.cyh_arg = dpr;
+        cyh.cyh_level = CY_LOCK_LEVEL;
 
-        /* Add the request to the timer */
-        return (add_req(req));
+        cyt.cyt_when = roundup(gethrtime() + dpr->dpr_interval,
+            ddi_periodic_resolution);
+        cyt.cyt_interval = dpr->dpr_interval;
+
+        mutex_enter(&cpu_lock);
+        dpr->dpr_cyclic_id = cyclic_add(&cyh, &cyt);
+        mutex_exit(&cpu_lock);
+
+        /*
+         * Make the id visible to ddi_periodic_delete(9F) before we
+         * return it:
+         */
+        mutex_enter(&periodics_lock);
+        list_insert_tail(&periodics, dpr);
+        mutex_exit(&periodics_lock);
+
+        return ((timeout_t)(uintptr_t)dpr->dpr_id);
 }
 
 /*
  *  void
  *  i_untimeout(timeout_t req)

@@ -849,53 +415,62 @@
  *  Note. This function is used by ddi_periodic_delete(), which cannot
  *  be called in interrupt context. As a result, this function is called
  *  in user or kernel context only in practice.
  */
 void
-i_untimeout(timeout_t timeout_req)
+i_untimeout(timeout_t id)
 {
-        timer_tw_t *tid;
-        tm_req_t *req;
-        timeout_t id;
+        ddi_periodic_impl_t *dpr;
 
-        /* Retrieve the id for this timeout request */
-        id = (timeout_t)timeout_req;
-        tid = &ddi_timer->idhash[TM_HASH((uintptr_t)id)];
-
-        mutex_enter(&tid->lock);
-        for (req = list_head(&tid->req); req != NULL;
-            req = list_next(&tid->req, req)) {
-                if (req->id == id)
+        /*
+         * Find the periodic in the list of all periodics and remove it.
+         * If we find in (and remove it from) the global list, we have
+         * license to free it once it is no longer busy.
+         */
+        mutex_enter(&periodics_lock);
+        for (dpr = list_head(&periodics); dpr != NULL; dpr =
+            list_next(&periodics, dpr)) {
+                if (dpr->dpr_id == (id_t)(uintptr_t)id) {
+                        list_remove(&periodics, dpr);
                         break;
         }
-        if (req == NULL) {
-                /* There is no requests with this id after all */
-                mutex_exit(&tid->lock);
-                return;
         }
-        mutex_enter(&req->lock);
+        mutex_exit(&periodics_lock);
 
-        /* Unregister this request first */
-        list_remove(&tid->req, req);
+        /*
+         * We could not find a periodic for this id, so bail out:
+         */
+        if (dpr == NULL)
+                return;
 
-        /* Notify that this request is canceled */
-        req->flags |= TM_CANCEL;
+        mutex_enter(&dpr->dpr_lock);
+        /*
+         * We should be the only one trying to cancel this periodic:
+         */
+        VERIFY(!(dpr->dpr_flags & DPF_CANCELLED));
+        /*
+         * Mark the periodic as cancelled:
+         */
+        dpr->dpr_flags |= DPF_CANCELLED;
+        mutex_exit(&dpr->dpr_lock);
 
-        /* Check if the handler is invoked */
-        if (req->flags & TM_INVOKING) {
                 /*
-                 * This request will be removed by timeout_execute() later,
-                 * so that there is no extra thing to do any more.
+         * Cancel our cyclic.  cyclic_remove() guarantees that the cyclic
+         * handler will not run again after it returns.  Note that the cyclic
+         * handler merely _dispatches_ the periodic, so this does _not_ mean
+         * the periodic handler is also finished running.
                  */
-                mutex_exit(&req->lock);
-                mutex_exit(&tid->lock);
-                return;
-        }
-        mutex_exit(&req->lock);
-        mutex_exit(&tid->lock);
+        mutex_enter(&cpu_lock);
+        cyclic_remove(dpr->dpr_cyclic_id);
+        mutex_exit(&cpu_lock);
 
         /*
-         * Notify untimeout() is about to be finished, and this request
-         * can be freed.
+         * Wait until the periodic handler is no longer running:
          */
-        atomic_or_uint(&req->flags, TM_UTMCOMP);
+        mutex_enter(&dpr->dpr_lock);
+        while (dpr->dpr_flags & (DPF_DISPATCHED | DPF_EXECUTING)) {
+                cv_wait(&dpr->dpr_cv, &dpr->dpr_lock);
+        }
+        mutex_exit(&dpr->dpr_lock);
+
+        periodic_destroy(dpr);
 }