rx: Don't adjust non-existent events
[openafs.git] / src / rx / rx_event.c
index 9d66070..0053b28 100644 (file)
 /*
- * Copyright 2000, International Business Machines Corporation and others.
- * All Rights Reserved.
- * 
- * This software has been released under the terms of the IBM Public
- * License.  For details, see the LICENSE file in the top-level source
- * directory or online at http://www.openafs.org/dl/license10.html
+ * Copyright (c) 2011 Your File System Inc. All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ *
+ * THIS SOFTWARE IS PROVIDED BY THE AUTHOR `AS IS'' AND ANY EXPRESS OR
+ * IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+ * IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT,
+ * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
+ * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+ * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+ * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF
+ * THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
+/* A reimplementation of the rx_event handler using red/black trees
+ *
+ * The first rx_event implementation used a simple sorted queue of all
+ * events, which lead to O(n^2) performance, where n is the number of
+ * outstanding events. This was found to scale poorly, so was replaced.
+ *
+ * The second implementation used a set of per-second buckets to store
+ * events. Each bucket (referred to as an epoch in the code) stored all
+ * of the events which expired in that second. However, on modern networks
+ * where RTT times are in the millisecond, most connections will have events
+ * expiring within the next second, so the problem reoccurs.
+ *
+ * This new implementation uses Red-Black trees to store a sorted list of
+ * events. Red Black trees are guaranteed to have no worse than O(log N)
+ * insertion, and are commonly used in timer applications
  */
 
 #include <afsconfig.h>
-#ifdef KERNEL
-#include "afs/param.h"
-#else
 #include <afs/param.h>
-#endif
-
-#ifdef AFS_SUN59_ENV
-#include <sys/time_impl.h>
-#endif
-
 
 #ifdef KERNEL
-#ifndef UKERNEL
-#include "afs/afs_osi.h"
-#else /* !UKERNEL */
-#include "afs/sysincludes.h"
-#include "afsincludes.h"
-#endif /* !UKERNEL */
-#include "rx/rx_clock.h"
-#include "rx/rx_queue.h"
-#include "rx/rx_event.h"
-#include "rx/rx_kernel.h"
-#include "rx_kmutex.h"
-#ifdef RX_ENABLE_LOCKS
-#include "rx/rx.h"
-#endif /* RX_ENABLE_LOCKS */
-#include "rx/rx_globals.h"
-#if defined(AFS_SGI_ENV)
-#include "sys/debug.h"
-/* These are necessary to get curproc (used by GLOCK asserts) to work. */
-#include "h/proc.h"
-#if !defined(AFS_SGI64_ENV) && !defined(UKERNEL)
-#include "h/user.h"
-#endif
-extern void *osi_Alloc();
-#endif
-#if defined(AFS_OBSD_ENV)
-#include "h/proc.h"
-#endif
-#else /* KERNEL */
-#include <stdio.h>
-#include "rx_clock.h"
-#include "rx_queue.h"
-#include "rx_event.h"
-#include "rx_user.h"
-#ifdef AFS_PTHREAD_ENV
-#include <rx/rx_pthread.h>
+# include "afs/sysincludes.h"
+# include "afsincludes.h"
 #else
-#include "rx_lwp.h"
+# include <roken.h>
 #endif
-#ifdef RX_ENABLE_LOCKS
+
+#include <afs/opr.h>
+#include <opr/queue.h>
+#include <opr/rbtree.h>
+
 #include "rx.h"
-#endif /* RX_ENABLE_LOCKS */
+#include "rx_atomic.h"
+#include "rx_call.h"
 #include "rx_globals.h"
