rx: Avoid osi_NetSend during rx shutdown
[openafs.git] / src / rx / rx.c
index 12ad917..78faf26 100644 (file)
@@ -73,6 +73,7 @@ extern afs_int32 afs_termState;
 #endif /* KERNEL */
 
 #include <opr/queue.h>
+#include <hcrypto/rand.h>
 
 #include "rx.h"
 #include "rx_clock.h"
@@ -125,7 +126,8 @@ 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);
+                           afs_uint32 epoch, int type, u_int securityIndex,
+                            int *unknownService);
 static struct rx_packet
        *rxi_ReceiveDataPacket(struct rx_call *call, struct rx_packet *np,
                               int istack, osi_socket socket,
@@ -150,21 +152,26 @@ 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_ChallengeOn(struct rx_connection *conn);
 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);
+static void rxi_AckAllInTransmitQueue(struct rx_call *call);
+static void rxi_CancelKeepAliveEvent(struct rx_call *call);
+static void rxi_CancelDelayedAbortEvent(struct rx_call *call);
+static void rxi_CancelGrowMTUEvent(struct rx_call *call);
+static void update_nextCid(void);
+
+#ifndef KERNEL
+static void rxi_Finalize_locked(void);
+#elif defined(UKERNEL)
+# define rxi_Finalize_locked() do { } while (0)
 #endif
 
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
 struct rx_tq_debug {
     rx_atomic_t rxi_start_aborted; /* rxi_start awoke after rxi_Send in error.*/
     rx_atomic_t rxi_start_in_error;
 } rx_tq_debug;
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
 
 /* Constant delay time before sending an acknowledge of the last packet
  * received.  This is to avoid sending an extra acknowledge when the
@@ -203,16 +210,6 @@ static unsigned int rxi_rpc_peer_stat_cnt;
 
 static unsigned int rxi_rpc_process_stat_cnt;
 
-/*
- * 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);
 
@@ -224,12 +221,16 @@ struct opr_queue rx_incomingCallQueue;
  * calls to process */
 struct opr_queue rx_idleServerQueue;
 
+/* List of free rx_serverQueueEntry structs */
+struct opr_queue rx_freeServerQueue;
+
 #if !defined(offsetof)
 #include <stddef.h>            /* for definition of offsetof() */
 #endif
 
 #ifdef RX_ENABLE_LOCKS
 afs_kmutex_t rx_atomic_mutex;
+static afs_kmutex_t freeSQEList_lock;
 #endif
 
 /* Forward prototypes */
@@ -255,6 +256,7 @@ extern afs_kmutex_t rx_packets_mutex;
 extern afs_kmutex_t rx_refcnt_mutex;
 extern afs_kmutex_t des_init_mutex;
 extern afs_kmutex_t des_random_mutex;
+#ifndef KERNEL
 extern afs_kmutex_t rx_clock_mutex;
 extern afs_kmutex_t rxi_connCacheMutex;
 extern afs_kmutex_t event_handler_mutex;
@@ -264,6 +266,7 @@ extern afs_kmutex_t rx_if_mutex;
 
 extern afs_kcondvar_t rx_event_handler_cond;
 extern afs_kcondvar_t rx_listener_cond;
+#endif /* !KERNEL */
 
 static afs_kmutex_t epoch_mutex;
 static afs_kmutex_t rx_init_mutex;
@@ -273,30 +276,37 @@ static afs_kmutex_t rx_rpc_stats;
 static void
 rxi_InitPthread(void)
 {
-    MUTEX_INIT(&rx_clock_mutex, "clock", MUTEX_DEFAULT, 0);
-    MUTEX_INIT(&rx_stats_mutex, "stats", MUTEX_DEFAULT, 0);
-    MUTEX_INIT(&rx_atomic_mutex, "atomic", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_quota_mutex, "quota", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_pthread_mutex, "pthread", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_packets_mutex, "packets", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_refcnt_mutex, "refcnts", MUTEX_DEFAULT, 0);
-    MUTEX_INIT(&epoch_mutex, "epoch", MUTEX_DEFAULT, 0);
-    MUTEX_INIT(&rx_init_mutex, "init", MUTEX_DEFAULT, 0);
-    MUTEX_INIT(&event_handler_mutex, "event handler", MUTEX_DEFAULT, 0);
+#ifndef KERNEL
+    MUTEX_INIT(&rx_clock_mutex, "clock", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rxi_connCacheMutex, "conn cache", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&event_handler_mutex, "event handler", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&listener_mutex, "listener", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_if_init_mutex, "if init", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_if_mutex, "if", MUTEX_DEFAULT, 0);
+#endif
+    MUTEX_INIT(&rx_stats_mutex, "stats", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&rx_atomic_mutex, "atomic", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&epoch_mutex, "epoch", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&rx_init_mutex, "init", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_debug_mutex, "debug", MUTEX_DEFAULT, 0);
 
+#ifndef KERNEL
     CV_INIT(&rx_event_handler_cond, "evhand", CV_DEFAULT, 0);
     CV_INIT(&rx_listener_cond, "rxlisten", CV_DEFAULT, 0);
+#endif
 
     osi_Assert(pthread_key_create(&rx_thread_id_key, NULL) == 0);
     osi_Assert(pthread_key_create(&rx_ts_info_key, NULL) == 0);
 
     MUTEX_INIT(&rx_rpc_stats, "rx_rpc_stats", MUTEX_DEFAULT, 0);
     MUTEX_INIT(&rx_freePktQ_lock, "rx_freePktQ_lock", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&rx_mallocedPktQ_lock, "rx_mallocedPktQ_lock", MUTEX_DEFAULT,
+              0);
+
 #ifdef RX_ENABLE_LOCKS
 #ifdef RX_LOCKS_DB
     rxdb_init();
@@ -311,7 +321,9 @@ rxi_InitPthread(void)
     MUTEX_INIT(&rx_connHashTable_lock, "rx_connHashTable_lock", MUTEX_DEFAULT,
               0);
     MUTEX_INIT(&rx_serverPool_lock, "rx_serverPool_lock", MUTEX_DEFAULT, 0);
+#ifndef KERNEL
     MUTEX_INIT(&rxi_keyCreate_lock, "rxi_keyCreate_lock", MUTEX_DEFAULT, 0);
+#endif
 #endif /* RX_ENABLE_LOCKS */
 }
 
@@ -395,6 +407,7 @@ struct rx_connection *rxLastConn = 0;
  * tiers:
  *
  * rx_connHashTable_lock - synchronizes conn creation, rx_connHashTable access
+ *                         also protects updates to rx_nextCid
  * conn_call_lock - used to synchonize rx_EndCall and rx_NewCall
  * call->lock - locks call data fields.
  * These are independent of each other:
@@ -438,36 +451,28 @@ static int rxdb_fileID = RXDB_FILE_RX;
 #define CLEAR_CALL_QUEUE_LOCK(C)
 #endif /* RX_ENABLE_LOCKS */
 struct rx_serverQueueEntry *rx_waitForPacket = 0;
-struct rx_serverQueueEntry *rx_waitingForPacket = 0;
-
-/* ------------Exported Interfaces------------- */
-
-/* This function allows rxkad to set the epoch to a suitably random number
- * which rx_NewConnection will use in the future.  The principle purpose is to
- * get rxnull connections to use the same epoch as the rxkad connections do, at
- * least once the first rxkad connection is established.  This is important now
- * that the host/port addresses aren't used in FindConnection: the uniqueness
- * of epoch/cid matters and the start time won't do. */
 
-#ifdef AFS_PTHREAD_ENV
 /*
- * This mutex protects the following global variables:
- * rx_epoch
+ * This mutex serializes calls to our initialization and shutdown routines
+ * (rx_InitHost, rx_Finalize and shutdown_rx). Only one thread can be running
+ * these at any time; all other threads must wait for it to finish running, and
+ * then examine the value of rxi_running afterwards.
  */
-
-#define LOCK_EPOCH MUTEX_ENTER(&epoch_mutex)
-#define UNLOCK_EPOCH MUTEX_EXIT(&epoch_mutex)
+#ifdef AFS_PTHREAD_ENV
+# define LOCK_RX_INIT MUTEX_ENTER(&rx_init_mutex)
+# define UNLOCK_RX_INIT MUTEX_EXIT(&rx_init_mutex)
 #else
-#define LOCK_EPOCH
-#define UNLOCK_EPOCH
-#endif /* AFS_PTHREAD_ENV */
+# define LOCK_RX_INIT
+# define UNLOCK_RX_INIT
+#endif
 
-void
-rx_SetEpoch(afs_uint32 epoch)
+/* ------------Exported Interfaces------------- */
+
+static rx_atomic_t rxi_running = RX_ATOMIC_INIT(0);
+int
+rxi_IsRunning(void)
 {
-    LOCK_EPOCH;
-    rx_epoch = epoch;
-    UNLOCK_EPOCH;
+    return rx_atomic_read(&rxi_running);
 }
 
 /* Initialize rx.  A port number may be mentioned, in which case this
@@ -476,23 +481,6 @@ rx_SetEpoch(afs_uint32 epoch)
  * by the kernel.  Whether this will ever overlap anything in
  * /etc/services is anybody's guess...  Returns 0 on success, -1 on
  * error. */
-#ifndef AFS_NT40_ENV
-static
-#endif
-int rxinit_status = 1;
-#ifdef AFS_PTHREAD_ENV
-/*
- * This mutex protects the following global variables:
- * rxinit_status
- */
-
-#define LOCK_RX_INIT MUTEX_ENTER(&rx_init_mutex)
-#define UNLOCK_RX_INIT MUTEX_EXIT(&rx_init_mutex)
-#else
-#define LOCK_RX_INIT
-#define UNLOCK_RX_INIT
-#endif
-
 int
 rx_InitHost(u_int host, u_int port)
 {
@@ -502,23 +490,21 @@ rx_InitHost(u_int host, u_int port)
     struct timeval tv;
 #endif /* KERNEL */
     char *htable, *ptable;
-    int tmp_status;
 
     SPLVAR;
 
     INIT_PTHREAD_LOCKS;
     LOCK_RX_INIT;
-    if (rxinit_status == 0) {
-       tmp_status = rxinit_status;
+    if (rxi_IsRunning()) {
        UNLOCK_RX_INIT;
-       return tmp_status;      /* Already started; return previous error code. */
+       return 0; /* already started */
     }
 #ifdef RXDEBUG
     rxi_DebugInit();
 #endif
 #ifdef AFS_NT40_ENV
     if (afs_winsockInit() < 0)
-       return -1;
+       goto error;
 #endif
 
 #ifndef KERNEL
@@ -534,8 +520,7 @@ rx_InitHost(u_int host, u_int port)
 
     rx_socket = rxi_GetHostUDPSocket(host, (u_short) port);
     if (rx_socket == OSI_NULLSOCKET) {
-       UNLOCK_RX_INIT;
-       return RX_ADDRINUSE;
+        goto addrinuse;
     }
 #if defined(RX_ENABLE_LOCKS) && defined(KERNEL)
 #ifdef RX_LOCKS_DB
@@ -559,6 +544,9 @@ rx_InitHost(u_int host, u_int port)
     MUTEX_INIT(&rx_connHashTable_lock, "rx_connHashTable_lock", MUTEX_DEFAULT,
               0);
     MUTEX_INIT(&rx_serverPool_lock, "rx_serverPool_lock", MUTEX_DEFAULT, 0);
+    MUTEX_INIT(&rx_mallocedPktQ_lock, "rx_mallocedPktQ_lock", MUTEX_DEFAULT,
+              0);
+
 #if defined(AFS_HPUX110_ENV)
     if (!uniprocessor)
        rx_sleepLock = alloc_spinlock(LAST_HELD_ORDER - 10, "rx_sleepLock");
@@ -581,6 +569,7 @@ rx_InitHost(u_int host, u_int port)
     opr_queue_Init(&rx_freePacketQueue);
     rxi_NeedMorePackets = FALSE;
     rx_nPackets = 0;   /* rx_nPackets is managed by rxi_MorePackets* */
+    opr_queue_Init(&rx_mallocedPacketQueue);
 
     /* enforce a minimum number of allocated packets */
     if (rx_extraPackets < rxi_nSendFrags * rx_maxSendWindow)
@@ -619,20 +608,20 @@ rx_InitHost(u_int host, u_int port)
        socklen_t addrlen = sizeof(addr);
 #endif
        if (getsockname((intptr_t)rx_socket, (struct sockaddr *)&addr, &addrlen)) {
-           rx_Finalize();
+           rxi_Finalize_locked();
            osi_Free(htable, rx_hashTableSize * sizeof(struct rx_connection *));
-           return -1;
+           goto error;
        }
        rx_port = addr.sin_port;
 #endif
     }
     rx_stats.minRtt.sec = 9999999;
-#ifdef KERNEL
-    rx_SetEpoch(tv.tv_sec | 0x80000000);
-#else
-    rx_SetEpoch(tv.tv_sec);    /* Start time of this package, rxkad
-                                * will provide a randomer value. */
-#endif
+    if (RAND_bytes(&rx_epoch, sizeof(rx_epoch)) != 1)
+       goto error;
+    rx_epoch  = (rx_epoch & ~0x40000000) | 0x80000000;
+    if (RAND_bytes(&rx_nextCid, sizeof(rx_nextCid)) != 1)
+       goto error;
+    rx_nextCid &= RX_CIDMASK;
     MUTEX_ENTER(&rx_quota_mutex);
     rxi_dataQuota += rx_extraQuota; /* + extra pkts caller asked to rsrv */
     MUTEX_EXIT(&rx_quota_mutex);
@@ -649,6 +638,7 @@ rx_InitHost(u_int host, u_int port)
 
     /* Initialize various global queues */
     opr_queue_Init(&rx_idleServerQueue);
+    opr_queue_Init(&rx_freeServerQueue);
     opr_queue_Init(&rx_incomingCallQueue);
     opr_queue_Init(&rx_freeCallQueue);
 
@@ -657,16 +647,24 @@ rx_InitHost(u_int host, u_int port)
     rx_GetIFInfo();
 #endif
 
-#if defined(RXK_LISTENER_ENV) || !defined(KERNEL)
     /* Start listener process (exact function is dependent on the
      * implementation environment--kernel or user space) */
     rxi_StartListener();
-#endif
 
     USERPRI;
-    tmp_status = rxinit_status = 0;
+
+    rx_atomic_set(&rxi_running, 1);
+    UNLOCK_RX_INIT;
+
+    return 0;
+
+ addrinuse:
     UNLOCK_RX_INIT;
-    return tmp_status;
+    return RX_ADDRINUSE;
+
+ error:
+    UNLOCK_RX_INIT;
+    return -1;
 }
 
 int
@@ -702,6 +700,7 @@ rxi_rto_startTimer(struct rx_call *call, int lastPacket, int istack)
 {
     struct clock now, retryTime;
 
+    MUTEX_ASSERT(&call->lock);
     clock_GetTime(&now);
     retryTime = now;
 
@@ -732,7 +731,9 @@ rxi_rto_startTimer(struct rx_call *call, int lastPacket, int istack)
 static_inline void
 rxi_rto_cancel(struct rx_call *call)
 {
-    rxevent_Cancel(&call->resendEvent, call, RX_CALL_REFCOUNT_RESEND);
+    MUTEX_ASSERT(&call->lock);
+    if (rxevent_Cancel(&call->resendEvent))
+       CALL_RELE(call, RX_CALL_REFCOUNT_RESEND);
 }
 
 /*!
@@ -806,20 +807,6 @@ rx_rto_setPeerTimeoutSecs(struct rx_peer *peer, int secs) {
 }
 
 /**
- * Enables or disables the busy call channel error (RX_CALL_BUSY).
- *
- * @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 onoff)
-{
-    osi_Assert(rxinit_status != 0);
-    rxi_busyChannelError = onoff ? 1 : 0;
-}
-
-/**
  * Set a delayed ack event on the specified call for the given time
  *
  * @param[in] call - the call on which to set the event
@@ -830,17 +817,21 @@ rxi_PostDelayedAckEvent(struct rx_call *call, struct clock *offset)
 {
     struct clock now, when;
 
+    MUTEX_ASSERT(&call->lock);
     clock_GetTime(&now);
     when = now;
     clock_Add(&when, offset);
 
-    if (!call->delayedAckEvent
-       || clock_Gt(&call->delayedAckTime, &when)) {
+    if (clock_Gt(&call->delayedAckTime, &when) &&
+       rxevent_Cancel(&call->delayedAckEvent)) {
+       /* We successfully cancelled an event too far in the future to install
+        * our new one; we can reuse the reference on the call. */
+       call->delayedAckEvent = rxevent_Post(&when, &now, rxi_SendDelayedAck,
+                                            call, NULL, 0);
 
-        rxevent_Cancel(&call->delayedAckEvent, call,
-                      RX_CALL_REFCOUNT_DELAY);
+       call->delayedAckTime = when;
+    } else if (call->delayedAckEvent == NULL) {
        CALL_HOLD(call, RX_CALL_REFCOUNT_DELAY);
-
        call->delayedAckEvent = rxevent_Post(&when, &now,
                                             rxi_SendDelayedAck,
                                             call, NULL, 0);
@@ -848,6 +839,15 @@ rxi_PostDelayedAckEvent(struct rx_call *call, struct clock *offset)
     }
 }
 
