fix-afsconfig-ordering-for-rx-20010816
[openafs.git] / src / rx / rx.c
index c34fcf9..83f7eb4 100644 (file)
@@ -1,28 +1,24 @@
 /*
-****************************************************************************
-*        Copyright IBM Corporation 1988, 1989 - All Rights Reserved        *
-*                                                                          *
-* Permission to use, copy, modify, and distribute this software and its    *
-* documentation for any purpose and without fee is hereby granted,         *
-* provided that the above copyright notice appear in all copies and        *
-* that both that copyright notice and this permission notice appear in     *
-* supporting documentation, and that the name of IBM not be used in        *
-* advertising or publicity pertaining to distribution of the software      *
-* without specific, written prior permission.                              *
-*                                                                          *
-* IBM DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE, INCLUDING ALL *
-* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS, IN NO EVENT SHALL IBM *
-* BE LIABLE FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY      *
-* DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER  *
-* IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING   *
-* OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.    *
-****************************************************************************
-*/
+ * Copyright 2000, International Business Machines Corporation and others.
+ * All Rights Reserved.
+ * 
+ * This software has been released under the terms of the IBM Public
+ * License.  For details, see the LICENSE file in the top-level source
+ * directory or online at http://www.openafs.org/dl/license10.html
+ */
 
 /* RX:  Extended Remote Procedure Call */
 
+#include <afsconfig.h>
 #ifdef KERNEL
 #include "../afs/param.h"
+#else
+#include <afs/param.h>
+#endif
+
+RCSID("$Header$");
+
+#ifdef KERNEL
 #include "../afs/sysincludes.h"
 #include "../afs/afsincludes.h"
 #ifndef UKERNEL
@@ -78,7 +74,6 @@ extern afs_int32 afs_termState;
 #endif /* AFS_AIX41_ENV */
 # include "../afsint/rxgen_consts.h"
 #else /* KERNEL */
-# include <afs/param.h>
 # include <sys/types.h>
 # include <errno.h>
 #ifdef AFS_NT40_ENV
@@ -93,6 +88,13 @@ extern afs_int32 afs_termState;
 # include <netinet/in.h>
 # include <sys/time.h>
 #endif
+#ifdef HAVE_STRING_H
+#include <string.h>
+#else
+#ifdef HAVE_STRINGS_H
+#include <strings.h>
+#endif
+#endif
 # include "rx.h"
 # include "rx_user.h"
 # include "rx_clock.h"
@@ -103,6 +105,9 @@ extern afs_int32 afs_termState;
 # include <afs/rxgen_consts.h>
 #endif /* KERNEL */
 