-#ifdef AFS_NT40_ENV
-#include <malloc.h>
-#endif
-#endif /* KERNEL */
-
-
-/* All event processing is relative to the apparent current time given by clock_GetTime */
-
-/* This should be static, but event_test wants to look at the free list... */
-struct rx_queue rxevent_free;  /* It's somewhat bogus to use a doubly-linked queue for the free list */
-struct rx_queue rxepoch_free;  /* It's somewhat bogus to use a doubly-linked queue for the free list */
-static struct rx_queue rxepoch_queue;  /* list of waiting epochs */
-static int rxevent_allocUnit = 10;     /* Allocation unit (number of event records to allocate at one time) */
-static int rxepoch_allocUnit = 10;     /* Allocation unit (number of epoch records to allocate at one time) */
-int rxevent_nFree;             /* Number of free event records */
-int rxevent_nPosted;           /* Current number of posted events */
-int rxepoch_nFree;             /* Number of free epoch records */
-static void (*rxevent_ScheduledEarlierEvent) (void);   /* Proc to call when an event is scheduled that is earlier than all other events */
-struct xfreelist {
+
+struct rxevent {
+    struct opr_queue q;
+    struct opr_rbtree_node node;
+    struct clock eventTime;
+    struct rxevent *next;
+    rx_atomic_t refcnt;
+    int handled;
+    void (*func)(struct rxevent *, void *, void *, int);
+    void *arg;
+    void *arg1;
+    int arg2;
+};
+
+struct malloclist {
     void *mem;
     int size;
-    struct xfreelist *next;
+    struct malloclist *next;
 };
-static struct xfreelist *xfreemallocs = 0, *xsp = 0;
-
-struct clock rxevent_nextRaiseEvents;  /* Time of next call to raise events */
-struct clock rxevent_lastEvent;        /* backwards time detection */
-int rxevent_raiseScheduled;    /* true if raise events is scheduled */
-
-#ifdef RX_ENABLE_LOCKS
-#ifdef RX_LOCKS_DB
-/* rxdb_fileID is used to identify the lock location, along with line#. */
-static int rxdb_fileID = RXDB_FILE_RX_EVENT;
-#endif /* RX_LOCKS_DB */
-#define RX_ENABLE_LOCKS  1
-afs_kmutex_t rxevent_lock;
-#endif /* RX_ENABLE_LOCKS */
-
-#ifdef AFS_PTHREAD_ENV
-/*
- * This mutex protects the following global variables:
- * rxevent_initialized
- */
 
-#include <assert.h>
-afs_kmutex_t rx_event_mutex;
-#define LOCK_EV_INIT MUTEX_ENTER(&rx_event_mutex)
-#define UNLOCK_EV_INIT MUTEX_EXIT(&rx_event_mutex)
+static struct {
+    afs_kmutex_t lock;
+    struct opr_queue list;
+    struct malloclist *mallocs;
+} freeEvents;
+
+static struct {
+    afs_kmutex_t lock;
+    struct opr_rbtree head;
+    struct rxevent *first;
+} eventTree;
+
+static struct {
+    afs_kmutex_t lock;
+    struct clock last;
+    struct clock next;
+    void (*func)(void);
+    int raised;
+} eventSchedule;
+
+static int allocUnit = 10;
+
+static struct rxevent *
+rxevent_alloc(void) {
+     struct rxevent *evlist;
+     struct rxevent *ev;
+     struct malloclist *mrec;
+     int i;
+
+     MUTEX_ENTER(&freeEvents.lock);
+     if (opr_queue_IsEmpty(&freeEvents.list)) {
+       MUTEX_EXIT(&freeEvents.lock);
+
+#if    defined(AFS_AIX32_ENV) && defined(KERNEL)
+       ev = rxi_Alloc(sizeof(struct rxevent));
 #else
-#define LOCK_EV_INIT
-#define UNLOCK_EV_INIT
-#endif /* AFS_PTHREAD_ENV */
+       evlist = osi_Alloc(sizeof(struct rxevent) * allocUnit);
+       mrec = osi_Alloc(sizeof(struct malloclist));
 
+       mrec->mem = evlist;
+       mrec->size = sizeof(struct rxevent) * allocUnit;
 
-int
-rxevent_adjTimes(struct clock *adjTime)
-{
-    /* backwards clock correction */
-    int nAdjusted = 0;
-    struct rxepoch *qep, *nqep;
-    struct rxevent *qev, *nqev;
-    
-    for (queue_Scan(&rxepoch_queue, qep, nqep, rxepoch)) {
-       for (queue_Scan(&qep->events, qev, nqev, rxevent)) {
-           if (clock_Gt(&qev->eventTime, adjTime)) {
-               clock_Sub(&qev->eventTime, adjTime); 
-               nAdjusted++;
-           }
-       }
-       if (qep->epochSec > adjTime->sec) {
-           qep->epochSec -= adjTime->sec;
+       MUTEX_ENTER(&freeEvents.lock);
+       for (i = 1; i < allocUnit; i++) {
+           opr_queue_Append(&freeEvents.list, &evlist[i].q);
        }
+       mrec->next = freeEvents.mallocs;
+       freeEvents.mallocs = mrec;
+       MUTEX_EXIT(&freeEvents.lock);
+#endif
+       ev = &evlist[0];
+    } else {
+       ev = opr_queue_First(&freeEvents.list, struct rxevent, q);
+       opr_queue_Remove(&ev->q);
+       MUTEX_EXIT(&freeEvents.lock);
     }
-    return nAdjusted;
+
+    memset(ev, 0, sizeof(struct rxevent));
+    rx_atomic_set(&ev->refcnt, 1);
+
+    return ev;
 }
 
-/* Pass in the number of events to allocate at a time */
-int rxevent_initialized = 0;
-void
-rxevent_Init(int nEvents, void (*scheduler) (void))
-{
-    LOCK_EV_INIT;
-    if (rxevent_initialized) {
-       UNLOCK_EV_INIT;
-       return;
-    }
-    MUTEX_INIT(&rxevent_lock, "rxevent_lock", MUTEX_DEFAULT, 0);
-    clock_Init();
-    if (nEvents)
-       rxevent_allocUnit = nEvents;
-    queue_Init(&rxevent_free);
-    queue_Init(&rxepoch_free);
-    queue_Init(&rxepoch_queue);
-    rxevent_nFree = rxevent_nPosted = 0;
-    rxepoch_nFree = 0;
-    rxevent_ScheduledEarlierEvent = scheduler;
-    rxevent_initialized = 1;
-    clock_Zero(&rxevent_nextRaiseEvents);
-    clock_Zero(&rxevent_lastEvent);
-    rxevent_raiseScheduled = 0;
-    UNLOCK_EV_INIT;
+static void
+rxevent_free(struct rxevent *ev) {
+    MUTEX_ENTER(&freeEvents.lock);
+    opr_queue_Prepend(&freeEvents.list, &ev->q);
+    MUTEX_EXIT(&freeEvents.lock);
 }
 
-/* Create and initialize new epoch structure */
-struct rxepoch *
-rxepoch_Allocate(struct clock *when)
-{
-    struct rxepoch *ep;
-    int i;
-
-    /* If we are short on free epoch entries, create a block of new oned
-     * and add them to the free queue */
-    if (queue_IsEmpty(&rxepoch_free)) {
-#if    defined(AFS_AIX32_ENV) && defined(KERNEL)
-       ep = (struct rxepoch *)rxi_Alloc(sizeof(struct rxepoch));
-       queue_Append(&rxepoch_free, &ep[0]), rxepoch_nFree++;
-#else
-#if defined(KERNEL) && !defined(UKERNEL) && defined(AFS_FBSD80_ENV)
-       ep = (struct rxepoch *)
-           afs_osi_Alloc_NoSleep(sizeof(struct rxepoch) * rxepoch_allocUnit);
-       xsp = xfreemallocs;
-       xfreemallocs =
-           (struct xfreelist *)afs_osi_Alloc_NoSleep(sizeof(struct xfreelist));
-#else
-       ep = (struct rxepoch *)
-           osi_Alloc(sizeof(struct rxepoch) * rxepoch_allocUnit);
-       xsp = xfreemallocs;
-       xfreemallocs =
-           (struct xfreelist *)osi_Alloc(sizeof(struct xfreelist));
-#endif
-       xfreemallocs->mem = (void *)ep;
-       xfreemallocs->size = sizeof(struct rxepoch) * rxepoch_allocUnit;
-       xfreemallocs->next = xsp;
-       for (i = 0; i < rxepoch_allocUnit; i++)
-           queue_Append(&rxepoch_free, &ep[i]), rxepoch_nFree++;
-#endif
+static_inline void
+rxevent_put(struct rxevent *ev) {
+    if (rx_atomic_dec_and_read(&ev->refcnt) == 0) {
+        rxevent_free(ev);
     }
-    ep = queue_First(&rxepoch_free, rxepoch);
-    queue_Remove(ep);
-    rxepoch_nFree--;
-    ep->epochSec = when->sec;
-    queue_Init(&ep->events);
-    return ep;
 }
 
-/* Add the indicated event (function, arg) at the specified clock time.  The
- * "when" argument specifies when "func" should be called, in clock (clock.h)
- * units. */
+void
+rxevent_Put(struct rxevent *ev) {
+    rxevent_put(ev);
+}
 
-static struct rxevent *
-_rxevent_Post(struct clock *when, struct clock *now, 
-             void (*func) (struct rxevent *, void *, void *, int), 
-             void *arg, void *arg1, int arg2, int newargs)
+static_inline struct rxevent *
+rxevent_get(struct rxevent *ev) {
+    rx_atomic_inc(&ev->refcnt);
+    return ev;
+}
+
+struct rxevent *
+rxevent_Get(struct rxevent *ev) {
+    return rxevent_get(ev);
+}
+
+/* Called if the time now is older than the last time we recorded running an
+ * event. This test catches machines where the system time has been set
+ * backwards, and avoids RX completely stalling when timers fail to fire.
+ *
+ * Take the different between now and the last event time, and subtract that
+ * from the timing of every event on the system. This does a relatively slow
+ * walk of the completely eventTree, but time-travel will hopefully be a pretty
+ * rare occurrence.
+ *
+ * This can only safely be called from the event thread, as it plays with the
+ * schedule directly.
+ *
+ */
+static void
+adjustTimes(void)
 {
-    struct rxevent *ev, *evqe, *evqpr;
-    struct rxepoch *ep, *epqe, *epqpr;
-    int isEarliest = 0;
-
-    MUTEX_ENTER(&rxevent_lock);
-#ifdef RXDEBUG
-    if (rx_Log_event) {
-       struct clock now1;
-       clock_GetTime(&now1);
-       fprintf(rx_Log_event, "%d.%d: rxevent_Post(%d.%d, %lp, %lp, %lp, %d)\n",
-               (int)now1.sec, (int)now1.usec, (int)when->sec, (int)when->usec,
-               func, arg,
-               arg1, arg2);
-    }
-#endif
-    /* If a time was provided, check for consistency */
-    if (now->sec) {
-       if (clock_Gt(&rxevent_lastEvent, now)) {
-           struct clock adjTime = rxevent_lastEvent;
-           clock_Sub(&adjTime, now);
-           rxevent_adjTimes(&adjTime);
-       }
-       rxevent_lastEvent = *now;
-    }
-    /* Get a pointer to the epoch for this event, if none is found then
-     * create a new epoch and insert it into the sorted list */
-    for (ep = NULL, queue_ScanBackwards(&rxepoch_queue, epqe, epqpr, rxepoch)) {
-       if (when->sec == epqe->epochSec) {
-           /* already have an structure for this epoch */
-           ep = epqe;
-           if (ep == queue_First(&rxepoch_queue, rxepoch))
-               isEarliest = 1;
-           break;
-       } else if (when->sec > epqe->epochSec) {
-           /* Create a new epoch and insert after qe */
-           ep = rxepoch_Allocate(when);
-           queue_InsertAfter(epqe, ep);
-           break;
-       }
-    }
-    if (ep == NULL) {
-       /* Create a new epoch and place it at the head of the list */
-       ep = rxepoch_Allocate(when);
-       queue_Prepend(&rxepoch_queue, ep);
-       isEarliest = 1;
-    }
+    struct opr_rbtree_node *node;
+    struct clock adjTime, now;
 
-    /* If we're short on free event entries, create a block of new ones and add
-     * them to the free queue */
-    if (queue_IsEmpty(&rxevent_free)) {
-       int i;
-#if    defined(AFS_AIX32_ENV) && defined(KERNEL)
-       ev = (struct rxevent *)rxi_Alloc(sizeof(struct rxevent));
-       queue_Append(&rxevent_free, &ev[0]), rxevent_nFree++;
-#else
+    MUTEX_ENTER(&eventTree.lock);
+    /* Time adjustment is expensive, make absolutely certain that we have
+     * to do it, by getting an up to date time to base our decision on
+     * once we've acquired the relevant locks.
+     */
+    clock_GetTime(&now);
+    if (!clock_Lt(&now, &eventSchedule.last))
+       goto out;
 
-#if defined(KERNEL) && !defined(UKERNEL) && defined(AFS_FBSD80_ENV)
-       ev = (struct rxevent *)afs_osi_Alloc_NoSleep(sizeof(struct rxevent) *
-                                        rxevent_allocUnit);
-       xsp = xfreemallocs;
-       xfreemallocs =
-           (struct xfreelist *)afs_osi_Alloc_NoSleep(sizeof(struct xfreelist));
-#else
-       ev = (struct rxevent *)osi_Alloc(sizeof(struct rxevent) *
-                                        rxevent_allocUnit);
-       xsp = xfreemallocs;
-       xfreemallocs =
-           (struct xfreelist *)osi_Alloc(sizeof(struct xfreelist));
-#endif
-       xfreemallocs->mem = (void *)ev;
-       xfreemallocs->size = sizeof(struct rxevent) * rxevent_allocUnit;
-       xfreemallocs->next = xsp;
-       for (i = 0; i < rxevent_allocUnit; i++)
-           queue_Append(&rxevent_free, &ev[i]), rxevent_nFree++;
-#endif
+    adjTime = eventSchedule.last;
+    clock_Zero(&eventSchedule.last);
+
+    clock_Sub(&adjTime, &now);
+
+    /* If there are no events in the tree, then there's nothing to adjust */
+    if (eventTree.first == NULL)
+       goto out;
+
+    node = opr_rbtree_first(&eventTree.head);
+    while(node) {
+       struct rxevent *event = opr_containerof(node, struct rxevent, node);
+
+       clock_Sub(&event->eventTime, &adjTime);
+       node = opr_rbtree_next(node);
     }
+    eventSchedule.next = eventTree.first->eventTime;
+
+out:
+    MUTEX_EXIT(&eventTree.lock);
+}
+
+static int initialised = 0;
+void
+rxevent_Init(int nEvents, void (*scheduler)(void))
+{
+    if (initialised)
+       return;
+
+    initialised = 1;
+
+    clock_Init();
+    MUTEX_INIT(&eventTree.lock, "event tree lock", MUTEX_DEFAULT, 0);
+    opr_rbtree_init(&eventTree.head);
+
+    MUTEX_INIT(&freeEvents.lock, "free events lock", MUTEX_DEFAULT, 0);
+    opr_queue_Init(&freeEvents.list);
+    freeEvents.mallocs = NULL;
 
-    /* Grab and initialize a new rxevent structure */
-    ev = queue_First(&rxevent_free, rxevent);
-    queue_Remove(ev);
-    rxevent_nFree--;
+    if (nEvents)
+       allocUnit = nEvents;
+
+    clock_Zero(&eventSchedule.next);
+    clock_Zero(&eventSchedule.last);
+    eventSchedule.raised = 0;
+    eventSchedule.func = scheduler;
+}
 
-    /* Record user defined event state */
+struct rxevent *
+rxevent_Post(struct clock *when, struct clock *now,
+            void (*func) (struct rxevent *, void *, void *, int),
+            void *arg, void *arg1, int arg2)
+{
+    struct rxevent *ev, *event;
+    struct opr_rbtree_node **childptr, *parent = NULL;
+
+    ev = rxevent_alloc();
     ev->eventTime = *when;
-    if (newargs) {
-       ev->func.newfunc = func;
-    } else {
-       ev->func.oldfunc = (void (*)(struct rxevent *, void *, void*))func;
-    }
+    ev->func = func;
     ev->arg = arg;
     ev->arg1 = arg1;
     ev->arg2 = arg2;
-    ev->newargs = newargs;
-    rxevent_nPosted += 1;      /* Rather than ++, to shut high-C up
-                                *  regarding never-set variables
-                                */
-
-    /* Insert the event into the sorted list of events for this epoch */
-    for (queue_ScanBackwards(&ep->events, evqe, evqpr, rxevent)) {
-       if (when->usec >= evqe->eventTime.usec) {
-           /* Insert event after evqe */
-           queue_InsertAfter(evqe, ev);
-           MUTEX_EXIT(&rxevent_lock);
-           return ev;
+
+    if (clock_Lt(now, &eventSchedule.last))
+       adjustTimes();
+
+    MUTEX_ENTER(&eventTree.lock);
+
+    /* Work out where in the tree we'll be storing this */
+    childptr = &eventTree.head.root;
+
+    while(*childptr) {
+       event = opr_containerof((*childptr), struct rxevent, node);
+
+       parent = *childptr;
+       if (clock_Lt(when, &event->eventTime))
+           childptr = &(*childptr)->left;
+       else if (clock_Gt(when, &event->eventTime))
+           childptr = &(*childptr)->right;
+       else {
+           opr_queue_Append(&event->q, &ev->q);
+           goto out;
        }
     }
-    /* Insert event at head of current epoch */
-    queue_Prepend(&ep->events, ev);
-    if (isEarliest && rxevent_ScheduledEarlierEvent
-       && (!rxevent_raiseScheduled
-           || clock_Lt(&ev->eventTime, &rxevent_nextRaiseEvents))) {
-       rxevent_raiseScheduled = 1;
-       clock_Zero(&rxevent_nextRaiseEvents);
-       MUTEX_EXIT(&rxevent_lock);
-       /* Notify our external scheduler */
-       (*rxevent_ScheduledEarlierEvent) ();
-       MUTEX_ENTER(&rxevent_lock);
+    opr_queue_Init(&ev->q);
+    opr_rbtree_insert(&eventTree.head, parent, childptr, &ev->node);
+
+    if (eventTree.first == NULL ||
+       clock_Lt(when, &(eventTree.first->eventTime))) {
+       eventTree.first = ev;
+       eventSchedule.raised = 1;
+       clock_Zero(&eventSchedule.next);
+       MUTEX_EXIT(&eventTree.lock);
+       if (eventSchedule.func != NULL)
+           (*eventSchedule.func)();
+       return rxevent_get(ev);
     }
-    MUTEX_EXIT(&rxevent_lock);
-    return ev;
-}
 
-struct rxevent *
-rxevent_Post(struct clock *when, 
-            void (*func) (struct rxevent *, void *, void *), 
-            void *arg, void *arg1)
-{
-    struct clock now;
-    clock_Zero(&now);
-    return _rxevent_Post(when, &now, 
-                        (void (*)(struct rxevent *, void *, void *, int))func,
-                        arg, arg1, 0, 0);
+out:
+    MUTEX_EXIT(&eventTree.lock);
+    return rxevent_get(ev);
 }
 
-struct rxevent *
-rxevent_Post2(struct clock *when, 
-             void (*func) (struct rxevent *, void *, void *, int), 
-             void *arg, void *arg1, int arg2)
+/* We're going to remove ev from the tree, so set the first pointer to the
+ * next event after it */
+static_inline void
+resetFirst(struct rxevent *ev)
 {
-    struct clock now;
-    clock_Zero(&now);
-    return _rxevent_Post(when, &now, func, arg, arg1, arg2, 1);
+    struct opr_rbtree_node *next = opr_rbtree_next(&ev->node);
+    if (next)
+       eventTree.first = opr_containerof(next, struct rxevent, node);
+    else
+       eventTree.first = NULL;
 }
 
-struct rxevent *
-rxevent_PostNow(struct clock *when, struct clock *now, 
-               void (*func) (struct rxevent *, void *, void *), 
-               void *arg, void *arg1)
+void
+rxevent_Cancel(struct rxevent **evp, struct rx_call *call, int type)
 {
-    return _rxevent_Post(when, now, 
-                        (void (*)(struct rxevent *, void *, void *, int))func,
-                        arg, arg1, 0, 0);
-}
+    struct rxevent *event;
 
-struct rxevent *
-rxevent_PostNow2(struct clock *when, struct clock *now, 
-                 void (*func) (struct rxevent *, void *, void *, int), 
-                void *arg, void *arg1, int arg2)
-{
-    return _rxevent_Post(when, now, func, arg, arg1, arg2, 1);
-}
+    if (!evp || !*evp)
+       return;
 
-/* Cancel an event by moving it from the event queue to the free list.
- * Warning, the event must be on the event queue!  If not, this should core
- * dump (reference through 0).  This routine should be called using the macro
- * event_Cancel, which checks for a null event and also nulls the caller's
- * event pointer after cancelling the event.
- */
-#ifdef RX_ENABLE_LOCKS
-#ifdef RX_REFCOUNT_CHECK
-int rxevent_Cancel_type = 0;
-#endif
-#endif
+    event = *evp;
+
+    MUTEX_ENTER(&eventTree.lock);
+
+    if (!event->handled) {
+       /* We're a node on the red/black tree. If our list is non-empty,
+        * then swap the first element in the list in in our place,
+        * promoting it to the list head */
+       if (event->node.parent == NULL
+           && eventTree.head.root != &event->node) {
+           /* Not in the rbtree, therefore must be a list element */
+           opr_queue_Remove(&event->q);
+       } else {
+           if (!opr_queue_IsEmpty(&event->q)) {
+               struct rxevent *next;
+
+               next = opr_queue_First(&event->q, struct rxevent, q);
+               opr_queue_Remove(&next->q); /* Remove ourselves from list */
+               if (event->q.prev == &event->q) {
+                   next->q.prev = next->q.next = &next->q;
+               } else {
+                   next->q = event->q;
+                   next->q.prev->next = &next->q;
+                   next->q.next->prev = &next->q;
+               }
 
-void
-rxevent_Cancel_1(struct rxevent *ev, struct rx_call *call,
-                int type)
-{
-#ifdef RXDEBUG
-    if (rx_Log_event) {
-       struct clock now;
-       clock_GetTime(&now);
-       fprintf(rx_Log_event, "%d.%d: rxevent_Cancel_1(%d.%d, %lp, %lp)\n",
-               (int)now.sec, (int)now.usec, (int)ev->eventTime.sec,
-               (int)ev->eventTime.usec, ev->func.newfunc,
-               ev->arg);
-    }
-#endif
-    /* Append it to the free list (rather than prepending) to keep the free
-     * list hot so nothing pages out
-     */
-    MUTEX_ENTER(&rxevent_lock);
-    if (!ev) {
-       MUTEX_EXIT(&rxevent_lock);
-       return;
-    }
-#ifdef RX_ENABLE_LOCKS
-    /* It's possible we're currently processing this event. */
-    if (queue_IsOnQueue(ev)) {
-       queue_MoveAppend(&rxevent_free, ev);
-       rxevent_nPosted--;
-       rxevent_nFree++;
-       if (call) {
-           call->refCount--;
-#ifdef RX_REFCOUNT_CHECK
-           call->refCDebug[type]--;
-           if (call->refCDebug[type] < 0) {
-               rxevent_Cancel_type = type;
-               osi_Panic("rxevent_Cancel: call refCount < 0");
+               opr_rbtree_replace(&eventTree.head, &event->node,
+                                  &next->node);
+
+               if (eventTree.first == event)
+                   eventTree.first = next;
+
+           } else {
+               if (eventTree.first == event)
+                   resetFirst(event);
+
+               opr_rbtree_remove(&eventTree.head, &event->node);
            }
-#endif /* RX_REFCOUNT_CHECK */
        }
+       event->handled = 1;
+       rxevent_put(event); /* Dispose of eventTree reference */
     }
-#else /* RX_ENABLE_LOCKS */
-    queue_MoveAppend(&rxevent_free, ev);
-    rxevent_nPosted--;
-    rxevent_nFree++;
-#endif /* RX_ENABLE_LOCKS */
-    MUTEX_EXIT(&rxevent_lock);
+
+    MUTEX_EXIT(&eventTree.lock);
+
+    *evp = NULL;
+    rxevent_put(event); /* Dispose of caller's reference */
+
+    if (call)
+       CALL_RELE(call, type);
 }
 
-/* Process all epochs that have expired relative to the current clock time
- * (which is not re-evaluated unless clock_NewTime has been called).  The
- * relative time to the next epoch is returned in the output parameter next
- * and the function returns 1.  If there are is no next epoch, the function
- * returns 0.
+/* Process all events which have expired. If events remain, then the relative
+ * time until the next event is returned in the parameter 'wait', and the
+ * function returns 1. If no events currently remain, the function returns 0
+ *
+ * If the current time is older than that of the last event processed, then we
+ * assume that time has gone backwards (for example, due to a system time reset)
+ * When this happens, all events in the current queue are rescheduled, using
+ * the difference between the current time and the last event time as a delta
  */
+
 int
-rxevent_RaiseEvents(struct clock *next)
+rxevent_RaiseEvents(struct clock *wait)
 {
-    struct rxepoch *ep;
-    struct rxevent *ev;
-    volatile struct clock now;
-    MUTEX_ENTER(&rxevent_lock);
-
-    /* Events are sorted by time, so only scan until an event is found that has
-     * not yet timed out */
-
-    clock_Zero(&now);
-    while (queue_IsNotEmpty(&rxepoch_queue)) {
-       ep = queue_First(&rxepoch_queue, rxepoch);
-       if (queue_IsEmpty(&ep->events)) {
-           queue_Remove(ep);
-           queue_Append(&rxepoch_free, ep);
-           rxepoch_nFree++;
-           continue;
+    struct clock now;
+    struct rxevent *event;
+    int ret;
+
+    clock_GetTime(&now);
+
+    /* Check for time going backwards */
+    if (clock_Lt(&now, &eventSchedule.last))
+         adjustTimes();
+    eventSchedule.last = now;
+
+    MUTEX_ENTER(&eventTree.lock);
+    /* Lock our event tree */
+    while (eventTree.first != NULL
+          && clock_Lt(&eventTree.first->eventTime, &now)) {
+
+       /* Grab the next node, either in the event's list, or in the tree node
+        * itself, and remove it from the event tree */
+       event = eventTree.first;
+       if (!opr_queue_IsEmpty(&event->q)) {
+           event = opr_queue_Last(&event->q, struct rxevent, q);
+           opr_queue_Remove(&event->q);
+       } else {
+           resetFirst(event);
+           opr_rbtree_remove(&eventTree.head, &event->node);
        }
-       do {
-       reraise:
-           ev = queue_First(&ep->events, rxevent);
-           if (clock_Lt(&now, &ev->eventTime)) {
-               clock_GetTime(&now);
-               if (clock_Gt(&rxevent_lastEvent, &now)) {
-                   struct clock adjTime = rxevent_lastEvent;
-                   int adjusted;
-                   clock_Sub(&adjTime, &now);
-                   adjusted = rxevent_adjTimes(&adjTime);
-                   rxevent_lastEvent = now;
-                   if (adjusted > 0)
-                       goto reraise;
-               }
-               if (clock_Lt(&now, &ev->eventTime)) {
-                    *next = rxevent_nextRaiseEvents = ev->eventTime;
-                    rxevent_raiseScheduled = 1;
-                    clock_Sub(next, &now);
-                    MUTEX_EXIT(&rxevent_lock);
-                    return 1;
-                }
-            }
-           queue_Remove(ev);
-           rxevent_nPosted--;
-           MUTEX_EXIT(&rxevent_lock);
-           if (ev->newargs) {
-               ev->func.newfunc(ev, ev->arg, ev->arg1, ev->arg2);
-           } else {
-               ev->func.oldfunc(ev, ev->arg, ev->arg1);
-           }
-           MUTEX_ENTER(&rxevent_lock);
-           queue_Append(&rxevent_free, ev);
-           rxevent_nFree++;
-       } while (queue_IsNotEmpty(&ep->events));
+       event->handled = 1;
+        MUTEX_EXIT(&eventTree.lock);
+
+        /* Fire the event, then free the structure */
+       event->func(event, event->arg, event->arg1, event->arg2);
+       rxevent_put(event);
+
+       MUTEX_ENTER(&eventTree.lock);
     }
-#ifdef RXDEBUG
-    if (rx_Log_event)
-       fprintf(rx_Log_event, "rxevent_RaiseEvents(%d.%d)\n", (int)now.sec,
-               (int)now.usec);
-#endif
-    rxevent_raiseScheduled = 0;
-    MUTEX_EXIT(&rxevent_lock);
-    return 0;
+
+    /* Figure out when we next need to be scheduled */
+    if (eventTree.first != NULL) {
+       *wait = eventSchedule.next = eventTree.first->eventTime;
+       ret = eventSchedule.raised = 1;
+       clock_Sub(wait, &now);
+    } else {
+       ret = eventSchedule.raised = 0;
+    }
+
+    MUTEX_EXIT(&eventTree.lock);
+
+    return ret;
 }
 
 void
 shutdown_rxevent(void)
 {
-    struct xfreelist *xp, *nxp;
+    struct malloclist *mrec, *nmrec;
 
-    LOCK_EV_INIT;
-    if (!rxevent_initialized) {
-       UNLOCK_EV_INIT;
+    if (!initialised) {
        return;
     }
-    rxevent_initialized = 0;
-    UNLOCK_EV_INIT;
-    MUTEX_DESTROY(&rxevent_lock);
-#if    defined(AFS_AIX32_ENV) && defined(KERNEL)
-    /* Everything is freed in afs_osinet.c */
-#else
-    xp = xfreemallocs;
-    while (xp) {
-       nxp = xp->next;
-       osi_Free((char *)xp->mem, xp->size);
-       osi_Free((char *)xp, sizeof(struct xfreelist));
-       xp = nxp;
+    MUTEX_DESTROY(&eventTree.lock);
+
+#if !defined(AFS_AIX32_ENV) || !defined(KERNEL)
+    MUTEX_DESTROY(&freeEvents.lock);
+    mrec = freeEvents.mallocs;
+    while (mrec) {
+       nmrec = mrec->next;
+       osi_Free(mrec->mem, mrec->size);
+       osi_Free(mrec, sizeof(struct malloclist));
+       mrec = nmrec;
     }
-    xfreemallocs = NULL;
+    mrec = NULL;
 #endif
-
 }