+void
+rxi_CancelDelayedAckEvent(struct rx_call *call)
+{
+    MUTEX_ASSERT(&call->lock);
+    /* Only drop the ref if we cancelled it before it could run. */
+    if (rxevent_Cancel(&call->delayedAckEvent))
+       CALL_RELE(call, RX_CALL_REFCOUNT_DELAY);
+}
+
 /* called with unincremented nRequestsRunning to see if it is OK to start
  * a new thread in this service.  Could be "no" for two reasons: over the
  * max quota, or would prevent others from reaching their min quota.
@@ -1049,8 +1049,8 @@ rx_NewConnection(afs_uint32 shost, u_short sport, u_short sservice,
                 int serviceSecurityIndex)
 {
     int hashindex, i;
-    afs_int32 cid;
     struct rx_connection *conn;
+    int code;
 
     SPLVAR;
 
@@ -1070,11 +1070,11 @@ rx_NewConnection(afs_uint32 shost, u_short sport, u_short sservice,
 #endif
     NETPRI;
     MUTEX_ENTER(&rx_connHashTable_lock);
-    cid = (rx_nextCid += RX_MAXCALLS);
     conn->type = RX_CLIENT_CONNECTION;
-    conn->cid = cid;
     conn->epoch = rx_epoch;
-    conn->peer = rxi_FindPeer(shost, sport, 0, 1);
+    conn->cid = rx_nextCid;
+    update_nextCid();
+    conn->peer = rxi_FindPeer(shost, sport, 1);
     conn->serviceId = sservice;
     conn->securityObject = securityObject;
     conn->securityData = (void *) 0;
@@ -1094,7 +1094,7 @@ rx_NewConnection(afs_uint32 shost, u_short sport, u_short sservice,
        conn->lastBusy[i] = 0;
     }
 
-    RXS_NewConnection(securityObject, conn);
+    code = RXS_NewConnection(securityObject, conn);
     hashindex =
        CONN_HASH(shost, sport, conn->cid, conn->epoch, RX_CLIENT_CONNECTION);
 
@@ -1105,6 +1105,9 @@ rx_NewConnection(afs_uint32 shost, u_short sport, u_short sservice,
        rx_atomic_inc(&rx_stats.nClientConns);
     MUTEX_EXIT(&rx_connHashTable_lock);
     USERPRI;
+    if (code) {
+       rxi_ConnectionError(conn, code);
+    }
     return conn;
 }
 
@@ -1162,7 +1165,6 @@ void
 rx_SetConnIdleDeadTime(struct rx_connection *conn, int seconds)
 {
     conn->idleDeadTime = seconds;
-    conn->idleDeadDetection = (seconds ? 1 : 0);
     rxi_CheckConnTimeouts(conn);
 }
 
@@ -1255,7 +1257,6 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
 {
     struct rx_connection **conn_ptr;
     int havecalls = 0;
-    struct rx_packet *packet;
     int i;
     SPLVAR;
 
@@ -1267,6 +1268,9 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
     if (conn->refCount > 0)
        conn->refCount--;
     else {
+#ifdef RX_REFCOUNT_CHECK
+       osi_Assert(conn->refCount == 0);
+#endif
         if (rx_stats_active) {
             MUTEX_ENTER(&rx_stats_mutex);
             rxi_lowConnRefCount++;
@@ -1288,6 +1292,7 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
     if ((conn->type == RX_CLIENT_CONNECTION)
        && (conn->flags & (RX_CONN_MAKECALL_WAITING|RX_CONN_MAKECALL_ACTIVE))) {
        conn->flags |= RX_CONN_DESTROY_ME;
+       MUTEX_EXIT(&rx_refcnt_mutex);
        MUTEX_EXIT(&conn->conn_data_lock);
        USERPRI;
        return;
@@ -1307,8 +1312,7 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
                    /* Push the final acknowledgment out now--there
                     * won't be a subsequent call to acknowledge the
                     * last reply packets */
-                   rxevent_Cancel(&call->delayedAckEvent, call,
-                                  RX_CALL_REFCOUNT_DELAY);
+                   rxi_CancelDelayedAckEvent(call);
                    if (call->state == RX_STATE_PRECALL
                        || call->state == RX_STATE_ACTIVE) {
                        rxi_SendAck(call, 0, 0, RX_ACK_DELAY, 0);
@@ -1343,21 +1347,6 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
        return;
     }
 