+int (*registerProgram)() = 0;
+int (*swapNameProgram)() = 0;
+
 #ifdef AFS_GLOBAL_RXLOCK_KERNEL
 struct rx_tq_debug {
     afs_int32 rxi_start_aborted; /* rxi_start awoke after rxi_Send in error. */
@@ -222,8 +227,6 @@ assert(pthread_once(&rx_once_init, rxi_InitPthread)==0);
 #define INIT_PTHREAD_LOCKS
 #endif
 
-extern void rxi_DeleteCachedConnections(void);
-
 
 /* Variables for handling the minProcs implementation.  availProcs gives the
  * number of threads available in the pool at this moment (not counting dudes
@@ -247,11 +250,8 @@ extern void rxi_DeleteCachedConnections(void);
  * to manipulate the queue.
  */
 
-extern void rxi_Delay(int);
-
-static int rxi_ServerThreadSelectingCall;
-
 #ifdef RX_ENABLE_LOCKS
+static int rxi_ServerThreadSelectingCall;
 static afs_kmutex_t rx_rpc_stats;
 void rxi_StartUnlocked();
 #endif
@@ -260,7 +260,7 @@ void rxi_StartUnlocked();
 ** pretty good that the next packet coming in is from the same connection 
 ** as the last packet, since we're send multiple packets in a transmit window.
 */
-struct rx_connection *rxLastConn; 
+struct rx_connection *rxLastConn = 0; 
 
 #ifdef RX_ENABLE_LOCKS
 /* The locking hierarchy for rx fine grain locking is composed of five
@@ -301,8 +301,6 @@ void osirx_AssertMine(afs_kmutex_t *lockaddr, char *msg);
 #define CLEAR_CALL_QUEUE_LOCK(C)
 #endif /* RX_ENABLE_LOCKS */
 static void rxi_DestroyConnectionNoLock();
-void rxi_DestroyConnection();
-void rxi_CleanupConnection();
 struct rx_serverQueueEntry *rx_waitForPacket = 0;
 
 /* ------------Exported Interfaces------------- */
@@ -365,6 +363,10 @@ int rx_Init(u_int port)
     char *htable, *ptable;
     int tmp_status;
 
+#if defined(AFS_DJGPP_ENV) && !defined(DEBUG)
+    __djgpp_set_quiet_socket(1);
+#endif
+
     SPLVAR;
 
     INIT_PTHREAD_LOCKS
@@ -429,14 +431,14 @@ int rx_Init(u_int port)
     rxi_nCalls = 0;
     rx_connDeadTime = 12;
     rx_tranquil     = 0;       /* reset flag */
-    bzero((char *)&rx_stats, sizeof(struct rx_stats));
+    memset((char *)&rx_stats, 0, sizeof(struct rx_stats));
     htable = (char *)
        osi_Alloc(rx_hashTableSize*sizeof(struct rx_connection *));
     PIN(htable, rx_hashTableSize*sizeof(struct rx_connection *));  /* XXXXX */
-    bzero(htable, rx_hashTableSize*sizeof(struct rx_connection *));
+    memset(htable, 0, rx_hashTableSize*sizeof(struct rx_connection *));
     ptable =  (char *) osi_Alloc(rx_hashTableSize*sizeof(struct rx_peer *));   
     PIN(ptable, rx_hashTableSize*sizeof(struct rx_peer *));      /* XXXXX */
-    bzero(ptable, rx_hashTableSize*sizeof(struct rx_peer *));
+    memset(ptable, 0, rx_hashTableSize*sizeof(struct rx_peer *));
 
     /* Malloc up a bunch of packets & buffers */
     rx_nFreePackets = 0;
@@ -458,9 +460,22 @@ int rx_Init(u_int port)
 #else
     osi_GetTime(&tv);
 #endif
-    /* *Slightly* random start time for the cid.  This is just to help
-     * out with the hashing function at the peer */
-    rx_port = port;
+    if (port) {
+       rx_port = port;
+    } else {
+#if defined(KERNEL) && !defined(UKERNEL)
+       /* Really, this should never happen in a real kernel */
+       rx_port = 0;
+#else
+       struct sockaddr_in addr;
+       int addrlen = sizeof(addr);
+       if (getsockname((int)rx_socket, (struct sockaddr *) &addr, &addrlen)) {
+           rx_Finalize();
+           return -1;
+       }
+       rx_port = addr.sin_port;
+#endif
+    }
     rx_stats.minRtt.sec = 9999999;
 #ifdef KERNEL
     rx_SetEpoch (tv.tv_sec | 0x80000000);
@@ -471,6 +486,8 @@ int rx_Init(u_int port)
     MUTEX_ENTER(&rx_stats_mutex);
     rxi_dataQuota += rx_extraQuota;    /* + extra pkts caller asked to rsrv */
     MUTEX_EXIT(&rx_stats_mutex);
+    /* *Slightly* random start time for the cid.  This is just to help
+     * out with the hashing function at the peer */
     rx_nextCid = ((tv.tv_sec ^ tv.tv_usec) << RX_CIDSHIFT);
     rx_connHashTable = (struct rx_connection **) htable;
     rx_peerHashTable = (struct rx_peer **) ptable;
@@ -552,7 +569,7 @@ register struct rx_service *aservice;
 }
 
 #else /* RX_ENABLE_LOCKS */
