rx: Don't double check conn->call
[openafs.git] / src / rx / rx.c
index e3a4c6a..269d215 100644 (file)
@@ -67,12 +67,15 @@ extern afs_int32 afs_termState;
 #  include <WINNT\afsreg.h>
 # endif
 
+# include <afs/opr.h>
+
 # include "rx_user.h"
 #endif /* KERNEL */
 
+#include <opr/queue.h>
+
 #include "rx.h"
 #include "rx_clock.h"
-#include "rx_queue.h"
 #include "rx_atomic.h"
 #include "rx_globals.h"
 #include "rx_trace.h"
@@ -80,9 +83,11 @@ extern afs_int32 afs_termState;
 #include "rx_stats.h"
 #include "rx_event.h"
 
+#include "rx_peer.h"
 #include "rx_conn.h"
 #include "rx_call.h"
 #include "rx_packet.h"
+#include "rx_server.h"
 
 #include <afs/rxgen_consts.h>
 
@@ -113,9 +118,45 @@ static void rxi_SendDelayedConnAbort(struct rxevent *event, void *arg1,
                                     void *unused, int unused2);
 static void rxi_ReapConnections(struct rxevent *unused, void *unused1,
                                void *unused2, int unused3);
+static struct rx_packet *rxi_SendCallAbort(struct rx_call *call,
+                                          struct rx_packet *packet,
+                                          int istack, int force);
+static void rxi_AckAll(struct rx_call *call);
+static struct rx_connection
+       *rxi_FindConnection(osi_socket socket, afs_uint32 host, u_short port,
+                           u_short serviceId, afs_uint32 cid,
+                           afs_uint32 epoch, int type, u_int securityIndex);
+static struct rx_packet
+       *rxi_ReceiveDataPacket(struct rx_call *call, struct rx_packet *np,
+                              int istack, osi_socket socket,
+                              afs_uint32 host, u_short port, int *tnop,
+                              struct rx_call **newcallp);
+static struct rx_packet
+       *rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
+                             int istack);
+static struct rx_packet
+       *rxi_ReceiveResponsePacket(struct rx_connection *conn,
+                                  struct rx_packet *np, int istack);
+static struct rx_packet
+       *rxi_ReceiveChallengePacket(struct rx_connection *conn,
+                                   struct rx_packet *np, int istack);
+static void rxi_AttachServerProc(struct rx_call *call, osi_socket socket,
+                                int *tnop, struct rx_call **newcallp);
+static void rxi_ClearTransmitQueue(struct rx_call *call, int force);
+static void rxi_ClearReceiveQueue(struct rx_call *call);
+static void rxi_ResetCall(struct rx_call *call, int newcall);
+static void rxi_ScheduleKeepAliveEvent(struct rx_call *call);
+static void rxi_ScheduleNatKeepAliveEvent(struct rx_connection *conn);
+static void rxi_ScheduleGrowMTUEvent(struct rx_call *call, int secs);
+static void rxi_KeepAliveOn(struct rx_call *call);
+static void rxi_GrowMTUOn(struct rx_call *call);
+static void rxi_ChallengeOn(struct rx_connection *conn);
 
 #ifdef RX_ENABLE_LOCKS
+static int rxi_CheckCall(struct rx_call *call, int haveCTLock);
 static void rxi_SetAcksInTransmitQueue(struct rx_call *call);
+#else
+static int rxi_CheckCall(struct rx_call *call);
 #endif
 
 #ifdef AFS_GLOBAL_RXLOCK_KERNEL
@@ -163,18 +204,26 @@ static unsigned int rxi_rpc_peer_stat_cnt;
 static unsigned int rxi_rpc_process_stat_cnt;
 
 /*
- * rxi_busyChannelError is the error to return to the application when a call
- * channel appears busy (inferred from the receipt of RX_PACKET_TYPE_BUSY
- * packets on the channel), and there are other call channels in the
- * connection that are not busy. If 0, we do not return errors upon receiving
- * busy packets; we just keep trying on the same call channel until we hit a
- * timeout.
+ * rxi_busyChannelError is a boolean.  It indicates whether or not RX_CALL_BUSY
+ * errors should be reported to the application when a call channel appears busy
+ * (inferred from the receipt of RX_PACKET_TYPE_BUSY packets on the channel),
+ * and there are other call channels in the connection that are not busy.
+ * If 0, we do not return errors upon receiving busy packets; we just keep
+ * trying on the same call channel until we hit a timeout.
  */
 static afs_int32 rxi_busyChannelError = 0;
 
 rx_atomic_t rx_nWaiting = RX_ATOMIC_INIT(0);
 rx_atomic_t rx_nWaited = RX_ATOMIC_INIT(0);
 
+/* Incoming calls wait on this queue when there are no available
+ * server processes */
+struct opr_queue rx_incomingCallQueue;
+
+/* Server processes wait on this queue when there are no appropriate
+ * calls to process */
+struct opr_queue rx_idleServerQueue;
+
 #if !defined(offsetof)
 #include <stddef.h>            /* for definition of offsetof() */
 #endif
@@ -494,6 +543,7 @@ rx_InitHost(u_int host, u_int port)
 #endif /* RX_LOCKS_DB */
     MUTEX_INIT(&rx_stats_mutex, "rx_stats_mutex", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_quota_mutex, "rx_quota_mutex", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&rx_atomic_mutex, "rx_atomic_mutex", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_pthread_mutex, "rx_pthread_mutex", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_packets_mutex, "rx_packets_mutex", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_refcnt_mutex, "rx_refcnt_mutex", MUTEX_DEFAULT, 0);
@@ -519,17 +569,16 @@ rx_InitHost(u_int host, u_int port)
     rx_connDeadTime = 12;
     rx_tranquil = 0;           /* reset flag */
     rxi_ResetStatistics();
-    htable = (char *)
-       osi_Alloc(rx_hashTableSize * sizeof(struct rx_connection *));
+    htable = osi_Alloc(rx_hashTableSize * sizeof(struct rx_connection *));
     PIN(htable, rx_hashTableSize * sizeof(struct rx_connection *));    /* XXXXX */
     memset(htable, 0, rx_hashTableSize * sizeof(struct rx_connection *));
-    ptable = (char *)osi_Alloc(rx_hashTableSize * sizeof(struct rx_peer *));
+    ptable = osi_Alloc(rx_hashTableSize * sizeof(struct rx_peer *));
     PIN(ptable, rx_hashTableSize * sizeof(struct rx_peer *));  /* XXXXX */
     memset(ptable, 0, rx_hashTableSize * sizeof(struct rx_peer *));
 
     /* Malloc up a bunch of packets & buffers */
     rx_nFreePackets = 0;
-    queue_Init(&rx_freePacketQueue);
+    opr_queue_Init(&rx_freePacketQueue);
     rxi_NeedMorePackets = FALSE;
     rx_nPackets = 0;   /* rx_nPackets is managed by rxi_MorePackets* */
 
@@ -571,6 +620,7 @@ rx_InitHost(u_int host, u_int port)
 #endif
        if (getsockname((intptr_t)rx_socket, (struct sockaddr *)&addr, &addrlen)) {
            rx_Finalize();
+           osi_Free(htable, rx_hashTableSize * sizeof(struct rx_connection *));
            return -1;
        }
        rx_port = addr.sin_port;
@@ -598,9 +648,9 @@ rx_InitHost(u_int host, u_int port)
     rxevent_Init(20, rxi_ReScheduleEvents);
 
     /* Initialize various global queues */
-    queue_Init(&rx_idleServerQueue);
-    queue_Init(&rx_incomingCallQueue);
-    queue_Init(&rx_freeCallQueue);
+    opr_queue_Init(&rx_idleServerQueue);
+    opr_queue_Init(&rx_incomingCallQueue);
+    opr_queue_Init(&rx_freeCallQueue);
 
 #if defined(AFS_NT40_ENV) && !defined(KERNEL)
     /* Initialize our list of usable IP addresses. */
