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);
}