-static QuotaOK(aservice)
+static int QuotaOK(aservice)
 register struct rx_service *aservice; {
     int rc=0;
     /* under min quota, we're OK */
@@ -611,7 +628,7 @@ void rxi_StartServerProcs(nExistingProcs)
 void rx_StartServer(donateMe)
 {
     register struct rx_service *service;
-    register int i;
+    register int i, nProcs=0;
     SPLVAR;
     clock_NewTime();
 
@@ -645,7 +662,25 @@ void rx_StartServer(donateMe)
     AFS_RXGUNLOCK();
     USERPRI;
 
-    if (donateMe) rx_ServerProc(); /* Never returns */
+    if (donateMe) {
+#ifndef AFS_NT40_ENV
+#ifndef KERNEL
+        char name[32];
+#ifdef AFS_PTHREAD_ENV
+        pid_t pid;
+        pid = (pid_t) pthread_self();
+#else /* AFS_PTHREAD_ENV */
+        PROCESS pid;
+        LWP_CurrentProcess(&pid);
+#endif /* AFS_PTHREAD_ENV */
+
+        sprintf(name,"srv_%d", ++nProcs);
+       if (registerProgram)
+            (*registerProgram)(pid, name);
+#endif /* KERNEL */
+#endif /* AFS_NT40_ENV */
+       rx_ServerProc(); /* Never returns */
+    }
     return;
 }
 
@@ -830,7 +865,7 @@ static void rxi_DestroyConnectionNoLock(conn)
        MUTEX_EXIT(&rx_stats_mutex);
     }
 