@@ -724,14 +774,15 @@ rxi_rto_packet_sent(struct rx_call *call, int lastPacket, int istack)
 static_inline void
 rxi_rto_packet_acked(struct rx_call *call, int istack)
 {
-    struct rx_packet *p, *nxp;
+    struct opr_queue *cursor;
 
     rxi_rto_cancel(call);
 
-    if (queue_IsEmpty(&call->tq))
+    if (opr_queue_IsEmpty(&call->tq))
        return;
 
-    for (queue_Scan(&call->tq, p, nxp, rx_packet)) {
+    for (opr_queue_Scan(&call->tq, cursor)) {
+       struct rx_packet *p = opr_queue_Entry(cursor, struct rx_packet, entry);
        if (p->header.seq > call->tfirst + call->twind)
            return;
 
@@ -755,17 +806,17 @@ rx_rto_setPeerTimeoutSecs(struct rx_peer *peer, int secs) {
 }
 
 /**
- * Sets the error generated when a busy call channel is detected.
+ * Enables or disables the busy call channel error (RX_CALL_BUSY).
  *
- * @param[in] error The error to return for a call on a busy channel.
+ * @param[in] onoff Non-zero to enable busy call channel errors.
  *
  * @pre Neither rx_Init nor rx_InitHost have been called yet
  */
 void
-rx_SetBusyChannelError(afs_int32 error)
+rx_SetBusyChannelError(afs_int32 onoff)
 {
     osi_Assert(rxinit_status != 0);
-    rxi_busyChannelError = error;
+    rxi_busyChannelError = onoff ? 1 : 0;
 }
 
 /**
@@ -1111,6 +1162,7 @@ void
 rx_SetConnIdleDeadTime(struct rx_connection *conn, int seconds)
 {
     conn->idleDeadTime = seconds;
+    conn->idleDeadDetection = (seconds ? 1 : 0);
     rxi_CheckConnTimeouts(conn);
 }
 
@@ -1261,7 +1313,7 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
                        || call->state == RX_STATE_ACTIVE) {
                        rxi_SendAck(call, 0, 0, RX_ACK_DELAY, 0);
                    } else {
-                       rxi_AckAll(NULL, call, 0);
+                       rxi_AckAll(call);
                    }
                }
                MUTEX_EXIT(&call->lock);
@@ -1494,10 +1546,10 @@ rx_NewCall(struct rx_connection *conn)
                          * effect on overall system performance.
                          */
                         call->state = RX_STATE_RESET;
+                        (*call->callNumber)++;
                         MUTEX_EXIT(&conn->conn_call_lock);
                         CALL_HOLD(call, RX_CALL_REFCOUNT_BEGIN);
                         rxi_ResetCall(call, 0);
-                        (*call->callNumber)++;
                         if (MUTEX_TRYENTER(&conn->conn_call_lock))
                             break;
 
@@ -1550,6 +1602,7 @@ rx_NewCall(struct rx_connection *conn)
        }
        if (i < RX_MAXCALLS) {
            conn->lastBusy[i] = 0;
+           call->flags &= ~RX_CALL_PEER_BUSY;
            break;
        }
         if (!wait)
@@ -1582,15 +1635,22 @@ rx_NewCall(struct rx_connection *conn)
     call->state = RX_STATE_ACTIVE;
     call->error = conn->error;
     if (call->error)
-       call->mode = RX_MODE_ERROR;
+       call->app.mode = RX_MODE_ERROR;
     else
-       call->mode = RX_MODE_SENDING;
+       call->app.mode = RX_MODE_SENDING;
+
+#ifdef AFS_RXERRQ_ENV
+    /* remember how many network errors the peer has when we started, so if
+     * more errors are encountered after the call starts, we know the other endpoint won't be
+     * responding to us */
+    call->neterr_gen = rx_atomic_read(&conn->peer->neterrs);
+#endif
 
     /* remember start time for call in case we have hard dead time limit */
     call->queueTime = queueTime;
     clock_GetTime(&call->startTime);
-    hzero(call->bytesSent);
-    hzero(call->bytesRcvd);
+    call->app.bytesSent = 0;
+    call->app.bytesRcvd = 0;
 
     /* Turn on busy protocol. */
     rxi_KeepAliveOn(call);
@@ -1659,12 +1719,14 @@ rxi_GetCallNumberVector(struct rx_connection *aconn,
     SPLVAR;
 
     NETPRI;
+    MUTEX_ENTER(&aconn->conn_call_lock);
     for (i = 0; i < RX_MAXCALLS; i++) {
        if ((tcall = aconn->call[i]) && (tcall->state == RX_STATE_DALLY))
            aint32s[i] = aconn->callNumber[i] + 1;
        else
            aint32s[i] = aconn->callNumber[i];
     }
+    MUTEX_EXIT(&aconn->conn_call_lock);
     USERPRI;
     return 0;
 }
@@ -1678,12 +1740,14 @@ rxi_SetCallNumberVector(struct rx_connection *aconn,
     SPLVAR;
 
     NETPRI;
+    MUTEX_ENTER(&aconn->conn_call_lock);
     for (i = 0; i < RX_MAXCALLS; i++) {
        if ((tcall = aconn->call[i]) && (tcall->state == RX_STATE_DALLY))
            aconn->callNumber[i] = aint32s[i] - 1;
        else
            aconn->callNumber[i] = aint32s[i];
     }
+    MUTEX_EXIT(&aconn->conn_call_lock);
     USERPRI;
     return 0;
 }
@@ -1901,6 +1965,7 @@ void
 rx_WakeupServerProcs(void)
 {
     struct rx_serverQueueEntry *np, *tqp;
+    struct opr_queue *cursor;
     SPLVAR;
 
     NETPRI;
@@ -1923,7 +1988,8 @@ rx_WakeupServerProcs(void)
 #endif /* RX_ENABLE_LOCKS */
     }
     MUTEX_EXIT(&freeSQEList_lock);
-    for (queue_Scan(&rx_idleServerQueue, np, tqp, rx_serverQueueEntry)) {
+    for (opr_queue_Scan(&rx_idleServerQueue, cursor)) {
+        np = opr_queue_Entry(cursor, struct rx_serverQueueEntry, entry);
 #ifdef RX_ENABLE_LOCKS
        CV_BROADCAST(&np->cv);
 #else /* RX_ENABLE_LOCKS */
@@ -1989,8 +2055,9 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
        ReturnToServerPool(cur_service);
     }
     while (1) {
-       if (queue_IsNotEmpty(&rx_incomingCallQueue)) {
-           struct rx_call *tcall, *ncall, *choice2 = NULL;
+       if (!opr_queue_IsEmpty(&rx_incomingCallQueue)) {
+           struct rx_call *tcall, *choice2 = NULL;
+           struct opr_queue *cursor;
 
            /* Scan for eligible incoming calls.  A call is not eligible
             * if the maximum number of calls for its service type are
@@ -1999,14 +2066,16 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
             * while the other threads may run ahead looking for calls which
             * have all their input data available immediately.  This helps
             * keep threads from blocking, waiting for data from the client. */
-           for (queue_Scan(&rx_incomingCallQueue, tcall, ncall, rx_call)) {
+           for (opr_queue_Scan(&rx_incomingCallQueue, cursor)) {
+               tcall = opr_queue_Entry(cursor, struct rx_call, entry);
+
                service = tcall->conn->service;
                if (!QuotaOK(service)) {
                    continue;
                }
                MUTEX_ENTER(&rx_pthread_mutex);
                if (tno == rxi_fcfs_thread_num
-                       || queue_IsLast(&rx_incomingCallQueue, tcall)) {
+                       || opr_queue_IsEnd(&rx_incomingCallQueue, cursor)) {
                    MUTEX_EXIT(&rx_pthread_mutex);
                    /* If we're the fcfs thread , then  we'll just use
                     * this call. If we haven't been able to find an optimal
@@ -2016,9 +2085,10 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
                    service = call->conn->service;
                } else {
                    MUTEX_EXIT(&rx_pthread_mutex);
-                   if (!queue_IsEmpty(&tcall->rq)) {
+                   if (!opr_queue_IsEmpty(&tcall->rq)) {
                        struct rx_packet *rp;
-                       rp = queue_First(&tcall->rq, rx_packet);
+                       rp = opr_queue_First(&tcall->rq, struct rx_packet,
+                                           entry);
                        if (rp->header.seq == 1) {
                            if (!meltdown_1pkt
                                || (rp->header.flags & RX_LAST_PACKET)) {
@@ -2041,7 +2111,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
        }
 
        if (call) {
-           queue_Remove(call);
+           opr_queue_Remove(&call->entry);
            MUTEX_EXIT(&rx_serverPool_lock);
            MUTEX_ENTER(&call->lock);
 
@@ -2058,8 +2128,8 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
                continue;
            }
 
-           if (queue_IsEmpty(&call->rq)
-               || queue_First(&call->rq, rx_packet)->header.seq != 1)
+           if (opr_queue_IsEmpty(&call->rq)
+               || opr_queue_First(&call->rq, struct rx_packet, entry)->header.seq != 1)
                rxi_SendAck(call, 0, 0, RX_ACK_DELAY, 0);
 
            CLEAR_CALL_QUEUE_LOCK(call);
@@ -2073,7 +2143,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
                *socketp = OSI_NULLSOCKET;
            }
            sq->socketp = socketp;
-           queue_Append(&rx_idleServerQueue, sq);
+           opr_queue_Append(&rx_idleServerQueue, &sq->entry);
 #ifndef AFS_AIX41_ENV
            rx_waitForPacket = sq;
 #else
@@ -2105,7 +2175,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
     if (call) {
        clock_GetTime(&call->startTime);
        call->state = RX_STATE_ACTIVE;
-       call->mode = RX_MODE_RECEIVING;
+       call->app.mode = RX_MODE_RECEIVING;
 #ifdef RX_KERNEL_TRACE
        if (ICL_SETACTIVE(afs_iclSetp)) {
            int glockOwner = ISAFS_GLOCK();
@@ -2163,8 +2233,9 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
        rxi_availProcs++;
         MUTEX_EXIT(&rx_quota_mutex);
     }
-    if (queue_IsNotEmpty(&rx_incomingCallQueue)) {
-       struct rx_call *tcall, *ncall;
+    if (!opr_queue_IsEmpty(&rx_incomingCallQueue)) {
+       struct rx_call *tcall;
+       struct opr_queue *cursor;
        /* Scan for eligible incoming calls.  A call is not eligible
         * if the maximum number of calls for its service type are
         * already executing */
@@ -2173,12 +2244,14 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
         * have all their input data available immediately.  This helps
         * keep threads from blocking, waiting for data from the client. */
        choice2 = (struct rx_call *)0;
-       for (queue_Scan(&rx_incomingCallQueue, tcall, ncall, rx_call)) {
+       for (opr_queue_Scan(&rx_incomingCallQueue, cursor)) {
+           tcall = opr_queue_Entry(cursor, struct rx_call, entry);
            service = tcall->conn->service;
            if (QuotaOK(service)) {
                MUTEX_ENTER(&rx_pthread_mutex);
-               if (tno == rxi_fcfs_thread_num
-                   || !tcall->queue_item_header.next) {
+               /* XXX - If tcall->entry.next is NULL, then we're no longer
+                * on a queue at all. This shouldn't happen. */
+               if (tno == rxi_fcfs_thread_num || !tcall->entry.next) {
                    MUTEX_EXIT(&rx_pthread_mutex);
                    /* If we're the fcfs thread, then  we'll just use
                     * this call. If we haven't been able to find an optimal
@@ -2188,9 +2261,10 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
                    service = call->conn->service;
                } else {
                    MUTEX_EXIT(&rx_pthread_mutex);
-                   if (!queue_IsEmpty(&tcall->rq)) {
+                   if (!opr_queue_IsEmpty(&tcall->rq)) {
                        struct rx_packet *rp;
-                       rp = queue_First(&tcall->rq, rx_packet);
+                       rp = opr_queue_First(&tcall->rq, struct rx_packet,
+                                           entry);
                        if (rp->header.seq == 1
                            && (!meltdown_1pkt
                                || (rp->header.flags & RX_LAST_PACKET))) {
@@ -2210,14 +2284,14 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
     }
 
     if (call) {
-       queue_Remove(call);
+       opr_queue_Remove(&call->entry);
        /* we can't schedule a call if there's no data!!! */
        /* send an ack if there's no data, if we're missing the
         * first packet, or we're missing something between first
         * and last -- there's a "hole" in the incoming data. */
-       if (queue_IsEmpty(&call->rq)
-           || queue_First(&call->rq, rx_packet)->header.seq != 1
-           || call->rprev != queue_Last(&call->rq, rx_packet)->header.seq)
+       if (opr_queue_IsEmpty(&call->rq)
+           || opr_queue_First(&call->rq, struct rx_packet, entry)->header.seq != 1
+           || call->rprev != opr_queue_Last(&call->rq, struct rx_packet, entry)->header.seq)
            rxi_SendAck(call, 0, 0, RX_ACK_DELAY, 0);
 
        call->flags &= (~RX_CALL_WAIT_PROC);
@@ -2239,7 +2313,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
            *socketp = OSI_NULLSOCKET;
        }
        sq->socketp = socketp;
-       queue_Append(&rx_idleServerQueue, sq);
+       opr_queue_Append(&rx_idleServerQueue, &sq->entry);
        do {
            osi_rxSleep(sq);
 #ifdef KERNEL
@@ -2262,7 +2336,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
     if (call) {
        clock_GetTime(&call->startTime);
        call->state = RX_STATE_ACTIVE;
-       call->mode = RX_MODE_RECEIVING;
+       call->app.mode = RX_MODE_RECEIVING;
 #ifdef RX_KERNEL_TRACE
        if (ICL_SETACTIVE(afs_iclSetp)) {
            int glockOwner = ISAFS_GLOCK();
@@ -2339,7 +2413,7 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
     call->arrivalProc = (void (*)())0;
     if (rc && call->error == 0) {
        rxi_CallError(call, rc);
-        call->mode = RX_MODE_ERROR;
+        call->app.mode = RX_MODE_ERROR;
        /* Send an abort message to the peer if this error code has
         * only just been set.  If it was set previously, assume the
         * peer has already been sent the error code or will request it
@@ -2348,12 +2422,12 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
     }
     if (conn->type == RX_SERVER_CONNECTION) {
        /* Make sure reply or at least dummy reply is sent */
-       if (call->mode == RX_MODE_RECEIVING) {
+       if (call->app.mode == RX_MODE_RECEIVING) {
            MUTEX_EXIT(&call->lock);
            rxi_WriteProc(call, 0, 0);
            MUTEX_ENTER(&call->lock);
        }
-       if (call->mode == RX_MODE_SENDING) {
+       if (call->app.mode == RX_MODE_SENDING) {
             MUTEX_EXIT(&call->lock);
            rxi_FlushWrite(call);
             MUTEX_ENTER(&call->lock);
@@ -2373,8 +2447,9 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
        char dummy;
        /* Make sure server receives input packets, in the case where
         * no reply arguments are expected */
-       if ((call->mode == RX_MODE_SENDING)
-           || (call->mode == RX_MODE_RECEIVING && call->rnext == 1)) {
+
+       if ((call->app.mode == RX_MODE_SENDING)
+           || (call->app.mode == RX_MODE_RECEIVING && call->rnext == 1)) {
            MUTEX_EXIT(&call->lock);
            (void)rxi_ReadProc(call, &dummy, 1);
            MUTEX_ENTER(&call->lock);
@@ -2429,21 +2504,21 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
      * ResetCall cannot: ResetCall may be called at splnet(), in the
      * kernel version, and may interrupt the macros rx_Read or
      * rx_Write, which run at normal priority for efficiency. */
-    if (call->currentPacket) {
+    if (call->app.currentPacket) {
 #ifdef RX_TRACK_PACKETS
-        call->currentPacket->flags &= ~RX_PKTFLAG_CP;
+        call->app.currentPacket->flags &= ~RX_PKTFLAG_CP;
 #endif
-       rxi_FreePacket(call->currentPacket);
-       call->currentPacket = (struct rx_packet *)0;
+       rxi_FreePacket(call->app.currentPacket);
+       call->app.currentPacket = (struct rx_packet *)0;
     }
 
-    call->nLeft = call->nFree = call->curlen = 0;
+    call->app.nLeft = call->app.nFree = call->app.curlen = 0;
 
     /* Free any packets from the last call to ReadvProc/WritevProc */
 #ifdef RXDEBUG_PACKET
     call->iovqc -=
 #endif /* RXDEBUG_PACKET */
-        rxi_FreePackets(0, &call->iovq);
+        rxi_FreePackets(0, &call->app.iovq);
     MUTEX_EXIT(&call->lock);
 
     CALL_RELE(call, RX_CALL_REFCOUNT_BEGIN);
@@ -2579,7 +2654,7 @@ rxi_NewCall(struct rx_connection *conn, int channel)
     struct rx_call *call;
 #ifdef AFS_GLOBAL_RXLOCK_KERNEL
     struct rx_call *cp;        /* Call pointer temp */
-    struct rx_call *nxp;       /* Next call pointer, for queue_Scan */
+    struct opr_queue *cursor;
 #endif /* AFS_GLOBAL_RXLOCK_KERNEL */
 
     dpf(("rxi_NewCall(conn %"AFS_PTR_FMT", channel %d)\n", conn, channel));
@@ -2595,7 +2670,8 @@ rxi_NewCall(struct rx_connection *conn, int channel)
      * Skip over those with in-use TQs.
      */
     call = NULL;
-    for (queue_Scan(&rx_freeCallQueue, cp, nxp, rx_call)) {
+    for (opr_queue_Scan(&rx_freeCallQueue, cursor)) {
+       cp = opr_queue_Entry(cursor, struct rx_call, entry);
        if (!(cp->flags & RX_CALL_TQ_BUSY)) {
            call = cp;
            break;
@@ -2603,10 +2679,10 @@ rxi_NewCall(struct rx_connection *conn, int channel)
     }
     if (call) {
 #else /* AFS_GLOBAL_RXLOCK_KERNEL */
-    if (queue_IsNotEmpty(&rx_freeCallQueue)) {
-       call = queue_First(&rx_freeCallQueue, rx_call);
+    if (!opr_queue_IsEmpty(&rx_freeCallQueue)) {
+       call = opr_queue_First(&rx_freeCallQueue, struct rx_call, entry);
 #endif /* AFS_GLOBAL_RXLOCK_KERNEL */
-       queue_Remove(call);
+       opr_queue_Remove(&call->entry);
         if (rx_stats_active)
            rx_atomic_dec(&rx_stats.nFreeCallStructs);
        MUTEX_EXIT(&rx_freeCallQueue_lock);
@@ -2643,9 +2719,9 @@ rxi_NewCall(struct rx_connection *conn, int channel)
        CV_INIT(&call->cv_tq, "call tq", CV_DEFAULT, 0);
 
        /* Initialize once-only items */
-       queue_Init(&call->tq);
-       queue_Init(&call->rq);
-       queue_Init(&call->iovq);
+       opr_queue_Init(&call->tq);
+       opr_queue_Init(&call->rq);
+       opr_queue_Init(&call->app.iovq);
 #ifdef RXDEBUG_PACKET
         call->rqc = call->tqc = call->iovqc = 0;
 #endif /* RXDEBUG_PACKET */
@@ -2675,16 +2751,16 @@ rxi_NewCall(struct rx_connection *conn, int channel)
  *
  * call->lock amd rx_refcnt_mutex are held upon entry.
  * haveCTLock is set when called from rxi_ReapConnections.
+ *
+ * return 1 if the call is freed, 0 if not.
  */
-static void
+static int
 rxi_FreeCall(struct rx_call *call, int haveCTLock)
 {
     int channel = call->channel;
     struct rx_connection *conn = call->conn;
+    u_char state = call->state;
 
-
-    if (call->state == RX_STATE_DALLY || call->state == RX_STATE_HOLD)
-       (*call->callNumber)++;
     /*
      * We are setting the state to RX_STATE_RESET to
      * ensure that no one else will attempt to use this
@@ -2697,10 +2773,24 @@ rxi_FreeCall(struct rx_call *call, int haveCTLock)
     MUTEX_EXIT(&rx_refcnt_mutex);
     rxi_ResetCall(call, 0);
 
-    MUTEX_ENTER(&conn->conn_call_lock);
-    if (call->conn->call[channel] == call)
-        call->conn->call[channel] = 0;
-    MUTEX_EXIT(&conn->conn_call_lock);
+    if (MUTEX_TRYENTER(&conn->conn_call_lock))
+    {
+        if (state == RX_STATE_DALLY || state == RX_STATE_HOLD)
+            (*call->callNumber)++;
+
+        if (call->conn->call[channel] == call)
+            call->conn->call[channel] = 0;
+        MUTEX_EXIT(&conn->conn_call_lock);
+    } else {
+        /*
+         * We couldn't obtain the conn_call_lock so we can't
+         * disconnect the call from the connection.  Set the
+         * call state to dally so that the call can be reused.
+         */
+        MUTEX_ENTER(&rx_refcnt_mutex);
+        call->state = RX_STATE_DALLY;
+        return 0;
+    }
 
     MUTEX_ENTER(&rx_freeCallQueue_lock);
     SET_CALL_QUEUE_LOCK(call, &rx_freeCallQueue_lock);
@@ -2710,11 +2800,11 @@ rxi_FreeCall(struct rx_call *call, int haveCTLock)
      * the head of the list, and idle calls at the tail.
      */
     if (call->flags & RX_CALL_TQ_BUSY)
-       queue_Prepend(&rx_freeCallQueue, call);
+       opr_queue_Prepend(&rx_freeCallQueue, &call->entry);
     else
-       queue_Append(&rx_freeCallQueue, call);
+       opr_queue_Append(&rx_freeCallQueue, &call->entry);
 #else /* AFS_GLOBAL_RXLOCK_KERNEL */
-    queue_Append(&rx_freeCallQueue, call);
+    opr_queue_Append(&rx_freeCallQueue, &call->entry);
 #endif /* AFS_GLOBAL_RXLOCK_KERNEL */
     if (rx_stats_active)
        rx_atomic_inc(&rx_stats.nFreeCallStructs);
@@ -2750,6 +2840,7 @@ rxi_FreeCall(struct rx_call *call, int haveCTLock)
        MUTEX_EXIT(&conn->conn_data_lock);
     }
     MUTEX_ENTER(&rx_refcnt_mutex);
+    return 1;
 }
 
 rx_atomic_t rxi_Allocsize = RX_ATOMIC_INIT(0);
@@ -2851,6 +2942,51 @@ rxi_SetPeerMtu(struct rx_peer *peer, afs_uint32 host, afs_uint32 port, int mtu)
     MUTEX_EXIT(&rx_peerHashTable_lock);
 }
 
+#ifdef AFS_RXERRQ_ENV
+static void
+rxi_SetPeerDead(afs_uint32 host, afs_uint16 port)
+{
+    int hashIndex = PEER_HASH(host, port);
+    struct rx_peer *peer;
+
+    MUTEX_ENTER(&rx_peerHashTable_lock);
+
+    for (peer = rx_peerHashTable[hashIndex]; peer; peer = peer->next) {
+       if (peer->host == host && peer->port == port) {
+           break;
+       }
+    }
+
+    if (peer) {
+       rx_atomic_inc(&peer->neterrs);
+    }
+
+    MUTEX_EXIT(&rx_peerHashTable_lock);
+}
+
+void
+rxi_ProcessNetError(struct sock_extended_err *err, afs_uint32 addr, afs_uint16 port)
+{
+# ifdef AFS_ADAPT_PMTU
+    if (err->ee_errno == EMSGSIZE && err->ee_info >= 68) {
+       rxi_SetPeerMtu(NULL, addr, port, err->ee_info - RX_IPUDP_SIZE);
+       return;
+    }
+# endif
+    if (err->ee_origin == SO_EE_ORIGIN_ICMP && err->ee_type == ICMP_DEST_UNREACH) {
+       switch (err->ee_code) {
+       case ICMP_NET_UNREACH:
+       case ICMP_HOST_UNREACH:
+       case ICMP_PORT_UNREACH:
+       case ICMP_NET_ANO:
+       case ICMP_HOST_ANO:
+           rxi_SetPeerDead(addr, port);
+           break;
+       }
+    }
+}
+#endif /* AFS_RXERRQ_ENV */
+
 /* Find the peer process represented by the supplied (host,port)
  * combination.  If there is no appropriate active peer structure, a
  * new one will be allocated and initialized
@@ -2874,9 +3010,11 @@ rxi_FindPeer(afs_uint32 host, u_short port,
            pp = rxi_AllocPeer();       /* This bzero's *pp */
            pp->host = host;    /* set here or in InitPeerParams is zero */
            pp->port = port;
+#ifdef AFS_RXERRQ_ENV
+           rx_atomic_set(&pp->neterrs, 0);
+#endif
            MUTEX_INIT(&pp->peer_lock, "peer_lock", MUTEX_DEFAULT, 0);
-           queue_Init(&pp->congestionQueue);
-           queue_Init(&pp->rpcStats);
+           opr_queue_Init(&pp->rpcStats);
            pp->next = rx_peerHashTable[hashIndex];
            rx_peerHashTable[hashIndex] = pp;
            rxi_InitPeerParams(pp);
@@ -2906,7 +3044,7 @@ rxi_FindPeer(afs_uint32 host, u_short port,
  * parameter must match the existing index for the connection.  If a
  * server connection is created, it will be created using the supplied
  * index, if the index is valid for this service */
-struct rx_connection *
+static struct rx_connection *
 rxi_FindConnection(osi_socket socket, afs_uint32 host,
                   u_short port, u_short serviceId, afs_uint32 cid,
                   afs_uint32 epoch, int type, u_int securityIndex)
@@ -2970,8 +3108,6 @@ rxi_FindConnection(osi_socket socket, afs_uint32 host,
        conn->lastSendTime = clock_Sec();       /* don't GC immediately */
        conn->epoch = epoch;
        conn->cid = cid & RX_CIDMASK;
-       /* conn->serial = conn->lastSerial = 0; */
-       /* conn->timeout = 0; */
        conn->ackRate = RX_FAST_ACK_RATE;
        conn->service = service;
        conn->serviceId = serviceId;
@@ -2980,8 +3116,8 @@ rxi_FindConnection(osi_socket socket, afs_uint32 host,
        conn->nSpecific = 0;
        conn->specific = NULL;
        rx_SetConnDeadTime(conn, service->connDeadTime);
-       rx_SetConnIdleDeadTime(conn, service->idleDeadTime);
-       rx_SetServerConnIdleDeadErr(conn, service->idleDeadErr);
+       conn->idleDeadTime = service->idleDeadTime;
+       conn->idleDeadDetection = service->idleDeadErr ? 1 : 0;
        for (i = 0; i < RX_MAXCALLS; i++) {
            conn->twind[i] = rx_initSendWindow;
            conn->rwind[i] = rx_initReceiveWindow;
@@ -3024,11 +3160,12 @@ rxi_CheckBusy(struct rx_call *call)
     int channel = call->channel;
     int freechannel = 0;
     int i;
-    afs_uint32 callNumber = *call->callNumber;
+    afs_uint32 callNumber;
 
     MUTEX_EXIT(&call->lock);
 
     MUTEX_ENTER(&conn->conn_call_lock);
+    callNumber = *call->callNumber;
 
     /* Are there any other call slots on this conn that we should try? Look for
      * slots that are empty and are either non-busy, or were marked as busy
@@ -3062,8 +3199,6 @@ rxi_CheckBusy(struct rx_call *call)
        }
     }
 
-    MUTEX_EXIT(&conn->conn_call_lock);
-
     MUTEX_ENTER(&call->lock);
 
     /* Since the call->lock and conn->conn_call_lock have been released it is
@@ -3081,8 +3216,9 @@ rxi_CheckBusy(struct rx_call *call)
         * rxi_busyChannelError so the application can retry the request,
         * presumably on a less-busy call channel. */
 
-       rxi_CallError(call, rxi_busyChannelError);
+       rxi_CallError(call, RX_CALL_BUSY);
     }
+    MUTEX_EXIT(&conn->conn_call_lock);
 }
 
 /* There are two packet tracing routines available for testing and monitoring
@@ -3112,7 +3248,6 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
     int channel;
     afs_uint32 currentCallNumber;
     int type;
-    int skew;
 #ifdef RXDEBUG
     char *packetType;
 #endif
@@ -3131,6 +3266,31 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
         np->header.seq, np->header.flags, np));
 #endif
 
+    /* Account for connectionless packets */
+    if (rx_stats_active &&
+       ((np->header.type == RX_PACKET_TYPE_VERSION) ||
+         (np->header.type == RX_PACKET_TYPE_DEBUG))) {
+       struct rx_peer *peer;
+
+       /* Try to look up the peer structure, but don't create one */
+       peer = rxi_FindPeer(host, port, 0, 0);
+
+       /* Since this may not be associated with a connection, it may have
+        * no refCount, meaning we could race with ReapConnections
+        */
+
+       if (peer && (peer->refCount > 0)) {
+#ifdef AFS_RXERRQ_ENV
+           if (rx_atomic_read(&peer->neterrs)) {
+               rx_atomic_set(&peer->neterrs, 0);
+           }
+#endif
+           MUTEX_ENTER(&peer->peer_lock);
+           peer->bytesReceived += np->length;
+           MUTEX_EXIT(&peer->peer_lock);
+       }
+    }
+
     if (np->header.type == RX_PACKET_TYPE_VERSION) {
        return rxi_ReceiveVersionPacket(np, socket, host, port, 1);
     }
@@ -3170,11 +3330,26 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
                           np->header.cid, np->header.epoch, type,
                           np->header.securityIndex);
 
+    /* To avoid having 2 connections just abort at each other,
+       don't abort an abort. */
     if (!conn) {
-       /* If no connection found or fabricated, just ignore the packet.
-        * (An argument could be made for sending an abort packet for
-        * the conn) */
-       return np;
+        if (np->header.type != RX_PACKET_TYPE_ABORT)
+            rxi_SendRawAbort(socket, host, port, RX_INVALID_OPERATION,
+                             np, 0);
+        return np;
+    }
+
+#ifdef AFS_RXERRQ_ENV
+    if (rx_atomic_read(&conn->peer->neterrs)) {
+       rx_atomic_set(&conn->peer->neterrs, 0);
+    }
+#endif
+
+    /* If we're doing statistics, then account for the incoming packet */
+    if (rx_stats_active) {
+       MUTEX_ENTER(&conn->peer->peer_lock);
+       conn->peer->bytesReceived += np->length;
+       MUTEX_EXIT(&conn->peer->peer_lock);
     }
 
     /* If the connection is in an error state, send an abort packet and ignore
@@ -3228,57 +3403,55 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
     }
 
     channel = np->header.cid & RX_CHANNELMASK;
+    MUTEX_ENTER(&conn->conn_call_lock);
     call = conn->call[channel];
 
     if (call) {
        MUTEX_ENTER(&call->lock);
         currentCallNumber = conn->callNumber[channel];
+        MUTEX_EXIT(&conn->conn_call_lock);
     } else if (type == RX_SERVER_CONNECTION) {  /* No call allocated */
-        MUTEX_ENTER(&conn->conn_call_lock);
-        call = conn->call[channel];
-        if (call) {
-            MUTEX_ENTER(&call->lock);
-            MUTEX_EXIT(&conn->conn_call_lock);
-            currentCallNumber = conn->callNumber[channel];
-        } else {
-            call = rxi_NewCall(conn, channel);  /* returns locked call */
-            MUTEX_EXIT(&conn->conn_call_lock);
-            *call->callNumber = currentCallNumber = np->header.callNumber;
+       call = rxi_NewCall(conn, channel);  /* returns locked call */
+       *call->callNumber = currentCallNumber = np->header.callNumber;
+       MUTEX_EXIT(&conn->conn_call_lock);
 #ifdef RXDEBUG
-            if (np->header.callNumber == 0)
-                dpf(("RecPacket call 0 %d %s: %x.%u.%u.%u.%u.%u.%u flags %d, packet %"AFS_PTR_FMT" len %d\n",
-                     np->header.serial, rx_packetTypes[np->header.type - 1], ntohl(conn->peer->host), ntohs(conn->peer->port),
-                     np->header.serial, np->header.epoch, np->header.cid, np->header.callNumber, np->header.seq,
-                     np->header.flags, np, np->length));
+       if (np->header.callNumber == 0)
+           dpf(("RecPacket call 0 %d %s: %x.%u.%u.%u.%u.%u.%u flags %d, "
+                "packet %"AFS_PTR_FMT" len %d\n",
+                np->header.serial, rx_packetTypes[np->header.type - 1], 
+                ntohl(conn->peer->host), ntohs(conn->peer->port),
+                np->header.serial, np->header.epoch, np->header.cid, 
+                np->header.callNumber, np->header.seq,
+                np->header.flags, np, np->length));
 #endif
-            call->state = RX_STATE_PRECALL;
-            clock_GetTime(&call->queueTime);
-            hzero(call->bytesSent);
-            hzero(call->bytesRcvd);
-            /*
-             * If the number of queued calls exceeds the overload
-             * threshold then abort this call.
-             */
-            if ((rx_BusyThreshold > 0) &&
-                (rx_atomic_read(&rx_nWaiting) > rx_BusyThreshold)) {
-                struct rx_packet *tp;
+       call->state = RX_STATE_PRECALL;
+       clock_GetTime(&call->queueTime);
+       call->app.bytesSent = 0;
+       call->app.bytesRcvd = 0;
+       /*
+        * If the number of queued calls exceeds the overload
+        * threshold then abort this call.
+        */
+       if ((rx_BusyThreshold > 0) &&
+           (rx_atomic_read(&rx_nWaiting) > rx_BusyThreshold)) {
+           struct rx_packet *tp;
 
-                rxi_CallError(call, rx_BusyError);
-                tp = rxi_SendCallAbort(call, np, 1, 0);
-                MUTEX_EXIT(&call->lock);
-               putConnection(conn);
-                if (rx_stats_active)
-                    rx_atomic_inc(&rx_stats.nBusies);
-                return tp;
-            }
-            rxi_KeepAliveOn(call);
+           rxi_CallError(call, rx_BusyError);
+           tp = rxi_SendCallAbort(call, np, 1, 0);
+           MUTEX_EXIT(&call->lock);
+           putConnection(conn);
+           if (rx_stats_active)
+               rx_atomic_inc(&rx_stats.nBusies);
+           return tp;
         }
+       rxi_KeepAliveOn(call);
     } else {    /* RX_CLIENT_CONNECTION and No call allocated */
         /* This packet can't be for this call. If the new call address is
          * 0 then no call is running on this channel. If there is a call
          * then, since this is a client connection we're getting data for
          * it must be for the previous call.
          */
+        MUTEX_EXIT(&conn->conn_call_lock);
         if (rx_stats_active)
             rx_atomic_inc(&rx_stats.spuriousPacketsRead);
        putConnection(conn);
@@ -3329,6 +3502,11 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
                return tp;
            }
            rxi_ResetCall(call, 0);
+            /*
+             * The conn_call_lock is not held but no one else should be
+             * using this call channel while we are processing this incoming
+             * packet.  This assignment should be safe.
+             */
            *call->callNumber = np->header.callNumber;
 #ifdef RXDEBUG
            if (np->header.callNumber == 0)
@@ -3339,8 +3517,8 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
 #endif
            call->state = RX_STATE_PRECALL;
            clock_GetTime(&call->queueTime);
-           hzero(call->bytesSent);
-           hzero(call->bytesRcvd);
+           call->app.bytesSent = 0;
+           call->app.bytesRcvd = 0;
            /*
             * If the number of queued calls exceeds the overload
             * threshold then abort this call.
@@ -3445,31 +3623,6 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
     /* Set remote user defined status from packet */
     call->remoteStatus = np->header.userStatus;
 
-    /* Note the gap between the expected next packet and the actual
-     * packet that arrived, when the new packet has a smaller serial number
-     * than expected.  Rioses frequently reorder packets all by themselves,
-     * so this will be quite important with very large window sizes.
-     * Skew is checked against 0 here to avoid any dependence on the type of
-     * inPacketSkew (which may be unsigned).  In C, -1 > (unsigned) 0 is always
-     * true!
-     * The inPacketSkew should be a smoothed running value, not just a maximum.  MTUXXX
-     * see CalculateRoundTripTime for an example of how to keep smoothed values.
-     * I think using a beta of 1/8 is probably appropriate.  93.04.21
-     */
-    MUTEX_ENTER(&conn->conn_data_lock);
-    skew = conn->lastSerial - np->header.serial;
-    conn->lastSerial = np->header.serial;
-    MUTEX_EXIT(&conn->conn_data_lock);
-    if (skew > 0) {
-       struct rx_peer *peer;
-       peer = conn->peer;
-       if (skew > peer->inPacketSkew) {
-           dpf(("*** In skew changed from %d to %d\n",
-                  peer->inPacketSkew, skew));
-           peer->inPacketSkew = skew;
-       }
-    }
-
     /* Now do packet type-specific processing */
     switch (np->header.type) {
     case RX_PACKET_TYPE_DATA:
@@ -3579,8 +3732,8 @@ rxi_IsConnInteresting(struct rx_connection *aconn)
            if ((tcall->state == RX_STATE_PRECALL)
                || (tcall->state == RX_STATE_ACTIVE))
                return 1;
-           if ((tcall->mode == RX_MODE_SENDING)
-               || (tcall->mode == RX_MODE_RECEIVING))
+           if ((tcall->app.mode == RX_MODE_SENDING)
+               || (tcall->app.mode == RX_MODE_RECEIVING))
                return 1;
        }
     }
@@ -3760,7 +3913,7 @@ TryAttach(struct rx_call *acall, osi_socket socket,
  * appropriate to the call (the call is in the right state, etc.).  This
  * routine can return a packet to the caller, for re-use */
 
-struct rx_packet *
+static struct rx_packet *
 rxi_ReceiveDataPacket(struct rx_call *call,
                      struct rx_packet *np, int istack,
                      osi_socket socket, afs_uint32 host, u_short port,
@@ -3787,15 +3940,12 @@ rxi_ReceiveDataPacket(struct rx_call *call,
        MUTEX_EXIT(&rx_freePktQ_lock);
         if (rx_stats_active)
             rx_atomic_inc(&rx_stats.noPacketBuffersOnRead);
-       call->rprev = np->header.serial;
        rxi_calltrace(RX_TRACE_DROP, call);
        dpf(("packet %"AFS_PTR_FMT" dropped on receipt - quota problems\n", np));
         /* We used to clear the receive queue here, in an attempt to free
          * packets. However this is unsafe if the queue has received a
          * soft ACK for the final packet */
        rxi_PostDelayedAckEvent(call, &rx_softAckDelay);
-
-       /* we've damaged this call already, might as well do it in. */
        return np;
     }
 #endif /* KERNEL */
@@ -3841,8 +3991,8 @@ rxi_ReceiveDataPacket(struct rx_call *call,
        if (seq == call->rnext) {
 
            /* Check to make sure it is not a duplicate of one already queued */
-           if (queue_IsNotEmpty(&call->rq)
-               && queue_First(&call->rq, rx_packet)->header.seq == seq) {
+           if (!opr_queue_IsEmpty(&call->rq)
+               && opr_queue_First(&call->rq, struct rx_packet, entry)->header.seq == seq) {
                 if (rx_stats_active)
                     rx_atomic_inc(&rx_stats.dupPacketsRead);
                dpf(("packet %"AFS_PTR_FMT" dropped on receipt - duplicate\n", np));
@@ -3860,7 +4010,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
 #ifdef RX_TRACK_PACKETS
            np->flags |= RX_PKTFLAG_RQ;
 #endif
-           queue_Prepend(&call->rq, np);
+           opr_queue_Prepend(&call->rq, &np->entry);
 #ifdef RXDEBUG_PACKET
             call->rqc++;
 #endif /* RXDEBUG_PACKET */
@@ -3883,10 +4033,12 @@ rxi_ReceiveDataPacket(struct rx_call *call,
            /* Check whether we have all of the packets for this call */
            if (call->flags & RX_CALL_HAVE_LAST) {
                afs_uint32 tseq;        /* temporary sequence number */
-               struct rx_packet *tp;   /* Temporary packet pointer */
-               struct rx_packet *nxp;  /* Next pointer, for queue_Scan */
+               struct opr_queue *cursor;
 
-               for (tseq = seq, queue_Scan(&call->rq, tp, nxp, rx_packet)) {
+               for (tseq = seq, opr_queue_Scan(&call->rq, cursor)) {
+                   struct rx_packet *tp;
+                   
+                   tp = opr_queue_Entry(cursor, struct rx_packet, entry);
                    if (tseq != tp->header.seq)
                        break;
                    if (tp->header.flags & RX_LAST_PACKET) {
@@ -3927,8 +4079,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
             * any of this packets predecessors are missing.  */
 
            afs_uint32 prev;    /* "Previous packet" sequence number */
-           struct rx_packet *tp;       /* Temporary packet pointer */
-           struct rx_packet *nxp;      /* Next pointer, for queue_Scan */
+           struct opr_queue *cursor;
            int missing;        /* Are any predecessors missing? */
 
            /* If the new packet's sequence number has been sent to the
@@ -3958,8 +4109,12 @@ rxi_ReceiveDataPacket(struct rx_call *call,
            }
 
            /* Look for the packet in the queue of old received packets */
-           for (prev = call->rnext - 1, missing =
-                0, queue_Scan(&call->rq, tp, nxp, rx_packet)) {
+           prev = call->rnext - 1;
+           missing = 0;
+           for (opr_queue_Scan(&call->rq, cursor)) {
+               struct rx_packet *tp
+                   = opr_queue_Entry(cursor, struct rx_packet, entry);
+
                /*Check for duplicate packet */
                if (seq == tp->header.seq) {
                     if (rx_stats_active)
@@ -4000,7 +4155,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
 #ifdef RXDEBUG_PACKET
             call->rqc++;
 #endif /* RXDEBUG_PACKET */
-           queue_InsertBefore(tp, np);
+           opr_queue_InsertBefore(cursor, &np->entry);
            call->nSoftAcks++;
            np = NULL;
 
@@ -4009,8 +4164,10 @@ rxi_ReceiveDataPacket(struct rx_call *call,
                && !(call->flags & RX_CALL_RECEIVE_DONE)) {
                afs_uint32 tseq;        /* temporary sequence number */
 
-               for (tseq =
-                    call->rnext, queue_Scan(&call->rq, tp, nxp, rx_packet)) {
+               tseq = call->rnext;
+               for (opr_queue_Scan(&call->rq, cursor)) {
+                   struct rx_packet *tp
+                        = opr_queue_Entry(cursor, struct rx_packet, entry);
                    if (tseq != tp->header.seq)
                        break;
                    if (tp->header.flags & RX_LAST_PACKET) {
@@ -4153,22 +4310,20 @@ rx_ack_reason(int reason)
 
 
 /* The real smarts of the whole thing.  */
-struct rx_packet *
+static struct rx_packet *
 rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
                     int istack)
 {
     struct rx_ackPacket *ap;
     int nAcks;
     struct rx_packet *tp;
-    struct rx_packet *nxp;     /* Next packet pointer for queue_Scan */
     struct rx_connection *conn = call->conn;
     struct rx_peer *peer = conn->peer;
+    struct opr_queue *cursor;
     struct clock now;          /* Current time, for RTT calculations */
     afs_uint32 first;
     afs_uint32 prev;
     afs_uint32 serial;
-    /* because there are CM's that are bogus, sending weird values for this. */
-    afs_uint32 skew = 0;
     int nbytes;
     int missing;
     int acked;
@@ -4190,12 +4345,15 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
     first = ntohl(ap->firstPacket);
     prev = ntohl(ap->previousPacket);
     serial = ntohl(ap->serial);
-    /* temporarily disabled -- needs to degrade over time
-     * skew = ntohs(ap->maxSkew); */
 
-    /* Ignore ack packets received out of order */
+    /*
+     * Ignore ack packets received out of order while protecting
+     * against peers that set the previousPacket field to a packet
+     * serial number instead of a sequence number.
+     */
     if (first < call->tfirst ||
-        (first == call->tfirst && prev < call->tprev)) {
+        (first == call->tfirst && prev < call->tprev && prev < call->tfirst
+        + call->twind)) {
        return np;
     }
 
@@ -4239,11 +4397,11 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        size_t len;
 
        len = _snprintf(msg, sizeof(msg),
-                       "tid[%d] RACK: reason %s serial %u previous %u seq %u skew %d first %u acks %u space %u ",
+                       "tid[%d] RACK: reason %s serial %u previous %u seq %u first %u acks %u space %u ",
                         GetCurrentThreadId(), rx_ack_reason(ap->reason),
                         ntohl(ap->serial), ntohl(ap->previousPacket),
-                        (unsigned int)np->header.seq, (unsigned int)skew,
-                        ntohl(ap->firstPacket), ap->nAcks, ntohs(ap->bufferSpace) );
+                        (unsigned int)np->header.seq, ntohl(ap->firstPacket),
+                        ap->nAcks, ntohs(ap->bufferSpace) );
        if (nAcks) {
            int offset;
 
@@ -4257,10 +4415,10 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
 #else /* AFS_NT40_ENV */
     if (rx_Log) {
        fprintf(rx_Log,
-               "RACK: reason %x previous %u seq %u serial %u skew %d first %u",
+               "RACK: reason %x previous %u seq %u serial %u first %u",
                ap->reason, ntohl(ap->previousPacket),
                (unsigned int)np->header.seq, (unsigned int)serial,
-               (unsigned int)skew, ntohl(ap->firstPacket));
+               ntohl(ap->firstPacket));
        if (nAcks) {
            int offset;
            for (offset = 0; offset < nAcks; offset++)
@@ -4291,13 +4449,6 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        }
     }
 
-    /* Update the outgoing packet skew value to the latest value of
-     * the peer's incoming packet skew value.  The ack packet, of
-     * course, could arrive out of order, but that won't affect things
-     * much */
-    peer->outPacketSkew = skew;
-
-
     clock_GetTime(&now);
 
     /* The transmit queue splits into 4 sections.
@@ -4324,11 +4475,11 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
      * disposed of
      */
 
-    tp = queue_First(&call->tq, rx_packet);
-    while(!queue_IsEnd(&call->tq, tp) && tp->header.seq < first) {
+    tp = opr_queue_First(&call->tq, struct rx_packet, entry);
+    while(!opr_queue_IsEnd(&call->tq, &tp->entry) && tp->header.seq < first) {
        struct rx_packet *next;
 
-       next = queue_Next(tp, rx_packet);
+       next = opr_queue_Next(&tp->entry, struct rx_packet, entry);
        call->tfirst = tp->header.seq + 1;
 
        if (!(tp->flags & RX_PKTFLAG_ACKED)) {
@@ -4356,7 +4507,7 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        } else
 #endif /* AFS_GLOBAL_RXLOCK_KERNEL */
        {
-           queue_Remove(tp);
+           opr_queue_Remove(&tp->entry);
 #ifdef RX_TRACK_PACKETS
            tp->flags &= ~RX_PKTFLAG_TQ;
 #endif
@@ -4384,7 +4535,8 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
 
     call->nSoftAcked = 0;
     missing = 0;
-    while (!queue_IsEnd(&call->tq, tp) && tp->header.seq < first + nAcks) {
+    while (!opr_queue_IsEnd(&call->tq, &tp->entry) 
+          && tp->header.seq < first + nAcks) {
        /* Set the acknowledge flag per packet based on the
         * information in the ack packet. An acknowlegded packet can
         * be downgraded when the server has discarded a packet it
@@ -4406,7 +4558,7 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
            missing = 1;
        }
 
-       tp = queue_Next(tp, rx_packet);
+       tp = opr_queue_Next(&tp->entry, struct rx_packet, entry);
     }
 
     /* We don't need to take any action with the 3rd or 4th section in the
@@ -4591,7 +4743,10 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
         * so we will retransmit as soon as the window permits
         */
 
-       for (acked = 0, queue_ScanBackwards(&call->tq, tp, nxp, rx_packet)) {
+       acked = 0;
+       for (opr_queue_ScanBackwards(&call->tq, cursor)) {
+           struct rx_packet *tp =
+               opr_queue_Entry(cursor, struct rx_packet, entry);
            if (acked) {
                if (!(tp->flags & RX_PKTFLAG_ACKED)) {
                    tp->flags &= ~RX_PKTFLAG_SENT;
@@ -4651,14 +4806,14 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        call->state = RX_STATE_DALLY;
        rxi_ClearTransmitQueue(call, 0);
         rxevent_Cancel(&call->keepAliveEvent, call, RX_CALL_REFCOUNT_ALIVE);
-    } else if (!queue_IsEmpty(&call->tq)) {
+    } else if (!opr_queue_IsEmpty(&call->tq)) {
        rxi_Start(call, istack);
     }
     return np;
 }
 
 /* Received a response to a challenge packet */
-struct rx_packet *
+static struct rx_packet *
 rxi_ReceiveResponsePacket(struct rx_connection *conn,
                          struct rx_packet *np, int istack)
 {
@@ -4715,7 +4870,7 @@ rxi_ReceiveResponsePacket(struct rx_connection *conn,
  * back to the server.  The server is responsible for retrying the
  * challenge if it fails to get a response. */
 
-struct rx_packet *
+static struct rx_packet *
 rxi_ReceiveChallengePacket(struct rx_connection *conn,
                           struct rx_packet *np, int istack)
 {
@@ -4753,7 +4908,7 @@ rxi_ReceiveChallengePacket(struct rx_connection *conn,
 /* Find an available server process to service the current request in
  * the given call structure.  If one isn't available, queue up this
  * call so it eventually gets one */
-void
+static void
 rxi_AttachServerProc(struct rx_call *call,
                     osi_socket socket, int *tnop,
                     struct rx_call **newcallp)
@@ -4769,7 +4924,7 @@ rxi_AttachServerProc(struct rx_call *call,
     MUTEX_ENTER(&rx_serverPool_lock);
 
     haveQuota = QuotaOK(service);
-    if ((!haveQuota) || queue_IsEmpty(&rx_idleServerQueue)) {
+    if ((!haveQuota) || opr_queue_IsEmpty(&rx_idleServerQueue)) {
        /* If there are no processes available to service this call,
         * put the call on the incoming call queue (unless it's
         * already on the queue).
@@ -4785,16 +4940,18 @@ rxi_AttachServerProc(struct rx_call *call,
            rx_atomic_inc(&rx_nWaited);
            rxi_calltrace(RX_CALL_ARRIVAL, call);
            SET_CALL_QUEUE_LOCK(call, &rx_serverPool_lock);
-           queue_Append(&rx_incomingCallQueue, call);
+           opr_queue_Append(&rx_incomingCallQueue, &call->entry);
        }
     } else {
-       sq = queue_Last(&rx_idleServerQueue, rx_serverQueueEntry);
+       sq = opr_queue_Last(&rx_idleServerQueue,
+                           struct rx_serverQueueEntry, entry);
 
        /* If hot threads are enabled, and both newcallp and sq->socketp
         * are non-null, then this thread will process the call, and the
         * idle server thread will start listening on this threads socket.
         */
-       queue_Remove(sq);
+       opr_queue_Remove(&sq->entry);
+
        if (rx_enable_hot_thread && newcallp && sq->socketp) {
            *newcallp = call;
            *tnop = sq->tno;
@@ -4807,14 +4964,13 @@ rxi_AttachServerProc(struct rx_call *call,
        if (call->flags & RX_CALL_WAIT_PROC) {
            /* Conservative:  I don't think this should happen */
            call->flags &= ~RX_CALL_WAIT_PROC;
-           if (queue_IsOnQueue(call)) {
-               queue_Remove(call);
-
-               rx_atomic_dec(&rx_nWaiting);
+           rx_atomic_dec(&rx_nWaiting);
+           if (opr_queue_IsOnQueue(&call->entry)) {
+               opr_queue_Remove(&call->entry);
            }
        }
        call->state = RX_STATE_ACTIVE;
-       call->mode = RX_MODE_RECEIVING;
+       call->app.mode = RX_MODE_RECEIVING;
 #ifdef RX_KERNEL_TRACE
        {
            int glockOwner = ISAFS_GLOCK();
@@ -4851,33 +5007,15 @@ rxi_AttachServerProc(struct rx_call *call,
  * a new call is being prepared (in the case of a client) or a reply
  * is being prepared (in the case of a server).  Rather than sending
  * an ack packet, an ACKALL packet is sent. */
-void
-rxi_AckAll(struct rxevent *event, struct rx_call *call, char *dummy)
+static void
+rxi_AckAll(struct rx_call *call)
 {
-#ifdef RX_ENABLE_LOCKS
-    if (event) {
-       MUTEX_ENTER(&call->lock);
-       rxevent_Put(call->delayedAckEvent);
-       call->delayedAckEvent = NULL;
-       CALL_RELE(call, RX_CALL_REFCOUNT_ACKALL);
-    }
-    rxi_SendSpecial(call, call->conn, (struct rx_packet *)0,
-                   RX_PACKET_TYPE_ACKALL, NULL, 0, 0);
-    call->flags |= RX_CALL_ACKALL_SENT;
-    if (event)
-       MUTEX_EXIT(&call->lock);
-#else /* RX_ENABLE_LOCKS */
-    if (event) {
-       rxevent_Put(call->delayedAckEvent);
-       call->delayedAckEvent = NULL;
-    }
-    rxi_SendSpecial(call, call->conn, (struct rx_packet *)0,
-                   RX_PACKET_TYPE_ACKALL, NULL, 0, 0);
+    rxi_SendSpecial(call, call->conn, NULL, RX_PACKET_TYPE_ACKALL, 
+                   NULL, 0, 0);
     call->flags |= RX_CALL_ACKALL_SENT;
-#endif /* RX_ENABLE_LOCKS */
 }
 
-void
+static void
 rxi_SendDelayedAck(struct rxevent *event, void *arg1, void *unused1,
                   int unused2)
 {
@@ -4911,13 +5049,17 @@ rxi_SendDelayedAck(struct rxevent *event, void *arg1, void *unused1,
 static void
 rxi_SetAcksInTransmitQueue(struct rx_call *call)
 {
-    struct rx_packet *p, *tp;
+    struct opr_queue *cursor;
     int someAcked = 0;
 
-    for (queue_Scan(&call->tq, p, tp, rx_packet)) {
+    for (opr_queue_Scan(&call->tq, cursor)) {
+       struct rx_packet *p 
+               = opr_queue_Entry(cursor, struct rx_packet, entry);
+
        p->flags |= RX_PKTFLAG_ACKED;
        someAcked = 1;
     }
+
     if (someAcked) {
        call->flags |= RX_CALL_TQ_CLEARME;
        call->flags |= RX_CALL_TQ_SOME_ACKED;
@@ -4940,15 +5082,17 @@ rxi_SetAcksInTransmitQueue(struct rx_call *call)
 
 /* Clear out the transmit queue for the current call (all packets have
  * been received by peer) */
-void
+static void
 rxi_ClearTransmitQueue(struct rx_call *call, int force)
 {
 #ifdef AFS_GLOBAL_RXLOCK_KERNEL
-    struct rx_packet *p, *tp;
-
+    struct opr_queue *cursor;
     if (!force && (call->flags & RX_CALL_TQ_BUSY)) {
        int someAcked = 0;
-       for (queue_Scan(&call->tq, p, tp, rx_packet)) {
+       for (opr_queue_Scan(&call->tq, cursor)) {
+           struct rx_packet *p 
+               = opr_queue_Entry(cursor, struct rx_packet, entry);
+
            p->flags |= RX_PKTFLAG_ACKED;
            someAcked = 1;
        }
@@ -4983,10 +5127,10 @@ rxi_ClearTransmitQueue(struct rx_call *call, int force)
 #endif
 }
 
-void
+static void
 rxi_ClearReceiveQueue(struct rx_call *call)
 {
-    if (queue_IsNotEmpty(&call->rq)) {
+    if (!opr_queue_IsEmpty(&call->rq)) {
         u_short count;
 
         count = rxi_FreePackets(0, &call->rq);
@@ -5004,22 +5148,31 @@ rxi_ClearReceiveQueue(struct rx_call *call)
 }
 
 /* Send an abort packet for the specified call */
-struct rx_packet *
+static struct rx_packet *
 rxi_SendCallAbort(struct rx_call *call, struct rx_packet *packet,
                  int istack, int force)
 {
-    afs_int32 error;
+    afs_int32 error, cerror;
     struct clock when, now;
 
     if (!call->error)
        return packet;
 
+    switch (call->error) {
+    case RX_CALL_IDLE:
+    case RX_CALL_BUSY:
+        cerror = RX_CALL_TIMEOUT;
+        break;
+    default:
+        cerror = call->error;
+    }
+
     /* Clients should never delay abort messages */
     if (rx_IsClientConn(call->conn))
        force = 1;
 
-    if (call->abortCode != call->error) {
-       call->abortCode = call->error;
+    if (call->abortCode != cerror) {
+       call->abortCode = cerror;
        call->abortCount = 0;
     }
 
@@ -5029,7 +5182,7 @@ rxi_SendCallAbort(struct rx_call *call, struct rx_packet *packet,
            rxevent_Cancel(&call->delayedAbortEvent, call,
                           RX_CALL_REFCOUNT_ABORT);
        }
-       error = htonl(call->error);
+       error = htonl(cerror);
        call->abortCount++;
        packet =
            rxi_SendSpecial(call, call->conn, packet, RX_PACKET_TYPE_ABORT,
@@ -5169,7 +5322,7 @@ rxi_CallError(struct rx_call *call, afs_int32 error)
  * unprotected macros, and may only be reset by non-interrupting code.
  */
 
-void
+static void
 rxi_ResetCall(struct rx_call *call, int newcall)
 {
     int flags;
@@ -5188,7 +5341,7 @@ rxi_ResetCall(struct rx_call *call, int newcall)
     }
 
 
-    rxevent_Cancel(&call->growMTUEvent, call, RX_CALL_REFCOUNT_ALIVE);
+    rxevent_Cancel(&call->growMTUEvent, call, RX_CALL_REFCOUNT_MTU);
 
     if (call->delayedAbortEvent) {
        rxevent_Cancel(&call->delayedAbortEvent, call, RX_CALL_REFCOUNT_ABORT);
@@ -5245,9 +5398,13 @@ rxi_ResetCall(struct rx_call *call, int newcall)
     }
     call->flags = 0;
 
-    if ((flags & RX_CALL_PEER_BUSY)) {
+    if (!newcall && (flags & RX_CALL_PEER_BUSY)) {
        /* The call channel is still busy; resetting the call doesn't change
-        * that */
+        * that. However, if 'newcall' is set, we are processing a call
+        * structure that has either been recycled from the free list, or has
+        * been newly allocated. So, RX_CALL_PEER_BUSY is not relevant if
+        * 'newcall' is set, since it describes a completely different call
+        * channel which we do not care about. */
        call->flags |= RX_CALL_PEER_BUSY;
     }
 
@@ -5291,6 +5448,9 @@ rxi_ResetCall(struct rx_call *call, int newcall)
        osi_rxWakeup(&call->twind);
 #endif
 
+    if (flags & RX_CALL_WAIT_PROC) {
+       rx_atomic_dec(&rx_nWaiting);
+    }
 #ifdef RX_ENABLE_LOCKS
     /* The following ensures that we don't mess with any queue while some
      * other thread might also be doing so. The call_queue_lock field is
@@ -5303,20 +5463,15 @@ rxi_ResetCall(struct rx_call *call, int newcall)
      */
     if (call->call_queue_lock) {
        MUTEX_ENTER(call->call_queue_lock);
-       if (queue_IsOnQueue(call)) {
-           queue_Remove(call);
-           if (flags & RX_CALL_WAIT_PROC) {
-               rx_atomic_dec(&rx_nWaiting);
-           }
+       if (opr_queue_IsOnQueue(&call->entry)) {
+           opr_queue_Remove(&call->entry);
        }
        MUTEX_EXIT(call->call_queue_lock);
        CLEAR_CALL_QUEUE_LOCK(call);
     }
 #else /* RX_ENABLE_LOCKS */
-    if (queue_IsOnQueue(call)) {
-       queue_Remove(call);
-       if (flags & RX_CALL_WAIT_PROC)
-           rx_atomic_dec(&rx_nWaiting);
+    if (opr_queue_IsOnQueue(&call->entry)) {
+       opr_queue_Remove(&call->entry);
     }
 #endif /* RX_ENABLE_LOCKS */
 
@@ -5354,9 +5509,8 @@ rxi_SendAck(struct rx_call *call,
            int istack)
 {
     struct rx_ackPacket *ap;
-    struct rx_packet *rqp;
-    struct rx_packet *nxp;     /* For queue_Scan */
     struct rx_packet *p;
+    struct opr_queue *cursor;
     u_char offset = 0;
     afs_int32 templ;
     afs_uint32 padbytes = 0;
@@ -5463,18 +5617,23 @@ rxi_SendAck(struct rx_call *call,
      * are packets in the receive queue awaiting processing.
      */
     if ((call->flags & RX_CALL_ACKALL_SENT) &&
-        !queue_IsEmpty(&call->rq)) {
-        ap->firstPacket = htonl(queue_Last(&call->rq, rx_packet)->header.seq + 1);
+        !opr_queue_IsEmpty(&call->rq)) {
+        ap->firstPacket = htonl(opr_queue_Last(&call->rq, struct rx_packet, entry)->header.seq + 1);
     } else {
         ap->firstPacket = htonl(call->rnext);
 
        ap->previousPacket = htonl(call->rprev);        /* Previous packet received */
 
-       /* No fear of running out of ack packet here because there can only be at most
-        * one window full of unacknowledged packets.  The window size must be constrained
-        * to be less than the maximum ack size, of course.  Also, an ack should always
-        * fit into a single packet -- it should not ever be fragmented.  */
-       for (offset = 0, queue_Scan(&call->rq, rqp, nxp, rx_packet)) {
+       /* No fear of running out of ack packet here because there can only 
+        * be at most one window full of unacknowledged packets.  The window
+        * size must be constrained to be less than the maximum ack size, 
+        * of course.  Also, an ack should always fit into a single packet 
+        * -- it should not ever be fragmented.  */
+       offset = 0;
+       for (opr_queue_Scan(&call->rq, cursor)) {
+           struct rx_packet *rqp
+               = opr_queue_Entry(cursor, struct rx_packet, entry);
+
            if (!rqp || !call->rq.next
                || (rqp->header.seq > (call->rnext + call->rwind))) {
 #ifndef RX_ENABLE_TSFPQ
@@ -5797,8 +5956,11 @@ rxi_SendXmitList(struct rx_call *call, struct rx_packet **list, int len,
     /* Send the whole list when the call is in receive mode, when
      * the call is in eof mode, when we are in fast recovery mode,
      * and when we have the last packet */
+    /* XXX - The accesses to app.mode aren't safe, as this may be called by
+     * the listener or event threads
+     */
     if ((list[len - 1]->header.flags & RX_LAST_PACKET)
-       || call->mode == RX_MODE_RECEIVING || call->mode == RX_MODE_EOF
+       || (call->flags & RX_CALL_FLUSH)
        || (call->flags & RX_CALL_FAST_RECOVER)) {
        /* Check for the case where the current list contains
         * an acked packet. Since we always send retransmissions
@@ -5824,12 +5986,56 @@ rxi_SendXmitList(struct rx_call *call, struct rx_packet **list, int len,
     }
 }
 
+/**
+ * Check if the peer for the given call is known to be dead
+ *
+ * If the call's peer appears dead (it has encountered fatal network errors
+ * since the call started) the call is killed with RX_CALL_DEAD if the call
+ * is active. Otherwise, we do nothing.
+ *
+ * @param[in] call  The call to check
+ *
+ * @return status
+ *  @retval 0 The call is fine, and we haven't done anything to the call
+ *  @retval nonzero The call's peer appears dead, and the call has been
+ *                  terminated if it was active
+ *
+ * @pre call->lock must be locked
+ */
+static int
+rxi_CheckPeerDead(struct rx_call *call)
+{
+#ifdef AFS_RXERRQ_ENV
+    int peererrs;
+
+    if (call->state == RX_STATE_DALLY) {
+       return 0;
+    }
+
+    peererrs = rx_atomic_read(&call->conn->peer->neterrs);
+    if (call->neterr_gen < peererrs) {
+       /* we have received network errors since this call started; kill
+        * the call */
+       if (call->state == RX_STATE_ACTIVE) {
+           rxi_CallError(call, RX_CALL_DEAD);
+       }
+       return -1;
+    }
+    if (call->neterr_gen > peererrs) {
+       /* someone has reset the number of peer errors; set the call error gen
+        * so we can detect if more errors are encountered */
+       call->neterr_gen = peererrs;
+    }
+#endif
+    return 0;
+}
+
 static void
 rxi_Resend(struct rxevent *event, void *arg0, void *arg1, int istack)
 {
     struct rx_call *call = arg0;
     struct rx_peer *peer;
-    struct rx_packet *p, *nxp;
+    struct opr_queue *cursor;
     struct clock maxTimeout = { 60, 0 };
 
     MUTEX_ENTER(&call->lock);
@@ -5845,11 +6051,13 @@ rxi_Resend(struct rxevent *event, void *arg0, void *arg1, int istack)
        call->resendEvent = NULL;
     }
 
+    rxi_CheckPeerDead(call);
+
     if (rxi_busyChannelError && (call->flags & RX_CALL_PEER_BUSY)) {
        rxi_CheckBusy(call);
     }
 
-    if (queue_IsEmpty(&call->tq)) {
+    if (opr_queue_IsEmpty(&call->tq)) {
        /* Nothing to do. This means that we've been raced, and that an
         * ACK has come in between when we were triggered, and when we
         * actually got to run. */
@@ -5860,7 +6068,8 @@ rxi_Resend(struct rxevent *event, void *arg0, void *arg1, int istack)
     call->flags |= RX_CALL_FAST_RECOVER;
 
     /* Mark all of the pending packets in the queue as being lost */
-    for (queue_Scan(&call->tq, p, nxp, rx_packet)) {
+    for (opr_queue_Scan(&call->tq, cursor)) {
+       struct rx_packet *p = opr_queue_Entry(cursor, struct rx_packet, entry);
        if (!(p->flags & RX_PKTFLAG_ACKED))
            p->flags &= ~RX_PKTFLAG_SENT;
     }
@@ -5908,9 +6117,10 @@ out:
 void
 rxi_Start(struct rx_call *call, int istack)
 {
-
-    struct rx_packet *p;
-    struct rx_packet *nxp;     /* Next pointer for queue_Scan */
+    struct opr_queue *cursor;
+#ifdef RX_ENABLE_LOCKS
+    struct opr_queue *store;
+#endif
     int nXmitPackets;
     int maxXmitPackets;
 
@@ -5922,8 +6132,7 @@ rxi_Start(struct rx_call *call, int istack)
        return;
     }
 
-    if (queue_IsNotEmpty(&call->tq)) { /* If we have anything to send */
-
+    if (!opr_queue_IsEmpty(&call->tq)) {       /* If we have anything to send */
        /* Send (or resend) any packets that need it, subject to
         * window restrictions and congestion burst control
         * restrictions.  Ask for an ack on the last packet sent in
@@ -5950,16 +6159,10 @@ rxi_Start(struct rx_call *call, int istack)
 #endif /* AFS_GLOBAL_RXLOCK_KERNEL */
                nXmitPackets = 0;
                maxXmitPackets = MIN(call->twind, call->cwind);
-               for (queue_Scan(&call->tq, p, nxp, rx_packet)) {
-#ifdef RX_TRACK_PACKETS
-                   if ((p->flags & RX_PKTFLAG_FREE)
-                       || (!queue_IsEnd(&call->tq, nxp)
-                           && (nxp->flags & RX_PKTFLAG_FREE))
-                       || (p == (struct rx_packet *)&rx_freePacketQueue)
-                       || (nxp == (struct rx_packet *)&rx_freePacketQueue)) {
-                       osi_Panic("rxi_Start: xmit queue clobbered");
-                   }
-#endif
+               for (opr_queue_Scan(&call->tq, cursor)) {
+                   struct rx_packet *p
+                       = opr_queue_Entry(cursor, struct rx_packet, entry);
+
                    if (p->flags & RX_PKTFLAG_ACKED) {
                        /* Since we may block, don't trust this */
                         if (rx_stats_active)
@@ -5996,7 +6199,7 @@ rxi_Start(struct rx_call *call, int istack)
                               *(call->callNumber), p));
                        call->xmitList[nXmitPackets++] = p;
                    }
-               }
+               } /* end of the queue_Scan */
 
                /* xmitList now hold pointers to all of the packets that are
                 * ready to send. Now we loop to send the packets */
@@ -6025,11 +6228,14 @@ rxi_Start(struct rx_call *call, int istack)
                    /* Some packets have received acks. If they all have, we can clear
                     * the transmit queue.
                     */
-                   for (missing =
-                        0, queue_Scan(&call->tq, p, nxp, rx_packet)) {
+                   missing = 0;
+                   for (opr_queue_ScanSafe(&call->tq, cursor, store)) {
+                       struct rx_packet *p
+                           = opr_queue_Entry(cursor, struct rx_packet, entry);
+
                        if (p->header.seq < call->tfirst
                            && (p->flags & RX_PKTFLAG_ACKED)) {
-                           queue_Remove(p);
+                           opr_queue_Remove(&p->entry);
 #ifdef RX_TRACK_PACKETS
                            p->flags &= ~RX_PKTFLAG_TQ;
 #endif
@@ -6114,10 +6320,10 @@ rxi_Send(struct rx_call *call, struct rx_packet *p,
  * haveCTLock Set if calling from rxi_ReapConnections
  */
 #ifdef RX_ENABLE_LOCKS
-int
+static int
 rxi_CheckCall(struct rx_call *call, int haveCTLock)
 #else /* RX_ENABLE_LOCKS */
-int
+static int
 rxi_CheckCall(struct rx_call *call)
 #endif                         /* RX_ENABLE_LOCKS */
 {
@@ -6127,6 +6333,36 @@ rxi_CheckCall(struct rx_call *call)
     afs_uint32 fudgeFactor;
     int cerror = 0;
     int newmtu = 0;
+    int idle_timeout = 0;
+    afs_int32  clock_diff = 0;
+
+    if (rxi_CheckPeerDead(call)) {
+       return -1;
+    }
+
+    now = clock_Sec();
+
+    /* Large swings in the clock can have a significant impact on
+     * the performance of RX call processing.  Forward clock shifts
+     * will result in premature event triggering or timeouts.
+     * Backward shifts can result in calls not completing until
+     * the clock catches up with the original start clock value.
+     *
+     * If a backward clock shift of more than five minutes is noticed,
+     * just fail the call.
+     */
+    if (now < call->lastSendTime)
+        clock_diff = call->lastSendTime - now;
+    if (now < call->startWait)
+        clock_diff = MAX(clock_diff, call->startWait - now);
+    if (now < call->lastReceiveTime)
+        clock_diff = MAX(clock_diff, call->lastReceiveTime - now);
+    if (clock_diff > 5 * 60)
+    {
+       if (call->state == RX_STATE_ACTIVE)
+           rxi_CallError(call, RX_CALL_TIMEOUT);
+       return -1;
+    }
 
 #ifdef AFS_GLOBAL_RXLOCK_KERNEL
     if (call->flags & RX_CALL_TQ_BUSY) {
@@ -6141,39 +6377,38 @@ rxi_CheckCall(struct rx_call *call)
                    ((afs_uint32) call->rtt_dev << 1) + 1023) >> 10;
 
     deadTime = conn->secondsUntilDead + fudgeFactor;
-    now = clock_Sec();
     /* These are computed to the second (+- 1 second).  But that's
      * good enough for these values, which should be a significant
      * number of seconds. */
     if (now > (call->lastReceiveTime + deadTime)) {
        if (call->state == RX_STATE_ACTIVE) {
-#ifdef ADAPT_PMTU
-#if defined(KERNEL) && defined(AFS_SUN5_ENV)
+#ifdef AFS_ADAPT_PMTU
+# if defined(KERNEL) && defined(AFS_SUN5_ENV)
            ire_t *ire;
-#if defined(AFS_SUN510_ENV) && defined(GLOBAL_NETSTACKID)
-           netstack_t *ns =  netstack_find_by_stackid(GLOBAL_NETSTACKID);
+#  if defined(AFS_SUN510_ENV) && defined(GLOBAL_NETSTACKID)
+           netstack_t *ns = netstack_find_by_stackid(GLOBAL_NETSTACKID);
            ip_stack_t *ipst = ns->netstack_ip;
-#endif
+#  endif
            ire = ire_cache_lookup(conn->peer->host
-#if defined(AFS_SUN510_ENV) && defined(ALL_ZONES)
+#  if defined(AFS_SUN510_ENV) && defined(ALL_ZONES)
                                   , ALL_ZONES
-#if defined(AFS_SUN510_ENV) && (defined(ICL_3_ARG) || defined(GLOBAL_NETSTACKID))
+#    if defined(ICL_3_ARG) || defined(GLOBAL_NETSTACKID)
                                   , NULL
-#if defined(AFS_SUN510_ENV) && defined(GLOBAL_NETSTACKID)
+#     if defined(GLOBAL_NETSTACKID)
                                   , ipst
-#endif
-#endif
-#endif
+#     endif
+#    endif
+#  endif
                );
 
            if (ire && ire->ire_max_frag > 0)
                rxi_SetPeerMtu(NULL, conn->peer->host, 0,
                               ire->ire_max_frag);
-#if defined(GLOBAL_NETSTACKID)
+#  if defined(GLOBAL_NETSTACKID)
            netstack_rele(ns);
-#endif
-#endif
-#endif /* ADAPT_PMTU */
+#  endif
+# endif
+#endif /* AFS_ADAPT_PMTU */
            cerror = RX_CALL_DEAD;
            goto mtuout;
        } else {
@@ -6185,12 +6420,14 @@ rxi_CheckCall(struct rx_call *call)
            rxevent_Cancel(&call->keepAliveEvent, call,
                           RX_CALL_REFCOUNT_ALIVE);
            rxevent_Cancel(&call->growMTUEvent, call,
-                          RX_CALL_REFCOUNT_ALIVE);
+                          RX_CALL_REFCOUNT_MTU);
             MUTEX_ENTER(&rx_refcnt_mutex);
-           if (call->refCount == 0) {
-               rxi_FreeCall(call, haveCTLock);
+            /* if rxi_FreeCall returns 1 it has freed the call */
+           if (call->refCount == 0 &&
+                rxi_FreeCall(call, haveCTLock))
+            {
                 MUTEX_EXIT(&rx_refcnt_mutex);
-               return -2;
+                return -2;
            }
             MUTEX_EXIT(&rx_refcnt_mutex);
            return -1;
@@ -6204,25 +6441,29 @@ rxi_CheckCall(struct rx_call *call)
         * attached process can die reasonably gracefully. */
     }
 
-    if (conn->idleDeadTime) {
-       idleDeadTime = conn->idleDeadTime + fudgeFactor;
-    }
+    if (conn->idleDeadDetection) {
+        if (conn->idleDeadTime) {
+            idleDeadTime = conn->idleDeadTime + fudgeFactor;
+        }
 
-    /* see if we have a non-activity timeout */
-    if (call->startWait && idleDeadTime
-       && ((call->startWait + idleDeadTime) < now) &&
-       (call->flags & RX_CALL_READER_WAIT)) {
-       if (call->state == RX_STATE_ACTIVE) {
-           cerror = RX_CALL_TIMEOUT;
-           goto mtuout;
-       }
-    }
-    if (call->lastSendData && idleDeadTime && (conn->idleDeadErr != 0)
-        && ((call->lastSendData + idleDeadTime) < now)) {
-       if (call->state == RX_STATE_ACTIVE) {
-           cerror = conn->idleDeadErr;
-           goto mtuout;
-       }
+        if (idleDeadTime) {
+            /* see if we have a non-activity timeout */
+            if (call->startWait && ((call->startWait + idleDeadTime) < now) &&
+                (call->flags & RX_CALL_READER_WAIT)) {
+                if (call->state == RX_STATE_ACTIVE) {
+                    cerror = RX_CALL_TIMEOUT;
+                    goto mtuout;
+                }
+            }
+
+            if (call->lastSendData && ((call->lastSendData + idleDeadTime) < now)) {
+                if (call->state == RX_STATE_ACTIVE) {
+                    cerror = conn->service ? conn->service->idleDeadErr : RX_CALL_IDLE;
+                    idle_timeout = 1;
+                    goto mtuout;
+                }
+            }
+        }
     }
 
     if (conn->hardDeadTime) {
@@ -6238,8 +6479,8 @@ rxi_CheckCall(struct rx_call *call)
     }
     return 0;
 mtuout:
-    if (conn->msgsizeRetryErr && cerror != RX_CALL_TIMEOUT
-       && call->lastReceiveTime) {
+    if (conn->msgsizeRetryErr && cerror != RX_CALL_TIMEOUT && !idle_timeout &&
+        call->lastReceiveTime) {
        int oldMTU = conn->peer->ifMTU;
 
        /* if we thought we could send more, perhaps things got worse */
@@ -6327,7 +6568,7 @@ rxi_NatKeepAliveEvent(struct rxevent *event, void *arg1,
     }
 }
 
-void
+static void
 rxi_ScheduleNatKeepAliveEvent(struct rx_connection *conn)
 {
     if (!conn->natKeepAliveEvent && conn->secondsUntilNatPing) {
@@ -6357,18 +6598,6 @@ rx_SetConnSecondsUntilNatPing(struct rx_connection *conn, afs_int32 seconds)
     MUTEX_EXIT(&conn->conn_data_lock);
 }
 
-void
-rxi_NatKeepAliveOn(struct rx_connection *conn)
-{
-    MUTEX_ENTER(&conn->conn_data_lock);
-    /* if it's already attached */
-    if (!(conn->flags & RX_CONN_ATTACHWAIT))
-       rxi_ScheduleNatKeepAliveEvent(conn);
-    else
-       conn->flags |= RX_CONN_NAT_PING;
-    MUTEX_EXIT(&conn->conn_data_lock);
-}
-
 /* When a call is in progress, this routine is called occasionally to
  * make sure that some traffic has arrived (or been sent to) the peer.
  * If nothing has arrived in a reasonable amount of time, the call is
@@ -6427,7 +6656,7 @@ rxi_GrowMTUEvent(struct rxevent *event, void *arg1, void *dummy, int dummy2)
     struct rx_call *call = arg1;
     struct rx_connection *conn;
 
-    CALL_RELE(call, RX_CALL_REFCOUNT_ALIVE);
+    CALL_RELE(call, RX_CALL_REFCOUNT_MTU);
     MUTEX_ENTER(&call->lock);
 
     if (event == call->growMTUEvent) {
@@ -6459,13 +6688,13 @@ rxi_GrowMTUEvent(struct rxevent *event, void *arg1, void *dummy, int dummy2)
      */
     if ((conn->peer->maxPacketSize != 0) &&
        (conn->peer->natMTU < RX_MAX_PACKET_SIZE) &&
-       (conn->idleDeadErr))
+       conn->idleDeadDetection)
        (void)rxi_SendAck(call, NULL, 0, RX_ACK_MTU, 0);
     rxi_ScheduleGrowMTUEvent(call, 0);
     MUTEX_EXIT(&call->lock);
 }
 
-void
+static void
 rxi_ScheduleKeepAliveEvent(struct rx_call *call)
 {
     if (!call->keepAliveEvent) {
@@ -6479,7 +6708,7 @@ rxi_ScheduleKeepAliveEvent(struct rx_call *call)
     }
 }
 
-void
+static void
 rxi_ScheduleGrowMTUEvent(struct rx_call *call, int secs)
 {
     if (!call->growMTUEvent) {
@@ -6496,14 +6725,14 @@ rxi_ScheduleGrowMTUEvent(struct rx_call *call, int secs)
        }
 
        when.sec += secs;
-       CALL_HOLD(call, RX_CALL_REFCOUNT_ALIVE);
+       CALL_HOLD(call, RX_CALL_REFCOUNT_MTU);
        call->growMTUEvent =
            rxevent_Post(&when, &now, rxi_GrowMTUEvent, call, NULL, 0);
     }
 }
 
 /* N.B. rxi_KeepAliveOff:  is defined earlier as a macro */
-void
+static void
 rxi_KeepAliveOn(struct rx_call *call)
 {
     /* Pretend last packet received was received now--i.e. if another
@@ -6529,7 +6758,7 @@ rx_KeepAliveOn(struct rx_call *call)
     rxi_KeepAliveOn(call);
 }
 
-void
+static void
 rxi_GrowMTUOn(struct rx_call *call)
 {
     struct rx_connection *conn = call->conn;
@@ -6541,7 +6770,7 @@ rxi_GrowMTUOn(struct rx_call *call)
 
 /* This routine is called to send connection abort messages
  * that have been delayed to throttle looping clients. */
-void
+static void
 rxi_SendDelayedConnAbort(struct rxevent *event, void *arg1, void *unused,
                         int unused2)
 {
@@ -6597,7 +6826,7 @@ rxi_SendDelayedCallAbort(struct rxevent *event, void *arg1, void *dummy,
  * seconds) to ask the client to authenticate itself.  The routine
  * issues a challenge to the client, which is obtained from the
  * security object associated with the connection */
-void
+static void
 rxi_ChallengeEvent(struct rxevent *event,
                   void *arg0, void *arg1, int tries)
 {
@@ -6658,7 +6887,7 @@ rxi_ChallengeEvent(struct rxevent *event,
  * security object associated with the connection is asked to create
  * the challenge at this time.  N.B.  rxi_ChallengeOff is a macro,
  * defined earlier. */
-void
+static void
 rxi_ChallengeOn(struct rx_connection *conn)
 {
     if (!conn->challengeEvent) {
@@ -6826,7 +7055,7 @@ rxi_ComputeRoundTripTime(struct rx_packet *p,
 
 /* Find all server connections that have not been active for a long time, and
  * toss them */
-void
+static void
 rxi_ReapConnections(struct rxevent *unused, void *unused1, void *unused2,
                    int unused3)
 {
@@ -6942,7 +7171,7 @@ rxi_ReapConnections(struct rxevent *unused, void *unused1, void *unused2,
                code = MUTEX_TRYENTER(&peer->peer_lock);
                if ((code) && (peer->refCount == 0)
                    && ((peer->idleWhen + rx_idlePeerTime) < now.sec)) {
-                   rx_interface_stat_p rpc_stat, nrpc_stat;
+                   struct opr_queue *cursor, *store;
                    size_t space;
 
                     /*
@@ -6974,14 +7203,18 @@ rxi_ReapConnections(struct rxevent *unused, void *unused1, void *unused2,
 
                    MUTEX_EXIT(&peer->peer_lock);
                    MUTEX_DESTROY(&peer->peer_lock);
-                   for (queue_Scan
-                        (&peer->rpcStats, rpc_stat, nrpc_stat,
-                         rx_interface_stat)) {
+
+                   for (opr_queue_ScanSafe(&peer->rpcStats, cursor, store)) {
                        unsigned int num_funcs;
+                       struct rx_interface_stat *rpc_stat
+                           = opr_queue_Entry(cursor, struct rx_interface_stat,
+                                            entry);
                        if (!rpc_stat)
                            break;
-                       queue_Remove(&rpc_stat->queue_header);
-                       queue_Remove(&rpc_stat->all_peers);
+
+                       opr_queue_Remove(&rpc_stat->entry);
+                       opr_queue_Remove(&rpc_stat->entryPeers);
+
                        num_funcs = rpc_stat->stats[0].func_total;
                        space =
                            sizeof(rx_interface_stat_t) +
@@ -7236,18 +7469,14 @@ rx_PrintStats(FILE * file)
 void
 rx_PrintPeerStats(FILE * file, struct rx_peer *peer)
 {
-    fprintf(file, "Peer %x.%d.  " "Burst size %d, " "burst wait %d.%06d.\n",
-           ntohl(peer->host), (int)ntohs(peer->port), (int)peer->burstSize,
-           (int)peer->burstWait.sec, (int)peer->burstWait.usec);
+    fprintf(file, "Peer %x.%d.\n",
+           ntohl(peer->host), (int)ntohs(peer->port));
 
     fprintf(file,
            "   Rtt %d, " "total sent %d, " "resent %d\n",
            peer->rtt, peer->nSent, peer->reSends);
 
-    fprintf(file,
-           "   Packet size %d, " "max in packet skew %d, "
-           "max out packet skew %d\n", peer->ifMTU, (int)peer->inPacketSkew,
-           (int)peer->outPacketSkew);
+    fprintf(file, "   Packet size %d\n", peer->ifMTU);
 }
 #endif
 
@@ -7616,16 +7845,12 @@ rx_GetServerPeers(osi_socket socket, afs_uint32 remoteAddr,
        peer->ifMTU = ntohs(peer->ifMTU);
        peer->idleWhen = ntohl(peer->idleWhen);
        peer->refCount = ntohs(peer->refCount);
-       peer->burstWait.sec = ntohl(peer->burstWait.sec);
-       peer->burstWait.usec = ntohl(peer->burstWait.usec);
        peer->rtt = ntohl(peer->rtt);
        peer->rtt_dev = ntohl(peer->rtt_dev);
        peer->timeout.sec = 0;
        peer->timeout.usec = 0;
        peer->nSent = ntohl(peer->nSent);
        peer->reSends = ntohl(peer->reSends);
-       peer->inPacketSkew = ntohl(peer->inPacketSkew);
-       peer->outPacketSkew = ntohl(peer->outPacketSkew);
        peer->natMTU = ntohs(peer->natMTU);
        peer->maxMTU = ntohs(peer->maxMTU);
        peer->maxDgramPackets = ntohs(peer->maxDgramPackets);
@@ -7672,18 +7897,16 @@ rx_GetLocalPeers(afs_uint32 peerHost, afs_uint16 peerPort,
                peerStats->ifMTU = tp->ifMTU;
                peerStats->idleWhen = tp->idleWhen;
                peerStats->refCount = tp->refCount;
-               peerStats->burstSize = tp->burstSize;
-               peerStats->burst = tp->burst;
-               peerStats->burstWait.sec = tp->burstWait.sec;
-               peerStats->burstWait.usec = tp->burstWait.usec;
+               peerStats->burstSize = 0;
+               peerStats->burst = 0;
+               peerStats->burstWait.sec = 0;
+               peerStats->burstWait.usec = 0;
                peerStats->rtt = tp->rtt;
                peerStats->rtt_dev = tp->rtt_dev;
                peerStats->timeout.sec = 0;
                peerStats->timeout.usec = 0;
                peerStats->nSent = tp->nSent;
                peerStats->reSends = tp->reSends;
-               peerStats->inPacketSkew = tp->inPacketSkew;
-               peerStats->outPacketSkew = tp->outPacketSkew;
                peerStats->natMTU = tp->natMTU;
                peerStats->maxMTU = tp->maxMTU;
                peerStats->maxDgramPackets = tp->maxDgramPackets;
@@ -7692,10 +7915,11 @@ rx_GetLocalPeers(afs_uint32 peerHost, afs_uint16 peerPort,
                peerStats->cwind = tp->cwind;
                peerStats->nDgramPackets = tp->nDgramPackets;
                peerStats->congestSeq = tp->congestSeq;
-               peerStats->bytesSent.high = tp->bytesSent.high;
-               peerStats->bytesSent.low = tp->bytesSent.low;
-               peerStats->bytesReceived.high = tp->bytesReceived.high;
-               peerStats->bytesReceived.low = tp->bytesReceived.low;
+               peerStats->bytesSent.high = tp->bytesSent >> 32;
+               peerStats->bytesSent.low = tp->bytesSent & MAX_AFS_UINT32;
+               peerStats->bytesReceived.high = tp->bytesReceived >> 32;
+               peerStats->bytesReceived.low
+                               = tp->bytesReceived & MAX_AFS_UINT32;
                 MUTEX_EXIT(&tp->peer_lock);
 
                 MUTEX_ENTER(&rx_peerHashTable_lock);
@@ -7738,15 +7962,16 @@ shutdown_rx(void)
 #endif /* AFS_USE_GETTIMEOFDAY */
 #endif /* AFS_PTHREAD_ENV */
 
-    while (!queue_IsEmpty(&rx_freeCallQueue)) {
-       call = queue_First(&rx_freeCallQueue, rx_call);
-       queue_Remove(call);
+    while (!opr_queue_IsEmpty(&rx_freeCallQueue)) {
+       call = opr_queue_First(&rx_freeCallQueue, struct rx_call, entry);
+       opr_queue_Remove(&call->entry);
        rxi_Free(call, sizeof(struct rx_call));
     }
 
-    while (!queue_IsEmpty(&rx_idleServerQueue)) {
-       sq = queue_First(&rx_idleServerQueue, rx_serverQueueEntry);
-       queue_Remove(sq);
+    while (!opr_queue_IsEmpty(&rx_idleServerQueue)) {
+       sq = opr_queue_First(&rx_idleServerQueue, struct rx_serverQueueEntry,
+                           entry);
+       opr_queue_Remove(&sq->entry);
     }
 #endif /* KERNEL */
 
@@ -7759,19 +7984,20 @@ shutdown_rx(void)
 
             MUTEX_ENTER(&rx_peerHashTable_lock);
             for (peer = *peer_ptr; peer; peer = next) {
-               rx_interface_stat_p rpc_stat, nrpc_stat;
+               struct opr_queue *cursor, *store;
                size_t space;
 
                 MUTEX_ENTER(&rx_rpc_stats);
                 MUTEX_ENTER(&peer->peer_lock);
-               for (queue_Scan
-                    (&peer->rpcStats, rpc_stat, nrpc_stat,
-                     rx_interface_stat)) {
+               for (opr_queue_ScanSafe(&peer->rpcStats, cursor, store)) {
                    unsigned int num_funcs;
+                   struct rx_interface_stat *rpc_stat
+                       = opr_queue_Entry(cursor, struct rx_interface_stat,
+                                        entry);
                    if (!rpc_stat)
                        break;
-                   queue_Remove(&rpc_stat->queue_header);
-                   queue_Remove(&rpc_stat->all_peers);
+                   opr_queue_Remove(&rpc_stat->entry);
+                   opr_queue_Remove(&rpc_stat->entryPeers);
                    num_funcs = rpc_stat->stats[0].func_total;
                    space =
                        sizeof(rx_interface_stat_t) +
@@ -7881,7 +8107,7 @@ rx_SetSpecific(struct rx_connection *conn, int key, void *ptr)
     int i;
     MUTEX_ENTER(&conn->conn_data_lock);
     if (!conn->specific) {
-       conn->specific = (void **)malloc((key + 1) * sizeof(void *));
+       conn->specific = malloc((key + 1) * sizeof(void *));
        for (i = 0; i < key; i++)
            conn->specific[i] = NULL;
        conn->nSpecific = key + 1;
@@ -7907,7 +8133,7 @@ rx_SetServiceSpecific(struct rx_service *svc, int key, void *ptr)
     int i;
     MUTEX_ENTER(&svc->svc_data_lock);
     if (!svc->specific) {
-       svc->specific = (void **)malloc((key + 1) * sizeof(void *));
+       svc->specific = malloc((key + 1) * sizeof(void *));
        for (i = 0; i < key; i++)
            svc->specific[i] = NULL;
        svc->nSpecific = key + 1;
@@ -7965,14 +8191,14 @@ rx_GetServiceSpecific(struct rx_service *svc, int key)
  * which can come and go based upon the peer lifetime.
  */
 
-static struct rx_queue processStats = { &processStats, &processStats };
+static struct opr_queue processStats = { &processStats, &processStats };
 
 /*
  * peerStats is a queue used to store the statistics for all peer structs.
  * Its contents are the union of all the peer rpcStats queues.
  */
 
-static struct rx_queue peerStats = { &peerStats, &peerStats };
+static struct opr_queue peerStats = { &peerStats, &peerStats };
 
 /*
  * rxi_monitor_processStats is used to turn process wide stat collection
@@ -7987,71 +8213,108 @@ static int rxi_monitor_processStats = 0;
 
 static int rxi_monitor_peerStats = 0;
 
-/*
- * rxi_AddRpcStat - given all of the information for a particular rpc
- * call, create (if needed) and update the stat totals for the rpc.
- *
- * PARAMETERS
- *
- * IN stats - the queue of stats that will be updated with the new value
- *
- * IN rxInterface - a unique number that identifies the rpc interface
- *
- * IN currentFunc - the index of the function being invoked
- *
- * IN totalFunc - the total number of functions in this interface
- *
- * IN queueTime - the amount of time this function waited for a thread
+
+void
+rxi_ClearRPCOpStat(rx_function_entry_v1_p rpc_stat)
+{
+    rpc_stat->invocations = 0;
+    rpc_stat->bytes_sent = 0;
+    rpc_stat->bytes_rcvd = 0;
+    rpc_stat->queue_time_sum.sec = 0;
+    rpc_stat->queue_time_sum.usec = 0;
+    rpc_stat->queue_time_sum_sqr.sec = 0;
+    rpc_stat->queue_time_sum_sqr.usec = 0;
+    rpc_stat->queue_time_min.sec = 9999999;
+    rpc_stat->queue_time_min.usec = 9999999;
+    rpc_stat->queue_time_max.sec = 0;
+    rpc_stat->queue_time_max.usec = 0;
+    rpc_stat->execution_time_sum.sec = 0;
+    rpc_stat->execution_time_sum.usec = 0;
+    rpc_stat->execution_time_sum_sqr.sec = 0;
+    rpc_stat->execution_time_sum_sqr.usec = 0;
+    rpc_stat->execution_time_min.sec = 9999999;
+    rpc_stat->execution_time_min.usec = 9999999;
+    rpc_stat->execution_time_max.sec = 0;
+    rpc_stat->execution_time_max.usec = 0;
+}
+
+/*!
+ * Given all of the information for a particular rpc
+ * call, find or create (if requested) the stat structure for the rpc.
  *
- * IN execTime - the amount of time this function invocation took to execute
+ * @param stats
+ *     the queue of stats that will be updated with the new value
  *
- * IN bytesSent - the number bytes sent by this invocation
+ * @param rxInterface
+ *     a unique number that identifies the rpc interface
  *
- * IN bytesRcvd - the number bytes received by this invocation
+ * @param totalFunc
+ *     the total number of functions in this interface. this is only
+ *      required if create is true
  *
- * IN isServer - if true, this invocation was made to a server
+ * @param isServer
+ *     if true, this invocation was made to a server
  *
- * IN remoteHost - the ip address of the remote host
+ * @param remoteHost
+ *     the ip address of the remote host. this is only required if create
+ *      and addToPeerList are true
  *
- * IN remotePort - the port of the remote host
+ * @param remotePort
+ *     the port of the remote host. this is only required if create
+ *      and addToPeerList are true
  *
- * IN addToPeerList - if != 0, add newly created stat to the global peer list
+ * @param addToPeerList
+ *     if != 0, add newly created stat to the global peer list
  *
- * INOUT counter - if a new stats structure is allocated, the counter will
- * be updated with the new number of allocated stat structures
+ * @param counter
+ *     if a new stats structure is allocated, the counter will
+ *     be updated with the new number of allocated stat structures.
+ *      only required if create is true
  *
- * RETURN CODES
+ * @param create
+ *     if no stats structure exists, allocate one
  *
- * Returns void.
  */
 
-static int
-rxi_AddRpcStat(struct rx_queue *stats, afs_uint32 rxInterface,
-              afs_uint32 currentFunc, afs_uint32 totalFunc,
-              struct clock *queueTime, struct clock *execTime,
-              afs_hyper_t * bytesSent, afs_hyper_t * bytesRcvd, int isServer,
-              afs_uint32 remoteHost, afs_uint32 remotePort,
-              int addToPeerList, unsigned int *counter)
+static rx_interface_stat_p
+rxi_FindRpcStat(struct opr_queue *stats, afs_uint32 rxInterface,
+               afs_uint32 totalFunc, int isServer, afs_uint32 remoteHost,
+               afs_uint32 remotePort, int addToPeerList,
+               unsigned int *counter, int create)
 {
-    int rc = 0;
-    rx_interface_stat_p rpc_stat, nrpc_stat;
+    rx_interface_stat_p rpc_stat = NULL;
+    struct opr_queue *cursor;
 
     /*
      * See if there's already a structure for this interface
      */
 
-    for (queue_Scan(stats, rpc_stat, nrpc_stat, rx_interface_stat)) {
+    for (opr_queue_Scan(stats, cursor)) {
+       rpc_stat = opr_queue_Entry(cursor, struct rx_interface_stat, entry);
+
        if ((rpc_stat->stats[0].interfaceId == rxInterface)
            && (rpc_stat->stats[0].remote_is_server == isServer))
            break;
     }
 
+    /* if they didn't ask us to create, we're done */
+    if (!create) {
+        if (opr_queue_IsEnd(stats, cursor))
+            return NULL;
+        else
+            return rpc_stat;
+    }
+
+    /* can't proceed without these */
+    if (!totalFunc || !counter)
+       return NULL;
+
     /*
      * Didn't find a match so allocate a new structure and add it to the
      * queue.
      */
 
-    if (queue_IsEnd(stats, rpc_stat) || (rpc_stat == NULL)
+    if (opr_queue_IsEnd(stats, cursor) || (rpc_stat == NULL)
        || (rpc_stat->stats[0].interfaceId != rxInterface)
        || (rpc_stat->stats[0].remote_is_server != isServer)) {
        int i;
@@ -8062,51 +8325,222 @@ rxi_AddRpcStat(struct rx_queue *stats, afs_uint32 rxInterface,
            totalFunc * sizeof(rx_function_entry_v1_t);
 
        rpc_stat = rxi_Alloc(space);
-       if (rpc_stat == NULL) {
-           rc = 1;
-           goto fail;
-       }
+       if (rpc_stat == NULL)
+           return NULL;
+
        *counter += totalFunc;
        for (i = 0; i < totalFunc; i++) {
+           rxi_ClearRPCOpStat(&(rpc_stat->stats[i]));
            rpc_stat->stats[i].remote_peer = remoteHost;
            rpc_stat->stats[i].remote_port = remotePort;
            rpc_stat->stats[i].remote_is_server = isServer;
            rpc_stat->stats[i].interfaceId = rxInterface;
            rpc_stat->stats[i].func_total = totalFunc;
            rpc_stat->stats[i].func_index = i;
-           hzero(rpc_stat->stats[i].invocations);
-           hzero(rpc_stat->stats[i].bytes_sent);
-           hzero(rpc_stat->stats[i].bytes_rcvd);
-           rpc_stat->stats[i].queue_time_sum.sec = 0;
-           rpc_stat->stats[i].queue_time_sum.usec = 0;
-           rpc_stat->stats[i].queue_time_sum_sqr.sec = 0;
-           rpc_stat->stats[i].queue_time_sum_sqr.usec = 0;
-           rpc_stat->stats[i].queue_time_min.sec = 9999999;
-           rpc_stat->stats[i].queue_time_min.usec = 9999999;
-           rpc_stat->stats[i].queue_time_max.sec = 0;
-           rpc_stat->stats[i].queue_time_max.usec = 0;
-           rpc_stat->stats[i].execution_time_sum.sec = 0;
-           rpc_stat->stats[i].execution_time_sum.usec = 0;
-           rpc_stat->stats[i].execution_time_sum_sqr.sec = 0;
-           rpc_stat->stats[i].execution_time_sum_sqr.usec = 0;
-           rpc_stat->stats[i].execution_time_min.sec = 9999999;
-           rpc_stat->stats[i].execution_time_min.usec = 9999999;
-           rpc_stat->stats[i].execution_time_max.sec = 0;
-           rpc_stat->stats[i].execution_time_max.usec = 0;
-       }
-       queue_Prepend(stats, rpc_stat);
+       }
+       opr_queue_Prepend(stats, &rpc_stat->entry);
        if (addToPeerList) {
-           queue_Prepend(&peerStats, &rpc_stat->all_peers);
+           opr_queue_Prepend(&peerStats, &rpc_stat->entryPeers);
        }
     }
+    return rpc_stat;
+}
+
+void
+rx_ClearProcessRPCStats(afs_int32 rxInterface)
+{
+    rx_interface_stat_p rpc_stat;
+    int totalFunc, i;
+
+    if (rxInterface == -1)
+        return;
+
+    MUTEX_ENTER(&rx_rpc_stats);
+    rpc_stat = rxi_FindRpcStat(&processStats, rxInterface, 0, 0,
+                              0, 0, 0, 0, 0);
+    if (rpc_stat) {
+       totalFunc = rpc_stat->stats[0].func_total;
+       for (i = 0; i < totalFunc; i++)
+           rxi_ClearRPCOpStat(&(rpc_stat->stats[i]));
+    }
+    MUTEX_EXIT(&rx_rpc_stats);
+    return;
+}
+
+void
+rx_ClearPeerRPCStats(afs_int32 rxInterface, afs_uint32 peerHost, afs_uint16 peerPort)
+{
+    rx_interface_stat_p rpc_stat;
+    int totalFunc, i;
+    struct rx_peer * peer;
+
+    if (rxInterface == -1)
+        return;
+
+    peer = rxi_FindPeer(peerHost, peerPort, 0, 0);
+    if (!peer)
+        return;
+
+    MUTEX_ENTER(&rx_rpc_stats);
+    rpc_stat = rxi_FindRpcStat(&peer->rpcStats, rxInterface, 0, 1,
+                              0, 0, 0, 0, 0);
+    if (rpc_stat) {
+       totalFunc = rpc_stat->stats[0].func_total;
+       for (i = 0; i < totalFunc; i++)
+           rxi_ClearRPCOpStat(&(rpc_stat->stats[i]));
+    }
+    MUTEX_EXIT(&rx_rpc_stats);
+    return;
+}
+
+void *
+rx_CopyProcessRPCStats(afs_uint64 op)
+{
+    rx_interface_stat_p rpc_stat;
+    rx_function_entry_v1_p rpcop_stat =
+       rxi_Alloc(sizeof(rx_function_entry_v1_t));
+    int currentFunc = (op & MAX_AFS_UINT32);
+    afs_int32 rxInterface = (op >> 32);
+
+    if (!rxi_monitor_processStats)
+        return NULL;
+
+    if (rxInterface == -1)
+        return NULL;
+
+    if (rpcop_stat == NULL)
+        return NULL;
+
+    MUTEX_ENTER(&rx_rpc_stats);
+    rpc_stat = rxi_FindRpcStat(&processStats, rxInterface, 0, 0,
+                              0, 0, 0, 0, 0);
+    if (rpc_stat)
+       memcpy(rpcop_stat, &(rpc_stat->stats[currentFunc]),
+              sizeof(rx_function_entry_v1_t));
+    MUTEX_EXIT(&rx_rpc_stats);
+    if (!rpc_stat) {
+       rxi_Free(rpcop_stat, sizeof(rx_function_entry_v1_t));
+       return NULL;
+    }
+    return rpcop_stat;
+}
+
+void *
+rx_CopyPeerRPCStats(afs_uint64 op, afs_uint32 peerHost, afs_uint16 peerPort)
+{
+    rx_interface_stat_p rpc_stat;
+    rx_function_entry_v1_p rpcop_stat =
+       rxi_Alloc(sizeof(rx_function_entry_v1_t));
+    int currentFunc = (op & MAX_AFS_UINT32);
+    afs_int32 rxInterface = (op >> 32);
+    struct rx_peer *peer;
+
+    if (!rxi_monitor_peerStats)
+        return NULL;
+
+    if (rxInterface == -1)
+        return NULL;
+
+    if (rpcop_stat == NULL)
+        return NULL;
+
+    peer = rxi_FindPeer(peerHost, peerPort, 0, 0);
+    if (!peer)
+        return NULL;
+
+    MUTEX_ENTER(&rx_rpc_stats);
+    rpc_stat = rxi_FindRpcStat(&peer->rpcStats, rxInterface, 0, 1,
+                              0, 0, 0, 0, 0);
+    if (rpc_stat)
+       memcpy(rpcop_stat, &(rpc_stat->stats[currentFunc]),
+              sizeof(rx_function_entry_v1_t));
+    MUTEX_EXIT(&rx_rpc_stats);
+    if (!rpc_stat) {
+       rxi_Free(rpcop_stat, sizeof(rx_function_entry_v1_t));
+       return NULL;
+    }
+    return rpcop_stat;
+}
+
+void
+rx_ReleaseRPCStats(void *stats)
+{
+    if (stats)
+       rxi_Free(stats, sizeof(rx_function_entry_v1_t));
+}
+
+/*!
+ * Given all of the information for a particular rpc
+ * call, create (if needed) and update the stat totals for the rpc.
+ *
+ * @param stats
+ *     the queue of stats that will be updated with the new value
+ *
+ * @param rxInterface
+ *     a unique number that identifies the rpc interface
+ *
+ * @param currentFunc
+ *     the index of the function being invoked
+ *
+ * @param totalFunc
+ *     the total number of functions in this interface
+ *
+ * @param queueTime
+ *     the amount of time this function waited for a thread
+ *
+ * @param execTime
+ *     the amount of time this function invocation took to execute
+ *
+ * @param bytesSent
+ *     the number bytes sent by this invocation
+ *
+ * @param bytesRcvd
+ *     the number bytes received by this invocation
+ *
+ * @param isServer
+ *     if true, this invocation was made to a server
+ *
+ * @param remoteHost
+ *     the ip address of the remote host
+ *
+ * @param remotePort
+ *     the port of the remote host
+ *
+ * @param addToPeerList
+ *     if != 0, add newly created stat to the global peer list
+ *
+ * @param counter
+ *     if a new stats structure is allocated, the counter will
+ *     be updated with the new number of allocated stat structures
+ *
+ */
+
+static int
+rxi_AddRpcStat(struct opr_queue *stats, afs_uint32 rxInterface,
+              afs_uint32 currentFunc, afs_uint32 totalFunc,
+              struct clock *queueTime, struct clock *execTime,
+              afs_uint64 bytesSent, afs_uint64 bytesRcvd, int isServer,
+              afs_uint32 remoteHost, afs_uint32 remotePort,
+              int addToPeerList, unsigned int *counter)
+{
+    int rc = 0;
+    rx_interface_stat_p rpc_stat;
+
+    rpc_stat = rxi_FindRpcStat(stats, rxInterface, totalFunc, isServer,
+                              remoteHost, remotePort, addToPeerList, counter,
+                              1);
+    if (!rpc_stat) {
+       rc = -1;
+       goto fail;
+    }
 
     /*
      * Increment the stats for this function
      */
 
-    hadd32(rpc_stat->stats[currentFunc].invocations, 1);
-    hadd(rpc_stat->stats[currentFunc].bytes_sent, *bytesSent);
-    hadd(rpc_stat->stats[currentFunc].bytes_rcvd, *bytesRcvd);
+    rpc_stat->stats[currentFunc].invocations++;
+    rpc_stat->stats[currentFunc].bytes_sent += bytesSent;
+    rpc_stat->stats[currentFunc].bytes_rcvd += bytesRcvd;
     clock_Add(&rpc_stat->stats[currentFunc].queue_time_sum, queueTime);
     clock_AddSq(&rpc_stat->stats[currentFunc].queue_time_sum_sqr, queueTime);
     if (clock_Lt(queueTime, &rpc_stat->stats[currentFunc].queue_time_min)) {
@@ -8129,41 +8563,12 @@ rxi_AddRpcStat(struct rx_queue *stats, afs_uint32 rxInterface,
     return rc;
 }
 
-/*
- * rx_IncrementTimeAndCount - increment the times and count for a particular
- * rpc function.
- *
- * PARAMETERS
- *
- * IN peer - the peer who invoked the rpc
- *
- * IN rxInterface - a unique number that identifies the rpc interface
- *
- * IN currentFunc - the index of the function being invoked
- *
- * IN totalFunc - the total number of functions in this interface
- *
- * IN queueTime - the amount of time this function waited for a thread
- *
- * IN execTime - the amount of time this function invocation took to execute
- *
- * IN bytesSent - the number bytes sent by this invocation
- *
- * IN bytesRcvd - the number bytes received by this invocation
- *
- * IN isServer - if true, this invocation was made to a server
- *
- * RETURN CODES
- *
- * Returns void.
- */
-
 void
-rx_IncrementTimeAndCount(struct rx_peer *peer, afs_uint32 rxInterface,
-                        afs_uint32 currentFunc, afs_uint32 totalFunc,
-                        struct clock *queueTime, struct clock *execTime,
-                        afs_hyper_t * bytesSent, afs_hyper_t * bytesRcvd,
-                        int isServer)
+rxi_IncrementTimeAndCount(struct rx_peer *peer, afs_uint32 rxInterface,
+                         afs_uint32 currentFunc, afs_uint32 totalFunc,
+                         struct clock *queueTime, struct clock *execTime,
+                         afs_uint64 bytesSent, afs_uint64 bytesRcvd,
+                         int isServer)
 {
 
     if (!(rxi_monitor_peerStats || rxi_monitor_processStats))
@@ -8186,9 +8591,63 @@ rx_IncrementTimeAndCount(struct rx_peer *peer, afs_uint32 rxInterface,
     }
 
     MUTEX_EXIT(&rx_rpc_stats);
+}
 
+/*!
+ * Increment the times and count for a particular rpc function.
+ *
+ * Traditionally this call was invoked from rxgen stubs. Modern stubs
+ * call rx_RecordCallStatistics instead, so the public version of this
+ * function is left purely for legacy callers.
+ *
+ * @param peer
+ *     The peer who invoked the rpc
+ *
+ * @param rxInterface
+ *     A unique number that identifies the rpc interface
+ *
+ * @param currentFunc
+ *     The index of the function being invoked
+ *
+ * @param totalFunc
+ *     The total number of functions in this interface
+ *
+ * @param queueTime
+ *     The amount of time this function waited for a thread
+ *
+ * @param execTime
+ *     The amount of time this function invocation took to execute
+ *
+ * @param bytesSent
+ *     The number bytes sent by this invocation
+ *
+ * @param bytesRcvd
+ *     The number bytes received by this invocation
+ *
+ * @param isServer
+ *     If true, this invocation was made to a server
+ *
+ */
+void
+rx_IncrementTimeAndCount(struct rx_peer *peer, afs_uint32 rxInterface,
+                        afs_uint32 currentFunc, afs_uint32 totalFunc,
+                        struct clock *queueTime, struct clock *execTime,
+                        afs_hyper_t * bytesSent, afs_hyper_t * bytesRcvd,
+                        int isServer)
+{
+    afs_uint64 sent64;
+    afs_uint64 rcvd64;
+
+    sent64 = ((afs_uint64)bytesSent->high << 32) + bytesSent->low;
+    rcvd64 = ((afs_uint64)bytesRcvd->high << 32) + bytesRcvd->low;
+
+    rxi_IncrementTimeAndCount(peer, rxInterface, currentFunc, totalFunc,
+                             queueTime, execTime, sent64, rcvd64,
+                             isServer);
 }
 
+
+
 /*
  * rx_MarshallProcessRPCStats - marshall an array of rpc statistics
  *
@@ -8223,12 +8682,12 @@ rx_MarshallProcessRPCStats(afs_uint32 callerVersion, int count,
        *(ptr++) = stats->interfaceId;
        *(ptr++) = stats->func_total;
        *(ptr++) = stats->func_index;
-       *(ptr++) = hgethi(stats->invocations);
-       *(ptr++) = hgetlo(stats->invocations);
-       *(ptr++) = hgethi(stats->bytes_sent);
-       *(ptr++) = hgetlo(stats->bytes_sent);
-       *(ptr++) = hgethi(stats->bytes_rcvd);
-       *(ptr++) = hgetlo(stats->bytes_rcvd);
+       *(ptr++) = stats->invocations >> 32;
+       *(ptr++) = stats->invocations & MAX_AFS_UINT32;
+       *(ptr++) = stats->bytes_sent >> 32;
+       *(ptr++) = stats->bytes_sent & MAX_AFS_UINT32;
+       *(ptr++) = stats->bytes_rcvd >> 32;
+       *(ptr++) = stats->bytes_rcvd & MAX_AFS_UINT32;
        *(ptr++) = stats->queue_time_sum.sec;
        *(ptr++) = stats->queue_time_sum.usec;
        *(ptr++) = stats->queue_time_sum_sqr.sec;
@@ -8328,11 +8787,11 @@ rx_RetrieveProcessRPCStats(afs_uint32 callerVersion, afs_uint32 * myVersion,
        ptr = *stats = rxi_Alloc(space);
 
        if (ptr != NULL) {
-           rx_interface_stat_p rpc_stat, nrpc_stat;
-
+           struct opr_queue *cursor;
 
-           for (queue_Scan
-                (&processStats, rpc_stat, nrpc_stat, rx_interface_stat)) {
+           for (opr_queue_Scan(&processStats, cursor)) {
+               struct rx_interface_stat *rpc_stat = 
+                   opr_queue_Entry(cursor, struct rx_interface_stat, entry);
                /*
                 * Copy the data based upon the caller version
                 */
@@ -8427,24 +8886,12 @@ rx_RetrievePeerRPCStats(afs_uint32 callerVersion, afs_uint32 * myVersion,
        ptr = *stats = rxi_Alloc(space);
 
        if (ptr != NULL) {
-           rx_interface_stat_p rpc_stat, nrpc_stat;
-           char *fix_offset;
+           struct opr_queue *cursor;
 
-           for (queue_Scan
-                (&peerStats, rpc_stat, nrpc_stat, rx_interface_stat)) {
-               /*
-                * We have to fix the offset of rpc_stat since we are
-                * keeping this structure on two rx_queues.  The rx_queue
-                * package assumes that the rx_queue member is the first
-                * member of the structure.  That is, rx_queue assumes that
-                * any one item is only on one queue at a time.  We are
-                * breaking that assumption and so we have to do a little
-                * math to fix our pointers.
-                */
-
-               fix_offset = (char *)rpc_stat;
-               fix_offset -= offsetof(rx_interface_stat_t, all_peers);
-               rpc_stat = (rx_interface_stat_p) fix_offset;
+           for (opr_queue_Scan(&peerStats, cursor)) {
+               struct rx_interface_stat *rpc_stat
+                   = opr_queue_Entry(cursor, struct rx_interface_stat,
+                                    entryPeers);
 
                /*
                 * Copy the data based upon the caller version
@@ -8575,7 +9022,7 @@ rx_enablePeerRPCStats(void)
 void
 rx_disableProcessRPCStats(void)
 {
-    rx_interface_stat_p rpc_stat, nrpc_stat;
+    struct opr_queue *cursor, *store;
     size_t space;
 
     MUTEX_ENTER(&rx_rpc_stats);
@@ -8590,11 +9037,13 @@ rx_disableProcessRPCStats(void)
        rx_enable_stats = 0;
     }
 
-    for (queue_Scan(&processStats, rpc_stat, nrpc_stat, rx_interface_stat)) {
-       unsigned int num_funcs = 0;
-       if (!rpc_stat)
-           break;
-       queue_Remove(rpc_stat);
+    for (opr_queue_ScanSafe(&processStats, cursor, store)) {
+       unsigned int num_funcs = 0;
+       struct rx_interface_stat *rpc_stat
+           = opr_queue_Entry(cursor, struct rx_interface_stat, entry);
+
+       opr_queue_Remove(&rpc_stat->entry);
+
        num_funcs = rpc_stat->stats[0].func_total;
        space =
            sizeof(rx_interface_stat_t) +
@@ -8643,8 +9092,8 @@ rx_disablePeerRPCStats(void)
            next = peer->next;
            code = MUTEX_TRYENTER(&peer->peer_lock);
            if (code) {
-               rx_interface_stat_p rpc_stat, nrpc_stat;
                size_t space;
+               struct opr_queue *cursor, *store;
 
                if (prev == *peer_ptr) {
                    *peer_ptr = next;
@@ -8659,14 +9108,14 @@ rx_disablePeerRPCStats(void)
                 peer->refCount++;
                 MUTEX_EXIT(&rx_peerHashTable_lock);
 
-                for (queue_Scan
-                    (&peer->rpcStats, rpc_stat, nrpc_stat,
-                     rx_interface_stat)) {
+                for (opr_queue_ScanSafe(&peer->rpcStats, cursor, store)) {
                    unsigned int num_funcs = 0;
-                   if (!rpc_stat)
-                       break;
-                   queue_Remove(&rpc_stat->queue_header);
-                   queue_Remove(&rpc_stat->all_peers);
+                   struct rx_interface_stat *rpc_stat
+                       = opr_queue_Entry(cursor, struct rx_interface_stat,
+                                        entry);
+
+                   opr_queue_Remove(&rpc_stat->entry);
+                   opr_queue_Remove(&rpc_stat->entryPeers);
                    num_funcs = rpc_stat->stats[0].func_total;
                    space =
                        sizeof(rx_interface_stat_t) +
@@ -8709,22 +9158,25 @@ rx_disablePeerRPCStats(void)
 void
 rx_clearProcessRPCStats(afs_uint32 clearFlag)
 {
-    rx_interface_stat_p rpc_stat, nrpc_stat;
+    struct opr_queue *cursor;
 
     MUTEX_ENTER(&rx_rpc_stats);
 
-    for (queue_Scan(&processStats, rpc_stat, nrpc_stat, rx_interface_stat)) {
+    for (opr_queue_Scan(&processStats, cursor)) {
        unsigned int num_funcs = 0, i;
+       struct rx_interface_stat *rpc_stat
+            = opr_queue_Entry(rpc_stat, struct rx_interface_stat, entry);
+
        num_funcs = rpc_stat->stats[0].func_total;
        for (i = 0; i < num_funcs; i++) {
            if (clearFlag & AFS_RX_STATS_CLEAR_INVOCATIONS) {
-               hzero(rpc_stat->stats[i].invocations);
+               rpc_stat->stats[i].invocations = 0;
            }
            if (clearFlag & AFS_RX_STATS_CLEAR_BYTES_SENT) {
-               hzero(rpc_stat->stats[i].bytes_sent);
+               rpc_stat->stats[i].bytes_sent = 0;
            }
            if (clearFlag & AFS_RX_STATS_CLEAR_BYTES_RCVD) {
-               hzero(rpc_stat->stats[i].bytes_rcvd);
+               rpc_stat->stats[i].bytes_rcvd = 0;
            }
            if (clearFlag & AFS_RX_STATS_CLEAR_QUEUE_TIME_SUM) {
                rpc_stat->stats[i].queue_time_sum.sec = 0;
@@ -8780,37 +9232,25 @@ rx_clearProcessRPCStats(afs_uint32 clearFlag)
 void
 rx_clearPeerRPCStats(afs_uint32 clearFlag)
 {
-    rx_interface_stat_p rpc_stat, nrpc_stat;
+    struct opr_queue *cursor;
 
     MUTEX_ENTER(&rx_rpc_stats);
 
-    for (queue_Scan(&peerStats, rpc_stat, nrpc_stat, rx_interface_stat)) {
-       unsigned int num_funcs = 0, i;
-       char *fix_offset;
-       /*
-        * We have to fix the offset of rpc_stat since we are
-        * keeping this structure on two rx_queues.  The rx_queue
-        * package assumes that the rx_queue member is the first
-        * member of the structure.  That is, rx_queue assumes that
-        * any one item is only on one queue at a time.  We are
-        * breaking that assumption and so we have to do a little
-        * math to fix our pointers.
-        */
-
-       fix_offset = (char *)rpc_stat;
-       fix_offset -= offsetof(rx_interface_stat_t, all_peers);
-       rpc_stat = (rx_interface_stat_p) fix_offset;
+    for (opr_queue_Scan(&peerStats, cursor)) {
+       unsigned int num_funcs, i;
+       struct rx_interface_stat *rpc_stat
+           = opr_queue_Entry(cursor, struct rx_interface_stat, entryPeers);
 
        num_funcs = rpc_stat->stats[0].func_total;
        for (i = 0; i < num_funcs; i++) {
            if (clearFlag & AFS_RX_STATS_CLEAR_INVOCATIONS) {
-               hzero(rpc_stat->stats[i].invocations);
+               rpc_stat->stats[i].invocations = 0;
            }
            if (clearFlag & AFS_RX_STATS_CLEAR_BYTES_SENT) {
-               hzero(rpc_stat->stats[i].bytes_sent);
+               rpc_stat->stats[i].bytes_sent = 0;
            }
            if (clearFlag & AFS_RX_STATS_CLEAR_BYTES_RCVD) {
-               hzero(rpc_stat->stats[i].bytes_rcvd);
+               rpc_stat->stats[i].bytes_rcvd = 0;
            }
            if (clearFlag & AFS_RX_STATS_CLEAR_QUEUE_TIME_SUM) {
                rpc_stat->stats[i].queue_time_sum.sec = 0;
@@ -8923,12 +9363,11 @@ int rx_DumpCalls(FILE *outputFile, char *cookie)
 
     for (c = rx_allCallsp; c; c = c->allNextp) {
         u_short rqc, tqc, iovqc;
-        struct rx_packet *p, *np;
 
         MUTEX_ENTER(&c->lock);
-        queue_Count(&c->rq, p, np, rx_packet, rqc);
-        queue_Count(&c->tq, p, np, rx_packet, tqc);
-        queue_Count(&c->iovq, p, np, rx_packet, iovqc);
+        rqc = opr_queue_Count(&c->rq);
+        tqc = opr_queue_Count(&c->tq);
+        iovqc = opr_queue_Count(&c->app.iovq);
 
        RXDPRINTF(RXDPRINTOUT, "%s - call=0x%p, id=%u, state=%u, mode=%u, conn=%p, epoch=%u, cid=%u, callNum=%u, connFlags=0x%x, flags=0x%x, "
                 "rqc=%u,%u, tqc=%u,%u, iovqc=%u,%u, "
@@ -8943,7 +9382,7 @@ int rx_DumpCalls(FILE *outputFile, char *cookie)
                 "refCountAlive=%u, refCountPacket=%u, refCountSend=%u, refCountAckAll=%u, refCountAbort=%u"
 #endif
                 "\r\n",
-                cookie, c, c->call_id, (afs_uint32)c->state, (afs_uint32)c->mode, c->conn, c->conn?c->conn->epoch:0, c->conn?c->conn->cid:0,
+                cookie, c, c->call_id, (afs_uint32)c->state, (afs_uint32)c->app.mode, c->conn, c->conn?c->conn->epoch:0, c->conn?c->conn->cid:0,
                 c->callNumber?*c->callNumber:0, c->conn?c->conn->flags:0, c->flags,
                 (afs_uint32)c->rqc, (afs_uint32)rqc, (afs_uint32)c->tqc, (afs_uint32)tqc, (afs_uint32)c->iovqc, (afs_uint32)iovqc,
                 (afs_uint32)c->localStatus, (afs_uint32)c->remoteStatus, c->error, c->timeout,