-    if (conn->natKeepAliveEvent) {
-       rxi_NatKeepAliveOff(conn);
-    }
-
-    if (conn->delayedAbortEvent) {
-       rxevent_Cancel(&conn->delayedAbortEvent, NULL, 0);
-       packet = rxi_AllocPacket(RX_PACKET_CLASS_SPECIAL);
-       if (packet) {
-           MUTEX_ENTER(&conn->conn_data_lock);
-           rxi_SendConnectionAbort(conn, packet, 0, 1);
-           MUTEX_EXIT(&conn->conn_data_lock);
-           rxi_FreePacket(packet);
-       }
-    }
-
     /* Remove from connection hash table before proceeding */
     conn_ptr =
        &rx_connHashTable[CONN_HASH
@@ -1375,10 +1364,13 @@ rxi_DestroyConnectionNoLock(struct rx_connection *conn)
        rxLastConn = 0;
 
     /* Make sure the connection is completely reset before deleting it. */
-    /* get rid of pending events that could zap us later */
-    rxevent_Cancel(&conn->challengeEvent, NULL, 0);
-    rxevent_Cancel(&conn->checkReachEvent, NULL, 0);
-    rxevent_Cancel(&conn->natKeepAliveEvent, NULL, 0);
+    /*
+     * Pending events hold a refcount, so we can't get here if they are
+     * non-NULL. */
+    osi_Assert(conn->challengeEvent == NULL);
+    osi_Assert(conn->delayedAbortEvent == NULL);
+    osi_Assert(conn->natKeepAliveEvent == NULL);
+    osi_Assert(conn->checkReachEvent == NULL);
 
     /* Add the connection to the list of destroyed connections that
      * need to be cleaned up. This is necessary to avoid deadlocks
@@ -1411,7 +1403,7 @@ rx_GetConnection(struct rx_connection *conn)
     USERPRI;
 }
 
-#ifdef  AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
 /* Wait for the transmit queue to no longer be busy.
  * requires the call->lock to be held */
 void
@@ -1419,12 +1411,8 @@ rxi_WaitforTQBusy(struct rx_call *call) {
     while (!call->error && (call->flags & RX_CALL_TQ_BUSY)) {
        call->flags |= RX_CALL_TQ_WAIT;
        call->tqWaiters++;
-#ifdef RX_ENABLE_LOCKS
-       osirx_AssertMine(&call->lock, "rxi_WaitforTQ lock");
+       MUTEX_ASSERT(&call->lock);
        CV_WAIT(&call->cv_tq, &call->lock);
-#else /* RX_ENABLE_LOCKS */
-       osi_rxSleep(&call->tq);
-#endif /* RX_ENABLE_LOCKS */
        call->tqWaiters--;
        if (call->tqWaiters == 0) {
            call->flags &= ~RX_CALL_TQ_WAIT;
@@ -1437,10 +1425,10 @@ static void
 rxi_WakeUpTransmitQueue(struct rx_call *call)
 {
     if (call->tqWaiters || (call->flags & RX_CALL_TQ_WAIT)) {
-       dpf(("call %"AFS_PTR_FMT" has %d waiters and flags %d\n",
+       dpf(("call %p has %d waiters and flags %d\n",
             call, call->tqWaiters, call->flags));
 #ifdef RX_ENABLE_LOCKS
-       osirx_AssertMine(&call->lock, "rxi_Start start");
+       MUTEX_ASSERT(&call->lock);
        CV_BROADCAST(&call->cv_tq);
 #else /* RX_ENABLE_LOCKS */
        osi_rxWakeup(&call->tq);
@@ -1467,7 +1455,7 @@ rx_NewCall(struct rx_connection *conn)
     SPLVAR;
 
     clock_NewTime();
-    dpf(("rx_NewCall(conn %"AFS_PTR_FMT")\n", conn));
+    dpf(("rx_NewCall(conn %p)\n", conn));
 
     NETPRI;
     clock_GetTime(&queueTime);
@@ -1602,7 +1590,6 @@ rx_NewCall(struct rx_connection *conn)
        }
        if (i < RX_MAXCALLS) {
            conn->lastBusy[i] = 0;
-           call->flags &= ~RX_CALL_PEER_BUSY;
            break;
        }
         if (!wait)
@@ -1635,9 +1622,9 @@ 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
@@ -1649,8 +1636,8 @@ rx_NewCall(struct rx_connection *conn)
     /* remember start time for call in case we have hard dead time limit */
     call->queueTime = queueTime;
     clock_GetTime(&call->startTime);
-    call->bytesSent = 0;
-    call->bytesRcvd = 0;
+    call->app.bytesSent = 0;
+    call->app.bytesRcvd = 0;
 
     /* Turn on busy protocol. */
     rxi_KeepAliveOn(call);
@@ -1670,22 +1657,19 @@ rx_NewCall(struct rx_connection *conn)
      * run (see code above that avoids resource starvation).
      */
 #ifdef RX_ENABLE_LOCKS
+    if (call->flags & (RX_CALL_TQ_BUSY | RX_CALL_TQ_CLEARME)) {
+        osi_Panic("rx_NewCall call about to be used without an empty tq");
+    }
+
     CV_BROADCAST(&conn->conn_call_cv);
 #else
     osi_rxWakeup(conn);
 #endif
     MUTEX_EXIT(&conn->conn_call_lock);
-
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
-    if (call->flags & (RX_CALL_TQ_BUSY | RX_CALL_TQ_CLEARME)) {
-        osi_Panic("rx_NewCall call about to be used without an empty tq");
-    }
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
-
     MUTEX_EXIT(&call->lock);
     USERPRI;
 
-    dpf(("rx_NewCall(call %"AFS_PTR_FMT")\n", call));
+    dpf(("rx_NewCall(call %p)\n", call));
     return call;
 }
 
@@ -1792,9 +1776,7 @@ rx_NewServiceHost(afs_uint32 host, u_short port, u_short serviceId,
     tservice = rxi_AllocService();
     NETPRI;
 
-#ifdef RX_ENABLE_LOCKS
     MUTEX_INIT(&tservice->svc_data_lock, "svc data lock", MUTEX_DEFAULT, 0);
-#endif
 
     for (i = 0; i < RX_MAX_SERVICES; i++) {
        struct rx_service *service = rx_services[i];
@@ -1838,7 +1820,6 @@ rx_NewServiceHost(afs_uint32 host, u_short port, u_short serviceId,
            service->minProcs = 0;
            service->maxProcs = 1;
            service->idleDeadTime = 60;
-           service->idleDeadErr = 0;
            service->connDeadTime = rx_connDeadTime;
            service->executeRequestProc = serviceProc;
            service->checkReach = 0;
@@ -1864,10 +1845,14 @@ rx_SetSecurityConfiguration(struct rx_service *service,
                            void *value)
 {
     int i;
+    int code;
     for (i = 0; i<service->nSecurityObjects; i++) {
        if (service->securityObjects[i]) {
-           RXS_SetConfiguration(service->securityObjects[i], NULL, type,
-                                value, NULL);
+           code = RXS_SetConfiguration(service->securityObjects[i], NULL, type,
+                                       value, NULL);
+           if (code) {
+               return code;
+           }
        }
     }
     return 0;
@@ -1964,7 +1949,7 @@ rxi_ServerProc(int threadID, struct rx_call *newcall, osi_socket * socketp)
 void
 rx_WakeupServerProcs(void)
 {
-    struct rx_serverQueueEntry *np, *tqp;
+    struct rx_serverQueueEntry *np;
     struct opr_queue *cursor;
     SPLVAR;
 
@@ -1979,8 +1964,8 @@ rx_WakeupServerProcs(void)
        osi_rxWakeup(rx_waitForPacket);
 #endif /* RX_ENABLE_LOCKS */
     MUTEX_ENTER(&freeSQEList_lock);
-    for (np = rx_FreeSQEList; np; np = tqp) {
-       tqp = *(struct rx_serverQueueEntry **)np;
+    for (opr_queue_Scan(&rx_freeServerQueue, cursor)) {
+        np = opr_queue_Entry(cursor, struct rx_serverQueueEntry, entry);
 #ifdef RX_ENABLE_LOCKS
        CV_BROADCAST(&np->cv);
 #else /* RX_ENABLE_LOCKS */
@@ -2040,8 +2025,10 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
 
     MUTEX_ENTER(&freeSQEList_lock);
 
-    if ((sq = rx_FreeSQEList)) {
-       rx_FreeSQEList = *(struct rx_serverQueueEntry **)sq;
+    if (!opr_queue_IsEmpty(&rx_freeServerQueue)) {
+       sq = opr_queue_First(&rx_freeServerQueue, struct rx_serverQueueEntry,
+                            entry);
+       opr_queue_Remove(&sq->entry);
        MUTEX_EXIT(&freeSQEList_lock);
     } else {                   /* otherwise allocate a new one and return that */
        MUTEX_EXIT(&freeSQEList_lock);
@@ -2075,7 +2062,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
                }
                MUTEX_ENTER(&rx_pthread_mutex);
                if (tno == rxi_fcfs_thread_num
-                       || opr_queue_IsEnd(&rx_incomingCallQueue, cursor)) {
+                       || opr_queue_IsLast(&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
@@ -2114,6 +2101,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
            opr_queue_Remove(&call->entry);
            MUTEX_EXIT(&rx_serverPool_lock);
            MUTEX_ENTER(&call->lock);
+           CLEAR_CALL_QUEUE_LOCK(call);
 
            if (call->flags & RX_CALL_WAIT_PROC) {
                call->flags &= ~RX_CALL_WAIT_PROC;
@@ -2132,7 +2120,6 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
                || 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);
            break;
        } else {
            /* If there are no eligible incoming calls, add this process
@@ -2146,19 +2133,19 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
            opr_queue_Append(&rx_idleServerQueue, &sq->entry);
 #ifndef AFS_AIX41_ENV
            rx_waitForPacket = sq;
-#else
-           rx_waitingForPacket = sq;
 #endif /* AFS_AIX41_ENV */
            do {
                CV_WAIT(&sq->cv, &rx_serverPool_lock);
 #ifdef KERNEL
                if (afs_termState == AFSOP_STOP_RXCALLBACK) {
-                   MUTEX_EXIT(&rx_serverPool_lock);
-                   return (struct rx_call *)0;
+                   break;
                }
 #endif
            } while (!(call = sq->newcall)
                     && !(socketp && *socketp != OSI_NULLSOCKET));
+           if (opr_queue_IsOnQueue(&sq->entry)) {
+               opr_queue_Remove(&sq->entry);
+           }
            MUTEX_EXIT(&rx_serverPool_lock);
            if (call) {
                MUTEX_ENTER(&call->lock);
@@ -2168,14 +2155,13 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
     }
 
     MUTEX_ENTER(&freeSQEList_lock);
-    *(struct rx_serverQueueEntry **)sq = rx_FreeSQEList;
-    rx_FreeSQEList = sq;
+    opr_queue_Prepend(&rx_freeServerQueue, &sq->entry);
     MUTEX_EXIT(&freeSQEList_lock);
 
     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();
@@ -2190,7 +2176,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
 #endif
 
        rxi_calltrace(RX_CALL_START, call);
-       dpf(("rx_GetCall(port=%d, service=%d) ==> call %"AFS_PTR_FMT"\n",
+       dpf(("rx_GetCall(port=%d, service=%d) ==> call %p\n",
             call->conn->service->servicePort, call->conn->service->serviceId,
             call));
 
@@ -2214,8 +2200,10 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
     NETPRI;
     MUTEX_ENTER(&freeSQEList_lock);
 
-    if ((sq = rx_FreeSQEList)) {
-       rx_FreeSQEList = *(struct rx_serverQueueEntry **)sq;
+    if (!opr_queue_IsEmpty(&rx_freeServerQueue)) {
+       sq = opr_queue_First(&rx_freeServerQueue, struct rx_serverQueueEntry,
+                            entry);
+       opr_queue_Remove(&sq->entry);
        MUTEX_EXIT(&freeSQEList_lock);
     } else {                   /* otherwise allocate a new one and return that */
        MUTEX_EXIT(&freeSQEList_lock);
@@ -2249,9 +2237,8 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
            service = tcall->conn->service;
            if (QuotaOK(service)) {
                MUTEX_ENTER(&rx_pthread_mutex);
-               /* 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) {
+               if (tno == rxi_fcfs_thread_num
+                       || opr_queue_IsLast(&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
@@ -2285,6 +2272,7 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
 
     if (call) {
        opr_queue_Remove(&call->entry);
+       CLEAR_CALL_QUEUE_LOCK(call);
        /* 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
@@ -2329,14 +2317,13 @@ rx_GetCall(int tno, struct rx_service *cur_service, osi_socket * socketp)
     MUTEX_EXIT(&sq->lock);
 
     MUTEX_ENTER(&freeSQEList_lock);
-    *(struct rx_serverQueueEntry **)sq = rx_FreeSQEList;
-    rx_FreeSQEList = sq;
+    opr_queue_Prepend(&rx_freeServerQueue, &sq->entry);
     MUTEX_EXIT(&freeSQEList_lock);
 
     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();
@@ -2399,7 +2386,7 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
     afs_int32 error;
     SPLVAR;
 
-    dpf(("rx_EndCall(call %"AFS_PTR_FMT" rc %d error %d abortCode %d)\n",
+    dpf(("rx_EndCall(call %p rc %d error %d abortCode %d)\n",
           call, rc, call->error, call->abortCode));
 
     NETPRI;
@@ -2410,10 +2397,10 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
        call->abortCount = 0;
     }
 
-    call->arrivalProc = (void (*)())0;
+    call->arrivalProc = NULL;
     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
@@ -2422,15 +2409,13 @@ 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) {
-            MUTEX_EXIT(&call->lock);
-           rxi_FlushWrite(call);
-            MUTEX_ENTER(&call->lock);
+       if (call->app.mode == RX_MODE_SENDING) {
+           rxi_FlushWriteLocked(call);
        }
        rxi_calltrace(RX_CALL_END, call);
        /* Call goes to hold state until reply packets are acknowledged */
@@ -2440,15 +2425,15 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
            call->state = RX_STATE_DALLY;
            rxi_ClearTransmitQueue(call, 0);
            rxi_rto_cancel(call);
-           rxevent_Cancel(&call->keepAliveEvent, call,
-                          RX_CALL_REFCOUNT_ALIVE);
+           rxi_CancelKeepAliveEvent(call);
        }
     } else {                   /* Client connection */
        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);
@@ -2458,8 +2443,7 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
         * and force-send it now.
         */
        if (call->delayedAckEvent) {
-           rxevent_Cancel(&call->delayedAckEvent, call,
-                          RX_CALL_REFCOUNT_DELAY);
+           rxi_CancelDelayedAckEvent(call);
            rxi_SendDelayedAck(NULL, call, NULL, 0);
        }
 
@@ -2476,8 +2460,18 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
         MUTEX_ENTER(&conn->conn_call_lock);
         MUTEX_ENTER(&call->lock);
 
-       if (!(call->flags & RX_CALL_PEER_BUSY)) {
+       if (!call->error) {
+           /* While there are some circumstances where a call with an error is
+            * obviously not on a "busy" channel, be conservative (clearing
+            * lastBusy is just best-effort to possibly speed up rx_NewCall).
+            * The call channel is definitely not busy if we just successfully
+            * completed a call on it. */
            conn->lastBusy[call->channel] = 0;
+
+       } else if (call->error == RX_CALL_TIMEOUT) {
+           /* The call is still probably running on the server side, so try to
+            * avoid this call channel in the future. */
+           conn->lastBusy[call->channel] = clock_Sec();
        }
 
        MUTEX_ENTER(&conn->conn_data_lock);
@@ -2503,21 +2497,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);
@@ -2532,6 +2526,10 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
      * Map errors to the local host's errno.h format.
      */
     error = ntoh_syserr_conv(error);
+
+    /* If the caller said the call failed with some error, we had better
+     * return an error code. */
+    osi_Assert(!rc || error);
     return error;
 }
 
@@ -2546,14 +2544,21 @@ rx_EndCall(struct rx_call *call, afs_int32 rc)
 void
 rx_Finalize(void)
 {
-    struct rx_connection **conn_ptr, **conn_end;
-
     INIT_PTHREAD_LOCKS;
     LOCK_RX_INIT;
-    if (rxinit_status == 1) {
+    if (!rxi_IsRunning()) {
        UNLOCK_RX_INIT;
        return;                 /* Already shutdown. */
     }
+    rxi_Finalize_locked();
+    UNLOCK_RX_INIT;
+}
+
+static void
+rxi_Finalize_locked(void)
+{
+    struct rx_connection **conn_ptr, **conn_end;
+    rx_atomic_set(&rxi_running, 0);
     rxi_DeleteCachedConnections();
     if (rx_connHashTable) {
        MUTEX_ENTER(&rx_connHashTable_lock);
@@ -2564,9 +2569,7 @@ rx_Finalize(void)
            for (conn = *conn_ptr; conn; conn = next) {
                next = conn->next;
                if (conn->type == RX_CLIENT_CONNECTION) {
-                    MUTEX_ENTER(&rx_refcnt_mutex);
-                   conn->refCount++;
-                    MUTEX_EXIT(&rx_refcnt_mutex);
+                    rx_GetConnection(conn);
 #ifdef RX_ENABLE_LOCKS
                    rxi_DestroyConnectionNoLock(conn);
 #else /* RX_ENABLE_LOCKS */
@@ -2592,9 +2595,6 @@ rx_Finalize(void)
 #ifdef AFS_NT40_ENV
     afs_winsockCleanup();
 #endif
-
-    rxinit_status = 1;
-    UNLOCK_RX_INIT;
 }
 #endif
 
@@ -2651,19 +2651,19 @@ static struct rx_call *
 rxi_NewCall(struct rx_connection *conn, int channel)
 {
     struct rx_call *call;
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
     struct rx_call *cp;        /* Call pointer temp */
     struct opr_queue *cursor;
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif
 
-    dpf(("rxi_NewCall(conn %"AFS_PTR_FMT", channel %d)\n", conn, channel));
+    dpf(("rxi_NewCall(conn %p, channel %d)\n", conn, channel));
 
     /* Grab an existing call structure, or allocate a new one.
      * Existing call structures are assumed to have been left reset by
      * rxi_FreeCall */
     MUTEX_ENTER(&rx_freeCallQueue_lock);
 
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
     /*
      * EXCEPT that the TQ might not yet be cleared out.
      * Skip over those with in-use TQs.
@@ -2677,24 +2677,24 @@ rxi_NewCall(struct rx_connection *conn, int channel)
        }
     }
     if (call) {
-#else /* AFS_GLOBAL_RXLOCK_KERNEL */
+#else /* RX_ENABLE_LOCKS */
     if (!opr_queue_IsEmpty(&rx_freeCallQueue)) {
        call = opr_queue_First(&rx_freeCallQueue, struct rx_call, entry);
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
        opr_queue_Remove(&call->entry);
         if (rx_stats_active)
            rx_atomic_dec(&rx_stats.nFreeCallStructs);
        MUTEX_EXIT(&rx_freeCallQueue_lock);
        MUTEX_ENTER(&call->lock);
        CLEAR_CALL_QUEUE_LOCK(call);
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
        /* Now, if TQ wasn't cleared earlier, do it now. */
        rxi_WaitforTQBusy(call);
        if (call->flags & RX_CALL_TQ_CLEARME) {
            rxi_ClearTransmitQueue(call, 1);
            /*queue_Init(&call->tq);*/
        }
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
        /* Bind the call to its connection structure */
        call->conn = conn;
        rxi_ResetCall(call, 1);
@@ -2720,7 +2720,7 @@ rxi_NewCall(struct rx_connection *conn, int channel)
        /* Initialize once-only items */
        opr_queue_Init(&call->tq);
        opr_queue_Init(&call->rq);
-       opr_queue_Init(&call->iovq);
+       opr_queue_Init(&call->app.iovq);
 #ifdef RXDEBUG_PACKET
         call->rqc = call->tqc = call->iovqc = 0;
 #endif /* RXDEBUG_PACKET */
@@ -2793,7 +2793,7 @@ rxi_FreeCall(struct rx_call *call, int haveCTLock)
 
     MUTEX_ENTER(&rx_freeCallQueue_lock);
     SET_CALL_QUEUE_LOCK(call, &rx_freeCallQueue_lock);
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
     /* A call may be free even though its transmit queue is still in use.
      * Since we search the call list from head to tail, put busy calls at
      * the head of the list, and idle calls at the tail.
@@ -2802,9 +2802,9 @@ rxi_FreeCall(struct rx_call *call, int haveCTLock)
        opr_queue_Prepend(&rx_freeCallQueue, &call->entry);
     else
        opr_queue_Append(&rx_freeCallQueue, &call->entry);
-#else /* AFS_GLOBAL_RXLOCK_KERNEL */
+#else /* RX_ENABLE_LOCKS */
     opr_queue_Append(&rx_freeCallQueue, &call->entry);
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
     if (rx_stats_active)
        rx_atomic_inc(&rx_stats.nFreeCallStructs);
     MUTEX_EXIT(&rx_freeCallQueue_lock);
@@ -2823,9 +2823,7 @@ rxi_FreeCall(struct rx_call *call, int haveCTLock)
      */
     MUTEX_ENTER(&conn->conn_data_lock);
     if (conn->flags & RX_CONN_DESTROY_ME && !(conn->flags & RX_CONN_MAKECALL_WAITING)) {
-        MUTEX_ENTER(&rx_refcnt_mutex);
-       conn->refCount++;
-        MUTEX_EXIT(&rx_refcnt_mutex);
+        rx_GetConnection(conn);
        MUTEX_EXIT(&conn->conn_data_lock);
 #ifdef RX_ENABLE_LOCKS
        if (haveCTLock)
@@ -2856,7 +2854,7 @@ rxi_Alloc(size_t size)
     }
 
 p = (char *)
-#if defined(KERNEL) && !defined(UKERNEL) && defined(AFS_FBSD80_ENV)
+#if defined(KERNEL) && !defined(UKERNEL) && defined(AFS_FBSD_ENV)
   afs_osi_Alloc_NoSleep(size);
 #else
   osi_Alloc(size);
@@ -2870,6 +2868,9 @@ p = (char *)
 void
 rxi_Free(void *addr, size_t size)
 {
+    if (!addr) {
+        return;
+    }
     if (rx_stats_active) {
        rx_atomic_sub(&rxi_Allocsize, (int) size);
         rx_atomic_dec(&rxi_Alloccnt);
@@ -2926,7 +2927,7 @@ rxi_SetPeerMtu(struct rx_peer *peer, afs_uint32 host, afs_uint32 port, int mtu)
        if (peer->ifMTU < OLD_MAX_PACKET_SIZE)
            peer->maxDgramPackets = 1;
        /* We no longer have valid peer packet information */
-       if (peer->maxPacketSize-RX_IPUDP_SIZE > peer->ifMTU)
+       if (peer->maxPacketSize + RX_HEADER_SIZE > peer->ifMTU)
            peer->maxPacketSize = 0;
         MUTEX_EXIT(&peer->peer_lock);
 
@@ -2943,7 +2944,7 @@ rxi_SetPeerMtu(struct rx_peer *peer, afs_uint32 host, afs_uint32 port, int mtu)
 
 #ifdef AFS_RXERRQ_ENV
 static void
-rxi_SetPeerDead(afs_uint32 host, afs_uint16 port)
+rxi_SetPeerDead(struct sock_extended_err *err, afs_uint32 host, afs_uint16 port)
 {
     int hashIndex = PEER_HASH(host, port);
     struct rx_peer *peer;
@@ -2952,15 +2953,25 @@ rxi_SetPeerDead(afs_uint32 host, afs_uint16 port)
 
     for (peer = rx_peerHashTable[hashIndex]; peer; peer = peer->next) {
        if (peer->host == host && peer->port == port) {
+           peer->refCount++;
            break;
        }
     }
 
+    MUTEX_EXIT(&rx_peerHashTable_lock);
+
     if (peer) {
        rx_atomic_inc(&peer->neterrs);
-    }
+       MUTEX_ENTER(&peer->peer_lock);
+       peer->last_err_origin = RX_NETWORK_ERROR_ORIGIN_ICMP;
+       peer->last_err_type = err->ee_type;
+       peer->last_err_code = err->ee_code;
+       MUTEX_EXIT(&peer->peer_lock);
 
-    MUTEX_EXIT(&rx_peerHashTable_lock);
+       MUTEX_ENTER(&rx_peerHashTable_lock);
+       peer->refCount--;
+       MUTEX_EXIT(&rx_peerHashTable_lock);
+    }
 }
 
 void
@@ -2979,22 +2990,93 @@ rxi_ProcessNetError(struct sock_extended_err *err, afs_uint32 addr, afs_uint16 p
        case ICMP_PORT_UNREACH:
        case ICMP_NET_ANO:
        case ICMP_HOST_ANO:
-           rxi_SetPeerDead(addr, port);
+           rxi_SetPeerDead(err, addr, port);
            break;
        }
     }
 }
+
+static const char *
+rxi_TranslateICMP(int type, int code)
+{
+    switch (type) {
+    case ICMP_DEST_UNREACH:
+       switch (code) {
+       case ICMP_NET_UNREACH:
+           return "Destination Net Unreachable";
+       case ICMP_HOST_UNREACH:
+           return "Destination Host Unreachable";
+       case ICMP_PROT_UNREACH:
+           return "Destination Protocol Unreachable";
+       case ICMP_PORT_UNREACH:
+           return "Destination Port Unreachable";
+       case ICMP_NET_ANO:
+           return "Destination Net Prohibited";
+       case ICMP_HOST_ANO:
+           return "Destination Host Prohibited";
+       }
+       break;
+    }
+    return NULL;
+}
 #endif /* AFS_RXERRQ_ENV */
 
+/**
+ * Get the last network error for a connection
+ *
+ * A "network error" here means an error retrieved from ICMP, or some other
+ * mechanism outside of Rx that informs us of errors in network reachability.
+ *
+ * If a peer associated with the given Rx connection has received a network
+ * error recently, this function allows the caller to know what error
+ * specifically occurred. This can be useful to know, since e.g. ICMP errors
+ * can cause calls to that peer to be quickly aborted. So, this function can
+ * help see why a call was aborted due to network errors.
+ *
+ * If we have received traffic from a peer since the last network error, we
+ * treat that peer as if we had not received an network error for it.
+ *
+ * @param[in] conn  The Rx connection to examine
+ * @param[out] err_origin  The origin of the last network error (e.g. ICMP);
+ *                         one of the RX_NETWORK_ERROR_ORIGIN_* constants
+ * @param[out] err_type  The type of the last error
+ * @param[out] err_code  The code of the last error
+ * @param[out] msg  Human-readable error message, if applicable; NULL otherwise
+ *
+ * @return If we have an error
+ *  @retval -1 No error to get; 'out' params are undefined
+ *  @retval 0 We have an error; 'out' params contain the last error
+ */
+int
+rx_GetNetworkError(struct rx_connection *conn, int *err_origin, int *err_type,
+                   int *err_code, const char **msg)
+{
+#ifdef AFS_RXERRQ_ENV
+    struct rx_peer *peer = conn->peer;
+    if (rx_atomic_read(&peer->neterrs)) {
+       MUTEX_ENTER(&peer->peer_lock);
+       *err_origin = peer->last_err_origin;
+       *err_type = peer->last_err_type;
+       *err_code = peer->last_err_code;
+       MUTEX_EXIT(&peer->peer_lock);
+
+       *msg = NULL;
+       if (*err_origin == RX_NETWORK_ERROR_ORIGIN_ICMP) {
+           *msg = rxi_TranslateICMP(*err_type, *err_code);
+       }
+
+       return 0;
+    }
+#endif
+    return -1;
+}
+
 /* 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
- * The origPeer, if set, is a pointer to a peer structure on which the
- * refcount will be be decremented. This is used to replace the peer
- * structure hanging off a connection structure */
+ */
 struct rx_peer *
-rxi_FindPeer(afs_uint32 host, u_short port,
-            struct rx_peer *origPeer, int create)
+rxi_FindPeer(afs_uint32 host, u_short port, int create)
 {
     struct rx_peer *pp;
     int hashIndex;
@@ -3024,8 +3106,6 @@ rxi_FindPeer(afs_uint32 host, u_short port,
     if (pp && create) {
        pp->refCount++;
     }
-    if (origPeer)
-       origPeer->refCount--;
     MUTEX_EXIT(&rx_peerHashTable_lock);
     return pp;
 }
@@ -3046,10 +3126,13 @@ rxi_FindPeer(afs_uint32 host, u_short port,
 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)
+                  afs_uint32 epoch, int type, u_int securityIndex,
+                   int *unknownService)
 {
     int hashindex, flag, i;
+    int code = 0;
     struct rx_connection *conn;
+    *unknownService = 0;
     hashindex = CONN_HASH(host, port, cid, epoch, type);
     MUTEX_ENTER(&rx_connHashTable_lock);
     rxLastConn ? (conn = rxLastConn, flag = 0) : (conn =
@@ -3094,6 +3177,7 @@ rxi_FindConnection(osi_socket socket, afs_uint32 host,
        if (!service || (securityIndex >= service->nSecurityObjects)
            || (service->securityObjects[securityIndex] == 0)) {
            MUTEX_EXIT(&rx_connHashTable_lock);
+            *unknownService = 1;
            return (struct rx_connection *)0;
        }
        conn = rxi_AllocConnection();   /* This bzero's the connection */
@@ -3102,7 +3186,7 @@ rxi_FindConnection(osi_socket socket, afs_uint32 host,
        CV_INIT(&conn->conn_call_cv, "conn call cv", CV_DEFAULT, 0);
        conn->next = rx_connHashTable[hashindex];
        rx_connHashTable[hashindex] = conn;
-       conn->peer = rxi_FindPeer(host, port, 0, 1);
+       conn->peer = rxi_FindPeer(host, port, 1);
        conn->type = RX_SERVER_CONNECTION;
        conn->lastSendTime = clock_Sec();       /* don't GC immediately */
        conn->epoch = epoch;
@@ -3115,14 +3199,13 @@ rxi_FindConnection(osi_socket socket, afs_uint32 host,
        conn->nSpecific = 0;
        conn->specific = NULL;
        rx_SetConnDeadTime(conn, service->connDeadTime);
-       conn->idleDeadTime = service->idleDeadTime;
-       conn->idleDeadDetection = service->idleDeadErr ? 1 : 0;
+       rx_SetConnIdleDeadTime(conn, service->idleDeadTime);
        for (i = 0; i < RX_MAXCALLS; i++) {
            conn->twind[i] = rx_initSendWindow;
            conn->rwind[i] = rx_initReceiveWindow;
        }
        /* Notify security object of the new connection */
-       RXS_NewConnection(conn->securityObject, conn);
+       code = RXS_NewConnection(conn->securityObject, conn);
        /* XXXX Connection timeout? */
        if (service->newConnProc)
            (*service->newConnProc) (conn);
@@ -3130,96 +3213,172 @@ rxi_FindConnection(osi_socket socket, afs_uint32 host,
             rx_atomic_inc(&rx_stats.nServerConns);
     }
 
-    MUTEX_ENTER(&rx_refcnt_mutex);
-    conn->refCount++;
-    MUTEX_EXIT(&rx_refcnt_mutex);
+    rx_GetConnection(conn);
 
     rxLastConn = conn;         /* store this connection as the last conn used */
     MUTEX_EXIT(&rx_connHashTable_lock);
+    if (code) {
+       rxi_ConnectionError(conn, code);
+    }
     return conn;
 }
 
-/**
- * Timeout a call on a busy call channel if appropriate.
- *
- * @param[in] call The busy call.
- *
- * @pre 'call' is marked as busy (namely,
- *      call->conn->lastBusy[call->channel] != 0)
- *
- * @pre call->lock is held
- * @pre rxi_busyChannelError is nonzero
- *
- * @note call->lock is dropped and reacquired
+/*!
+ * Abort the call if the server is over the busy threshold. This
+ * can be used without requiring a call structure be initialised,
+ * or connected to a particular channel
  */
-static void
-rxi_CheckBusy(struct rx_call *call)
+static_inline int
+rxi_AbortIfServerBusy(osi_socket socket, struct rx_connection *conn,
+                     struct rx_packet *np)
 {
-    struct rx_connection *conn = call->conn;
-    int channel = call->channel;
-    int freechannel = 0;
-    int i;
-    afs_uint32 callNumber;
+    afs_uint32 serial;
 
-    MUTEX_EXIT(&call->lock);
+    if ((rx_BusyThreshold > 0) &&
+       (rx_atomic_read(&rx_nWaiting) > rx_BusyThreshold)) {
+       MUTEX_ENTER(&conn->conn_data_lock);
+       serial = ++conn->serial;
+       MUTEX_EXIT(&conn->conn_data_lock);
+       rxi_SendRawAbort(socket, conn->peer->host, conn->peer->port,
+                        serial, rx_BusyError, np, 0);
+       if (rx_stats_active)
+           rx_atomic_inc(&rx_stats.nBusies);
+       return 1;
+    }
+
+    return 0;
+}
+
+static_inline struct rx_call *
+rxi_ReceiveClientCall(struct rx_packet *np, struct rx_connection *conn)
+{
+    int channel;
+    struct rx_call *call;
 
+    channel = np->header.cid & RX_CHANNELMASK;
     MUTEX_ENTER(&conn->conn_call_lock);
-    callNumber = *call->callNumber;
+    call = conn->call[channel];
+    if (np->header.type == RX_PACKET_TYPE_BUSY) {
+       conn->lastBusy[channel] = clock_Sec();
+    }
+    if (!call || conn->callNumber[channel] != np->header.callNumber) {
+       MUTEX_EXIT(&conn->conn_call_lock);
+       if (rx_stats_active)
+           rx_atomic_inc(&rx_stats.spuriousPacketsRead);
+       return NULL;
+    }
 
-    /* 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
-     * longer than conn->secondsUntilDead seconds before this call started. */
+    MUTEX_ENTER(&call->lock);
+    MUTEX_EXIT(&conn->conn_call_lock);
 
-    for (i = 0; i < RX_MAXCALLS && !freechannel; i++) {
-       if (i == channel) {
-           /* only look at channels that aren't us */
-           continue;
-       }
+    if ((call->state == RX_STATE_DALLY)
+       && np->header.type == RX_PACKET_TYPE_ACK) {
+       if (rx_stats_active)
+           rx_atomic_inc(&rx_stats.ignorePacketDally);
+        MUTEX_EXIT(&call->lock);
+       return NULL;
+    }
 
-       if (conn->lastBusy[i]) {
-           /* if this channel looked busy too recently, don't look at it */
-           if (conn->lastBusy[i] >= call->startTime.sec) {
-               continue;
-           }
-           if (call->startTime.sec - conn->lastBusy[i] < conn->secondsUntilDead) {
-               continue;
-           }
-       }
+    return call;
+}
 
-       if (conn->call[i]) {
-           struct rx_call *tcall = conn->call[i];
-           MUTEX_ENTER(&tcall->lock);
-           if (tcall->state == RX_STATE_DALLY) {
-               freechannel = 1;
-           }
-           MUTEX_EXIT(&tcall->lock);
-       } else {
-           freechannel = 1;
+static_inline struct rx_call *
+rxi_ReceiveServerCall(osi_socket socket, struct rx_packet *np,
+                     struct rx_connection *conn)
+{
+    int channel;
+    struct rx_call *call;
+
+    channel = np->header.cid & RX_CHANNELMASK;
+    MUTEX_ENTER(&conn->conn_call_lock);
+    call = conn->call[channel];
+
+    if (!call) {
+       if (rxi_AbortIfServerBusy(socket, conn, np)) {
+           MUTEX_EXIT(&conn->conn_call_lock);
+           return NULL;
        }
-    }
 
-    MUTEX_ENTER(&call->lock);
+       call = rxi_NewCall(conn, channel);  /* returns locked call */
+       *call->callNumber = np->header.callNumber;
+       MUTEX_EXIT(&conn->conn_call_lock);
 
-    /* Since the call->lock and conn->conn_call_lock have been released it is
-     * possible that (1) the call may no longer be busy and/or (2) the call may
-     * have been reused by another waiting thread. Therefore, we must confirm
-     * that the call state has not changed when deciding whether or not to
-     * force this application thread to retry by forcing a Timeout error. */
+       call->state = RX_STATE_PRECALL;
+       clock_GetTime(&call->queueTime);
+       call->app.bytesSent = 0;
+       call->app.bytesRcvd = 0;
+       rxi_KeepAliveOn(call);
 
-    if (freechannel && *call->callNumber == callNumber &&
-        (call->flags & RX_CALL_PEER_BUSY)) {
-       /* Since 'freechannel' is set, there exists another channel in this
-        * rx_conn that the application thread might be able to use. We know
-        * that we have the correct call since callNumber is unchanged, and we
-        * know that the call is still busy. So, set the call error state to
-        * rxi_busyChannelError so the application can retry the request,
-        * presumably on a less-busy call channel. */
+       return call;
+    }
 
-       rxi_CallError(call, RX_CALL_BUSY);
+    if (np->header.callNumber == conn->callNumber[channel]) {
+       MUTEX_ENTER(&call->lock);
+       MUTEX_EXIT(&conn->conn_call_lock);
+       return call;
     }
+
+    if (np->header.callNumber < conn->callNumber[channel]) {
+       MUTEX_EXIT(&conn->conn_call_lock);
+       if (rx_stats_active)
+           rx_atomic_inc(&rx_stats.spuriousPacketsRead);
+       return NULL;
+    }
+
+    MUTEX_ENTER(&call->lock);
     MUTEX_EXIT(&conn->conn_call_lock);
+
+    /* Wait until the transmit queue is idle before deciding
+     * whether to reset the current call. Chances are that the
+     * call will be in ether DALLY or HOLD state once the TQ_BUSY
+     * flag is cleared.
+     */
+#ifdef RX_ENABLE_LOCKS
+    if (call->state == RX_STATE_ACTIVE && !call->error) {
+       rxi_WaitforTQBusy(call);
+        /* If we entered error state while waiting,
+         * must call rxi_CallError to permit rxi_ResetCall
+         * to processed when the tqWaiter count hits zero.
+         */
+        if (call->error) {
+           rxi_CallError(call, call->error);
+           MUTEX_EXIT(&call->lock);
+            return NULL;
+        }
+    }
+#endif /* RX_ENABLE_LOCKS */
+    /* If the new call cannot be taken right now send a busy and set
+     * the error condition in this call, so that it terminates as
+     * quickly as possible */
+    if (call->state == RX_STATE_ACTIVE) {
+       rxi_CallError(call, RX_CALL_DEAD);
+       rxi_SendSpecial(call, conn, NULL, RX_PACKET_TYPE_BUSY,
+                       NULL, 0, 1);
+       MUTEX_EXIT(&call->lock);
+       return NULL;
+    }
+
+    if (rxi_AbortIfServerBusy(socket, conn, np)) {
+       MUTEX_EXIT(&call->lock);
+       return NULL;
+    }
+
+    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;
+    call->state = RX_STATE_PRECALL;
+    clock_GetTime(&call->queueTime);
+    call->app.bytesSent = 0;
+    call->app.bytesRcvd = 0;
+    rxi_KeepAliveOn(call);
+
+    return call;
 }
 
+
 /* There are two packet tracing routines available for testing and monitoring
  * Rx.  One is called just after every packet is received and the other is
  * called just before every packet is sent.  Received packets, have had their
@@ -3244,9 +3403,8 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
 {
     struct rx_call *call;
     struct rx_connection *conn;
-    int channel;
-    afs_uint32 currentCallNumber;
     int type;
+    int unknownService = 0;
 #ifdef RXDEBUG
     char *packetType;
 #endif
@@ -3259,7 +3417,7 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
  * this is the first time the packet has been seen */
     packetType = (np->header.type > 0 && np->header.type < RX_N_PACKET_TYPES)
        ? rx_packetTypes[np->header.type - 1] : "*UNKNOWN*";
-    dpf(("R %d %s: %x.%d.%d.%d.%d.%d.%d flags %d, packet %"AFS_PTR_FMT"\n",
+    dpf(("R %d %s: %x.%d.%d.%d.%d.%d.%d flags %d, packet %p\n",
         np->header.serial, packetType, ntohl(host), ntohs(port), np->header.serviceId,
         np->header.epoch, np->header.cid, np->header.callNumber,
         np->header.seq, np->header.flags, np));
@@ -3272,7 +3430,7 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
        struct rx_peer *peer;
 
        /* Try to look up the peer structure, but don't create one */
-       peer = rxi_FindPeer(host, port, 0, 0);
+       peer = rxi_FindPeer(host, port, 0);
 
        /* Since this may not be associated with a connection, it may have
         * no refCount, meaning we could race with ReapConnections
@@ -3306,9 +3464,10 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
        addr.sin_family = AF_INET;
        addr.sin_port = port;
        addr.sin_addr.s_addr = host;
+       memset(&addr.sin_zero, 0, sizeof(addr.sin_zero));
 #ifdef STRUCT_SOCKADDR_HAS_SA_LEN
        addr.sin_len = sizeof(addr);
-#endif /* AFS_OSF_ENV */
+#endif
        drop = (*rx_justReceived) (np, &addr);
        /* drop packet if return value is non-zero */
        if (drop)
@@ -3327,13 +3486,13 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
     conn =
        rxi_FindConnection(socket, host, port, np->header.serviceId,
                           np->header.cid, np->header.epoch, type,
-                          np->header.securityIndex);
+                          np->header.securityIndex, &unknownService);
 
     /* To avoid having 2 connections just abort at each other,
        don't abort an abort. */
     if (!conn) {
-        if (np->header.type != RX_PACKET_TYPE_ABORT)
-            rxi_SendRawAbort(socket, host, port, RX_INVALID_OPERATION,
+        if (unknownService && (np->header.type != RX_PACKET_TYPE_ABORT))
+           rxi_SendRawAbort(socket, host, port, 0, RX_INVALID_OPERATION,
                              np, 0);
         return np;
     }
@@ -3401,234 +3560,28 @@ 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 (type == RX_SERVER_CONNECTION)
+       call = rxi_ReceiveServerCall(socket, np, conn);
+    else
+       call = rxi_ReceiveClientCall(np, conn);
 
-    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 */
-        call = conn->call[channel];
-        if (call) {
-            MUTEX_ENTER(&call->lock);
-            currentCallNumber = conn->callNumber[channel];
-            MUTEX_EXIT(&conn->conn_call_lock);
-        } else {
-            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));
-#endif
-            call->state = RX_STATE_PRECALL;
-            clock_GetTime(&call->queueTime);
-            call->bytesSent = 0;
-            call->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);
-        }
-    } 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);
+    if (call == NULL) {
        putConnection(conn);
-        return np;
-    }
-
-    /* There is a non-NULL locked call at this point */
-    if (type == RX_SERVER_CONNECTION) {        /* We're the server */
-        if (np->header.callNumber < currentCallNumber) {
-            MUTEX_EXIT(&call->lock);
-            if (rx_stats_active)
-                rx_atomic_inc(&rx_stats.spuriousPacketsRead);
-           putConnection(conn);
-            return np;
-        } else if (np->header.callNumber != currentCallNumber) {
-           /* Wait until the transmit queue is idle before deciding
-            * whether to reset the current call. Chances are that the
-            * call will be in ether DALLY or HOLD state once the TQ_BUSY
-            * flag is cleared.
-            */
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
-            if (call->state == RX_STATE_ACTIVE) {
-                rxi_WaitforTQBusy(call);
-                /*
-                 * If we entered error state while waiting,
-                 * must call rxi_CallError to permit rxi_ResetCall
-                 * to processed when the tqWaiter count hits zero.
-                 */
-                if (call->error) {
-                    rxi_CallError(call, call->error);
-                    MUTEX_EXIT(&call->lock);
-                   putConnection(conn);
-                    return np;
-                }
-            }
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
-           /* If the new call cannot be taken right now send a busy and set
-            * the error condition in this call, so that it terminates as
-            * quickly as possible */
-           if (call->state == RX_STATE_ACTIVE) {
-               struct rx_packet *tp;
-
-               rxi_CallError(call, RX_CALL_DEAD);
-               tp = rxi_SendSpecial(call, conn, np, RX_PACKET_TYPE_BUSY,
-                                    NULL, 0, 1);
-               MUTEX_EXIT(&call->lock);
-               putConnection(conn);
-               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)
-               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);
-           call->bytesSent = 0;
-           call->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);
-       } else {
-           /* Continuing call; do nothing here. */
-       }
-    } else {                   /* we're the client */
-       /* Ignore all incoming acknowledgements for calls in DALLY state */
-       if ((call->state == RX_STATE_DALLY)
-           && (np->header.type == RX_PACKET_TYPE_ACK)) {
-            if (rx_stats_active)
-                rx_atomic_inc(&rx_stats.ignorePacketDally);
-            MUTEX_EXIT(&call->lock);
-           putConnection(conn);
-           return np;
-       }
-
-       /* Ignore anything that's not relevant to the current call.  If there
-        * isn't a current call, then no packet is relevant. */
-       if (np->header.callNumber != currentCallNumber) {
-            if (rx_stats_active)
-                rx_atomic_inc(&rx_stats.spuriousPacketsRead);
-            MUTEX_EXIT(&call->lock);
-           putConnection(conn);
-           return np;
-       }
-       /* If the service security object index stamped in the packet does not
-        * match the connection's security index, ignore the packet */
-       if (np->header.securityIndex != conn->securityIndex) {
-           MUTEX_EXIT(&call->lock);
-           putConnection(conn);
-           return np;
-       }
-
-       /* If we're receiving the response, then all transmit packets are
-        * implicitly acknowledged.  Get rid of them. */
-       if (np->header.type == RX_PACKET_TYPE_DATA) {
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
-           /* XXX Hack. Because we must release the global rx lock when
-            * sending packets (osi_NetSend) we drop all acks while we're
-            * traversing the tq in rxi_Start sending packets out because
-            * packets may move to the freePacketQueue as result of being here!
-            * So we drop these packets until we're safely out of the
-            * traversing. Really ugly!
-            * For fine grain RX locking, we set the acked field in the
-            * packets and let rxi_Start remove them from the transmit queue.
-            */
-           if (call->flags & RX_CALL_TQ_BUSY) {
-#ifdef RX_ENABLE_LOCKS
-               rxi_SetAcksInTransmitQueue(call);
-#else
-               putConnection(conn);
-               return np;      /* xmitting; drop packet */
-#endif
-           } else {
-               rxi_ClearTransmitQueue(call, 0);
-           }
-#else /* AFS_GLOBAL_RXLOCK_KERNEL */
-           rxi_ClearTransmitQueue(call, 0);
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
-       } else {
-           if (np->header.type == RX_PACKET_TYPE_ACK) {
-               /* now check to see if this is an ack packet acknowledging that the
-                * server actually *lost* some hard-acked data.  If this happens we
-                * ignore this packet, as it may indicate that the server restarted in
-                * the middle of a call.  It is also possible that this is an old ack
-                * packet.  We don't abort the connection in this case, because this
-                * *might* just be an old ack packet.  The right way to detect a server
-                * restart in the midst of a call is to notice that the server epoch
-                * changed, btw.  */
-               /* XXX I'm not sure this is exactly right, since tfirst **IS**
-                * XXX unacknowledged.  I think that this is off-by-one, but
-                * XXX I don't dare change it just yet, since it will
-                * XXX interact badly with the server-restart detection
-                * XXX code in receiveackpacket.  */
-               if (ntohl(rx_GetInt32(np, FIRSTACKOFFSET)) < call->tfirst) {
-                    if (rx_stats_active)
-                        rx_atomic_inc(&rx_stats.spuriousPacketsRead);
-                   MUTEX_EXIT(&call->lock);
-                   putConnection(conn);
-                   return np;
-               }
-           }
-       }                       /* else not a data packet */
+       return np;
     }
 
-    osirx_AssertMine(&call->lock, "rxi_ReceivePacket middle");
+    MUTEX_ASSERT(&call->lock);
     /* Set remote user defined status from packet */
     call->remoteStatus = np->header.userStatus;
 
     /* Now do packet type-specific processing */
     switch (np->header.type) {
     case RX_PACKET_TYPE_DATA:
+       /* If we're a client, and receiving a response, then all the packets
+        * we transmitted packets are implicitly acknowledged. */
+       if (type == RX_CLIENT_CONNECTION && !opr_queue_IsEmpty(&call->tq))
+           rxi_AckAllInTransmitQueue(call);
+
        np = rxi_ReceiveDataPacket(call, np, 1, socket, host, port, tnop,
                                   newcallp);
        break;
@@ -3655,49 +3608,18 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
        putConnection(conn);
        return np;              /* xmitting; drop packet */
     }
-    case RX_PACKET_TYPE_BUSY: {
-       struct clock busyTime;
-       clock_NewTime();
-       clock_GetTime(&busyTime);
-
-       MUTEX_EXIT(&call->lock);
-
-       MUTEX_ENTER(&conn->conn_call_lock);
-       MUTEX_ENTER(&call->lock);
-       conn->lastBusy[call->channel] = busyTime.sec;
-       call->flags |= RX_CALL_PEER_BUSY;
-       MUTEX_EXIT(&call->lock);
-       MUTEX_EXIT(&conn->conn_call_lock);
-
-       putConnection(conn);
-       return np;
-    }
+    case RX_PACKET_TYPE_BUSY:
+       /* Mostly ignore BUSY packets. We will update lastReceiveTime below,
+        * so we don't think the endpoint is completely dead, but otherwise
+        * just act as if we never saw anything. If all we get are BUSY packets
+        * back, then we will eventually error out with RX_CALL_TIMEOUT if the
+        * connection is configured with idle/hard timeouts. */
+       break;
 
     case RX_PACKET_TYPE_ACKALL:
        /* All packets acknowledged, so we can drop all packets previously
         * readied for sending */
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
-       /* XXX Hack. We because we can't release the global rx lock when
-        * sending packets (osi_NetSend) we drop all ack pkts while we're
-        * traversing the tq in rxi_Start sending packets out because
-        * packets may move to the freePacketQueue as result of being
-        * here! So we drop these packets until we're safely out of the
-        * traversing. Really ugly!
-        * For fine grain RX locking, we set the acked field in the packets
-        * and let rxi_Start remove the packets from the transmit queue.
-        */
-       if (call->flags & RX_CALL_TQ_BUSY) {
-#ifdef RX_ENABLE_LOCKS
-           rxi_SetAcksInTransmitQueue(call);
-           break;
-#else /* RX_ENABLE_LOCKS */
-           MUTEX_EXIT(&call->lock);
-           putConnection(conn);
-           return np;          /* xmitting; drop packet */
-#endif /* RX_ENABLE_LOCKS */
-       }
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
-       rxi_ClearTransmitQueue(call, 0);
+       rxi_AckAllInTransmitQueue(call);
        break;
     default:
        /* Should not reach here, unless the peer is broken: send an abort
@@ -3711,8 +3633,6 @@ rxi_ReceivePacket(struct rx_packet *np, osi_socket socket,
      * the packet will be delivered to the user before any get time is required
      * (if not, then the time won't actually be re-evaluated here). */
     call->lastReceiveTime = clock_Sec();
-    /* we've received a legit packet, so the channel is not busy */
-    call->flags &= ~RX_CALL_PEER_BUSY;
     MUTEX_EXIT(&call->lock);
     putConnection(conn);
     return np;
@@ -3735,8 +3655,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;
        }
     }
@@ -3796,6 +3716,14 @@ rxi_ConnClearAttachWait(struct rx_connection *conn)
     }
 }
 
+/*
+ * Event handler function for connection-specific events for checking
+ * reachability.  Also called directly from main code with |event| == NULL
+ * in order to trigger the initial reachability check.
+ *
+ * When |event| == NULL, must be called with the connection data lock held,
+ * but returns with the lock unlocked.
+ */
 static void
 rxi_CheckReachEvent(struct rxevent *event, void *arg1, void *arg2, int dummy)
 {
@@ -3805,17 +3733,14 @@ rxi_CheckReachEvent(struct rxevent *event, void *arg1, void *arg2, int dummy)
     struct clock when, now;
     int i, waiting;
 
-    MUTEX_ENTER(&conn->conn_data_lock);
-
-    if (event) {
-       rxevent_Put(conn->checkReachEvent);
-       conn->checkReachEvent = NULL;
-    }
+    if (event != NULL)
+       MUTEX_ENTER(&conn->conn_data_lock);
+    else
+       MUTEX_ASSERT(&conn->conn_data_lock);
 
+    if (event != NULL && event == conn->checkReachEvent)
+       rxevent_Put(&conn->checkReachEvent);
     waiting = conn->flags & RX_CONN_ATTACHWAIT;
-    if (event) {
-       putConnection(conn);
-    }
     MUTEX_EXIT(&conn->conn_data_lock);
 
     if (waiting) {
@@ -3847,9 +3772,7 @@ rxi_CheckReachEvent(struct rxevent *event, void *arg1, void *arg2, int dummy)
            when.sec += RX_CHECKREACH_TIMEOUT;
            MUTEX_ENTER(&conn->conn_data_lock);
            if (!conn->checkReachEvent) {
-                MUTEX_ENTER(&rx_refcnt_mutex);
-               conn->refCount++;
-                MUTEX_EXIT(&rx_refcnt_mutex);
+                rx_GetConnection(conn);
                conn->checkReachEvent = rxevent_Post(&when, &now,
                                                     rxi_CheckReachEvent, conn,
                                                     NULL, 0);
@@ -3857,6 +3780,9 @@ rxi_CheckReachEvent(struct rxevent *event, void *arg1, void *arg2, int dummy)
            MUTEX_EXIT(&conn->conn_data_lock);
        }
     }
+    /* If fired as an event handler, drop our refcount on the connection. */
+    if (event != NULL)
+       putConnection(conn);
 }
 
 static int
@@ -3882,9 +3808,12 @@ rxi_CheckConnReach(struct rx_connection *conn, struct rx_call *call)
        return 1;
     }
     conn->flags |= RX_CONN_ATTACHWAIT;
-    MUTEX_EXIT(&conn->conn_data_lock);
-    if (!conn->checkReachEvent)
+    if (conn->checkReachEvent == NULL) {
+       /* rxi_CheckReachEvent(NULL, ...) will drop the lock. */
        rxi_CheckReachEvent(NULL, conn, call, 0);
+    } else {
+       MUTEX_EXIT(&conn->conn_data_lock);
+    }
 
     return 1;
 }
@@ -3893,7 +3822,7 @@ rxi_CheckConnReach(struct rx_connection *conn, struct rx_call *call)
 static void
 TryAttach(struct rx_call *acall, osi_socket socket,
          int *tnop, struct rx_call **newcallp,
-         int reachOverride)
+         int reachOverride, int istack)
 {
     struct rx_connection *conn = acall->conn;
 
@@ -3907,7 +3836,19 @@ TryAttach(struct rx_call *acall, osi_socket socket,
             * may not any proc available
             */
        } else {
-           rxi_ChallengeOn(acall->conn);
+           int code;
+           code = rxi_ChallengeOn(acall->conn);
+           if (code) {
+               /*
+                * Ideally we would rxi_ConnectionError here, but doing that is
+                * difficult, because some callers may have locked 'call',
+                * _and_ another call on the same conn. So we cannot
+                * rxi_ConnectionError, since that needs to lock every call on
+                * the conn. But we can at least abort the call we have.
+                */
+               rxi_CallError(acall, code);
+               rxi_SendCallAbort(acall, NULL, istack, 0);
+           }
        }
     }
 }
@@ -3944,7 +3885,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
         if (rx_stats_active)
             rx_atomic_inc(&rx_stats.noPacketBuffersOnRead);
        rxi_calltrace(RX_TRACE_DROP, call);
-       dpf(("packet %"AFS_PTR_FMT" dropped on receipt - quota problems\n", np));
+       dpf(("packet %p 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 */
@@ -3998,9 +3939,8 @@ rxi_ReceiveDataPacket(struct rx_call *call,
                && 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));
-               rxevent_Cancel(&call->delayedAckEvent, call,
-                              RX_CALL_REFCOUNT_DELAY);
+               dpf(("packet %p dropped on receipt - duplicate\n", np));
+               rxi_CancelDelayedAckEvent(call);
                np = rxi_SendAck(call, np, serial, RX_ACK_DUPLICATE, istack);
                ackNeeded = 0;
                call->rprev = seq;
@@ -4057,7 +3997,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
            if (call->arrivalProc) {
                (*call->arrivalProc) (call, call->arrivalProcHandle,
                                      call->arrivalProcArg);
-               call->arrivalProc = (void (*)())0;
+               call->arrivalProc = NULL;
            }
 
            /* Update last packet received */
@@ -4068,7 +4008,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
             * server thread is available, this thread becomes a server
             * thread and the server thread becomes a listener thread. */
            if (isFirst) {
-               TryAttach(call, socket, tnop, newcallp, 0);
+               TryAttach(call, socket, tnop, newcallp, 0, istack);
            }
        }
        /* This is not the expected next packet. */
@@ -4090,8 +4030,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
            if (seq < call->rnext) {
                 if (rx_stats_active)
                     rx_atomic_inc(&rx_stats.dupPacketsRead);
-               rxevent_Cancel(&call->delayedAckEvent, call,
-                              RX_CALL_REFCOUNT_DELAY);
+               rxi_CancelDelayedAckEvent(call);
                np = rxi_SendAck(call, np, serial, RX_ACK_DUPLICATE, istack);
                ackNeeded = 0;
                call->rprev = seq;
@@ -4102,8 +4041,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
             * accomodated by the current window, then send a negative
             * acknowledge and drop the packet */
            if ((call->rnext + call->rwind) <= seq) {
-               rxevent_Cancel(&call->delayedAckEvent, call,
-                              RX_CALL_REFCOUNT_DELAY);
+               rxi_CancelDelayedAckEvent(call);
                np = rxi_SendAck(call, np, serial, RX_ACK_EXCEEDS_WINDOW,
                                 istack);
                ackNeeded = 0;
@@ -4122,8 +4060,7 @@ rxi_ReceiveDataPacket(struct rx_call *call,
                if (seq == tp->header.seq) {
                     if (rx_stats_active)
                         rx_atomic_inc(&rx_stats.dupPacketsRead);
-                   rxevent_Cancel(&call->delayedAckEvent, call,
-                                  RX_CALL_REFCOUNT_DELAY);
+                   rxi_CancelDelayedAckEvent(call);
                    np = rxi_SendAck(call, np, serial, RX_ACK_DUPLICATE,
                                     istack);
                    ackNeeded = 0;
@@ -4234,10 +4171,10 @@ rxi_ReceiveDataPacket(struct rx_call *call,
      * received. Always send a soft ack for the last packet in
      * the server's reply. */
     if (ackNeeded) {
-       rxevent_Cancel(&call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
+       rxi_CancelDelayedAckEvent(call);
        np = rxi_SendAck(call, np, serial, ackNeeded, istack);
     } else if (call->nSoftAcks > (u_short) rxi_SoftAckRate) {
-       rxevent_Cancel(&call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
+       rxi_CancelDelayedAckEvent(call);
        np = rxi_SendAck(call, np, serial, RX_ACK_IDLE, istack);
     } else if (call->nSoftAcks) {
        if (haveLast && !(flags & RX_CLIENT_INITIATED))
@@ -4245,14 +4182,15 @@ rxi_ReceiveDataPacket(struct rx_call *call,
        else
            rxi_PostDelayedAckEvent(call, &rx_softAckDelay);
     } else if (call->flags & RX_CALL_RECEIVE_DONE) {
-       rxevent_Cancel(&call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
+       rxi_CancelDelayedAckEvent(call);
     }
 
     return np;
 }
 
 static void
-rxi_UpdatePeerReach(struct rx_connection *conn, struct rx_call *acall)
+rxi_UpdatePeerReach(struct rx_connection *conn, struct rx_call *acall,
+                   int istack)
 {
     struct rx_peer *peer = conn->peer;
 
@@ -4273,7 +4211,7 @@ rxi_UpdatePeerReach(struct rx_connection *conn, struct rx_call *acall)
                if (call != acall)
                    MUTEX_ENTER(&call->lock);
                /* tnop can be null if newcallp is null */
-               TryAttach(call, (osi_socket) - 1, NULL, NULL, 1);
+               TryAttach(call, (osi_socket) - 1, NULL, NULL, 1, istack);
                if (call != acall)
                    MUTEX_EXIT(&call->lock);
            }
@@ -4367,7 +4305,7 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
     }
 
     if (ap->reason == RX_ACK_PING_RESPONSE)
-       rxi_UpdatePeerReach(conn, call);
+       rxi_UpdatePeerReach(conn, call, istack);
 
     if (conn->lastPacketSizeSeq) {
        MUTEX_ENTER(&conn->conn_data_lock);
@@ -4440,12 +4378,12 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
         * but we are clearly receiving.
         */
        if (!peer->maxPacketSize)
-           peer->maxPacketSize = RX_MIN_PACKET_SIZE+RX_IPUDP_SIZE;
+           peer->maxPacketSize = RX_MIN_PACKET_SIZE - RX_HEADER_SIZE;
 
        if (pktsize > peer->maxPacketSize) {
            peer->maxPacketSize = pktsize;
-           if ((pktsize-RX_IPUDP_SIZE > peer->ifMTU)) {
-               peer->ifMTU=pktsize-RX_IPUDP_SIZE;
+           if ((pktsize + RX_HEADER_SIZE > peer->ifMTU)) {
+               peer->ifMTU = pktsize + RX_HEADER_SIZE;
                peer->natMTU = rxi_AdjustIfMTU(peer->ifMTU);
                rxi_ScheduleGrowMTUEvent(call, 1);
            }
@@ -4490,9 +4428,9 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
            rxi_ComputeRoundTripTime(tp, ap, call, peer, &now);
        }
 
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
-       /* XXX Hack. Because we have to release the global rx lock when sending
-        * packets (osi_NetSend) we drop all acks while we're traversing the tq
+#ifdef RX_ENABLE_LOCKS
+       /* XXX Hack. Because we have to release the global call lock when sending
+        * packets (rxi_NetSend) we drop all acks while we're traversing the tq
         * in rxi_Start sending packets out because packets may move to the
         * freePacketQueue as result of being here! So we drop these packets until
         * we're safely out of the traversing. Really ugly!
@@ -4501,14 +4439,10 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
         * when it's done transmitting.
         */
        if (call->flags & RX_CALL_TQ_BUSY) {
-#ifdef RX_ENABLE_LOCKS
            tp->flags |= RX_PKTFLAG_ACKED;
            call->flags |= RX_CALL_TQ_SOME_ACKED;
-#else /* RX_ENABLE_LOCKS */
-           break;
-#endif /* RX_ENABLE_LOCKS */
        } else
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
        {
            opr_queue_Remove(&tp->entry);
 #ifdef RX_TRACK_PACKETS
@@ -4566,25 +4500,7 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
 
     /* We don't need to take any action with the 3rd or 4th section in the
      * queue - they're not addressed by the contents of this ACK packet.
-     */
-
-    /* If the window has been extended by this acknowledge packet,
-     * then wakeup a sender waiting in alloc for window space, or try
-     * sending packets now, if he's been sitting on packets due to
-     * lack of window space */
-    if (call->tnext < (call->tfirst + call->twind)) {
-#ifdef RX_ENABLE_LOCKS
-       CV_SIGNAL(&call->cv_twind);
-#else
-       if (call->flags & RX_CALL_WAIT_WINDOW_ALLOC) {
-           call->flags &= ~RX_CALL_WAIT_WINDOW_ALLOC;
-           osi_rxWakeup(&call->twind);
-       }
-#endif
-       if (call->flags & RX_CALL_WAIT_WINDOW_SEND) {
-           call->flags &= ~RX_CALL_WAIT_WINDOW_SEND;
-       }
-    }
+     */
 
     /* if the ack packet has a receivelen field hanging off it,
      * update our state */
@@ -4596,12 +4512,20 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        rx_packetread(np, rx_AckDataSize(ap->nAcks) + (int)sizeof(afs_int32),
                      (int)sizeof(afs_int32), &tSize);
        tSize = (afs_uint32) ntohl(tSize);
+       if (tSize > RX_MAX_PACKET_SIZE)
+           tSize = RX_MAX_PACKET_SIZE;
+       if (tSize < RX_MIN_PACKET_SIZE)
+           tSize = RX_MIN_PACKET_SIZE;
        peer->natMTU = rxi_AdjustIfMTU(MIN(tSize, peer->ifMTU));
 
        /* Get the maximum packet size to send to this peer */
        rx_packetread(np, rx_AckDataSize(ap->nAcks), (int)sizeof(afs_int32),
                      &tSize);
        tSize = (afs_uint32) ntohl(tSize);
+       if (tSize > RX_MAX_PACKET_SIZE)
+           tSize = RX_MAX_PACKET_SIZE;
+       if (tSize < RX_MIN_PACKET_SIZE)
+           tSize = RX_MIN_PACKET_SIZE;
        tSize = (afs_uint32) MIN(tSize, rx_MyMaxSendSize);
        tSize = rxi_AdjustMaxMTU(peer->natMTU, tSize);
 
@@ -4623,6 +4547,10 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
                          rx_AckDataSize(ap->nAcks) + 2 * (int)sizeof(afs_int32),
                          (int)sizeof(afs_int32), &tSize);
            tSize = (afs_uint32) ntohl(tSize);  /* peer's receive window, if it's */
+           if (tSize == 0)
+               tSize = 1;
+           if (tSize >= rx_maxSendWindow)
+               tSize = rx_maxSendWindow;
            if (tSize < call->twind) {  /* smaller than our send */
                call->twind = tSize;    /* window, we must send less... */
                call->ssthresh = MIN(call->twind, call->ssthresh);
@@ -4644,6 +4572,10 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
                          rx_AckDataSize(ap->nAcks) + 2 * (int)sizeof(afs_int32),
                          sizeof(afs_int32), &tSize);
            tSize = (afs_uint32) ntohl(tSize);
+           if (tSize == 0)
+               tSize = 1;
+           if (tSize >= rx_maxSendWindow)
+               tSize = rx_maxSendWindow;
            /*
             * As of AFS 3.5 we set the send window to match the receive window.
             */
@@ -4692,6 +4624,24 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        call->MTU = OLD_MAX_PACKET_SIZE;
     }
 
+    /* If the window has been extended by this acknowledge packet,
+     * then wakeup a sender waiting in alloc for window space, or try
+     * sending packets now, if he's been sitting on packets due to
+     * lack of window space */
+    if (call->tnext < (call->tfirst + call->twind)) {
+#ifdef RX_ENABLE_LOCKS
+       CV_SIGNAL(&call->cv_twind);
+#else
+       if (call->flags & RX_CALL_WAIT_WINDOW_ALLOC) {
+           call->flags &= ~RX_CALL_WAIT_WINDOW_ALLOC;
+           osi_rxWakeup(&call->twind);
+       }
+#endif
+       if (call->flags & RX_CALL_WAIT_WINDOW_SEND) {
+           call->flags &= ~RX_CALL_WAIT_WINDOW_SEND;
+       }
+    }
+
     if (nNacked) {
        /*
         * Calculate how many datagrams were successfully received after
@@ -4808,13 +4758,40 @@ rxi_ReceiveAckPacket(struct rx_call *call, struct rx_packet *np,
        && call->tfirst + call->nSoftAcked >= call->tnext) {
        call->state = RX_STATE_DALLY;
        rxi_ClearTransmitQueue(call, 0);
-        rxevent_Cancel(&call->keepAliveEvent, call, RX_CALL_REFCOUNT_ALIVE);
+       rxi_CancelKeepAliveEvent(call);
     } else if (!opr_queue_IsEmpty(&call->tq)) {
        rxi_Start(call, istack);
     }
     return np;
 }
 
+/**
+ * Schedule a connection abort to be sent after some delay.
+ *
+ * @param[in] conn The connection to send the abort on.
+ * @param[in] msec The number of milliseconds to wait before sending.
+ *
+ * @pre conn_data_lock must be held
+ */
+static void
+rxi_SendConnectionAbortLater(struct rx_connection *conn, int msec)
+{
+    struct clock when, now;
+
+    MUTEX_ASSERT(&conn->conn_data_lock);
+    if (!conn->error) {
+       return;
+    }
+    if (!conn->delayedAbortEvent) {
+       clock_GetTime(&now);
+       when = now;
+       clock_Addmsec(&when, msec);
+       rx_GetConnection(conn);
+       conn->delayedAbortEvent =
+           rxevent_Post(&when, &now, rxi_SendDelayedConnAbort, conn, NULL, 0);
+    }
+}
+
 /* Received a response to a challenge packet */
 static struct rx_packet *
 rxi_ReceiveResponsePacket(struct rx_connection *conn,
@@ -4830,17 +4807,26 @@ rxi_ReceiveResponsePacket(struct rx_connection *conn,
     if (RXS_CheckAuthentication(conn->securityObject, conn) == 0)
        return np;
 
+    if (!conn->securityChallengeSent) {
+       /* We've never sent out a challenge for this connection, so this
+        * response cannot possibly be correct; ignore it. This can happen
+        * if we sent a challenge to the client, then we were restarted, and
+        * then the client sent us a response. If we ignore the response, the
+        * client will eventually resend a data packet, causing us to send a
+        * new challenge and the client to send a new response. */
+       return np;
+    }
+
     /* Otherwise, have the security object evaluate the response packet */
     error = RXS_CheckResponse(conn->securityObject, conn, np);
     if (error) {
        /* If the response is invalid, reset the connection, sending
-        * an abort to the peer */
-#ifndef KERNEL
-       rxi_Delay(1);
-#endif
+        * an abort to the peer. Send the abort with a 1 second delay,
+        * to avoid a peer hammering us by constantly recreating a
+        * connection with bad credentials. */
        rxi_ConnectionError(conn, error);
        MUTEX_ENTER(&conn->conn_data_lock);
-       np = rxi_SendConnectionAbort(conn, np, istack, 0);
+       rxi_SendConnectionAbortLater(conn, 1000);
        MUTEX_EXIT(&conn->conn_data_lock);
        return np;
     } else {
@@ -4863,7 +4849,7 @@ rxi_ReceiveResponsePacket(struct rx_connection *conn,
         * some calls went into attach-wait while we were waiting
         * for authentication..
         */
-       rxi_UpdatePeerReach(conn, NULL);
+       rxi_UpdatePeerReach(conn, NULL, istack);
     }
     return np;
 }
@@ -4971,9 +4957,10 @@ rxi_AttachServerProc(struct rx_call *call,
            if (opr_queue_IsOnQueue(&call->entry)) {
                opr_queue_Remove(&call->entry);
            }
+           CLEAR_CALL_QUEUE_LOCK(call);
        }
        call->state = RX_STATE_ACTIVE;
-       call->mode = RX_MODE_RECEIVING;
+       call->app.mode = RX_MODE_RECEIVING;
 #ifdef RX_KERNEL_TRACE
        {
            int glockOwner = ISAFS_GLOCK();
@@ -5018,6 +5005,11 @@ rxi_AckAll(struct rx_call *call)
     call->flags |= RX_CALL_ACKALL_SENT;
 }
 
+/*
+ * Event handler for per-call delayed acks.
+ * Also called synchronously, with |event| == NULL, to send a "delayed" ack
+ * immediately.
+ */
 static void
 rxi_SendDelayedAck(struct rxevent *event, void *arg1, void *unused1,
                   int unused2)
@@ -5026,25 +5018,22 @@ rxi_SendDelayedAck(struct rxevent *event, void *arg1, void *unused1,
 #ifdef RX_ENABLE_LOCKS
     if (event) {
        MUTEX_ENTER(&call->lock);
-       if (event == call->delayedAckEvent) {
-           rxevent_Put(call->delayedAckEvent);
-           call->delayedAckEvent = NULL;
-       }
-       CALL_RELE(call, RX_CALL_REFCOUNT_DELAY);
+       if (event == call->delayedAckEvent)
+           rxevent_Put(&call->delayedAckEvent);
     }
     (void)rxi_SendAck(call, 0, 0, RX_ACK_DELAY, 0);
     if (event)
        MUTEX_EXIT(&call->lock);
 #else /* RX_ENABLE_LOCKS */
-    if (event) {
-       rxevent_Put(call->delayedAckEvent);
-       call->delayedAckEvent = NULL;
-    }
+    if (event)
+       rxevent_Put(&call->delayedAckEvent);
     (void)rxi_SendAck(call, 0, 0, RX_ACK_DELAY, 0);
 #endif /* RX_ENABLE_LOCKS */
+    /* Release the call reference for the event that fired. */
+    if (event)
+       CALL_RELE(call, RX_CALL_REFCOUNT_DELAY);
 }
 
-
 #ifdef RX_ENABLE_LOCKS
 /* Set ack in all packets in transmit queue. rxi_Start will deal with
  * clearing them out.
@@ -5083,12 +5072,30 @@ rxi_SetAcksInTransmitQueue(struct rx_call *call)
 }
 #endif /* RX_ENABLE_LOCKS */
 
+/*!
+ * Acknowledge the whole transmit queue.
+ *
+ * If we're running without locks, or the transmit queue isn't busy, then
+ * we can just clear the queue now. Otherwise, we have to mark all of the
+ * packets as acknowledged, and let rxi_Start clear it later on
+ */
+static void
+rxi_AckAllInTransmitQueue(struct rx_call *call)
+{
+#ifdef RX_ENABLE_LOCKS
+    if (call->flags & RX_CALL_TQ_BUSY) {
+       rxi_SetAcksInTransmitQueue(call);
+       return;
+    }
+#endif
+    rxi_ClearTransmitQueue(call, 0);
+}
 /* Clear out the transmit queue for the current call (all packets have
  * been received by peer) */
 static void
 rxi_ClearTransmitQueue(struct rx_call *call, int force)
 {
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
     struct opr_queue *cursor;
     if (!force && (call->flags & RX_CALL_TQ_BUSY)) {
        int someAcked = 0;
@@ -5104,16 +5111,16 @@ rxi_ClearTransmitQueue(struct rx_call *call, int force)
            call->flags |= RX_CALL_TQ_SOME_ACKED;
        }
     } else {
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
 #ifdef RXDEBUG_PACKET
         call->tqc -=
 #endif /* RXDEBUG_PACKET */
             rxi_FreePackets(0, &call->tq);
        rxi_WakeUpTransmitQueue(call);
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
        call->flags &= ~RX_CALL_TQ_CLEARME;
     }
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif
 
     rxi_rto_cancel(call);
     call->tfirst = call->tnext;        /* implicitly acknowledge all data already sent */
@@ -5141,7 +5148,7 @@ rxi_ClearReceiveQueue(struct rx_call *call)
 #ifdef RXDEBUG_PACKET
         call->rqc -= count;
         if ( call->rqc != 0 )
-            dpf(("rxi_ClearReceiveQueue call %"AFS_PTR_FMT" rqc %u != 0\n", call, call->rqc));
+         dpf(("rxi_ClearReceiveQueue call %p rqc %u != 0\n", call, call->rqc));
 #endif
        call->flags &= ~(RX_CALL_RECEIVE_DONE | RX_CALL_HAVE_LAST);
     }
@@ -5155,38 +5162,34 @@ static struct rx_packet *
 rxi_SendCallAbort(struct rx_call *call, struct rx_packet *packet,
                  int istack, int force)
 {
-    afs_int32 error, cerror;
+    afs_int32 error;
     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 != cerror) {
-       call->abortCode = cerror;
+    /*
+     * An opcode that has been deprecated or has yet to be implemented is not
+     * a misbehavior of the client.  Do not punish the client by introducing
+     * delays.
+     */
+    if (call->error == RXGEN_OPCODE) {
+       force = 1;
+    } else if (call->abortCode != call->error) {
+       call->abortCode = call->error;
        call->abortCount = 0;
     }
 
     if (force || rxi_callAbortThreshhold == 0
        || call->abortCount < rxi_callAbortThreshhold) {
-       if (call->delayedAbortEvent) {
-           rxevent_Cancel(&call->delayedAbortEvent, call,
-                          RX_CALL_REFCOUNT_ABORT);
-       }
-       error = htonl(cerror);
-       call->abortCount++;
+       rxi_CancelDelayedAbortEvent(call);
+       error = htonl(call->error);
+       if (!force)
+           call->abortCount++;
        packet =
            rxi_SendSpecial(call, call->conn, packet, RX_PACKET_TYPE_ABORT,
                            (char *)&error, sizeof(error), istack);
@@ -5201,6 +5204,14 @@ rxi_SendCallAbort(struct rx_call *call, struct rx_packet *packet,
     return packet;
 }
 
+static void
+rxi_CancelDelayedAbortEvent(struct rx_call *call)
+{
+    MUTEX_ASSERT(&call->lock);
+    if (rxevent_Cancel(&call->delayedAbortEvent))
+       CALL_RELE(call, RX_CALL_REFCOUNT_ABORT);
+}
+
 /* Send an abort packet for the specified connection.  Packet is an
  * optional pointer to a packet that can be used to send the abort.
  * Once the number of abort messages reaches the threshhold, an
@@ -5215,7 +5226,6 @@ rxi_SendConnectionAbort(struct rx_connection *conn,
                        struct rx_packet *packet, int istack, int force)
 {
     afs_int32 error;
-    struct clock when, now;
 
     if (!conn->error)
        return packet;
@@ -5227,7 +5237,8 @@ rxi_SendConnectionAbort(struct rx_connection *conn,
     if (force || rxi_connAbortThreshhold == 0
        || conn->abortCount < rxi_connAbortThreshhold) {
 
-       rxevent_Cancel(&conn->delayedAbortEvent, NULL, 0);
+       if (rxevent_Cancel(&conn->delayedAbortEvent))
+           putConnection(conn);
        error = htonl(conn->error);
        conn->abortCount++;
        MUTEX_EXIT(&conn->conn_data_lock);
@@ -5236,12 +5247,8 @@ rxi_SendConnectionAbort(struct rx_connection *conn,
                            RX_PACKET_TYPE_ABORT, (char *)&error,
                            sizeof(error), istack);
        MUTEX_ENTER(&conn->conn_data_lock);
-    } else if (!conn->delayedAbortEvent) {
-       clock_GetTime(&now);
-       when = now;
-       clock_Addmsec(&when, rxi_connAbortDelay);
-       conn->delayedAbortEvent =
-           rxevent_Post(&when, &now, rxi_SendDelayedConnAbort, conn, NULL, 0);
+    } else {
+       rxi_SendConnectionAbortLater(conn, rxi_connAbortDelay);
     }
     return packet;
 }
@@ -5258,13 +5265,14 @@ rxi_ConnectionError(struct rx_connection *conn,
     if (error) {
        int i;
 
-       dpf(("rxi_ConnectionError conn %"AFS_PTR_FMT" error %d\n", conn, error));
+       dpf(("rxi_ConnectionError conn %p error %d\n", conn, error));
 
        MUTEX_ENTER(&conn->conn_data_lock);
-       rxevent_Cancel(&conn->challengeEvent, NULL, 0);
-       rxevent_Cancel(&conn->natKeepAliveEvent, NULL, 0);
-       if (conn->checkReachEvent) {
-           rxevent_Cancel(&conn->checkReachEvent, NULL, 0);
+       if (rxevent_Cancel(&conn->challengeEvent))
+           putConnection(conn);
+       if (rxevent_Cancel(&conn->natKeepAliveEvent))
+           putConnection(conn);
+       if (rxevent_Cancel(&conn->checkReachEvent)) {
            conn->flags &= ~(RX_CONN_ATTACHWAIT|RX_CONN_NAT_PING);
            putConnection(conn);
        }
@@ -5301,14 +5309,12 @@ rx_InterruptCall(struct rx_call *call, afs_int32 error)
 void
 rxi_CallError(struct rx_call *call, afs_int32 error)
 {
-#ifdef DEBUG
-    osirx_AssertMine(&call->lock, "rxi_CallError");
-#endif
-    dpf(("rxi_CallError call %"AFS_PTR_FMT" error %d call->error %d\n", call, error, call->error));
+    MUTEX_ASSERT(&call->lock);
+    dpf(("rxi_CallError call %p error %d call->error %d\n", call, error, call->error));
     if (call->error)
        error = call->error;
 
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
     if (!((call->flags & RX_CALL_TQ_BUSY) || (call->tqWaiters > 0))) {
        rxi_ResetCall(call, 0);
     }
@@ -5331,23 +5337,22 @@ rxi_ResetCall(struct rx_call *call, int newcall)
     int flags;
     struct rx_peer *peer;
     struct rx_packet *packet;
-#ifdef DEBUG
-    osirx_AssertMine(&call->lock, "rxi_ResetCall");
-#endif
-    dpf(("rxi_ResetCall(call %"AFS_PTR_FMT", newcall %d)\n", call, newcall));
+
+    MUTEX_ASSERT(&call->lock);
+    dpf(("rxi_ResetCall(call %p, newcall %d)\n", call, newcall));
 
     /* Notify anyone who is waiting for asynchronous packet arrival */
     if (call->arrivalProc) {
        (*call->arrivalProc) (call, call->arrivalProcHandle,
                              call->arrivalProcArg);
-       call->arrivalProc = (void (*)())0;
+       call->arrivalProc = NULL;
     }
 
 
-    rxevent_Cancel(&call->growMTUEvent, call, RX_CALL_REFCOUNT_MTU);
+    rxi_CancelGrowMTUEvent(call);
 
     if (call->delayedAbortEvent) {
-       rxevent_Cancel(&call->delayedAbortEvent, call, RX_CALL_REFCOUNT_ABORT);
+       rxi_CancelDelayedAbortEvent(call);
        packet = rxi_AllocPacket(RX_PACKET_CLASS_SPECIAL);
        if (packet) {
            rxi_SendCallAbort(call, packet, 0, 1);
@@ -5391,26 +5396,14 @@ rxi_ResetCall(struct rx_call *call, int newcall)
     MUTEX_EXIT(&peer->peer_lock);
 
     flags = call->flags;
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
     rxi_WaitforTQBusy(call);
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
 
     rxi_ClearTransmitQueue(call, 1);
     if (call->tqWaiters || (flags & RX_CALL_TQ_WAIT)) {
-        dpf(("rcall %"AFS_PTR_FMT" has %d waiters and flags %d\n", call, call->tqWaiters, call->flags));
+       dpf(("rcall %p has %d waiters and flags %d\n", call, call->tqWaiters, call->flags));
     }
     call->flags = 0;
 
-    if (!newcall && (flags & RX_CALL_PEER_BUSY)) {
-       /* The call channel is still busy; resetting the call doesn't change
-        * 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;
-    }
-
     rxi_ClearReceiveQueue(call);
     /* why init the queue if you just emptied it? queue_Init(&call->rq); */
 
@@ -5478,8 +5471,8 @@ rxi_ResetCall(struct rx_call *call, int newcall)
     }
 #endif /* RX_ENABLE_LOCKS */
 
-    rxi_KeepAliveOff(call);
-    rxevent_Cancel(&call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
+    rxi_CancelKeepAliveEvent(call);
+    rxi_CancelDelayedAckEvent(call);
 }
 
 /* Send an acknowledge for the indicated packet (seq,serial) of the
@@ -5506,6 +5499,9 @@ rxi_ResetCall(struct rx_call *call, int newcall)
     int        reason;                  Reason an acknowledge was prompted
 */
 
+#define RX_ZEROS 1024
+static char rx_zeros[RX_ZEROS];
+
 struct rx_packet *
 rxi_SendAck(struct rx_call *call,
            struct rx_packet *optionalPacket, int serial, int reason,
@@ -5536,7 +5532,7 @@ rxi_SendAck(struct rx_call *call,
         */
        if (call->conn->peer->maxPacketSize &&
            (call->conn->peer->maxPacketSize < OLD_MAX_PACKET_SIZE
-            +RX_IPUDP_SIZE))
+            - RX_HEADER_SIZE))
            padbytes = call->conn->peer->maxPacketSize+16;
        else
            padbytes = call->conn->peer->maxMTU + 128;
@@ -5665,6 +5661,11 @@ rxi_SendAck(struct rx_call *call,
     ap->nAcks = offset;
     p->length = rx_AckDataSize(offset) + 4 * sizeof(afs_int32);
 
+    /* Must zero the 3 octets that rx_AckDataSize skips at the end of the
+     * ACK list.
+     */
+    rx_packetwrite(p, rx_AckDataSize(offset) - 3, 3, rx_zeros);
+
     /* these are new for AFS 3.3 */
     templ = rxi_AdjustMaxMTU(call->conn->peer->ifMTU, rx_maxReceiveSize);
     templ = htonl(templ);
@@ -5683,6 +5684,8 @@ rxi_SendAck(struct rx_call *call,
     rx_packetwrite(p, rx_AckDataSize(offset) + 3 * sizeof(afs_int32),
                   sizeof(afs_int32), &templ);
 
+    p->length = rx_AckDataSize(offset) + 4 * sizeof(afs_int32);
+
     p->header.serviceId = call->conn->serviceId;
     p->header.cid = (call->conn->cid | call->channel);
     p->header.callNumber = *call->callNumber;
@@ -5691,21 +5694,21 @@ rxi_SendAck(struct rx_call *call,
     p->header.epoch = call->conn->epoch;
     p->header.type = RX_PACKET_TYPE_ACK;
     p->header.flags = RX_SLOW_START_OK;
-    if (reason == RX_ACK_PING) {
+    if (reason == RX_ACK_PING)
        p->header.flags |= RX_REQUEST_ACK;
-       if (padbytes) {
-           p->length = padbytes +
-               rx_AckDataSize(call->rwind) + 4 * sizeof(afs_int32);
 
-           while (padbytes--)
-               /* not fast but we can potentially use this if truncated
-                * fragments are delivered to figure out the mtu.
-                */
-               rx_packetwrite(p, rx_AckDataSize(offset) + 4 *
-                              sizeof(afs_int32), sizeof(afs_int32),
-                              &padbytes);
+    while (padbytes > 0) {
+       if (padbytes > RX_ZEROS) {
+           rx_packetwrite(p, p->length, RX_ZEROS, rx_zeros);
+           p->length += RX_ZEROS;
+           padbytes -= RX_ZEROS;
+       } else {
+           rx_packetwrite(p, p->length, padbytes, rx_zeros);
+           p->length += padbytes;
+           padbytes = 0;
        }
     }
+
     if (call->conn->type == RX_CLIENT_CONNECTION)
        p->header.flags |= RX_CLIENT_INITIATED;
 
@@ -5846,7 +5849,7 @@ rxi_SendList(struct rx_call *call, struct xmitlist *xmit,
 
     /* Since we're about to send a data packet to the peer, it's
      * safe to nuke any scheduled end-of-packets ack */
-    rxevent_Cancel(&call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
+    rxi_CancelDelayedAckEvent(call);
 
     MUTEX_EXIT(&call->lock);
     CALL_HOLD(call, RX_CALL_REFCOUNT_SEND);
@@ -5866,9 +5869,6 @@ rxi_SendList(struct rx_call *call, struct xmitlist *xmit,
      * processing), and for the connection (so that we can discover
      * idle connections) */
     conn->lastSendTime = call->lastSendTime = clock_Sec();
-    /* Let a set of retransmits trigger an idle timeout */
-    if (!xmit->resending)
-       call->lastSendData = call->lastSendTime;
 }
 
 /* When sending packets we need to follow these rules:
@@ -5959,8 +5959,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
@@ -6045,18 +6048,11 @@ rxi_Resend(struct rxevent *event, void *arg0, void *arg1, int istack)
     /* Make sure that the event pointer is removed from the call
      * structure, since there is no longer a per-call retransmission
      * event pending. */
-    if (event == call->resendEvent) {
-       CALL_RELE(call, RX_CALL_REFCOUNT_RESEND);
-       rxevent_Put(call->resendEvent);
-       call->resendEvent = NULL;
-    }
+    if (event == call->resendEvent)
+       rxevent_Put(&call->resendEvent);
 
     rxi_CheckPeerDead(call);
 
-    if (rxi_busyChannelError && (call->flags & RX_CALL_PEER_BUSY)) {
-       rxi_CheckBusy(call);
-    }
-
     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
@@ -6106,6 +6102,7 @@ rxi_Resend(struct rxevent *event, void *arg0, void *arg1, int istack)
     rxi_Start(call, istack);
 
 out:
+    CALL_RELE(call, RX_CALL_REFCOUNT_RESEND);
     MUTEX_EXIT(&call->lock);
 }
 
@@ -6125,7 +6122,7 @@ rxi_Start(struct rx_call *call, int istack)
     int maxXmitPackets;
 
     if (call->error) {
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
         if (rx_stats_active)
             rx_atomic_inc(&rx_tq_debug.rxi_start_in_error);
 #endif
@@ -6148,15 +6145,15 @@ rxi_Start(struct rx_call *call, int istack)
         * But check whether we're here recursively, and let the other guy
         * do the work.
         */
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
        if (!(call->flags & RX_CALL_TQ_BUSY)) {
            call->flags |= RX_CALL_TQ_BUSY;
            do {
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
            restart:
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
                call->flags &= ~RX_CALL_NEED_START;
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
                nXmitPackets = 0;
                maxXmitPackets = MIN(call->twind, call->cwind);
                for (opr_queue_Scan(&call->tq, cursor)) {
@@ -6195,7 +6192,7 @@ rxi_Start(struct rx_call *call, int istack)
                                             nXmitPackets, istack);
                            goto restart;
                        }
-                        dpf(("call %d xmit packet %"AFS_PTR_FMT"\n",
+                      dpf(("call %d xmit packet %p\n",
                               *(call->callNumber), p));
                        call->xmitList[nXmitPackets++] = p;
                    }
@@ -6208,7 +6205,7 @@ rxi_Start(struct rx_call *call, int istack)
                                     istack);
                }
 
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
                if (call->error) {
                    /* We went into the error state while sending packets. Now is
                     * the time to reset the call. This will also inform the using
@@ -6221,7 +6218,7 @@ rxi_Start(struct rx_call *call, int istack)
                    rxi_CallError(call, call->error);
                    return;
                }
-#ifdef RX_ENABLE_LOCKS
+
                if (call->flags & RX_CALL_TQ_SOME_ACKED) {
                    int missing;
                    call->flags &= ~RX_CALL_TQ_SOME_ACKED;
@@ -6249,20 +6246,19 @@ rxi_Start(struct rx_call *call, int istack)
                    if (!missing)
                        call->flags |= RX_CALL_TQ_CLEARME;
                }
-#endif /* RX_ENABLE_LOCKS */
                if (call->flags & RX_CALL_TQ_CLEARME)
                    rxi_ClearTransmitQueue(call, 1);
            } while (call->flags & RX_CALL_NEED_START);
            /*
             * TQ references no longer protected by this flag; they must remain
-            * protected by the global lock.
+            * protected by the call lock.
             */
            call->flags &= ~RX_CALL_TQ_BUSY;
            rxi_WakeUpTransmitQueue(call);
        } else {
            call->flags |= RX_CALL_NEED_START;
        }
-#endif /* AFS_GLOBAL_RXLOCK_KERNEL */
+#endif /* RX_ENABLE_LOCKS */
     } else {
        rxi_rto_cancel(call);
     }
@@ -6275,6 +6271,7 @@ void
 rxi_Send(struct rx_call *call, struct rx_packet *p,
         int istack)
 {
+    int code;
     struct rx_connection *conn = call->conn;
 
     /* Stamp each packet with the user supplied status */
@@ -6282,11 +6279,19 @@ rxi_Send(struct rx_call *call, struct rx_packet *p,
 
     /* Allow the security object controlling this call's security to
      * make any last-minute changes to the packet */
-    RXS_SendPacket(conn->securityObject, call, p);
+    code = RXS_SendPacket(conn->securityObject, call, p);
+    if (code) {
+       MUTEX_EXIT(&call->lock);
+       CALL_HOLD(call, RX_CALL_REFCOUNT_SEND);
+       rxi_ConnectionError(conn, code);
+       CALL_RELE(call, RX_CALL_REFCOUNT_SEND);
+       MUTEX_ENTER(&call->lock);
+       return;
+    }
 
     /* Since we're about to send SOME sort of packet to the peer, it's
      * safe to nuke any scheduled end-of-packets ack */
-    rxevent_Cancel(&call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
+    rxi_CancelDelayedAckEvent(call);
 
     /* Actually send the packet, filling in more connection-specific fields */
     MUTEX_EXIT(&call->lock);
@@ -6303,12 +6308,6 @@ rxi_Send(struct rx_call *call, struct rx_packet *p,
        (p->length <= (rx_AckDataSize(call->rwind) + 4 * sizeof(afs_int32))))
     {
        conn->lastSendTime = call->lastSendTime = clock_Sec();
-       /* Don't count keepalive ping/acks here, so idleness can be tracked. */
-       if ((p->header.type != RX_PACKET_TYPE_ACK) ||
-           ((((struct rx_ackPacket *)rx_DataOf(p))->reason != RX_ACK_PING) &&
-            (((struct rx_ackPacket *)rx_DataOf(p))->reason !=
-             RX_ACK_PING_RESPONSE)))
-           call->lastSendData = call->lastSendTime;
     }
 }
 
@@ -6319,13 +6318,8 @@ rxi_Send(struct rx_call *call, struct rx_packet *p,
  *  may be freed!
  * haveCTLock Set if calling from rxi_ReapConnections
  */
-#ifdef RX_ENABLE_LOCKS
 static int
 rxi_CheckCall(struct rx_call *call, int haveCTLock)
-#else /* RX_ENABLE_LOCKS */
-static int
-rxi_CheckCall(struct rx_call *call)
-#endif                         /* RX_ENABLE_LOCKS */
 {
     struct rx_connection *conn = call->conn;
     afs_uint32 now;
@@ -6364,7 +6358,7 @@ rxi_CheckCall(struct rx_call *call)
        return -1;
     }
 
-#ifdef AFS_GLOBAL_RXLOCK_KERNEL
+#ifdef RX_ENABLE_LOCKS
     if (call->flags & RX_CALL_TQ_BUSY) {
        /* Call is active and will be reset by rxi_Start if it's
         * in an error state.
@@ -6382,45 +6376,15 @@ rxi_CheckCall(struct rx_call *call)
      * number of seconds. */
     if (now > (call->lastReceiveTime + deadTime)) {
        if (call->state == RX_STATE_ACTIVE) {
-#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);
-           ip_stack_t *ipst = ns->netstack_ip;
-#  endif
-           ire = ire_cache_lookup(conn->peer->host
-#  if defined(AFS_SUN510_ENV) && defined(ALL_ZONES)
-                                  , ALL_ZONES
-#    if defined(ICL_3_ARG) || defined(GLOBAL_NETSTACKID)
-                                  , NULL
-#     if defined(GLOBAL_NETSTACKID)
-                                  , ipst
-#     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)
-           netstack_rele(ns);
-#  endif
-# endif
-#endif /* AFS_ADAPT_PMTU */
            cerror = RX_CALL_DEAD;
            goto mtuout;
        } else {
 #ifdef RX_ENABLE_LOCKS
            /* Cancel pending events */
-           rxevent_Cancel(&call->delayedAckEvent, call,
-                          RX_CALL_REFCOUNT_DELAY);
+           rxi_CancelDelayedAckEvent(call);
            rxi_rto_cancel(call);
-           rxevent_Cancel(&call->keepAliveEvent, call,
-                          RX_CALL_REFCOUNT_ALIVE);
-           rxevent_Cancel(&call->growMTUEvent, call,
-                          RX_CALL_REFCOUNT_MTU);
+           rxi_CancelKeepAliveEvent(call);
+           rxi_CancelGrowMTUEvent(call);
             MUTEX_ENTER(&rx_refcnt_mutex);
             /* if rxi_FreeCall returns 1 it has freed the call */
            if (call->refCount == 0 &&
@@ -6441,29 +6405,18 @@ rxi_CheckCall(struct rx_call *call)
         * attached process can die reasonably gracefully. */
     }
 
-    if (conn->idleDeadDetection) {
-        if (conn->idleDeadTime) {
-            idleDeadTime = conn->idleDeadTime + fudgeFactor;
-        }
-
-        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 (conn->idleDeadTime) {
+       idleDeadTime = conn->idleDeadTime + fudgeFactor;
+    }
 
-            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 (idleDeadTime) {
+       /* see if we have a non-activity timeout */
+       if (call->startWait && ((call->startWait + idleDeadTime) < now)) {
+           if (call->state == RX_STATE_ACTIVE) {
+               cerror = RX_CALL_TIMEOUT;
+               goto mtuout;
+           }
+       }
     }
 
     if (conn->hardDeadTime) {
@@ -6483,19 +6436,20 @@ mtuout:
         call->lastReceiveTime) {
        int oldMTU = conn->peer->ifMTU;
 
-       /* if we thought we could send more, perhaps things got worse */
-       if (conn->peer->maxPacketSize > conn->lastPacketSize)
-           /* maxpacketsize will be cleared in rxi_SetPeerMtu */
-           newmtu = MAX(conn->peer->maxPacketSize-RX_IPUDP_SIZE,
-                        conn->lastPacketSize-(128+RX_IPUDP_SIZE));
+       /* If we thought we could send more, perhaps things got worse.
+        * Shrink by 128 bytes and try again. */
+       if (conn->peer->maxPacketSize < conn->lastPacketSize)
+           /* maxPacketSize will be cleared in rxi_SetPeerMtu */
+           newmtu = MAX(conn->peer->maxPacketSize + RX_HEADER_SIZE,
+                        conn->lastPacketSize - 128 + RX_HEADER_SIZE);
        else
-           newmtu = conn->lastPacketSize-(128+RX_IPUDP_SIZE);
+           newmtu = conn->lastPacketSize - 128 + RX_HEADER_SIZE;
 
        /* minimum capped in SetPeerMtu */
        rxi_SetPeerMtu(conn->peer, 0, 0, newmtu);
 
        /* clean up */
-       conn->lastPacketSize = 0;
+       conn->lastPacketSize = conn->lastPacketSizeSeq = 0;
 
        /* needed so ResetCall doesn't clobber us. */
        call->MTU = conn->peer->ifMTU;
@@ -6519,6 +6473,7 @@ rxi_NatKeepAliveEvent(struct rxevent *event, void *arg1,
     struct sockaddr_in taddr;
     char *tp;
     char a[1] = { 0 };
+    int resched = 0;
     struct iovec tmpiov[2];
     osi_socket socket =
         (conn->type ==
@@ -6529,6 +6484,7 @@ rxi_NatKeepAliveEvent(struct rxevent *event, void *arg1,
     taddr.sin_family = AF_INET;
     taddr.sin_port = rx_PortOf(rx_PeerOf(conn));
     taddr.sin_addr.s_addr = rx_HostOf(rx_PeerOf(conn));
+    memset(&taddr.sin_zero, 0, sizeof(taddr.sin_zero));
 #ifdef STRUCT_SOCKADDR_HAS_SA_LEN
     taddr.sin_len = sizeof(struct sockaddr_in);
 #endif
@@ -6547,38 +6503,43 @@ rxi_NatKeepAliveEvent(struct rxevent *event, void *arg1,
     tmpiov[0].iov_base = tbuffer;
     tmpiov[0].iov_len = 1 + sizeof(struct rx_header);
 
-    osi_NetSend(socket, &taddr, tmpiov, 1, 1 + sizeof(struct rx_header), 1);
+    rxi_NetSend(socket, &taddr, tmpiov, 1, 1 + sizeof(struct rx_header), 1);
 
     MUTEX_ENTER(&conn->conn_data_lock);
+    /* We ran, so the handle is no longer needed to try to cancel ourselves. */
+    if (event == conn->natKeepAliveEvent)
+       rxevent_Put(&conn->natKeepAliveEvent);
     MUTEX_ENTER(&rx_refcnt_mutex);
     /* Only reschedule ourselves if the connection would not be destroyed */
-    if (conn->refCount <= 1) {
-       rxevent_Put(conn->natKeepAliveEvent);
-       conn->natKeepAliveEvent = NULL;
-        MUTEX_EXIT(&rx_refcnt_mutex);
-       MUTEX_EXIT(&conn->conn_data_lock);
-       rx_DestroyConnection(conn); /* drop the reference for this */
-    } else {
-       conn->refCount--; /* drop the reference for this */
-        MUTEX_EXIT(&rx_refcnt_mutex);
-       rxevent_Put(conn->natKeepAliveEvent);
-       conn->natKeepAliveEvent = NULL;
-       rxi_ScheduleNatKeepAliveEvent(conn);
-       MUTEX_EXIT(&conn->conn_data_lock);
+    if (conn->refCount > 1)
+       resched = 1;
+    if (conn->refCount <= 0) {
+#ifdef RX_REFCOUNT_CHECK
+       osi_Assert(conn->refCount == 0);
+#endif
+       if (rx_stats_active) {
+           MUTEX_ENTER(&rx_stats_mutex);
+           rxi_lowConnRefCount++;
+           MUTEX_EXIT(&rx_stats_mutex);
+       }
     }
+    MUTEX_EXIT(&rx_refcnt_mutex);
+    if (resched)
+       rxi_ScheduleNatKeepAliveEvent(conn);
+    MUTEX_EXIT(&conn->conn_data_lock);
+    putConnection(conn);
 }
 
 static void
 rxi_ScheduleNatKeepAliveEvent(struct rx_connection *conn)
 {
+    MUTEX_ASSERT(&conn->conn_data_lock);
     if (!conn->natKeepAliveEvent && conn->secondsUntilNatPing) {
        struct clock when, now;
        clock_GetTime(&now);
        when = now;
        when.sec += conn->secondsUntilNatPing;
-        MUTEX_ENTER(&rx_refcnt_mutex);
-       conn->refCount++; /* hold a reference for this */
-        MUTEX_EXIT(&rx_refcnt_mutex);
+        rx_GetConnection(conn);
        conn->natKeepAliveEvent =
            rxevent_Post(&when, &now, rxi_NatKeepAliveEvent, conn, NULL, 0);
     }
@@ -6612,29 +6573,23 @@ rxi_KeepAliveEvent(struct rxevent *event, void *arg1, void *dummy,
     struct rx_connection *conn;
     afs_uint32 now;
 
-    CALL_RELE(call, RX_CALL_REFCOUNT_ALIVE);
     MUTEX_ENTER(&call->lock);
 
-    if (event == call->keepAliveEvent) {
-       rxevent_Put(call->keepAliveEvent);
-       call->keepAliveEvent = NULL;
-    }
+    if (event == call->keepAliveEvent)
+       rxevent_Put(&call->keepAliveEvent);
 
     now = clock_Sec();
 
-#ifdef RX_ENABLE_LOCKS
     if (rxi_CheckCall(call, 0)) {
        MUTEX_EXIT(&call->lock);
+       CALL_RELE(call, RX_CALL_REFCOUNT_ALIVE);
        return;
     }
-#else /* RX_ENABLE_LOCKS */
-    if (rxi_CheckCall(call))
-       return;
-#endif /* RX_ENABLE_LOCKS */
 
     /* Don't try to keep alive dallying calls */
     if (call->state == RX_STATE_DALLY) {
        MUTEX_EXIT(&call->lock);
+       CALL_RELE(call, RX_CALL_REFCOUNT_ALIVE);
        return;
     }
 
@@ -6647,6 +6602,7 @@ rxi_KeepAliveEvent(struct rxevent *event, void *arg1, void *dummy,
     }
     rxi_ScheduleKeepAliveEvent(call);
     MUTEX_EXIT(&call->lock);
+    CALL_RELE(call, RX_CALL_REFCOUNT_ALIVE);
 }
 
 /* Does what's on the nameplate. */
@@ -6656,29 +6612,17 @@ 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_MTU);
     MUTEX_ENTER(&call->lock);
 
-    if (event == call->growMTUEvent) {
-       rxevent_Put(call->growMTUEvent);
-       call->growMTUEvent = NULL;
-    }
+    if (event == call->growMTUEvent)
+       rxevent_Put(&call->growMTUEvent);
 
-#ifdef RX_ENABLE_LOCKS
-    if (rxi_CheckCall(call, 0)) {
-       MUTEX_EXIT(&call->lock);
-       return;
-    }
-#else /* RX_ENABLE_LOCKS */
-    if (rxi_CheckCall(call))
-       return;
-#endif /* RX_ENABLE_LOCKS */
+    if (rxi_CheckCall(call, 0))
+       goto out;
 
     /* Don't bother with dallying calls */
-    if (call->state == RX_STATE_DALLY) {
-       MUTEX_EXIT(&call->lock);
-       return;
-    }
+    if (call->state == RX_STATE_DALLY)
+       goto out;
 
     conn = call->conn;
 
@@ -6688,15 +6632,18 @@ rxi_GrowMTUEvent(struct rxevent *event, void *arg1, void *dummy, int dummy2)
      */
     if ((conn->peer->maxPacketSize != 0) &&
        (conn->peer->natMTU < RX_MAX_PACKET_SIZE) &&
-       conn->idleDeadDetection)
+       conn->idleDeadTime)
        (void)rxi_SendAck(call, NULL, 0, RX_ACK_MTU, 0);
     rxi_ScheduleGrowMTUEvent(call, 0);
+out:
     MUTEX_EXIT(&call->lock);
+    CALL_RELE(call, RX_CALL_REFCOUNT_MTU);
 }
 
 static void
 rxi_ScheduleKeepAliveEvent(struct rx_call *call)
 {
+    MUTEX_ASSERT(&call->lock);
     if (!call->keepAliveEvent) {
        struct clock when, now;
        clock_GetTime(&now);
@@ -6709,8 +6656,16 @@ rxi_ScheduleKeepAliveEvent(struct rx_call *call)
 }
 
 static void
+rxi_CancelKeepAliveEvent(struct rx_call *call) {
+    MUTEX_ASSERT(&call->lock);
+    if (rxevent_Cancel(&call->keepAliveEvent))
+       CALL_RELE(call, RX_CALL_REFCOUNT_ALIVE);
+}
+
+static void
 rxi_ScheduleGrowMTUEvent(struct rx_call *call, int secs)
 {
+    MUTEX_ASSERT(&call->lock);
     if (!call->growMTUEvent) {
        struct clock when, now;
 
@@ -6731,7 +6686,27 @@ rxi_ScheduleGrowMTUEvent(struct rx_call *call, int secs)
     }
 }
 
-/* N.B. rxi_KeepAliveOff:  is defined earlier as a macro */
+static void
+rxi_CancelGrowMTUEvent(struct rx_call *call)
+{
+    MUTEX_ASSERT(&call->lock);
+    if (rxevent_Cancel(&call->growMTUEvent))
+       CALL_RELE(call, RX_CALL_REFCOUNT_MTU);
+}
+
+/*
+ * Increment the counter for the next connection ID, handling overflow.
+ */
+static void
+update_nextCid(void)
+{
+    /* Overflow is technically undefined behavior; avoid it. */
+    if (rx_nextCid > MAX_AFS_INT32 - (1 << RX_CIDSHIFT))
+       rx_nextCid = -1 * ((MAX_AFS_INT32 / RX_CIDSHIFT) * RX_CIDSHIFT);
+    else
+       rx_nextCid += 1 << RX_CIDSHIFT;
+}
+
 static void
 rxi_KeepAliveOn(struct rx_call *call)
 {
@@ -6744,20 +6719,6 @@ rxi_KeepAliveOn(struct rx_call *call)
     rxi_ScheduleKeepAliveEvent(call);
 }
 
-/*
- * Solely in order that callers not need to include rx_call.h
- */
-void
-rx_KeepAliveOff(struct rx_call *call)
-{
-    rxi_KeepAliveOff(call);
-}
-void
-rx_KeepAliveOn(struct rx_call *call)
-{
-    rxi_KeepAliveOn(call);
-}
-
 static void
 rxi_GrowMTUOn(struct rx_call *call)
 {
@@ -6780,8 +6741,8 @@ rxi_SendDelayedConnAbort(struct rxevent *event, void *arg1, void *unused,
     struct rx_packet *packet;
 
     MUTEX_ENTER(&conn->conn_data_lock);
-    rxevent_Put(conn->delayedAbortEvent);
-    conn->delayedAbortEvent = NULL;
+    if (event == conn->delayedAbortEvent)
+       rxevent_Put(&conn->delayedAbortEvent);
     error = htonl(conn->error);
     conn->abortCount++;
     MUTEX_EXIT(&conn->conn_data_lock);
@@ -6793,6 +6754,7 @@ rxi_SendDelayedConnAbort(struct rxevent *event, void *arg1, void *unused,
                            sizeof(error), 0);
        rxi_FreePacket(packet);
     }
+    putConnection(conn);
 }
 
 /* This routine is called to send call abort messages
@@ -6807,8 +6769,8 @@ rxi_SendDelayedCallAbort(struct rxevent *event, void *arg1, void *dummy,
     struct rx_packet *packet;
 
     MUTEX_ENTER(&call->lock);
-    rxevent_Put(call->delayedAbortEvent);
-    call->delayedAbortEvent = NULL;
+    if (event == call->delayedAbortEvent)
+       rxevent_Put(&call->delayedAbortEvent);
     error = htonl(call->error);
     call->abortCount++;
     packet = rxi_AllocPacket(RX_PACKET_CLASS_SPECIAL);
@@ -6822,20 +6784,38 @@ rxi_SendDelayedCallAbort(struct rxevent *event, void *arg1, void *dummy,
     CALL_RELE(call, RX_CALL_REFCOUNT_ABORT);
 }
 
-/* This routine is called periodically (every RX_AUTH_REQUEST_TIMEOUT
+/*
+ * This routine is called periodically (every RX_AUTH_REQUEST_TIMEOUT
  * 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 */
+ * security object associated with the connection
+ *
+ * This routine is both an event handler and a function called directly;
+ * when called directly the passed |event| is NULL and the
+ * conn->conn->data>lock must must not be held.  Also, when called as an
+ * an event handler, we must putConnection before we exit; but when called
+ * directly (the first challenge), we must NOT putConnection.
+ */
 static void
 rxi_ChallengeEvent(struct rxevent *event,
                   void *arg0, void *arg1, int tries)
 {
     struct rx_connection *conn = arg0;
+    int event_raised = 0;      /* assume we were called directly */
 
-    if (event) {
-       rxevent_Put(conn->challengeEvent);
-       conn->challengeEvent = NULL;
+    MUTEX_ENTER(&conn->conn_data_lock);
+    if (event != NULL && event == conn->challengeEvent) {
+       event_raised = 1;       /* called as an event */
+       rxevent_Put(&conn->challengeEvent);
     }
+    MUTEX_EXIT(&conn->conn_data_lock);
+
+    /* If there are no active calls it is not worth re-issuing the
+     * challenge.  If the client issues another call on this connection
+     * the challenge can be requested at that time.
+     */
+    if (!rxi_HasActiveCalls(conn))
+       goto done;
 
     if (RXS_CheckAuthentication(conn->securityObject, conn) != 0) {
        struct rx_packet *packet;
@@ -6861,39 +6841,74 @@ rxi_ChallengeEvent(struct rxevent *event,
                }
            }
            MUTEX_EXIT(&conn->conn_call_lock);
-           return;
+           goto done;
        }
 
        packet = rxi_AllocPacket(RX_PACKET_CLASS_SPECIAL);
        if (packet) {
-           /* If there's no packet available, do this later. */
-           RXS_GetChallenge(conn->securityObject, conn, packet);
-           rxi_SendSpecial((struct rx_call *)0, conn, packet,
-                           RX_PACKET_TYPE_CHALLENGE, NULL, -1, 0);
+           int code;
+           code = RXS_GetChallenge(conn->securityObject, conn, packet);
+           if (code && event_raised) {
+               /*
+                * We can only rxi_ConnectionError the connection if we are
+                * running as an event. Otherwise, the caller may have our call
+                * locked, and so we cannot call rxi_ConnectionError (since it
+                * tries to lock each call in the conn).
+                */
+               rxi_FreePacket(packet);
+               rxi_ConnectionError(conn, code);
+               goto done;
+           }
+           if (code == 0) {
+               /* Only send a challenge packet if we were able to allocate a
+                * packet, and the security layer successfully populated the
+                * challenge. */
+               rxi_SendSpecial((struct rx_call *)0, conn, packet,
+                               RX_PACKET_TYPE_CHALLENGE, NULL, -1, 0);
+               conn->securityChallengeSent = 1;
+           }
            rxi_FreePacket(packet);
        }
        clock_GetTime(&now);
        when = now;
        when.sec += RX_CHALLENGE_TIMEOUT;
-       conn->challengeEvent =
-           rxevent_Post(&when, &now, rxi_ChallengeEvent, conn, 0,
-                        (tries - 1));
+       MUTEX_ENTER(&conn->conn_data_lock);
+       /* Only reschedule ourselves if not already pending. */
+       if (conn->challengeEvent == NULL) {
+           rx_GetConnection(conn);
+           conn->challengeEvent =
+               rxevent_Post(&when, &now, rxi_ChallengeEvent, conn, 0,
+                            (tries - 1));
+       }
+       MUTEX_EXIT(&conn->conn_data_lock);
     }
+ done:
+    if (event_raised)
+       putConnection(conn);
 }
 
 /* Call this routine to start requesting the client to authenticate
  * itself.  This will continue until authentication is established,
  * the call times out, or an invalid response is returned.  The
  * security object associated with the connection is asked to create
- * the challenge at this time.  N.B.  rxi_ChallengeOff is a macro,
- * defined earlier. */
-static void
+ * the challenge at this time. */
+static int
 rxi_ChallengeOn(struct rx_connection *conn)
 {
-    if (!conn->challengeEvent) {
-       RXS_CreateChallenge(conn->securityObject, conn);
+    int start = 0;
+    MUTEX_ENTER(&conn->conn_data_lock);
+    if (!conn->challengeEvent)
+       start = 1;
+    MUTEX_EXIT(&conn->conn_data_lock);
+    if (start) {
+       int code;
+       code = RXS_CreateChallenge(conn->securityObject, conn);
+       if (code) {
+           return code;
+       }
        rxi_ChallengeEvent(NULL, conn, 0, RX_CHALLENGE_MAXTRIES);
-    };
+    }
+    return 0;
 }
 
 
@@ -6949,7 +6964,7 @@ rxi_ComputeRoundTripTime(struct rx_packet *p,
        return;                 /* somebody set the clock back, don't count this time. */
 
     clock_Sub(&thisRtt, sentp);
-    dpf(("rxi_ComputeRoundTripTime(call=%d packet=%"AFS_PTR_FMT" rttp=%d.%06d sec)\n",
+    dpf(("rxi_ComputeRoundTripTime(call=%d packet=%p rttp=%d.%06d sec)\n",
           p->header.callNumber, p, thisRtt.sec, thisRtt.usec));
 
     if (clock_IsZero(&thisRtt)) {
@@ -7048,8 +7063,10 @@ rxi_ComputeRoundTripTime(struct rx_packet *p,
     peer->rtt_dev = call->rtt_dev;
     peer->rtt = call->rtt;
 
-    dpf(("rxi_ComputeRoundTripTime(call=%d packet=%"AFS_PTR_FMT" rtt=%d ms, srtt=%d ms, rtt_dev=%d ms, timeout=%d.%06d sec)\n",
-          p->header.callNumber, p, MSEC(&thisRtt), call->rtt >> 3, call->rtt_dev >> 2, (call->rto.sec), (call->rto.usec)));
+    dpf(("rxi_ComputeRoundTripTime(call=%d packet=%p rtt=%d ms, srtt=%d ms, "
+        "rtt_dev=%d ms, timeout=%d.%06d sec)\n",
+        p->header.callNumber, p, MSEC(&thisRtt), call->rtt >> 3,
+        call->rtt_dev >> 2, (call->rto.sec), (call->rto.usec)));
 }
 
 
@@ -7060,6 +7077,7 @@ rxi_ReapConnections(struct rxevent *unused, void *unused1, void *unused2,
                    int unused3)
 {
     struct clock now, when;
+    struct rxevent *event;
     clock_GetTime(&now);
 
     /* Find server connection structures that haven't been used for
@@ -7088,11 +7106,7 @@ rxi_ReapConnections(struct rxevent *unused, void *unused1, void *unused2,
                        code = MUTEX_TRYENTER(&call->lock);
                        if (!code)
                            continue;
-#ifdef RX_ENABLE_LOCKS
                        result = rxi_CheckCall(call, 1);
-#else /* RX_ENABLE_LOCKS */
-                       result = rxi_CheckCall(call);
-#endif /* RX_ENABLE_LOCKS */
                        MUTEX_EXIT(&call->lock);
                        if (result == -2) {
                            /* If CheckCall freed the call, it might
@@ -7268,7 +7282,8 @@ rxi_ReapConnections(struct rxevent *unused, void *unused1, void *unused2,
 
     when = now;
     when.sec += RX_REAP_TIME;  /* Check every RX_REAP_TIME seconds */
-    rxevent_Put(rxevent_Post(&when, &now, rxi_ReapConnections, 0, NULL, 0));
+    event = rxevent_Post(&when, &now, rxi_ReapConnections, 0, NULL, 0);
+    rxevent_Put(&event);
 }
 
 
@@ -7523,6 +7538,7 @@ MakeDebugCall(osi_socket socket, afs_uint32 remoteAddr, afs_uint16 remotePort,
     taddr.sin_family = AF_INET;
     taddr.sin_port = remotePort;
     taddr.sin_addr.s_addr = remoteAddr;
+    memset(&taddr.sin_zero, 0, sizeof(taddr.sin_zero));
 #ifdef STRUCT_SOCKADDR_HAS_SA_LEN
     taddr.sin_len = sizeof(struct sockaddr_in);
 #endif
@@ -7941,10 +7957,11 @@ shutdown_rx(void)
 #endif /* KERNEL */
 
     LOCK_RX_INIT;
-    if (rxinit_status == 1) {
+    if (!rxi_IsRunning()) {
        UNLOCK_RX_INIT;
        return;                 /* Already shutdown. */
     }
+    rx_atomic_set(&rxi_running, 0);
 #ifndef KERNEL
     rx_port = 0;
 #ifndef AFS_PTHREAD_ENV
@@ -7955,7 +7972,7 @@ shutdown_rx(void)
     rxi_StopListener();
 #endif /* AFS_PTHREAD_ENV */
     shutdown_rxevent();
-    rx_SetEpoch(0);
+    rx_epoch = 0;
 #ifndef AFS_PTHREAD_ENV
 #ifndef AFS_USE_GETTIMEOFDAY
     clock_UnInit();
@@ -8041,8 +8058,10 @@ shutdown_rx(void)
 
     MUTEX_ENTER(&freeSQEList_lock);
 
-    while ((np = rx_FreeSQEList)) {
-       rx_FreeSQEList = *(struct rx_serverQueueEntry **)np;
+    while (!opr_queue_IsEmpty(&rx_freeServerQueue)) {
+       np = opr_queue_First(&rx_freeServerQueue, struct rx_serverQueueEntry,
+                            entry);
+       opr_queue_Remove(&np->entry);
        MUTEX_DESTROY(&np->lock);
        rxi_Free(np, sizeof(*np));
     }
@@ -8062,25 +8081,13 @@ shutdown_rx(void)
          rx_hashTableSize * sizeof(struct rx_connection *));
     UNPIN(rx_peerHashTable, rx_hashTableSize * sizeof(struct rx_peer *));
 
-    rxi_FreeAllPackets();
-
     MUTEX_ENTER(&rx_quota_mutex);
     rxi_dataQuota = RX_MAX_QUOTA;
     rxi_availProcs = rxi_totalMin = rxi_minDeficit = 0;
     MUTEX_EXIT(&rx_quota_mutex);
-    rxinit_status = 1;
     UNLOCK_RX_INIT;
 }
 
-#ifdef RX_ENABLE_LOCKS
-void
-osirx_AssertMine(afs_kmutex_t * lockaddr, char *msg)
-{
-    if (!MUTEX_ISMINE(lockaddr))
-       osi_Panic("Lock not held: %s", msg);
-}
-#endif /* RX_ENABLE_LOCKS */
-
 #ifndef KERNEL
 
 /*
@@ -8377,7 +8384,7 @@ rx_ClearPeerRPCStats(afs_int32 rxInterface, afs_uint32 peerHost, afs_uint16 peer
     if (rxInterface == -1)
         return;
 
-    peer = rxi_FindPeer(peerHost, peerPort, 0, 0);
+    peer = rxi_FindPeer(peerHost, peerPort, 0);
     if (!peer)
         return;
 
@@ -8444,7 +8451,7 @@ rx_CopyPeerRPCStats(afs_uint64 op, afs_uint32 peerHost, afs_uint16 peerPort)
     if (rpcop_stat == NULL)
         return NULL;
 
-    peer = rxi_FindPeer(peerHost, peerPort, 0, 0);
+    peer = rxi_FindPeer(peerHost, peerPort, 0);
     if (!peer)
         return NULL;
 
@@ -9165,7 +9172,7 @@ rx_clearProcessRPCStats(afs_uint32 clearFlag)
     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);
+            = opr_queue_Entry(cursor, struct rx_interface_stat, entry);
 
        num_funcs = rpc_stat->stats[0].func_total;
        for (i = 0; i < num_funcs; i++) {
@@ -9367,13 +9374,13 @@ int rx_DumpCalls(FILE *outputFile, char *cookie)
         MUTEX_ENTER(&c->lock);
         rqc = opr_queue_Count(&c->rq);
         tqc = opr_queue_Count(&c->tq);
-        iovqc = opr_queue_Count(&c->iovq);
+        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, "
                 "lstatus=%u, rstatus=%u, error=%d, timeout=%u, "
-                "resendEvent=%d, timeoutEvt=%d, keepAliveEvt=%d, delayedAckEvt=%d, delayedAbortEvt=%d, abortCode=%d, abortCount=%d, "
-                "lastSendTime=%u, lastRecvTime=%u, lastSendData=%u"
+                "resendEvent=%d, keepAliveEvt=%d, delayedAckEvt=%d, delayedAbortEvt=%d, abortCode=%d, abortCount=%d, "
+                "lastSendTime=%u, lastRecvTime=%u"
 #ifdef RX_ENABLE_LOCKS
                 ", refCount=%u"
 #endif
@@ -9382,12 +9389,12 @@ 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,
-                c->resendEvent?1:0, c->timeoutEvent?1:0, c->keepAliveEvent?1:0, c->delayedAckEvent?1:0, c->delayedAbortEvent?1:0,
-                c->abortCode, c->abortCount, c->lastSendTime, c->lastReceiveTime, c->lastSendData
+                c->resendEvent?1:0, c->keepAliveEvent?1:0, c->delayedAckEvent?1:0, c->delayedAbortEvent?1:0,
+                c->abortCode, c->abortCount, c->lastSendTime, c->lastReceiveTime
 #ifdef RX_ENABLE_LOCKS
                 , (afs_uint32)c->refCount
 #endif
@@ -9409,3 +9416,17 @@ int rx_DumpCalls(FILE *outputFile, char *cookie)
     return 0;
 }
 #endif
+
+int
+rxi_NetSend(osi_socket socket, void *addr, struct iovec *dvec,
+           int nvecs, int length, int istack)
+{
+    if (rxi_IsRunning()) {
+       return osi_NetSend(socket, addr, dvec, nvecs, length, istack);
+    }
+#ifdef AFS_NT40_ENV
+    return WSAESHUTDOWN;
+#else
+    return ESHUTDOWN;
+#endif
+}