-    if (conn->refCount > 0) {
+    if ((conn->refCount > 0) || (conn->flags & RX_CONN_BUSY)) {
        /* Busy; wait till the last guy before proceeding */
        MUTEX_EXIT(&conn->conn_data_lock);
        USERPRI;
@@ -1040,6 +1075,7 @@ struct rx_call *rx_NewCall(conn)
     return call;
 }
 
+int
 rxi_HasActiveCalls(aconn)
 register struct rx_connection *aconn; {
     register int i;
@@ -1048,7 +1084,7 @@ register struct rx_connection *aconn; {
 
     NETPRI;
     for(i=0; i<RX_MAXCALLS; i++) {
-      if (tcall = aconn->call[i]) {
+      if ((tcall = aconn->call[i])) {
        if ((tcall->state == RX_STATE_ACTIVE) 
            || (tcall->state == RX_STATE_PRECALL)) {
          USERPRI;
@@ -1060,6 +1096,7 @@ register struct rx_connection *aconn; {
     return 0;
 }
 
+int
 rxi_GetCallNumberVector(aconn, aint32s)
 register struct rx_connection *aconn;
 register afs_int32 *aint32s; {
@@ -1078,6 +1115,7 @@ register afs_int32 *aint32s; {
     return 0;
 }
 
+int
 rxi_SetCallNumberVector(aconn, aint32s)
 register struct rx_connection *aconn;
 register afs_int32 *aint32s; {
@@ -1343,12 +1381,12 @@ osi_socket *socketp;
 {
     struct rx_serverQueueEntry *sq;
     register struct rx_call *call = (struct rx_call *) 0, *choice2;
-    struct rx_service *service;
+    struct rx_service *service = NULL;
     SPLVAR;
 
     MUTEX_ENTER(&freeSQEList_lock);
 
-    if (sq = rx_FreeSQEList) {
+    if ((sq = rx_FreeSQEList)) {
        rx_FreeSQEList = *(struct rx_serverQueueEntry **)sq;
        MUTEX_EXIT(&freeSQEList_lock);
     } else {    /* otherwise allocate a new one and return that */
@@ -1499,14 +1537,14 @@ rx_GetCall(tno, cur_service, socketp)
 {
     struct rx_serverQueueEntry *sq;
     register struct rx_call *call = (struct rx_call *) 0, *choice2;
-    struct rx_service *service;
+    struct rx_service *service = NULL;
     SPLVAR;
 
     NETPRI;
     AFS_RXGLOCK();
     MUTEX_ENTER(&freeSQEList_lock);
 
-    if (sq = rx_FreeSQEList) {
+    if ((sq = rx_FreeSQEList)) {
        rx_FreeSQEList = *(struct rx_serverQueueEntry **)sq;
        MUTEX_EXIT(&freeSQEList_lock);
     } else {    /* otherwise allocate a new one and return that */
@@ -1730,6 +1768,7 @@ afs_int32 rx_EndCall(call, rc)
        MUTEX_ENTER(&conn->conn_call_lock);
        MUTEX_ENTER(&call->lock);
        MUTEX_ENTER(&conn->conn_data_lock);
+       conn->flags |= RX_CONN_BUSY;
        if (conn->flags & RX_CONN_MAKECALL_WAITING) {
            conn->flags &= (~RX_CONN_MAKECALL_WAITING);
            MUTEX_EXIT(&conn->conn_data_lock);
@@ -1768,8 +1807,10 @@ afs_int32 rx_EndCall(call, rc)
 
     CALL_RELE(call, RX_CALL_REFCOUNT_BEGIN);
     MUTEX_EXIT(&call->lock);
-    if (conn->type == RX_CLIENT_CONNECTION)
+    if (conn->type == RX_CLIENT_CONNECTION) {
        MUTEX_EXIT(&conn->conn_call_lock);
+       conn->flags &= ~RX_CONN_BUSY;
+    }
     AFS_RXGUNLOCK();
     USERPRI;
     /*
@@ -2062,7 +2103,7 @@ register size_t size;
     p = (char *) osi_Alloc(size);
 #endif
     if (!p) osi_Panic("rxi_Alloc error");
-    bzero(p, size);
+    memset(p, 0, size);
     return p;
 }
 
@@ -3144,7 +3185,7 @@ nextloop:;
     } else if (call->nSoftAcks > (u_short)rxi_SoftAckRate) {
        rxevent_Cancel(call->delayedAckEvent, call, RX_CALL_REFCOUNT_DELAY);
        np = rxi_SendAck(call, np, seq, serial, flags,
-                        RX_ACK_DELAY, istack);
+                        RX_ACK_IDLE, istack);
     } else if (call->nSoftAcks) {
        clock_GetTime(&when);
        if (haveLast && !(flags & RX_CLIENT_INITIATED)) {
@@ -3187,7 +3228,6 @@ struct rx_packet *rxi_ReceiveAckPacket(call, np, istack)
     afs_uint32 serial;
     /* because there are CM's that are bogus, sending weird values for this. */
     afs_uint32 skew = 0;
-    int needRxStart = 0;
     int nbytes;
     int missing;
     int acked;
@@ -3224,8 +3264,9 @@ struct rx_packet *rxi_ReceiveAckPacket(call, np, istack)
     if (rx_Log) {
       fprintf( rx_Log, 
              "RACK: reason %x previous %u seq %u serial %u skew %d first %u",
-             ap->reason, ntohl(ap->previousPacket), np->header.seq, serial, 
-             skew, ntohl(ap->firstPacket));
+             ap->reason, ntohl(ap->previousPacket), 
+             (unsigned int) np->header.seq, (unsigned int) serial, 
+             (unsigned int) skew, ntohl(ap->firstPacket));
        if (nAcks) {
            int offset;
            for (offset = 0; offset < nAcks; offset++) 
@@ -3260,13 +3301,17 @@ struct rx_packet *rxi_ReceiveAckPacket(call, np, istack)
        if (tp->header.seq >= first) break;
        call->tfirst = tp->header.seq + 1;
        if (tp->header.serial == serial) {
-         rxi_ComputeRoundTripTime(tp, &tp->timeSent, peer);
+         /* Use RTT if not delayed by client. */
+         if (ap->reason != RX_ACK_DELAY)
+             rxi_ComputeRoundTripTime(tp, &tp->timeSent, peer);
 #ifdef ADAPT_WINDOW
          rxi_ComputeRate(peer, call, tp, np, ap->reason);
 #endif
        }
-       else if ((tp->firstSerial == serial)) {
-         rxi_ComputeRoundTripTime(tp, &tp->firstSent, peer);
+       else if (tp->firstSerial == serial) {
+           /* Use RTT if not delayed by client. */
+           if (ap->reason != RX_ACK_DELAY)
+               rxi_ComputeRoundTripTime(tp, &tp->firstSent, peer);
 #ifdef ADAPT_WINDOW
          rxi_ComputeRate(peer, call, tp, np, ap->reason);
 #endif
@@ -3327,13 +3372,17 @@ struct rx_packet *rxi_ReceiveAckPacket(call, np, istack)
 #endif /* RX_ENABLE_LOCKS */
 #endif /* AFS_GLOBAL_RXLOCK_KERNEL */
        if (tp->header.serial == serial) {
-         rxi_ComputeRoundTripTime(tp, &tp->timeSent, peer);
+           /* Use RTT if not delayed by client. */
+           if (ap->reason != RX_ACK_DELAY)
+               rxi_ComputeRoundTripTime(tp, &tp->timeSent, peer);
 #ifdef ADAPT_WINDOW
          rxi_ComputeRate(peer, call, tp, np, ap->reason);
 #endif
        }
        else if ((tp->firstSerial == serial)) {
-         rxi_ComputeRoundTripTime(tp, &tp->firstSent, peer);
+           /* Use RTT if not delayed by client. */
+           if (ap->reason != RX_ACK_DELAY)
+               rxi_ComputeRoundTripTime(tp, &tp->firstSent, peer);
 #ifdef ADAPT_WINDOW
          rxi_ComputeRate(peer, call, tp, np, ap->reason);
 #endif
@@ -3813,7 +3862,7 @@ register struct rx_call **newcallp;
        if (call->flags & RX_CALL_CLEARED) {
            /* send an ack now to start the packet flow up again */
            call->flags &= ~RX_CALL_CLEARED;
-           rxi_SendAck(call, 0, 0, 0, 0, RX_ACK_DELAY, 0);
+           rxi_SendAck(call, 0, 0, 0, 0, RX_ACK_IDLE, 0);
        }
 #ifdef RX_ENABLE_LOCKS
        CV_SIGNAL(&sq->cv);
@@ -4403,8 +4452,8 @@ struct rx_packet *rxi_SendAck(call, optionalPacket, seq, serial, pflags, reason,
 #ifdef RXDEBUG
     if (rx_Log) {
        fprintf(rx_Log, "SACK: reason %x previous %u seq %u first %u",
-               ap->reason, ntohl(ap->previousPacket), p->header.seq, 
-               ntohl(ap->firstPacket));
+               ap->reason, ntohl(ap->previousPacket), 
+               (unsigned int) p->header.seq, ntohl(ap->firstPacket));
        if (ap->nAcks) {
            for (offset = 0; offset < ap->nAcks; offset++) 
                putc(ap->acks[offset] == RX_ACK_TYPE_NACK? '-' : '*', rx_Log);
@@ -4440,7 +4489,7 @@ struct rx_packet *rxi_SendAck(call, optionalPacket, seq, serial, pflags, reason,
 }
 
 /* Send all of the packets in the list in single datagram */
-static void rxi_SendList(call, list, len, istack, moreFlag, now, retryTime)
+static void rxi_SendList(call, list, len, istack, moreFlag, now, retryTime, resending)
   struct rx_call *call;
   struct rx_packet **list;
   int len;
@@ -4448,6 +4497,7 @@ static void rxi_SendList(call, list, len, istack, moreFlag, now, retryTime)
   int moreFlag;
   struct clock *now;
   struct clock *retryTime;
+  int resending;
 {
     int i;
     int requestAck = 0;
@@ -4457,6 +4507,7 @@ static void rxi_SendList(call, list, len, istack, moreFlag, now, retryTime)
 
     MUTEX_ENTER(&peer->peer_lock);
     peer->nSent += len;
+    if (resending) peer->reSends += len;
     MUTEX_ENTER(&rx_stats_mutex);
     rx_stats.dataPacketsSent += len;
     MUTEX_EXIT(&rx_stats_mutex);
@@ -4510,6 +4561,7 @@ static void rxi_SendList(call, list, len, istack, moreFlag, now, retryTime)
 
        MUTEX_ENTER(&peer->peer_lock);
        peer->nSent++;
+       if (resending) peer->reSends++;
        MUTEX_ENTER(&rx_stats_mutex);
        rx_stats.dataPacketsSent++;
        MUTEX_EXIT(&rx_stats_mutex);
@@ -4558,13 +4610,14 @@ static void rxi_SendList(call, list, len, istack, moreFlag, now, retryTime)
  * We always keep the last list we should have sent so we
  * can set the RX_MORE_PACKETS flags correctly.
  */
-static void rxi_SendXmitList(call, list, len, istack, now, retryTime)
+static void rxi_SendXmitList(call, list, len, istack, now, retryTime, resending)
   struct rx_call *call;
   struct rx_packet **list;
   int len;
   int istack;
   struct clock *now;
   struct clock *retryTime;
+  int resending;
 {
     int i, cnt, lastCnt = 0;
     struct rx_packet **listP, **lastP = 0;
@@ -4578,7 +4631,7 @@ static void rxi_SendXmitList(call, list, len, istack, now, retryTime)
                || list[i]->acked
                || list[i]->length > RX_JUMBOBUFFERSIZE)) {
            if (lastCnt > 0) {
-               rxi_SendList(call, lastP, lastCnt, istack, 1, now, retryTime);
+               rxi_SendList(call, lastP, lastCnt, istack, 1, now, retryTime, resending);
                /* If the call enters an error state stop sending, or if
                 * we entered congestion recovery mode, stop sending */
                if (call->error || (call->flags & RX_CALL_FAST_RECOVER_WAIT))
@@ -4601,7 +4654,7 @@ static void rxi_SendXmitList(call, list, len, istack, now, retryTime)
                || list[i]->length != RX_JUMBOBUFFERSIZE) {
                if (lastCnt > 0) {
                    rxi_SendList(call, lastP, lastCnt, istack, 1,
-                                now, retryTime);
+                                now, retryTime, resending);
                    /* If the call enters an error state stop sending, or if
                     * we entered congestion recovery mode, stop sending */
                    if (call->error || (call->flags&RX_CALL_FAST_RECOVER_WAIT))
@@ -4636,17 +4689,17 @@ static void rxi_SendXmitList(call, list, len, istack, now, retryTime)
        }
        if (lastCnt > 0) {
            rxi_SendList(call, lastP, lastCnt, istack, morePackets,
-                        now, retryTime);
+                        now, retryTime, resending);
            /* If the call enters an error state stop sending, or if
             * we entered congestion recovery mode, stop sending */
            if (call->error || (call->flags & RX_CALL_FAST_RECOVER_WAIT))
                return;
        }
        if (morePackets) {
-           rxi_SendList(call, listP, cnt, istack, 0, now, retryTime);
+           rxi_SendList(call, listP, cnt, istack, 0, now, retryTime, resending);
        }
     } else if (lastCnt > 0) {
-       rxi_SendList(call, lastP, lastCnt, istack, 0, now, retryTime);
+       rxi_SendList(call, lastP, lastCnt, istack, 0, now, retryTime, resending);
     }
 }
 
@@ -4681,6 +4734,7 @@ void rxi_Start(event, call, istack)
     int nXmitPackets;
     int maxXmitPackets;
     struct rx_packet **xmitList;
+    int resending = 0;
 
     /* If rxi_Start is being called as a result of a resend event,
      * then make sure that the event pointer is removed from the call
@@ -4689,6 +4743,7 @@ void rxi_Start(event, call, istack)
     if (event && event == call->resendEvent) {
        CALL_RELE(call, RX_CALL_REFCOUNT_RESEND);
        call->resendEvent = NULL;
+       resending = 1;
        if (queue_IsEmpty(&call->tq)) {
            /* Nothing to do */
            return;
@@ -4838,7 +4893,7 @@ void rxi_Start(event, call, istack)
         * ready to send. Now we loop to send the packets */
        if (nXmitPackets > 0) {
            rxi_SendXmitList(call, xmitList, nXmitPackets, istack,
-                            &now, &retryTime);
+                            &now, &retryTime, resending);
        }
        osi_Free(xmitList, maxXmitPackets * sizeof(struct rx_packet *));
 
@@ -4939,7 +4994,7 @@ void rxi_Start(event, call, istack)
                                             (char *)call, istack);
 #else /* RX_ENABLE_LOCKS */
            call->resendEvent = rxevent_Post(&retryTime, rxi_Start,
-                                            (char *)call, (void*)istack);
+                                            (char *)call, (void*)(long)istack);
 #endif /* RX_ENABLE_LOCKS */
          }
        }
@@ -5258,10 +5313,20 @@ void rxi_ComputeRoundTripTime(p, sentp, peer)
 {
        struct clock thisRtt, *rttp = &thisRtt;
 
+#if defined(AFS_ALPHA_LINUX22_ENV) && defined(AFS_PTHREAD_ENV) && !defined(KERNEL)
+       /* making year 2038 bugs to get this running now - stroucki */
+       struct timeval temptime;
+#endif
       register int rtt_timeout;
-      static char id[]="@(#)adaptive RTO";
 
-    clock_GetTime(rttp);
+#if defined(AFS_ALPHA_LINUX20_ENV) && defined(AFS_PTHREAD_ENV) && !defined(KERNEL)
+      /* yet again. This was the worst Heisenbug of the port - stroucki */
+      clock_GetTime(&temptime);
+      rttp->sec=(afs_int32)temptime.tv_sec;
+      rttp->usec=(afs_int32)temptime.tv_usec;
+#else
+      clock_GetTime(rttp);
+#endif
     if (clock_Lt(rttp, sentp)) {
       clock_Zero(rttp);
       return;     /* somebody set the clock back, don't count this time. */
@@ -5682,7 +5747,7 @@ rxi_DebugPrint(format, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10,
 {
     struct clock now;
     clock_GetTime(&now);
-    fprintf(rx_Log, " %u.%.3u:", now.sec, now.usec/1000);
+    fprintf(rx_Log, " %u.%.3u:", (unsigned int) now.sec, (unsigned int) now.usec/1000);
     fprintf(rx_Log, format, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15);
     putc('\n', rx_Log);
 }
@@ -5711,9 +5776,8 @@ void rx_PrintTheseStats (file, s, size, freePackets, version)
     }
 
     fprintf(file,
-           "rx stats: free packets %d, "
-           "allocs %d, ",
-           freePackets,
+           "rx stats: free packets %d, allocs %d, ",
+           (int) freePackets,
            s->packetRequests);
 
     if (version >= RX_DEBUGI_VERSION_W_NEWPACKETTYPES) {
@@ -5793,7 +5857,7 @@ void rx_PrintTheseStats (file, s, size, freePackets, version)
            "   \t(these should be small) sendFailed %d, "
            "fatalErrors %d\n", 
            s->netSendFailures,
-           s->fatalErrors);
+           (int) s->fatalErrors);
 
     if (s->nRttSamples) {
        fprintf(file,
@@ -5845,10 +5909,10 @@ struct rx_peer *peer;
            "Burst size %d, "
            "burst wait %u.%d.\n",
            ntohl(peer->host),
-           peer->port,
-           peer->burstSize,
-           peer->burstWait.sec,
-           peer->burstWait.usec);
+           (int) peer->port,
+           (int) peer->burstSize,
+           (int) peer->burstWait.sec,
+           (int) peer->burstWait.usec);
 
     fprintf(file,
            "   Rtt %d, "
@@ -5856,8 +5920,8 @@ struct rx_peer *peer;
            "total sent %d, "
            "resent %d\n",
            peer->rtt,
-           peer->timeout.sec,
-           peer->timeout.usec,
+           (int) peer->timeout.sec,
+           (int) peer->timeout.usec,
            peer->nSent,
            peer->reSends);
 
@@ -5866,8 +5930,8 @@ struct rx_peer *peer;
            "max in packet skew %d, "
            "max out packet skew %d\n",
            peer->ifMTU,
-           peer->inPacketSkew,
-           peer->outPacketSkew);
+           (int) peer->inPacketSkew,
+           (int) peer->outPacketSkew);
 }
 
 #ifdef AFS_PTHREAD_ENV
@@ -5924,8 +5988,8 @@ static int MakeDebugCall(
        theader.flags = RX_CLIENT_INITIATED | RX_LAST_PACKET;
        theader.serviceId = 0;
        
-       bcopy(&theader, tbuffer, sizeof(theader));
-       bcopy(inputData, tp, inputLength);
+       memcpy(tbuffer, &theader, sizeof(theader));
+       memcpy(tp, inputData, inputLength);
        code = sendto(socket, tbuffer, inputLength+sizeof(struct rx_header), 0,
                      (struct sockaddr *) &taddr, sizeof(struct sockaddr_in));
        
@@ -5941,7 +6005,7 @@ static int MakeDebugCall(
            code = recvfrom(socket, tbuffer, sizeof(tbuffer), 0,
                            (struct sockaddr *) &faddr, &faddrLen);
 
-           bcopy(tbuffer, &theader, sizeof(struct rx_header));
+           memcpy(&theader, tbuffer, sizeof(struct rx_header));
            if (counter == ntohl(theader.callNumber)) break;
        }
 
@@ -5950,7 +6014,7 @@ static int MakeDebugCall(
     }
     code -= sizeof(struct rx_header);
     if (code > outputLength) code = outputLength;
-    bcopy(tp, outputData, code);
+    memcpy(outputData, tp, code);
     return code;
 }
 
@@ -6187,7 +6251,6 @@ afs_int32 rx_GetServerPeers(
 {
     struct rx_debugIn in;
     afs_int32 rc = 0;
-    int i;
 
     /*
      * supportedValues is currently unused, but added to allow future
@@ -6253,6 +6316,8 @@ void shutdown_rx(void)
 {
     struct rx_serverQueueEntry *np;
     register int i, j;
+    register struct rx_call *call;
+    register struct rx_serverQueueEntry *sq;
 
     LOCK_RX_INIT
     if (rxinit_status == 1) {
@@ -6260,6 +6325,35 @@ void shutdown_rx(void)
        return; /* Already shutdown. */
     }
 
+#ifndef KERNEL
+    rx_port = 0;
+#ifndef AFS_PTHREAD_ENV
+    FD_ZERO(&rx_selectMask);
+#endif /* AFS_PTHREAD_ENV */
+    rxi_dataQuota = RX_MAX_QUOTA;
+#ifndef AFS_PTHREAD_ENV
+    rxi_StopListener();
+#endif /* AFS_PTHREAD_ENV */
+    shutdown_rxevent();
+    rx_SetEpoch(0);
+#ifndef AFS_PTHREAD_ENV
+#ifndef AFS_USE_GETTIMEOFDAY
+    clock_UnInit();
+#endif /* AFS_USE_GETTIMEOFDAY */
+#endif /* AFS_PTHREAD_ENV */
+
+    while (!queue_IsEmpty(&rx_freeCallQueue)) {
+        call = queue_First(&rx_freeCallQueue, rx_call);
+        queue_Remove(call);
+        rxi_Free(call, sizeof(struct rx_call));
+    }
+
+    while (!queue_IsEmpty(&rx_idleServerQueue)) {
+        sq = queue_First(&rx_idleServerQueue, rx_serverQueueEntry);
+        queue_Remove(sq);                                                    
+    }
+#endif /* KERNEL */
+
     {  
        struct rx_peer **peer_ptr, **peer_end;
        for (peer_ptr = &rx_peerHashTable[0], 
@@ -6314,7 +6408,7 @@ void shutdown_rx(void)
 
     MUTEX_ENTER(&freeSQEList_lock);
 
-    while (np = rx_FreeSQEList) {
+    while ((np = rx_FreeSQEList)) {
        rx_FreeSQEList = *(struct rx_serverQueueEntry **)np;
        MUTEX_DESTROY(&np->lock);
        rxi_Free(np, sizeof(*np));
@@ -6511,7 +6605,8 @@ static int rxi_AddRpcStat(
      * queue.
      */
 
-    if ((rpc_stat == NULL) ||
+    if (queue_IsEnd(stats, rpc_stat) ||
+       (rpc_stat == NULL) ||
        (rpc_stat->stats[0].interfaceId != rxInterface) ||
        (rpc_stat->stats[0].remote_is_server != isServer)) {
        int i;
@@ -6812,9 +6907,6 @@ int rx_RetrieveProcessRPCStats(
        ptr = *stats = (afs_uint32 *) rxi_Alloc(space);
 
        if (ptr != NULL) {
-           register struct rx_peer *pp;
-           int i;
-           int num_copied = 0;
            rx_interface_stat_p rpc_stat, nrpc_stat;
 
 
@@ -6917,8 +7009,6 @@ int rx_RetrievePeerRPCStats(
        ptr = *stats = (afs_uint32 *) rxi_Alloc(space);
 
        if (ptr != NULL) {
-           int i;
-           int num_copied = 0;
            rx_interface_stat_p rpc_stat, nrpc_stat;
            char *fix_offset;