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

Split Close
Expand all
Collapse all
          --- old/usr/src/uts/common/os/ddi_timer.c
          +++ new/usr/src/uts/common/os/ddi_timer.c
↓ open down ↓ 15 lines elided ↑ open up ↑
  16   16   * fields enclosed by brackets "[]" replaced with your own identifying
  17   17   * information: Portions Copyright [yyyy] [name of copyright owner]
  18   18   *
  19   19   * CDDL HEADER END
  20   20   */
  21   21  
  22   22  /*
  23   23   * Copyright 2009 Sun Microsystems, Inc.  All rights reserved.
  24   24   * Use is subject to license terms.
  25   25   */
       26 +/*
       27 + * Copyright (c) 2013, Joyent, Inc. All rights reserved.
       28 + */
  26   29  
  27      -#include <sys/atomic.h>
  28      -#include <sys/callb.h>
  29      -#include <sys/conf.h>
  30   30  #include <sys/cmn_err.h>
  31      -#include <sys/taskq.h>
  32      -#include <sys/dditypes.h>
  33   31  #include <sys/ddi_timer.h>
  34      -#include <sys/disp.h>
       32 +#include <sys/id_space.h>
  35   33  #include <sys/kobj.h>
  36      -#include <sys/note.h>
  37      -#include <sys/param.h>
  38   34  #include <sys/sysmacros.h>
  39   35  #include <sys/systm.h>
       36 +#include <sys/taskq.h>
       37 +#include <sys/taskq_impl.h>
  40   38  #include <sys/time.h>
  41   39  #include <sys/types.h>
       40 +#include <sys/sdt.h>
  42   41  
  43   42  /*
  44      - * global variables for timeout request
  45      - */
  46      -static kmem_cache_t *req_cache;         /* kmem cache for timeout request */
  47      -
  48      -/*
  49      - * taskq parameters for cyclic_timer
       43 + * The ddi_periodic_add(9F) Implementation
  50   44   *
  51      - * timer_taskq_num:
  52      - * timer_taskq_num represents the number of taskq threads.
  53      - * Currently 4 threads are pooled to handle periodic timeout requests.
  54      - * This number is chosen based on the fact that the callout (one-time
  55      - * timeout framework) uses 8 threads with TQ_NOSLEEP; the periodic timeout
  56      - * calls taskq_dispatch() with TQ_SLEEP instead, and in this case, 4 threads
  57      - * should be sufficient to handle periodic timeout requests. (see also
  58      - * timer_taskq_max_num below)
       45 + * This file contains the implementation of the ddi_periodic_add(9F) interface.
       46 + * It is a thin wrapper around the cyclic subsystem (see documentation in
       47 + * uts/common/os/cyclic.c), providing a DDI interface for registering
       48 + * (and unregistering) callbacks for periodic invocation at arbitrary
       49 + * interrupt levels, or in kernel context.
  59   50   *
  60      - * timer_taskq_min_num:
  61      - * timer_taskq_min_num represents the number of pre-populated taskq_ent
  62      - * structures, and this variable holds the same value as timer_taskq_num does.
       51 + * Each call to ddi_periodic_add will result in a new opaque handle, as
       52 + * allocated from an id_space, a new "periodic" object (ddi_periodic_impl_t)
       53 + * and a registered cyclic.
  63   54   *
  64      - * timer_taskq_max_num:
  65      - * Since TQ_SLEEP is set when taskq_dispatch() is called, the framework waits
  66      - * for one second if more taskq_ent structures than timer_taskq_max_num are
  67      - * required. However, from the timeout point of view, one second is much longer
  68      - * than expected, and to prevent this occurrence, timer_taskq_max_num should
  69      - * hold a sufficiently-large value, which is 128 here. Note that since the size
  70      - * of taskq_ent_t is relatively small, this doesn't use up the resource so much.
  71      - * (Currently the size is less than 8k at most)
       55 + * Operation
  72   56   *
  73      - * About the detailed explanation of the taskq function arguments, please see
  74      - * usr/src/uts/common/os/taskq.c.
       57 + * Whenever the cyclic fires, our cyclic handler checks that the particular
       58 + * periodic is not dispatched already (we do not support overlapping execution
       59 + * of the consumer's handler function), and not yet cancelled.  If both of
       60 + * these conditions hold, we mark the periodic as DPF_DISPATCHED and enqueue it
       61 + * to either the taskq (for DDI_IPL_0) or to one of the soft interrupt queues
       62 + * (DDI_IPL_1 to DDI_IPL_10).
       63 + *
       64 + * While the taskq (or soft interrupt handler) is handling a particular
       65 + * periodic, we mark it as DPF_EXECUTING.  When complete, we reset both
       66 + * DPF_DISPATCHED and DPF_EXECUTING.
       67 + *
       68 + * Cancellation
       69 + *
       70 + * ddi_periodic_delete(9F) historically had spectacularly loose semantics with
       71 + * respect to cancellation concurrent with handler execution.  These semantics
       72 + * are now tighter:
       73 + *
       74 + *   1. At most one invocation of ddi_periodic_delete(9F) will actually
       75 + *      perform the deletion, all others will return immediately.
       76 + *   2. The invocation that performs the deletion will _block_ until
       77 + *      the handler is no longer running, and all resources have been
       78 + *      released.
       79 + *
       80 + * We affect this model by removing the cancelling periodic from the
       81 + * global list and marking it DPF_CANCELLED.  This will prevent further
       82 + * execution of the handler.  We then wait on a CV until the DPF_EXECUTING
       83 + * and DPF_DISPATCHED flags are clear, which means the periodic is removed
       84 + * from all request queues, is no longer executing, and may be freed.  At this
       85 + * point we return the opaque ID to the id_space and free the memory.
       86 + *
       87 + * NOTE:
       88 + * The ddi_periodic_add(9F) interface is presently limited to a minimum period
       89 + * of 10ms between firings.
  75   90   */
  76      -int timer_taskq_num = 4;                /* taskq thread number */
  77      -int timer_taskq_min_num = 4;            /* min. number of taskq_ent structs */
  78      -int timer_taskq_max_num = 128;          /* max. number of taskq_ent structs */
  79      -static taskq_t *tm_taskq;               /* taskq thread pool */
  80      -static kthread_t *tm_work_thread;       /* work thread invoking taskq */
  81   91  
  82   92  /*
  83      - * timer variables
       93 + * Tuneables:
  84   94   */
  85      -static cyc_timer_t *ddi_timer;          /* ddi timer based on the cyclic */
  86      -static volatile hrtime_t timer_hrtime;  /* current tick time on the timer */
       95 +int ddi_periodic_max_id = 1024;
       96 +int ddi_periodic_taskq_threadcount = 4;
       97 +hrtime_t ddi_periodic_resolution = 10000000;
  87   98  
  88   99  /*
  89      - * Variable used for the suspend/resume.
      100 + * Globals:
  90  101   */
  91      -static volatile boolean_t timer_suspended;
      102 +static kmem_cache_t *periodic_cache;
      103 +static id_space_t *periodic_id_space;
      104 +static taskq_t *periodic_taskq;
  92  105  
  93  106  /*
  94      - * Kernel taskq queue to ddi timer
      107 + * periodics_lock protects the list of all periodics (periodics), and
      108 + * each of the soft interrupt request queues (periodic_softint_queue).
      109 + *
      110 + * Do not hold an individual periodic's lock while obtaining periodics_lock.
      111 + * While in the periodic_softint_queue list, the periodic will be marked
      112 + * DPF_DISPATCHED, and thus safe from frees.  Only the invocation of
      113 + * i_untimeout() that removes the periodic from the global list is allowed
      114 + * to free it.
  95  115   */
  96      -static list_t kern_queue;       /* kernel thread request queue */
  97      -static kcondvar_t kern_cv;      /* condition variable for taskq queue */
      116 +static kmutex_t periodics_lock;
      117 +static list_t periodics;
      118 +static list_t periodic_softint_queue[10]; /* for IPL1 up to IPL10 */
  98  119  
  99      -/*
 100      - * Software interrupt queue dedicated to ddi timer
 101      - */
 102      -static list_t intr_queue;       /* software interrupt request queue */
 103      -static uint_t intr_state;       /* software interrupt state */
      120 +typedef enum periodic_ipl {
      121 +        PERI_IPL_0 = 0,
      122 +        PERI_IPL_1,
      123 +        PERI_IPL_2,
      124 +        PERI_IPL_3,
      125 +        PERI_IPL_4,
      126 +        PERI_IPL_5,
      127 +        PERI_IPL_6,
      128 +        PERI_IPL_7,
      129 +        PERI_IPL_8,
      130 +        PERI_IPL_9,
      131 +        PERI_IPL_10,
      132 +} periodic_ipl_t;
 104  133  
 105  134  /*
 106      - * This lock is used to protect the intr_queue and kern_queue.
 107      - * It's also used to protect the intr_state which represents the software
 108      - * interrupt state for the timer.
      135 + * This function may be called either from a soft interrupt handler
      136 + * (ddi_periodic_softintr), or as a taskq worker function.
 109  137   */
 110      -static kmutex_t disp_req_lock;
 111      -
 112      -/*
 113      - * the periodic timer interrupt priority level
 114      - */
 115      -enum {
 116      -        TM_IPL_0 = 0,                   /* kernel context */
 117      -        TM_IPL_1, TM_IPL_2, TM_IPL_3,   /* level 1-3 */
 118      -        TM_IPL_4, TM_IPL_5, TM_IPL_6,   /* level 4-6 */
 119      -        TM_IPL_7, TM_IPL_8, TM_IPL_9,   /* level 7-9 */
 120      -        TM_IPL_10                       /* level 10 */
 121      -};
 122      -
 123      -/*
 124      - * A callback handler used by CPR to stop and resume callouts.
 125      - * Since the taskq uses TASKQ_CPR_SAFE, the function just set the boolean
 126      - * flag to timer_suspended here.
 127      - */
 128      -/*ARGSUSED*/
 129      -static boolean_t
 130      -timer_cpr_callb(void *arg, int code)
      138 +static void
      139 +periodic_execute(void *arg)
 131  140  {
 132      -        timer_suspended = (code == CB_CODE_CPR_CHKPT);
 133      -        return (B_TRUE);
 134      -}
      141 +        ddi_periodic_impl_t *dpr = arg;
      142 +        mutex_enter(&dpr->dpr_lock);
 135  143  
 136      -/*
 137      - * Return a proposed timeout request id. add_req() determines whether
 138      - * or not the proposed one is used. If it's not suitable, add_req()
 139      - * recalls get_req_cnt(). To reduce the lock contention between the
 140      - * timer and i_untimeout(), the atomic instruction should be used here.
 141      - */
 142      -static timeout_t
 143      -get_req_cnt(void)
 144      -{
 145      -        static volatile ulong_t timeout_cnt = 0;
 146      -        return ((timeout_t)atomic_inc_ulong_nv(&timeout_cnt));
 147      -}
      144 +        /*
      145 +         * We must be DISPATCHED, but not yet EXECUTING:
      146 +         */
      147 +        VERIFY((dpr->dpr_flags & (DPF_DISPATCHED | DPF_EXECUTING)) ==
      148 +            DPF_DISPATCHED);
 148  149  
 149      -/*
 150      - * Get the system resolution.
 151      - * Note. currently there is a restriction about the system resolution, and
 152      - * the 10ms tick (the default clock resolution) is only supported now.
 153      - */
 154      -static hrtime_t
 155      -i_get_res(void)
 156      -{
 157      -        return ((hrtime_t)10000000); /* 10ms tick only */
 158      -}
      150 +        if (!(dpr->dpr_flags & DPF_CANCELLED)) {
      151 +                int level = dpr->dpr_level;
      152 +                uint64_t count = dpr->dpr_fire_count;
      153 +                /*
      154 +                 * If we have not yet been cancelled, then
      155 +                 * mark us executing:
      156 +                 */
      157 +                dpr->dpr_flags |= DPF_EXECUTING;
      158 +                mutex_exit(&dpr->dpr_lock);
 159  159  
 160      -/*
 161      - * Return the value for the cog of the timing wheel.
 162      - * TICK_FACTOR is used to gain a finer cog on the clock resolution.
 163      - */
 164      -static hrtime_t
 165      -tw_tick(hrtime_t time)
 166      -{
 167      -        return ((time << TICK_FACTOR) / ddi_timer->res);
 168      -}
      160 +                /*
      161 +                 * Execute the handler, without holding locks:
      162 +                 */
      163 +                DTRACE_PROBE4(ddi__periodic__execute, void *, dpr->dpr_handler,
      164 +                    void *, dpr->dpr_arg, int, level, uint64_t, count);
      165 +                (*dpr->dpr_handler)(dpr->dpr_arg);
      166 +                DTRACE_PROBE4(ddi__periodic__done, void *, dpr->dpr_handler,
      167 +                    void *, dpr->dpr_arg, int, level, uint64_t, count);
 169  168  
 170      -/*
 171      - * Calculate the expiration time for the timeout request.
 172      - */
 173      -static hrtime_t
 174      -expire_tick(tm_req_t *req)
 175      -{
 176      -        return (tw_tick(req->exp_time));
 177      -}
      169 +                mutex_enter(&dpr->dpr_lock);
      170 +                dpr->dpr_fire_count++;
      171 +        }
 178  172  
 179      -/*
 180      - * Register a timeout request to the timer. This function is used
 181      - * in i_timeout().
 182      - */
 183      -static timeout_t
 184      -add_req(tm_req_t *req)
 185      -{
 186      -        timer_tw_t *tid, *tw;
 187      -        tm_req_t *next;
 188      -        timeout_t id;
 189      -
 190      -retry:
 191  173          /*
 192      -         * Retrieve a timeout request id. Since i_timeout() needs to return
 193      -         * a non-zero value, re-try if the zero is gotten.
      174 +         * We're done with this periodic for now, so release it and
      175 +         * wake anybody that was waiting for us to be finished:
 194  176           */
 195      -        if ((id = get_req_cnt()) == 0)
 196      -                id = get_req_cnt();
      177 +        dpr->dpr_flags &= ~(DPF_DISPATCHED | DPF_EXECUTING);
      178 +        cv_broadcast(&dpr->dpr_cv);
      179 +        mutex_exit(&dpr->dpr_lock);
      180 +}
 197  181  
 198      -        /*
 199      -         * Check if the id is not used yet. Since the framework now deals
 200      -         * with the periodic timeout requests, we cannot assume the id
 201      -         * allocated (long) before doesn't exist any more when it will
 202      -         * be re-assigned again (especially on 32bit) but need to handle
 203      -         * this case to solve the conflicts. If it's used already, retry
 204      -         * another.
 205      -         */
 206      -        tid = &ddi_timer->idhash[TM_HASH((uintptr_t)id)];
 207      -        mutex_enter(&tid->lock);
 208      -        for (next = list_head(&tid->req); next != NULL;
 209      -            next = list_next(&tid->req, next)) {
 210      -                if (next->id == id) {
 211      -                        mutex_exit(&tid->lock);
 212      -                        goto retry;
 213      -                }
 214      -        }
 215      -        /* Nobody uses this id yet */
 216      -        req->id = id;
      182 +void
      183 +ddi_periodic_softintr(int level)
      184 +{
      185 +        ddi_periodic_impl_t *dpr;
      186 +        VERIFY(level >= PERI_IPL_1 && level <= PERI_IPL_10);
 217  187  
      188 +        mutex_enter(&periodics_lock);
 218  189          /*
 219      -         * Register this request to the timer.
 220      -         * The list operation must be list_insert_head().
 221      -         * Other operations can degrade performance.
      190 +         * Pull the first scheduled periodic off the queue for this priority
      191 +         * level:
 222  192           */
 223      -        list_insert_head(&tid->req, req);
 224      -        mutex_exit(&tid->lock);
 225      -
 226      -        tw = &ddi_timer->exhash[TM_HASH(expire_tick(req))];
 227      -        mutex_enter(&tw->lock);
 228      -        /*
 229      -         * Other operations than list_insert_head() can
 230      -         * degrade performance here.
 231      -         */
 232      -        list_insert_head(&tw->req, req);
 233      -        mutex_exit(&tw->lock);
 234      -
 235      -        return (id);
      193 +        while ((dpr = list_remove_head(&periodic_softint_queue[level - 1]))
      194 +            != NULL) {
      195 +                mutex_exit(&periodics_lock);
      196 +                /*
      197 +                 * And execute it:
      198 +                 */
      199 +                periodic_execute(dpr);
      200 +                mutex_enter(&periodics_lock);
      201 +        }
      202 +        mutex_exit(&periodics_lock);
 236  203  }
 237  204  
 238      -/*
 239      - * Periodic timeout requests cannot be removed until they are canceled
 240      - * explicitly. Until then, they need to be re-registerd after they are
 241      - * fired. transfer_req() re-registers the requests for the next fires.
 242      - * Note. transfer_req() sends the cv_signal to timeout_execute(), which
 243      - * runs in interrupt context. Make sure this function will not be blocked,
 244      - * otherwise the deadlock situation can occur.
 245      - */
 246      -static void
 247      -transfer_req(tm_req_t *req, timer_tw_t *tw)
      205 +void
      206 +ddi_periodic_init(void)
 248  207  {
 249      -        timer_tw_t *new_tw;
 250      -        hrtime_t curr_time;
 251      -        ASSERT(tw && MUTEX_HELD(&tw->lock));
      208 +        int i;
 252  209  
 253      -        /* Calculate the next expiration time by interval */
 254      -        req->exp_time += req->interval;
 255      -        curr_time = gethrtime();
 256      -
 257  210          /*
 258      -         * If a long time (more than 1 clock resolution) has already
 259      -         * passed for some reason (e.g. debugger or high interrupt),
 260      -         * round up the next expiration to the appropriate one
 261      -         * since this request is periodic and never catches with it.
      211 +         * Create a kmem_cache for request tracking objects, and a list
      212 +         * to store them in so we can later delete based on opaque handles:
 262  213           */
 263      -        if (curr_time - req->exp_time >= ddi_timer->res) {
 264      -                req->exp_time = roundup(curr_time + req->interval,
 265      -                    ddi_timer->res);
 266      -        }
      214 +        periodic_cache = kmem_cache_create("ddi_periodic",
      215 +            sizeof (ddi_periodic_impl_t), 0, NULL, NULL, NULL, NULL, NULL, 0);
      216 +        list_create(&periodics, sizeof (ddi_periodic_impl_t),
      217 +            offsetof(ddi_periodic_impl_t, dpr_link));
 267  218  
 268  219          /*
 269      -         * Re-register this request.
 270      -         * Note. since it is guaranteed that the timer is invoked on only
 271      -         * one CPU at any time (by the cyclic subsystem), a deadlock
 272      -         * cannot occur regardless of the lock order here.
      220 +         * Initialise the identifier space for ddi_periodic_add(9F):
 273  221           */
 274      -        new_tw = &ddi_timer->exhash[TM_HASH(expire_tick(req))];
      222 +        periodic_id_space = id_space_create("ddi_periodic", 1,
      223 +            ddi_periodic_max_id);
 275  224  
 276  225          /*
 277      -         * If it's on the timer cog already, there is nothing
 278      -         * to do. Just return.
      226 +         * Initialise the request queue for each soft interrupt level:
 279  227           */
 280      -        if (new_tw == tw)
 281      -                return;
      228 +        for (i = PERI_IPL_1; i <= PERI_IPL_10; i++) {
      229 +                list_create(&periodic_softint_queue[i - 1],
      230 +                    sizeof (ddi_periodic_impl_t), offsetof(ddi_periodic_impl_t,
      231 +                    dpr_softint_link));
      232 +        }
 282  233  
 283      -        /* Remove this request from the timer */
 284      -        list_remove(&tw->req, req);
 285      -
 286      -        /* Re-register this request to the timer */
 287      -        mutex_enter(&new_tw->lock);
 288      -
 289  234          /*
 290      -         * Other operations than list_insert_head() can
 291      -         * degrade performance here.
      235 +         * Create the taskq for running PERI_IPL_0 handlers.  This taskq will
      236 +         * _only_ be used with taskq_dispatch_ent(), and a taskq_ent_t
      237 +         * pre-allocated with the ddi_periodic_impl_t.
 292  238           */
 293      -        list_insert_head(&new_tw->req, req);
 294      -        mutex_exit(&new_tw->lock);
      239 +        periodic_taskq = taskq_create_instance("ddi_periodic_taskq", -1,
      240 +            ddi_periodic_taskq_threadcount, maxclsyspri, 0, 0, 0);
 295  241  
 296  242          /*
 297      -         * Set the TM_TRANSFER flag and notify the request is transfered
 298      -         * completely. This prevents a race in the case that this request
 299      -         * is serviced on another CPU already.
      243 +         * Initialize the mutex lock used for the soft interrupt request
      244 +         * queues.
 300  245           */
 301      -        mutex_enter(&req->lock);
 302      -        req->flags |= TM_TRANSFER;
 303      -        cv_signal(&req->cv);
 304      -        mutex_exit(&req->lock);
      246 +        mutex_init(&periodics_lock, NULL, MUTEX_ADAPTIVE, NULL);
 305  247  }
 306  248  
 307      -/*
 308      - * Execute timeout requests.
 309      - * Note. since timeout_execute() can run in interrupt context and block
 310      - * on condition variables, there are restrictions on the timer code that
 311      - * signals these condition variables (see i_untimeout(), transfer_req(),
 312      - * and condvar(9F)). Functions that signal these cvs must ensure that
 313      - * they will not be blocked (for memory allocations or any other reason)
 314      - * since condition variables don't support priority inheritance.
 315      - */
 316  249  static void
 317      -timeout_execute(void *arg)
      250 +periodic_cyclic_handler(void *arg)
 318  251  {
 319      -        tm_req_t *req = (tm_req_t *)arg;
 320      -        ASSERT(req->flags & TM_INVOKING && !(req->flags & TM_EXECUTING));
      252 +        extern void sir_on(int);
      253 +        ddi_periodic_impl_t *dpr = arg;
 321  254  
 322      -        for (;;) {
 323      -                /*
 324      -                 * Check if this request is canceled. If it's canceled, do not
 325      -                 * execute this request.
 326      -                 */
 327      -                mutex_enter(&req->lock);
 328      -                if (!(req->flags & TM_CANCEL)) {
 329      -                        /*
 330      -                         * Set the current thread to prevent a dead lock
 331      -                         * situation in case that this timeout request is
 332      -                         * canceled in the handler being invoked now.
 333      -                         * (this doesn't violate the spec) Set TM_EXECUTING
 334      -                         * to show this handler is invoked soon.
 335      -                         */
 336      -                        req->h_thread = curthread;
 337      -                        req->flags |= TM_EXECUTING;
 338      -                        mutex_exit(&req->lock);
      255 +        mutex_enter(&dpr->dpr_lock);
      256 +        /*
      257 +         * If we've been cancelled, or we're already dispatched, then exit
      258 +         * immediately:
      259 +         */
      260 +        if (dpr->dpr_flags & (DPF_CANCELLED | DPF_DISPATCHED)) {
      261 +                mutex_exit(&dpr->dpr_lock);
      262 +                return;
      263 +        }
      264 +        VERIFY(!(dpr->dpr_flags & DPF_EXECUTING));
 339  265  
 340      -                        /* The handler is invoked without holding any locks */
 341      -                        (*req->handler)(req->arg);
      266 +        /*
      267 +         * This periodic is not presently dispatched, so dispatch it now:
      268 +         */
      269 +        dpr->dpr_flags |= DPF_DISPATCHED;
      270 +        mutex_exit(&dpr->dpr_lock);
 342  271  
 343      -                        mutex_enter(&req->lock);
 344      -                }
 345      -
      272 +        if (dpr->dpr_level == PERI_IPL_0) {
 346  273                  /*
 347      -                 * Check if this request is canceled or not. If not, prepare
 348      -                 * for the next fire.
      274 +                 * DDI_IPL_0 periodics are dispatched onto the taskq:
 349  275                   */
 350      -                if (req->flags & TM_CANCEL) {
 351      -                        timer_tw_t *tw;
 352      -                        /*
 353      -                         * Wait until the timer finishes all things for
 354      -                         * this request.
 355      -                         */
 356      -                        while (!(req->flags & TM_TRANSFER))
 357      -                                cv_wait(&req->cv, &req->lock);
 358      -                        mutex_exit(&req->lock);
 359      -                        ASSERT(req->flags & TM_TRANSFER);
 360      -
 361      -                        /* Remove this request from the timer */
 362      -                        tw = &ddi_timer->exhash[TM_HASH(expire_tick(req))];
 363      -                        mutex_enter(&tw->lock);
 364      -                        list_remove(&tw->req, req);
 365      -                        mutex_exit(&tw->lock);
 366      -
 367      -                        /* Free this request */
 368      -                        kmem_cache_free(req_cache, req);
 369      -                        return;
 370      -                }
 371      -                ASSERT(req->flags & TM_EXECUTING);
 372      -
      276 +                taskq_dispatch_ent(periodic_taskq, periodic_execute,
      277 +                    dpr, 0, &dpr->dpr_taskq_ent);
      278 +        } else {
 373  279                  /*
 374      -                 * TM_EXECUTING must be set at this point.
 375      -                 * Unset the flag.
      280 +                 * Higher priority periodics are handled by a soft
      281 +                 * interrupt handler.  Enqueue us for processing and
      282 +                 * fire the soft interrupt:
 376  283                   */
 377      -                req->flags &= ~(TM_EXECUTING | TM_TRANSFER);
      284 +                mutex_enter(&periodics_lock);
      285 +                list_insert_tail(&periodic_softint_queue[dpr->dpr_level - 1],
      286 +                    dpr);
      287 +                mutex_exit(&periodics_lock);
 378  288  
 379  289                  /*
 380      -                 * Decrease the request cnt. The reqest cnt shows
 381      -                 * how many times this request is executed now.
 382      -                 * If this counter becomes the zero, drop TM_INVOKING
 383      -                 * to show there is no requests to do now.
      290 +                 * Raise the soft interrupt level for this periodic:
 384  291                   */
 385      -                req->cnt--;
 386      -                if (req->cnt == 0) {
 387      -                        req->flags &= ~TM_INVOKING;
 388      -                        mutex_exit(&req->lock);
 389      -                        return;
 390      -                }
 391      -                mutex_exit(&req->lock);
      292 +                sir_on(dpr->dpr_level);
 392  293          }
 393  294  }
 394  295  
 395      -/*
 396      - * Timeout worker thread for processing task queue.
 397      - */
 398  296  static void
 399      -timeout_taskq_thread(void *arg)
      297 +periodic_destroy(ddi_periodic_impl_t *dpr)
 400  298  {
 401      -        _NOTE(ARGUNUSED(arg));
 402      -        tm_req_t *kern_req;
 403      -        callb_cpr_t cprinfo;
 404      -
 405      -        CALLB_CPR_INIT(&cprinfo, &disp_req_lock, callb_generic_cpr,
 406      -            "timeout_taskq_thread");
 407      -
 408      -        /*
 409      -         * This thread is wakened up when a new request is added to
 410      -         * the queue. Then pick up all requests and dispatch them
 411      -         * via taskq_dispatch().
 412      -         */
 413      -        for (;;) {
 414      -                /*
 415      -                 * Check the queue and pick up a request if the queue
 416      -                 * is not NULL.
 417      -                 */
 418      -                mutex_enter(&disp_req_lock);
 419      -                while ((kern_req = list_head(&kern_queue)) == NULL) {
 420      -                        CALLB_CPR_SAFE_BEGIN(&cprinfo);
 421      -                        cv_wait(&kern_cv, &disp_req_lock);
 422      -                        CALLB_CPR_SAFE_END(&cprinfo, &disp_req_lock);
 423      -                }
 424      -                list_remove(&kern_queue, kern_req);
 425      -                mutex_exit(&disp_req_lock);
 426      -
 427      -                /* Execute the timeout request via the taskq thread */
 428      -                (void) taskq_dispatch(tm_taskq, timeout_execute,
 429      -                    (void *)kern_req, TQ_SLEEP);
 430      -        }
 431      -}
 432      -
 433      -/*
 434      - * Dispatch the timeout request based on the level specified.
 435      - * If the level is equal to zero, notify the worker thread to
 436      - * call taskq_dispatch() in kernel context. If the level is bigger
 437      - * than zero, add a software interrupt request to the queue and raise
 438      - * the interrupt level to the specified one.
 439      - */
 440      -static void
 441      -timeout_dispatch(tm_req_t *req)
 442      -{
 443      -        int level = req->level;
 444      -        extern void sir_on(int);
 445      -
 446      -        if (level == TM_IPL_0) {
 447      -                /* Add a new request to the tail */
 448      -                mutex_enter(&disp_req_lock);
 449      -                list_insert_tail(&kern_queue, req);
 450      -                mutex_exit(&disp_req_lock);
 451      -
 452      -                /*
 453      -                 * notify the worker thread that this request
 454      -                 * is newly added to the queue.
 455      -                 * Note. this cv_signal() can be called after the
 456      -                 * mutex_lock.
 457      -                 */
 458      -                cv_signal(&kern_cv);
 459      -        } else {
 460      -                /* Add a new request to the tail */
 461      -                mutex_enter(&disp_req_lock);
 462      -                list_insert_tail(&intr_queue, req);
 463      -
 464      -                /* Issue the software interrupt */
 465      -                if (intr_state & TM_INTR_START(level)) {
 466      -                        /*
 467      -                         * timer_softintr() is already running; no need to
 468      -                         * raise a siron. Due to lock protection of
 469      -                         * the intr_queue and intr_state, we know that
 470      -                         * timer_softintr() will see the new addition to
 471      -                         * the intr_queue.
 472      -                         */
 473      -                        mutex_exit(&disp_req_lock);
 474      -                } else {
 475      -                        intr_state |= TM_INTR_SET(level);
 476      -                        mutex_exit(&disp_req_lock);
 477      -
 478      -                        /* Raise an interrupt to execute timeout requests */
 479      -                        sir_on(level);
 480      -                }
 481      -        }
 482      -}
 483      -
 484      -/*
 485      - * Check the software interrupt queue and invoke requests at the specified
 486      - * interrupt level.
 487      - * Note that the queue may change during call so that the disp_req_lock
 488      - * and the intr_state are used to protect it.
 489      - * The software interrupts supported here are up to the level 10. Higher
 490      - * than 10 interrupts cannot be supported.
 491      - */
 492      -void
 493      -timer_softintr(int level)
 494      -{
 495      -        tm_req_t *intr_req;
 496      -        ASSERT(level >= TM_IPL_1 && level <= TM_IPL_10);
 497      -
 498      -        /* Check if we are asked to process the softcall list */
 499      -        mutex_enter(&disp_req_lock);
 500      -        if (!(intr_state & TM_INTR_SET(level))) {
 501      -                mutex_exit(&disp_req_lock);
      299 +        if (dpr == NULL)
 502  300                  return;
 503      -        }
 504  301  
 505      -        /* Notify this software interrupt request will be executed soon */
 506      -        intr_state |= TM_INTR_START(level);
 507      -        intr_state &= ~TM_INTR_SET(level);
 508      -
 509      -        /* loop the link until there is no requests */
 510      -        for (intr_req = list_head(&intr_queue); intr_req != NULL;
 511      -            /* Nothing */) {
 512      -
 513      -                /* Check the interrupt level */
 514      -                if (intr_req->level != level) {
 515      -                        intr_req = list_next(&intr_queue, intr_req);
 516      -                        continue;
 517      -                }
 518      -                list_remove(&intr_queue, intr_req);
 519      -                mutex_exit(&disp_req_lock);
 520      -
 521      -                /* Execute the software interrupt request */
 522      -                timeout_execute(intr_req);
 523      -
 524      -                mutex_enter(&disp_req_lock);
 525      -                /* Restart the loop since new requests might be added */
 526      -                intr_req = list_head(&intr_queue);
 527      -        }
 528      -
 529      -        /* reset the interrupt state */
 530      -        intr_state &= ~TM_INTR_START(level);
 531      -        mutex_exit(&disp_req_lock);
 532      -}
 533      -
 534      -/*
 535      - *  void
 536      - *  cyclic_timer(void)
 537      - *
 538      - *  Overview
 539      - *   cyclic_timer() is a function invoked periodically by the cyclic
 540      - *   subsystem.
 541      - *
 542      - *   The function calls timeout_invoke() with timeout requests whose
 543      - *   expiration time is already reached.
 544      - *
 545      - *  Arguments
 546      - *   Nothing
 547      - *
 548      - *  Return value
 549      - *   Nothing
 550      - */
 551      -void
 552      -cyclic_timer(void)
 553      -{
 554      -        tm_req_t *req;
 555      -        timer_tw_t *tw;
 556      -        hrtime_t curr_tick, curr;
 557      -
 558      -        /* If the system is suspended, just return */
 559      -        if (timer_suspended)
 560      -                return;
 561      -
 562      -        /* Get the current time */
 563      -        timer_hrtime = ddi_timer->tick_time = curr = gethrtime();
 564      -        curr_tick = tw_tick(ddi_timer->tick_time);
 565      -
 566      -restart:
 567  302          /*
 568      -         * Check the timer cogs to see if there are timeout requests
 569      -         * who reach the expiration time. Call timeout_invoke() to execute
 570      -         * the requests, then.
      303 +         * By now, we should have a periodic that is not busy, and has been
      304 +         * cancelled:
 571  305           */
 572      -        while (curr_tick >= ddi_timer->tick) {
 573      -                tm_req_t *next;
 574      -                tw = &ddi_timer->exhash[TM_HASH(ddi_timer->tick)];
 575      -                mutex_enter(&tw->lock);
 576      -                for (req = list_head(&tw->req); req != NULL; req = next) {
 577      -                        next = list_next(&tw->req, req);
 578      -                        /*
 579      -                         * If this request is already obsolete, free
 580      -                         * it here.
 581      -                         */
 582      -                        if (req->flags & TM_UTMCOMP) {
 583      -                                /*
 584      -                                 * Remove this request from the timer,
 585      -                                 * then free it.
 586      -                                 */
 587      -                                list_remove(&tw->req, req);
 588      -                                kmem_cache_free(req_cache, req);
 589      -                        } else if (curr >= req->exp_time) {
 590      -                                mutex_enter(&req->lock);
 591      -                                /*
 592      -                                 * Check if this request is canceled, but not
 593      -                                 * being executed now.
 594      -                                 */
 595      -                                if (req->flags & TM_CANCEL &&
 596      -                                    !(req->flags & TM_INVOKING)) {
 597      -                                        mutex_exit(&req->lock);
 598      -                                        continue;
 599      -                                }
 600      -                                /*
 601      -                                 * Record how many times timeout_execute()
 602      -                                 * must be invoked.
 603      -                                 */
 604      -                                req->cnt++;
 605      -                                /*
 606      -                                 * Invoke timeout_execute() via taskq or
 607      -                                 * software interrupt.
 608      -                                 */
 609      -                                if (req->flags & TM_INVOKING) {
 610      -                                        /*
 611      -                                         * If it's already invoked,
 612      -                                         * There is nothing to do.
 613      -                                         */
 614      -                                        mutex_exit(&req->lock);
 615      -                                } else {
 616      -                                        req->flags |= TM_INVOKING;
 617      -                                        mutex_exit(&req->lock);
 618      -                                        /*
 619      -                                         * Dispatch this timeout request.
 620      -                                         * timeout_dispatch() chooses either
 621      -                                         * a software interrupt or taskq thread
 622      -                                         * based on the level.
 623      -                                         */
 624      -                                        timeout_dispatch(req);
 625      -                                }
 626      -                                /*
 627      -                                 * Periodic timeout requests must prepare for
 628      -                                 * the next fire.
 629      -                                 */
 630      -                                transfer_req(req, tw);
 631      -                        }
 632      -                }
 633      -                mutex_exit(&tw->lock);
 634      -                ddi_timer->tick++;
 635      -        }
      306 +        VERIFY(dpr->dpr_flags == DPF_CANCELLED);
 636  307  
 637      -        /*
 638      -         * Check the current time. If we spend some amount of time,
 639      -         * double-check if some of the requests reaches the expiration
 640      -         * time during the work.
 641      -         */
 642      -        curr = gethrtime();
 643      -        curr_tick = tw_tick(curr);
 644      -        if (curr_tick >= ddi_timer->tick) {
 645      -                ddi_timer->tick -= 1;
 646      -                goto restart;
 647      -        }
 648      -        /* Adjustment for the next rolling */
 649      -        ddi_timer->tick -= 1;
      308 +        id_free(periodic_id_space, dpr->dpr_id);
      309 +        kmem_cache_free(periodic_cache, dpr);
 650  310  }
 651  311  
 652      -/*
 653      - *  void
 654      - *  timer_init(void)
 655      - *
 656      - *  Overview
 657      - *    timer_init() allocates the internal data structures used by
 658      - *    i_timeout(), i_untimeout() and the timer.
 659      - *
 660      - *  Arguments
 661      - *    Nothing
 662      - *
 663      - *  Return value
 664      - *    Nothing
 665      - *
 666      - *  Caller's context
 667      - *    timer_init() can be called in kernel context only.
 668      - */
 669      -void
 670      -timer_init(void)
      312 +static ddi_periodic_impl_t *
      313 +periodic_create(void)
 671  314  {
 672      -        int i;
      315 +        ddi_periodic_impl_t *dpr;
 673  316  
 674      -        /* Create kmem_cache for timeout requests */
 675      -        req_cache = kmem_cache_create("timeout_request", sizeof (tm_req_t),
 676      -            0, NULL, NULL, NULL, NULL, NULL, 0);
      317 +        dpr = kmem_cache_alloc(periodic_cache, KM_SLEEP);
      318 +        bzero(dpr, sizeof (*dpr));
      319 +        dpr->dpr_id = id_alloc(periodic_id_space);
      320 +        mutex_init(&dpr->dpr_lock, NULL, MUTEX_ADAPTIVE, NULL);
      321 +        cv_init(&dpr->dpr_cv, NULL, CV_DEFAULT, NULL);
 677  322  
 678      -        /* Initialize the timer which is invoked by the cyclic subsystem */
 679      -        ddi_timer = kmem_alloc(sizeof (cyc_timer_t), KM_SLEEP);
 680      -        ddi_timer->res = nsec_per_tick;
 681      -        ddi_timer->tick = tw_tick(gethrtime());
 682      -        ddi_timer->tick_time = 0;
 683      -
 684      -        /* Initialize the timing wheel */
 685      -        bzero((char *)&ddi_timer->idhash[0], TM_HASH_SZ * sizeof (timer_tw_t));
 686      -        bzero((char *)&ddi_timer->exhash[0], TM_HASH_SZ * sizeof (timer_tw_t));
 687      -
 688      -        for (i = 0; i < TM_HASH_SZ; i++) {
 689      -                list_create(&ddi_timer->idhash[i].req, sizeof (tm_req_t),
 690      -                    offsetof(tm_req_t, id_req));
 691      -                mutex_init(&ddi_timer->idhash[i].lock, NULL, MUTEX_ADAPTIVE,
 692      -                    NULL);
 693      -
 694      -                list_create(&ddi_timer->exhash[i].req, sizeof (tm_req_t),
 695      -                    offsetof(tm_req_t, ex_req));
 696      -                mutex_init(&ddi_timer->exhash[i].lock, NULL, MUTEX_ADAPTIVE,
 697      -                    NULL);
 698      -        }
 699      -
 700      -        /* Create a taskq thread pool */
 701      -        tm_taskq = taskq_create_instance("timeout_taskq", 0,
 702      -            timer_taskq_num, MAXCLSYSPRI,
 703      -            timer_taskq_min_num, timer_taskq_max_num,
 704      -            TASKQ_PREPOPULATE | TASKQ_CPR_SAFE);
 705      -
 706      -        /*
 707      -         * Initialize the taskq queue which is dedicated to this timeout
 708      -         * interface/timer.
 709      -         */
 710      -        list_create(&kern_queue, sizeof (tm_req_t),
 711      -            offsetof(tm_req_t, disp_req));
 712      -
 713      -        /* Create a worker thread to dispatch the taskq thread */
 714      -        tm_work_thread = thread_create(NULL, 0, timeout_taskq_thread, NULL,
 715      -            0, &p0, TS_RUN, MAXCLSYSPRI);
 716      -
 717      -        /*
 718      -         * Initialize the software interrupt queue which is dedicated to
 719      -         * this timeout interface/timer.
 720      -         */
 721      -        list_create(&intr_queue, sizeof (tm_req_t),
 722      -            offsetof(tm_req_t, disp_req));
 723      -
 724      -        /*
 725      -         * Initialize the mutex lock used for both of kern_queue and
 726      -         * intr_queue.
 727      -         */
 728      -        mutex_init(&disp_req_lock, NULL, MUTEX_ADAPTIVE, NULL);
 729      -        cv_init(&kern_cv, NULL, CV_DEFAULT, NULL);
 730      -
 731      -        /* Register the callback handler for the system suspend/resume */
 732      -        (void) callb_add(timer_cpr_callb, 0, CB_CL_CPR_CALLOUT, "cyclicTimer");
      323 +        return (dpr);
 733  324  }
 734  325  
 735      -/*
 736      - *  timeout_t
 737      - *  i_timeout(void (*func)(void *), void *arg,  hrtime_t interval,
 738      - *      int level, int flags)
 739      - *
 740      - *  Overview
 741      - *    i_timeout() is an internal function scheduling the passed function
 742      - *    to be invoked in the interval in nanoseconds. The callback function
 743      - *    keeps invoked until the request is explicitly canceled by i_untimeout().
 744      - *    This function is used for ddi_periodic_add(9F).
 745      - *
 746      - *  Arguments
 747      - *
 748      - *    func: the callback function
 749      - *          the callback function will be invoked in kernel context if
 750      - *          the level passed is the zero. Otherwise be invoked in interrupt
 751      - *          context at the specified level by the argument "level".
 752      - *
 753      - *          Note that It's guaranteed by the cyclic subsystem that the
 754      - *          function is invoked on the only one CPU and is never executed
 755      - *          simultaneously even on MP system.
 756      - *
 757      - *     arg: the argument passed to the callback function
 758      - *
 759      - * interval: interval time in nanoseconds
 760      - *          if the interval is the zero, the timer resolution is used.
 761      - *
 762      - *  level : callback interrupt level
 763      - *          If the value is 0 (the zero), the callback function is invoked
 764      - *          in kernel context. If the value is more than 0 (the zero), but
 765      - *          less than or equal to 10, the callback function is invoked in
 766      - *          interrupt context at the specified interrupt level.
 767      - *          This value must be in range of 0-10.
 768      - *
 769      - *  Return value
 770      - *    returns a non-zero opaque value (timeout_t) on success.
 771      - *
 772      - *  Caller's context
 773      - *    i_timeout() can be called in user or kernel context.
 774      - */
 775  326  timeout_t
 776  327  i_timeout(void (*func)(void *), void *arg, hrtime_t interval, int level)
 777  328  {
 778      -        hrtime_t start_time = gethrtime(), res;
 779      -        tm_req_t *req = NULL;
      329 +        cyc_handler_t cyh;
      330 +        cyc_time_t cyt;
      331 +        ddi_periodic_impl_t *dpr;
 780  332  
 781      -        /* Allocate and initialize the timeout request */
 782      -        req = kmem_cache_alloc(req_cache, KM_SLEEP);
 783      -        req->handler = func;
 784      -        req->arg = arg;
 785      -        req->h_thread = NULL;
 786      -        req->level = level;
 787      -        req->flags = 0;
 788      -        req->cnt = 0;
 789      -        mutex_init(&req->lock, NULL, MUTEX_ADAPTIVE, NULL);
 790      -        cv_init(&req->cv, NULL, CV_DEFAULT, NULL);
      333 +        VERIFY(func != NULL);
      334 +        VERIFY(level >= 0 && level <= 10);
 791  335  
 792  336          /*
      337 +         * Allocate object to track this periodic:
      338 +         */
      339 +        dpr = periodic_create();
      340 +        dpr->dpr_level = level;
      341 +        dpr->dpr_handler = func;
      342 +        dpr->dpr_arg = arg;
      343 +
      344 +        /*
 793  345           * The resolution must be finer than or equal to
 794  346           * the requested interval. If it's not, set the resolution
 795  347           * to the interval.
 796  348           * Note. There is a restriction currently. Regardless of the
 797  349           * clock resolution used here, 10ms is set as the timer resolution.
 798  350           * Even on the 1ms resolution timer, the minimum interval is 10ms.
 799  351           */
 800      -        if ((res = i_get_res()) > interval) {
 801      -                uintptr_t pc = (uintptr_t)req->handler;
      352 +        if (ddi_periodic_resolution > interval) {
      353 +                uintptr_t pc = (uintptr_t)dpr->dpr_handler;
 802  354                  ulong_t off;
 803  355                  cmn_err(CE_WARN,
 804  356                      "The periodic timeout (handler=%s, interval=%lld) "
 805  357                      "requests a finer interval than the supported resolution. "
 806  358                      "It rounds up to %lld\n", kobj_getsymname(pc, &off),
 807      -                    interval, res);
 808      -                interval = res;
      359 +                    interval, ddi_periodic_resolution);
      360 +                interval = ddi_periodic_resolution;
 809  361          }
 810  362  
 811  363          /*
 812  364           * If the specified interval is already multiples of
 813  365           * the resolution, use it as is. Otherwise, it rounds
 814  366           * up to multiples of the timer resolution.
 815  367           */
 816      -        req->interval = roundup(interval, i_get_res());
      368 +        dpr->dpr_interval = roundup(interval, ddi_periodic_resolution);
 817  369  
 818  370          /*
 819      -         * For the periodic timeout requests, the first expiration time will
 820      -         * be adjusted to the timer tick edge to take advantage of the cyclic
 821      -         * subsystem. In that case, the first fire is likely not an expected
 822      -         * one, but the fires later can be more accurate due to this.
      371 +         * Create the underlying cyclic:
 823  372           */
 824      -        req->exp_time = roundup(start_time + req->interval, i_get_res());
      373 +        cyh.cyh_func = periodic_cyclic_handler;
      374 +        cyh.cyh_arg = dpr;
      375 +        cyh.cyh_level = CY_LOCK_LEVEL;
 825  376  
 826      -        /* Add the request to the timer */
 827      -        return (add_req(req));
      377 +        cyt.cyt_when = roundup(gethrtime() + dpr->dpr_interval,
      378 +            ddi_periodic_resolution);
      379 +        cyt.cyt_interval = dpr->dpr_interval;
      380 +
      381 +        mutex_enter(&cpu_lock);
      382 +        dpr->dpr_cyclic_id = cyclic_add(&cyh, &cyt);
      383 +        mutex_exit(&cpu_lock);
      384 +
      385 +        /*
      386 +         * Make the id visible to ddi_periodic_delete(9F) before we
      387 +         * return it:
      388 +         */
      389 +        mutex_enter(&periodics_lock);
      390 +        list_insert_tail(&periodics, dpr);
      391 +        mutex_exit(&periodics_lock);
      392 +
      393 +        return ((timeout_t)(uintptr_t)dpr->dpr_id);
 828  394  }
 829  395  
 830  396  /*
 831  397   *  void
 832  398   *  i_untimeout(timeout_t req)
 833  399   *
 834  400   *  Overview
 835  401   *    i_untimeout() is an internal function canceling the i_timeout()
 836  402   *    request previously issued.
 837  403   *    This function is used for ddi_periodic_delete(9F).
↓ open down ↓ 6 lines elided ↑ open up ↑
 844  410   *
 845  411   *  Caller's context
 846  412   *    i_untimeout() can be called in user, kernel or interrupt context.
 847  413   *    It cannot be called in high interrupt context.
 848  414   *
 849  415   *  Note. This function is used by ddi_periodic_delete(), which cannot
 850  416   *  be called in interrupt context. As a result, this function is called
 851  417   *  in user or kernel context only in practice.
 852  418   */
 853  419  void
 854      -i_untimeout(timeout_t timeout_req)
      420 +i_untimeout(timeout_t id)
 855  421  {
 856      -        timer_tw_t *tid;
 857      -        tm_req_t *req;
 858      -        timeout_t id;
      422 +        ddi_periodic_impl_t *dpr;
 859  423  
 860      -        /* Retrieve the id for this timeout request */
 861      -        id = (timeout_t)timeout_req;
 862      -        tid = &ddi_timer->idhash[TM_HASH((uintptr_t)id)];
 863      -
 864      -        mutex_enter(&tid->lock);
 865      -        for (req = list_head(&tid->req); req != NULL;
 866      -            req = list_next(&tid->req, req)) {
 867      -                if (req->id == id)
      424 +        /*
      425 +         * Find the periodic in the list of all periodics and remove it.
      426 +         * If we find in (and remove it from) the global list, we have
      427 +         * license to free it once it is no longer busy.
      428 +         */
      429 +        mutex_enter(&periodics_lock);
      430 +        for (dpr = list_head(&periodics); dpr != NULL; dpr =
      431 +            list_next(&periodics, dpr)) {
      432 +                if (dpr->dpr_id == (id_t)(uintptr_t)id) {
      433 +                        list_remove(&periodics, dpr);
 868  434                          break;
      435 +                }
 869  436          }
 870      -        if (req == NULL) {
 871      -                /* There is no requests with this id after all */
 872      -                mutex_exit(&tid->lock);
      437 +        mutex_exit(&periodics_lock);
      438 +
      439 +        /*
      440 +         * We could not find a periodic for this id, so bail out:
      441 +         */
      442 +        if (dpr == NULL)
 873  443                  return;
 874      -        }
 875      -        mutex_enter(&req->lock);
 876  444  
 877      -        /* Unregister this request first */
 878      -        list_remove(&tid->req, req);
      445 +        mutex_enter(&dpr->dpr_lock);
      446 +        /*
      447 +         * We should be the only one trying to cancel this periodic:
      448 +         */
      449 +        VERIFY(!(dpr->dpr_flags & DPF_CANCELLED));
      450 +        /*
      451 +         * Mark the periodic as cancelled:
      452 +         */
      453 +        dpr->dpr_flags |= DPF_CANCELLED;
      454 +        mutex_exit(&dpr->dpr_lock);
 879  455  
 880      -        /* Notify that this request is canceled */
 881      -        req->flags |= TM_CANCEL;
      456 +        /*
      457 +         * Cancel our cyclic.  cyclic_remove() guarantees that the cyclic
      458 +         * handler will not run again after it returns.  Note that the cyclic
      459 +         * handler merely _dispatches_ the periodic, so this does _not_ mean
      460 +         * the periodic handler is also finished running.
      461 +         */
      462 +        mutex_enter(&cpu_lock);
      463 +        cyclic_remove(dpr->dpr_cyclic_id);
      464 +        mutex_exit(&cpu_lock);
 882  465  
 883      -        /* Check if the handler is invoked */
 884      -        if (req->flags & TM_INVOKING) {
 885      -                /*
 886      -                 * This request will be removed by timeout_execute() later,
 887      -                 * so that there is no extra thing to do any more.
 888      -                 */
 889      -                mutex_exit(&req->lock);
 890      -                mutex_exit(&tid->lock);
 891      -                return;
 892      -        }
 893      -        mutex_exit(&req->lock);
 894      -        mutex_exit(&tid->lock);
 895      -
 896  466          /*
 897      -         * Notify untimeout() is about to be finished, and this request
 898      -         * can be freed.
      467 +         * Wait until the periodic handler is no longer running:
 899  468           */
 900      -        atomic_or_uint(&req->flags, TM_UTMCOMP);
      469 +        mutex_enter(&dpr->dpr_lock);
      470 +        while (dpr->dpr_flags & (DPF_DISPATCHED | DPF_EXECUTING)) {
      471 +                cv_wait(&dpr->dpr_cv, &dpr->dpr_lock);
      472 +        }
      473 +        mutex_exit(&dpr->dpr_lock);
      474 +
      475 +        periodic_destroy(dpr);
 901  476  }
    
XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX