You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tomcat.apache.org by mt...@apache.org on 2005/04/24 11:54:47 UTC

cvs commit: jakarta-tomcat-connectors/jk/native/common jk_ajp12_worker.c jk_ajp13_worker.c jk_ajp14_worker.c jk_ajp_common.c jk_ajp_common.h jk_lb_worker.c jk_service.h jk_worker.c jk_worker.h

mturk       2005/04/24 02:54:47

  Modified:    jk/native/common jk_ajp12_worker.c jk_ajp13_worker.c
                        jk_ajp14_worker.c jk_ajp_common.c jk_ajp_common.h
                        jk_lb_worker.c jk_service.h jk_worker.c jk_worker.h
  Log:
  Added wc_maintain, that will recycle all connections regardless of the
  elected worker. Added new worker callback 'maintain' that does the
  actual job.
  
  Revision  Changes    Path
  1.28      +3 -2      jakarta-tomcat-connectors/jk/native/common/jk_ajp12_worker.c
  
  Index: jk_ajp12_worker.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_ajp12_worker.c,v
  retrieving revision 1.27
  retrieving revision 1.28
  diff -u -r1.27 -r1.28
  --- jk_ajp12_worker.c	17 Feb 2005 07:10:23 -0000	1.27
  +++ jk_ajp12_worker.c	24 Apr 2005 09:54:47 -0000	1.28
  @@ -16,7 +16,7 @@
   
   /***************************************************************************
    * Description: ajpv1.2 worker, used to call local or remote jserv hosts   *
  - *              This worker is deprecated                                  * 
  + *              This worker is deprecated                                  *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
    * Based on:    jserv_ajpv12.c from Jserv                                  *
    * Version:     $Revision$                                          *
  @@ -254,6 +254,7 @@
                   private_data->worker.init = init;
                   private_data->worker.get_endpoint = get_endpoint;
                   private_data->worker.destroy = destroy;
  +                private_data->worker.maintain = NULL;
                   private_data->worker.retries = JK_RETRIES;
   
                   *w = &private_data->worker;
  
  
  
  1.25      +3 -2      jakarta-tomcat-connectors/jk/native/common/jk_ajp13_worker.c
  
  Index: jk_ajp13_worker.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_ajp13_worker.c,v
  retrieving revision 1.24
  retrieving revision 1.25
  diff -u -r1.24 -r1.25
  --- jk_ajp13_worker.c	15 Feb 2005 08:52:53 -0000	1.24
  +++ jk_ajp13_worker.c	24 Apr 2005 09:54:47 -0000	1.25
  @@ -86,7 +86,7 @@
           JK_TRACE_EXIT(l);
           return 0;
       }
  -    
  +
       aw = (ajp_worker_t *) calloc(1, sizeof(ajp_worker_t));
       if (!aw) {
           jk_log(l, JK_LOG_ERROR,
  @@ -108,6 +108,7 @@
       aw->worker.init = init;
       aw->worker.get_endpoint = get_endpoint;
       aw->worker.destroy = destroy;
  +    aw->worker.maintain = ajp_maintain;
       aw->worker.retries = JK_RETRIES;
   
       aw->logon = NULL;           /* No Logon on AJP13 */
  
  
  
  1.29      +5 -4      jakarta-tomcat-connectors/jk/native/common/jk_ajp14_worker.c
  
  Index: jk_ajp14_worker.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_ajp14_worker.c,v
  retrieving revision 1.28
  retrieving revision 1.29
  diff -u -r1.28 -r1.29
  --- jk_ajp14_worker.c	15 Feb 2005 08:52:53 -0000	1.28
  +++ jk_ajp14_worker.c	24 Apr 2005 09:54:47 -0000	1.29
  @@ -133,10 +133,10 @@
       return JK_TRUE;
   }
   
  -/* 
  - * AJP14 Logon Phase 
  +/*
  + * AJP14 Logon Phase
    *
  - * INIT + REPLY / NEGO + REPLY 
  + * INIT + REPLY / NEGO + REPLY
    */
   
   static int handle_logon(ajp_endpoint_t * ae,
  @@ -423,6 +423,7 @@
       aw->worker.init = init;
       aw->worker.get_endpoint = get_endpoint;
       aw->worker.destroy = destroy;
  +    aw->worker.maintain = ajp_maintain;
       aw->worker.retries = JK_RETRIES;
   
       aw->logon = logon;          /* LogOn Handler for AJP14 */
  
  
  
  1.102     +143 -115  jakarta-tomcat-connectors/jk/native/common/jk_ajp_common.c
  
  Index: jk_ajp_common.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_ajp_common.c,v
  retrieving revision 1.101
  retrieving revision 1.102
  diff -u -r1.101 -r1.102
  --- jk_ajp_common.c	21 Apr 2005 12:04:18 -0000	1.101
  +++ jk_ajp_common.c	24 Apr 2005 09:54:47 -0000	1.102
  @@ -52,7 +52,7 @@
       tv.tv_sec = 0;
       select(0, NULL, NULL, NULL, &tv);
   #endif
  -} 
  +}
   
   const char *response_trans_headers[] = {
       "Content-Type",
  @@ -68,7 +68,7 @@
       "WWW-Authenticate"
   };
   
  -static const char *long_res_header_for_sc(int sc) 
  +static const char *long_res_header_for_sc(int sc)
   {
       const char *rc = NULL;
       sc = sc & 0X00FF;
  @@ -248,7 +248,7 @@
       }
   
       /* NOTREACHED */
  -} 
  +}
   
   static int sc_for_req_header(const char *header_name)
   {
  @@ -262,7 +262,7 @@
        */
       if (len < 4 || len > 15)
           return UNKNOWN_METHOD;
  -    
  +
       while (*p)
           header[i++] = toupper((unsigned char)*p++);
       header[i] = '\0';
  @@ -563,10 +563,10 @@
       *body_chunk
       terminator      boolean <! -- recycle connection or not  -->
   
  -req_header_name := 
  +req_header_name :=
       sc_req_header_name | (string)
   
  -res_header_name := 
  +res_header_name :=
       sc_res_header_name | (string)
   
   header_value :=
  @@ -685,10 +685,10 @@
   static void ajp_reset_endpoint(ajp_endpoint_t * ae, jk_logger_t *l)
   {
       if (ae->sd > 0 && !ae->reuse) {
  -        jk_close_socket(ae->sd);
  +        jk_shutdown_socket(ae->sd);
           if (JK_IS_DEBUG_LEVEL(l))
               jk_log(l, JK_LOG_DEBUG,
  -                   "closed socket with sd = %d", ae->sd);
  +                   "reset socket with sd = %d", ae->sd);
           ae->sd = -1;
       }
       jk_reset_pool(&(ae->pool));
  @@ -703,7 +703,7 @@
       JK_TRACE_ENTER(l);
   
       if (ae->sd > 0) {
  -        jk_close_socket(ae->sd);
  +        jk_shutdown_socket(ae->sd);
           if (JK_IS_DEBUG_LEVEL(l))
               jk_log(l, JK_LOG_DEBUG,
                      "closed socket with sd = %d", ae->sd);
  @@ -805,7 +805,7 @@
           return JK_FALSE;
       }
   
  -    /* Read and check for Pong reply 
  +    /* Read and check for Pong reply
        */
       if (ajp_connection_tcp_get_message(ae, msg, l) != JK_TRUE) {
           jk_log(l, JK_LOG_ERROR,
  @@ -829,55 +829,53 @@
   int ajp_connect_to_endpoint(ajp_endpoint_t * ae, jk_logger_t *l)
   {
       char buf[32];
  -    unsigned attempt;
       int rc = JK_TRUE;
   
       JK_TRACE_ENTER(l);
   
  -    for (attempt = 0; attempt < ae->worker->connect_retry_attempts; attempt++) {
  -        ae->sd = jk_open_socket(&ae->worker->worker_inet_addr,
  -                                ae->worker->keepalive,
  -                                ae->worker->socket_timeout,
  -                                ae->worker->socket_buf, l);
  -        if (ae->sd >= 0) {
  -            jk_log(l, JK_LOG_DEBUG,
  -                   "connected sd = %d to %s",
  -                   ae->sd, jk_dump_hinfo(&ae->worker->worker_inet_addr, buf));
  +    ae->sd = jk_open_socket(&ae->worker->worker_inet_addr,
  +                            ae->worker->keepalive,
  +                            ae->worker->socket_timeout,
  +                            ae->worker->socket_buf, l);
  +    if (ae->sd >= 0) {
  +        jk_log(l, JK_LOG_DEBUG,
  +               "Connected socket %d to (%s)",
  +               ae->sd, jk_dump_hinfo(&ae->worker->worker_inet_addr, buf));
   
  -            /* set last_access only if needed */
  -            if (ae->worker->cache_timeout > 0 || ae->worker->recycle_timeout > 0)
  -                ae->last_access = time(NULL);
  -            if (ae->worker->socket_timeout > 0) {
  -                if (!jk_is_socket_connected(ae->sd)) {
  -                    jk_log(l, JK_LOG_INFO,
  -                           "Socket is not connected any more (errno=%d)", errno);
  -                    jk_close_socket(ae->sd);
  -                    ae->sd = -1;
  -                    JK_TRACE_EXIT(l);
  -                    return JK_FALSE;
  -                }
  -            }
  -            /* Check if we must execute a logon after the physical connect */
  -            if (ae->worker->logon != NULL) {
  -                rc = ae->worker->logon(ae, l);
  -                JK_TRACE_EXIT(l);
  -                return rc;
  -            }
  -            /* should we send a CPING to validate connection ? */
  -            if (ae->worker->connect_timeout > 0) {
  -                rc = ajp_handle_cping_cpong (ae,
  -                            ae->worker->connect_timeout, l);
  +        /* set last_access only if needed */
  +        if (ae->worker->cache_timeout > 0 || ae->worker->recycle_timeout > 0)
  +            ae->last_access = time(NULL);
  +        if (ae->worker->socket_timeout > 0) {
  +            if (!jk_is_socket_connected(ae->sd)) {
  +                jk_log(l, JK_LOG_INFO,
  +                       "Socket %d to (%s) is not connected any more (errno=%d)",
  +                       ae->sd, jk_dump_hinfo(&ae->worker->worker_inet_addr, buf),
  +                       errno);
  +                jk_close_socket(ae->sd);
  +                ae->sd = -1;
                   JK_TRACE_EXIT(l);
  -                return rc;
  +                return JK_FALSE;
               }
  +        }
  +        /* Check if we must execute a logon after the physical connect */
  +        if (ae->worker->logon != NULL) {
  +            rc = ae->worker->logon(ae, l);
  +            JK_TRACE_EXIT(l);
  +            return rc;
  +        }
  +        /* should we send a CPING to validate connection ? */
  +        if (ae->worker->connect_timeout > 0) {
  +            rc = ajp_handle_cping_cpong (ae,
  +                        ae->worker->connect_timeout, l);
               JK_TRACE_EXIT(l);
  -            return JK_TRUE;
  +            return rc;
           }
  +        JK_TRACE_EXIT(l);
  +        return JK_TRUE;
       }
   
       jk_log(l, JK_LOG_INFO,
  -           "Failed connecting to tomcat. Tomcat is probably not started or is "
  -           "listening on the wrong host/port (%s). Failed errno = %d",
  +           "Failed opening socket to (%s) with (errno=%d)",
              jk_dump_hinfo(&ae->worker->worker_inet_addr, buf), errno);
       JK_TRACE_EXIT(l);
       return JK_FALSE;
  @@ -892,7 +890,7 @@
   {
       int rc;
   
  -    JK_TRACE_ENTER(l); 
  +    JK_TRACE_ENTER(l);
       if (ae->proto == AJP13_PROTO) {
           jk_b_end(msg, AJP13_WS_HEADER);
           if (JK_IS_DEBUG_LEVEL(l))
  @@ -1036,7 +1034,7 @@
    * Read all the data from the socket.
    *
    * Socket API doesn't guaranty that all the data will be kept in a
  - * single read, so we must loop until all awaited data is received 
  + * single read, so we must loop until all awaited data is received
    */
   
   static int ajp_read_fully_from_server(jk_ws_service_t *s, jk_logger_t *l,
  @@ -1104,7 +1102,7 @@
   
       if ((len = ajp_read_fully_from_server(r, l, read_buf, len)) < 0) {
           jk_log(l, JK_LOG_INFO,
  -               "Error receiving data from client failed. "
  +               "Receiving data from client failed. "
                  "Connection aborted or network problems");
           JK_TRACE_EXIT(l);
           return JK_CLIENT_ERROR;
  @@ -1119,7 +1117,7 @@
              an empty body packet */
           if (0 != jk_b_append_int(msg, (unsigned short)len)) {
               jk_log(l, JK_LOG_INFO,
  -                   "failed appending message length");
  +                   "Failed appending message length");
               JK_TRACE_EXIT(l);
               return JK_CLIENT_ERROR;
           }
  @@ -1139,7 +1137,7 @@
    * - send request, but send must be see as asynchronous,
    *   since send() call will return noerror about 95% of time
    *   Hopefully we'll get more information on next read.
  - * 
  + *
    * nb: reqmsg is the original request msg buffer
    *     repmsg is the reply msg buffer which could be scratched
    */
  @@ -1164,7 +1162,8 @@
           if (ae->worker->socket_timeout) {
               if (!jk_is_socket_connected(ae->sd)) {
                   jk_log(l, JK_LOG_INFO,
  -                       "Socket is not connected any more (errno=%d)", errno);
  +                       "Socket %d is not connected any more (errno=%d)",
  +                       ae->sd, errno);
                   jk_close_socket(ae->sd);
                   ae->sd = -1;
                   err++;
  @@ -1203,7 +1202,7 @@
        * If we failed to reuse a connection, try to reconnect.
        */
       if (ae->sd < 0) {
  -        
  +
           if (err) {
               /* XXX: If err is set, the tomcat is either dead or disconnected */
               jk_log(l, JK_LOG_INFO,
  @@ -1226,6 +1225,9 @@
               }
           }
           else {
  +            /* Close the socket if unable to connect */
  +            jk_close_socket(ae->sd);
  +            ae->sd = -1;
               jk_log(l, JK_LOG_INFO,
                      "Error connecting to the Tomcat process.");
               JK_TRACE_EXIT(l);
  @@ -1244,7 +1246,7 @@
                  ae->left_bytes_to_send, op->reply->len - AJP_HEADER_LEN);
   
       /*
  -     * POST recovery job is done here and will work when data to 
  +     * POST recovery job is done here and will work when data to
        * POST are less than 8k, since it's the maximum size of op-post buffer.
        * We send here the first part of data which was sent previously to the
        * remote Tomcat
  @@ -1289,7 +1291,7 @@
           /* We never sent any POST data and we check if we have to send at
            * least one block of data (max 8k). These data will be kept in reply
            * for resend if the remote Tomcat is down, a fact we will learn only
  -         * doing a read (not yet) 
  +         * doing a read (not yet)
            */
           /* || s->is_chunked - this can't be done here. The original protocol
              sends the first chunk of post data ( based on Content-Length ),
  @@ -1433,7 +1435,7 @@
    * the connection (half-closed state - FIN-WAIT2). In that case
    * we must close our side of the socket and abort emission.
    * We will need another connection to send the request
  - * There is need of refactoring here since we mix 
  + * There is need of refactoring here since we mix
    * reply reception (tomcat -> apache) and request send (apache -> tomcat)
    * and everything using the same buffer (repmsg)
    * ajp13/ajp14 is async but handling read/send this way prevent nice recovery
  @@ -1482,22 +1484,22 @@
                          "Tomcat is down or network problems. "
                          "No response has been sent to the client (yet)");
                   /*
  -                 * communication with tomcat has been interrupted BEFORE 
  +                 * communication with tomcat has been interrupted BEFORE
                    * headers have been sent to the client.
                    * DISCUSSION: As we suppose that tomcat has already started
                    * to process the query we think it's unrecoverable (and we
  -                 * should not retry or switch to another tomcat in the 
  -                 * cluster). 
  +                 * should not retry or switch to another tomcat in the
  +                 * cluster).
                    */
   
                   /*
  -                 * We mark it unrecoverable if recovery_opts set to RECOVER_ABORT_IF_TCGETREQUEST 
  +                 * We mark it unrecoverable if recovery_opts set to RECOVER_ABORT_IF_TCGETREQUEST
                    */
                   if (p->worker->recovery_opts & RECOVER_ABORT_IF_TCGETREQUEST)
                       op->recoverable = JK_FALSE;
  -                /* 
  +                /*
                    * we want to display the webservers error page, therefore
  -                 * we return JK_FALSE 
  +                 * we return JK_FALSE
                    */
                   JK_TRACE_EXIT(l);
                   return JK_FALSE;
  @@ -1507,21 +1509,21 @@
                          "Tomcat is down or network problems. "
                          "Part of the response has already been sent to the client");
   
  -                /* communication with tomcat has been interrupted AFTER 
  +                /* communication with tomcat has been interrupted AFTER
                    * headers have been sent to the client.
                    * headers (and maybe parts of the body) have already been
                    * sent, therefore the response is "complete" in a sense
  -                 * that nobody should append any data, especially no 500 error 
  -                 * page of the webserver! 
  +                 * that nobody should append any data, especially no 500 error
  +                 * page of the webserver!
                    *
                    * BUT if you retrun JK_TRUE you have a 200 (OK) code in your
  -                 * in your apache access.log instead of a 500 (Error). 
  +                 * in your apache access.log instead of a 500 (Error).
                    * Therefore return FALSE/FALSE
  -                 * return JK_TRUE; 
  +                 * return JK_TRUE;
                    */
   
                   /*
  -                 * We mark it unrecoverable if recovery_opts set to RECOVER_ABORT_IF_TCSENDHEADER 
  +                 * We mark it unrecoverable if recovery_opts set to RECOVER_ABORT_IF_TCSENDHEADER
                    */
                   if (p->worker->recovery_opts & RECOVER_ABORT_IF_TCSENDHEADER)
                       op->recoverable = JK_FALSE;
  @@ -1542,11 +1544,11 @@
               headeratclient = JK_TRUE;
           }
           else if (JK_AJP13_HAS_RESPONSE == rc) {
  -            /* 
  +            /*
                * in upload-mode there is no second chance since
  -             * we may have allready sent part of the uploaded data 
  +             * we may have allready sent part of the uploaded data
                * to Tomcat.
  -             * In this case if Tomcat connection is broken we must 
  +             * In this case if Tomcat connection is broken we must
                * abort request and indicate error.
                * A possible work-around could be to store the uploaded
                * data to file and replay for it
  @@ -1632,7 +1634,7 @@
   
           s->secret = p->worker->secret;
   
  -        /* 
  +        /*
            * We get here initial request (in reqmsg)
            */
           if (!ajp_marshal_into_msgb(op->request, s, l, p)) {
  @@ -1646,15 +1648,15 @@
   
           if (JK_IS_DEBUG_LEVEL(l))
               jk_log(l, JK_LOG_DEBUG, "processing with %d retries", s->retries);
  -        /* 
  +        /*
            * JK_RETRIES could be replaced by the number of workers in
  -         * a load-balancing configuration 
  +         * a load-balancing configuration
            */
           for (i = 0; i < s->retries; i++) {
               /*
                * We're using reqmsg which hold initial request
                * if Tomcat is stopped or restarted, we will pass reqmsg
  -             * to next valid tomcat. 
  +             * to next valid tomcat.
                */
               err = ajp_send_request(e, s, l, p, op);
               if (err == JK_TRUE) {
  @@ -1683,7 +1685,7 @@
                   }
   
                   if (err != JK_CLIENT_ERROR) {
  -                    /* if we can't get reply, check if no recover flag was set 
  +                    /* if we can't get reply, check if no recover flag was set
                        * if is_recoverable_error is cleared, we have started
                        * receiving upload data and we must consider that
                        * operation is no more recoverable
  @@ -1945,7 +1947,7 @@
                      "setting number of retries to %d",
                       pThis->retries);
           }
  -        /* 
  +        /*
            *  Need to initialize secret here since we could return from inside
            *  of the following loop
            */
  @@ -2028,7 +2030,7 @@
           JK_ENTER_CS(&w->cs, rc);
           if (rc) {
               int i, sock = -1;
  -            
  +
               if (p->sd > 0 && !p->reuse) {
                   sock  = p->sd;
                   p->sd = -1;
  @@ -2043,7 +2045,7 @@
               *e = NULL;
               JK_LEAVE_CS(&w->cs, rc);
               if (sock >= 0)
  -                jk_close_socket(sock);
  +                jk_shutdown_socket(sock);
               if (i >= 0) {
                   if (JK_IS_DEBUG_LEVEL(l))
                       jk_log(l, JK_LOG_DEBUG,
  @@ -2086,7 +2088,7 @@
   
           JK_ENTER_CS(&aw->cs, rc);
           if (rc) {
  -            unsigned int i, slot;
  +            unsigned int slot;
               for (slot = 0; slot < aw->ep_cache_sz; slot++) {
                   if (aw->ep_cache[slot]) {
                       ae = aw->ep_cache[slot];
  @@ -2094,40 +2096,7 @@
                       break;
                   }
               }
  -            /* Handle enpoint cache timeouts */
  -            if (aw->cache_timeout) {
  -                for (i = 0; i < aw->ep_cache_sz; i++) {
  -                    /* Skip the cached enty */
  -                    if (aw->ep_cache[i]) {
  -                        int elapsed = (int)(now - aw->ep_cache[i]->last_access);
  -                        if (elapsed > aw->cache_timeout) {
  -                            aw->ep_cache[i]->reuse = JK_FALSE;
  -                            ajp_reset_endpoint(aw->ep_cache[i], l);
  -                            if (JK_IS_DEBUG_LEVEL(l))
  -                                jk_log(l, JK_LOG_DEBUG,
  -                                        "cleaning cache slot=%d elapsed %u",
  -                                        i, elapsed);
  -                        }
  -                    }
  -                }
  -            }
               if (ae) {
  -                if (ae->sd > 0 && aw->recycle_timeout > 0) {
  -                    /* Handle timeouts for open sockets */
  -                    int elapsed = (int)(now - ae->last_access);
  -                    if (JK_IS_DEBUG_LEVEL(l))
  -                        jk_log(l, JK_LOG_DEBUG,
  -                               "time elapsed since last request = %u seconds",
  -                               elapsed);
  -                    if (elapsed > aw->recycle_timeout) {
  -                        ae->reuse = JK_FALSE;
  -                        ajp_reset_endpoint(ae, l);
  -                        if (JK_IS_DEBUG_LEVEL(l))
  -                            jk_log(l, JK_LOG_DEBUG,
  -                                    "reached connection recycle timeout, closed cache slot=%d",
  -                                    slot);
  -                    }
  -                }
                   ae->last_access = now;
                   *je = &ae->endpoint;
                   JK_LEAVE_CS(&aw->cs, rc);
  @@ -2158,3 +2127,62 @@
       JK_TRACE_EXIT(l);
       return JK_FALSE;
   }
  +
  +int JK_METHOD ajp_maintain(jk_worker_t *pThis, jk_logger_t *l)
  +{
  +    JK_TRACE_ENTER(l);
  +
  +    if (pThis && pThis->worker_private) {
  +        ajp_worker_t *aw = pThis->worker_private;
  +        time_t now;
  +        int rc;
  +        /* Obtain current time only if needed */
  +        if (aw->cache_timeout > 0 || aw->recycle_timeout > 0)
  +            now = time(NULL);
  +        else {
  +            /* Nothing to do. */
  +            JK_TRACE_EXIT(l);
  +            return JK_TRUE;
  +        }
  +        JK_ENTER_CS(&aw->cs, rc);
  +        if (rc) {
  +            unsigned int i;
  +            /* Handle worker cache and recycle timeouts */
  +            for (i = 0; i < aw->ep_cache_sz; i++) {
  +                /* Skip the closed sockets */
  +                if (aw->ep_cache[i] && aw->ep_cache[i]->sd >= 0) {
  +                    int elapsed = (int)difftime(now, aw->ep_cache[i]->last_access);
  +                    if (((aw->cache_timeout > 0) && (elapsed > aw->cache_timeout)) ||
  +                        ((aw->recycle_timeout > 0) && (elapsed > aw->recycle_timeout))) {
  +                        aw->ep_cache[i]->reuse = JK_FALSE;
  +                        ajp_reset_endpoint(aw->ep_cache[i], l);
  +                        if (JK_IS_DEBUG_LEVEL(l))
  +                            jk_log(l, JK_LOG_DEBUG,
  +                                    "cleaning cache slot=%d elapsed %u",
  +                                    i, elapsed);
  +                    }
  +                }
  +            }
  +            JK_LEAVE_CS(&aw->cs, rc);
  +            JK_TRACE_EXIT(l);
  +            return JK_TRUE;
  +        }
  +        else {
  +           jk_log(l, JK_LOG_ERROR,
  +                  "locking thread with errno=%d",
  +                  errno);
  +            JK_TRACE_EXIT(l);
  +            return JK_FALSE;
  +
  +        }
  +    }
  +    else {
  +        JK_LOG_NULL_PARAMS(l);
  +    }
  +
  +    JK_TRACE_EXIT(l);
  +    return JK_FALSE;
  +}
  +
  +
  +
  
  
  
  1.32      +18 -16    jakarta-tomcat-connectors/jk/native/common/jk_ajp_common.h
  
  Index: jk_ajp_common.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_ajp_common.h,v
  retrieving revision 1.31
  retrieving revision 1.32
  diff -u -r1.31 -r1.32
  --- jk_ajp_common.h	17 Feb 2005 07:08:09 -0000	1.31
  +++ jk_ajp_common.h	24 Apr 2005 09:54:47 -0000	1.32
  @@ -35,7 +35,7 @@
   
   /*
    * Conditional request attributes
  - * 
  + *
    */
   #define SC_A_CONTEXT            (unsigned char)1
   #define SC_A_SERVLET_PATH       (unsigned char)2
  @@ -56,12 +56,12 @@
    * The list of methods was taken from Section 5.1.1 of RFC 2616,
    * RFC 2518, the ACL IETF draft, and the DeltaV IESG Proposed Standard.
    *          Method        = "OPTIONS"
  - *                        | "GET"    
  - *                        | "HEAD"   
  - *                        | "POST"   
  - *                        | "PUT"    
  - *                        | "DELETE" 
  - *                        | "TRACE"  
  + *                        | "GET"
  + *                        | "HEAD"
  + *                        | "POST"
  + *                        | "PUT"
  + *                        | "DELETE"
  + *                        | "TRACE"
    *                        | "PROPFIND"
    *                        | "PROPPATCH"
    *                        | "MKCOL"
  @@ -82,7 +82,7 @@
    *                        | "MERGE"
    *                        | "BASELINE-CONTROL"
    *                        | "MKACTIVITY"
  - * 
  + *
    */
   #define SC_M_OPTIONS            (unsigned char)1
   #define SC_M_GET                (unsigned char)2
  @@ -116,7 +116,7 @@
   /*
    * Frequent request headers, these headers are coded as numbers
    * instead of strings.
  - * 
  + *
    * Accept
    * Accept-Charset
    * Accept-Encoding
  @@ -131,7 +131,7 @@
    * Pragma
    * Referer
    * User-Agent
  - * 
  + *
    */
   
   #define SC_ACCEPT               (unsigned short)0xA001
  @@ -152,7 +152,7 @@
   /*
    * Frequent response headers, these headers are coded as numbers
    * instead of strings.
  - * 
  + *
    * Content-Type
    * Content-Language
    * Content-Length
  @@ -163,7 +163,7 @@
    * Servlet-Engine
    * Status
    * WWW-Authenticate
  - * 
  + *
    */
   
   #define SC_RESP_CONTENT_TYPE        (unsigned short)0xA001
  @@ -232,11 +232,11 @@
       const char *name;
       const char *host;
       int port;
  -    /* 
  +    /*
        * Open connections cache...
        *
        * 1. Critical section object to protect the cache.
  -     * 2. Cache size. 
  +     * 2. Cache size.
        * 3. An array of "open" endpoints.
        */
       JK_CRIT_SEC cs;
  @@ -259,7 +259,7 @@
        * AJP14 will set here its login handler
        */
       int (*logon) (ajp_endpoint_t * ae, jk_logger_t *l);
  -    
  +
       /* Reclycle inactive connections */
       int recycle_timeout;
       /*
  @@ -355,6 +355,8 @@
   int ajp_connection_tcp_get_message(ajp_endpoint_t * ae,
                                      jk_msg_buf_t *msg, jk_logger_t *l);
   
  +int JK_METHOD ajp_maintain(jk_worker_t *pThis, jk_logger_t *l);
  +
   #ifdef __cplusplus
   }
   #endif                          /* __cplusplus */
  
  
  
  1.78      +15 -2     jakarta-tomcat-connectors/jk/native/common/jk_lb_worker.c
  
  Index: jk_lb_worker.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_lb_worker.c,v
  retrieving revision 1.77
  retrieving revision 1.78
  diff -u -r1.77 -r1.78
  --- jk_lb_worker.c	21 Apr 2005 11:18:44 -0000	1.77
  +++ jk_lb_worker.c	24 Apr 2005 09:54:47 -0000	1.78
  @@ -161,6 +161,18 @@
       }
   }
   
  +static int JK_METHOD maintain_workers(jk_worker_t *p, jk_logger_t *l)
  +{
  +    unsigned int i = 0;
  +    lb_worker_t *lb = (lb_worker_t *)p->worker_private;
  +    for (i = 0; i < lb->num_of_workers; i++) {
  +        if (lb->lb_workers[i].w->maintain) {
  +            lb->lb_workers[i].w->maintain(lb->lb_workers[i].w, l);
  +        }
  +    }
  +    return JK_TRUE;
  +}
  +
   static void retry_worker(worker_record_t *w,
                            int recover_wait_time,
                            jk_logger_t *l)
  @@ -609,7 +621,7 @@
                       rec->s->errors++;
                       rec->s->in_error_state = JK_TRUE;
                       rec->s->in_recovering = JK_FALSE;
  -                    rec->s->error_time = time(0);
  +                    rec->s->error_time = time(NULL);
   
                       if (is_service_error > JK_HTTP_OK) {
                           /*
  @@ -911,6 +923,7 @@
           private_data->worker.init = init;
           private_data->worker.get_endpoint = get_endpoint;
           private_data->worker.destroy = destroy;
  +        private_data->worker.maintain = maintain_workers;
           private_data->worker.retries = JK_RETRIES;
           private_data->s->recover_wait_time = WAIT_BEFORE_RECOVER;
           *w = &private_data->worker;
  
  
  
  1.31      +38 -32    jakarta-tomcat-connectors/jk/native/common/jk_service.h
  
  Index: jk_service.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_service.h,v
  retrieving revision 1.30
  retrieving revision 1.31
  diff -u -r1.30 -r1.31
  --- jk_service.h	19 Feb 2005 10:54:49 -0000	1.30
  +++ jk_service.h	24 Apr 2005 09:54:47 -0000	1.31
  @@ -97,28 +97,28 @@
    * seeing the internal vtables of your favorite OO language.  Whatever
    * works for you.
    *
  - * See apache1.3/mod_jk.c and iis/jk_isapi_plugin.c for examples.  
  + * See apache1.3/mod_jk.c and iis/jk_isapi_plugin.c for examples.
    */
   struct jk_ws_service
   {
   
  -    /* 
  +    /*
        * A 'this' pointer which is used by the subclasses of this class to
        * point to data which is specific to a given web server platform
  -     * (e.g. Apache or IIS).  
  +     * (e.g. Apache or IIS).
        */
       void *ws_private;
   
       /*
        * Provides memory management.  All data specific to this request is
        * allocated within this pool, which can then be reclaimed at the end
  -     * of the request handling cycle. 
  +     * of the request handling cycle.
        *
  -     * Alive as long as the request is alive.  
  +     * Alive as long as the request is alive.
        */
       jk_pool_t *pool;
   
  -    /* 
  +    /*
        * CGI Environment needed by servlets
        */
       const char *method;
  @@ -147,7 +147,7 @@
        * ssl_cipher   - The ssl cipher suite in use.
        * ssl_session  - The ssl session string
        *
  -     * In some servers it is impossible to extract all this information, in this 
  +     * In some servers it is impossible to extract all this information, in this
        * case, we are passing NULL.
        */
       int is_ssl;
  @@ -158,7 +158,7 @@
   
       /*
        * SSL extra information for Servlet 2.3 API
  -     * 
  +     *
        * ssl_key_size - ssl key size in use
        */
       int ssl_key_size;
  @@ -172,13 +172,13 @@
   
   
       /*
  -     * Request attributes. 
  +     * Request attributes.
        *
  -     * These attributes that were extracted from the web server and are 
  +     * These attributes that were extracted from the web server and are
        * sent to Tomcat.
        *
        * The developer should be able to read them from the ServletRequest
  -     * attributes. Tomcat is required to append org.apache.tomcat. to 
  +     * attributes. Tomcat is required to append org.apache.tomcat. to
        * these attrinbute names.
        */
       char **attributes_names;        /* Names of the request attributes  */
  @@ -188,7 +188,7 @@
       /*
        * The jvm route is in use when the adapter load balance among
        * several JVMs. It is the ID of a specific JVM in the load balance
  -     * group. We are using this variable to implement JVM session 
  +     * group. We are using this variable to implement JVM session
        * affinity
        */
       const char *jvm_route;
  @@ -215,7 +215,7 @@
       /* Number of retries. Defaults to JK_RETRIES
        */
       int retries;
  -    
  +
       /* Uri worker map. Added for virtual host support
        */
       jk_uri_worker_map_t *uw_map;
  @@ -233,7 +233,7 @@
       /*
        * Read a chunk of the request body into a buffer.  Attempt to read len
        * bytes into the buffer.  Write the number of bytes actually read into
  -     * actually_read.  
  +     * actually_read.
        */
       int (JK_METHOD * read) (jk_ws_service_t *s,
                               void *buffer,
  @@ -274,17 +274,17 @@
    * imagine that you are seeing the internal vtables of your favorite OO
    * language.  Whatever works for you.
    *
  - * See jk_ajp13_worker.c/jk_ajp14_worker.c and jk_ajp12_worker.c for examples.  
  + * See jk_ajp13_worker.c/jk_ajp14_worker.c and jk_ajp12_worker.c for examples.
    */
   struct jk_endpoint
   {
       size_t rd;
       size_t wr;
   
  -    /* 
  +    /*
        * A 'this' pointer which is used by the subclasses of this class to
  -     * point to data/functions which are specific to a given protocol 
  -     * (e.g. ajp12 or ajp13 or ajp14).  
  +     * point to data/functions which are specific to a given protocol
  +     * (e.g. ajp12 or ajp13 or ajp14).
        */
       void *endpoint_private;
   
  @@ -302,7 +302,7 @@
        * Called when this particular endpoint has finished processing a
        * request.  For some protocols (e.g. ajp12), this frees the memory
        * associated with the endpoint.  For others (e.g. ajp13/ajp14), this can
  -     * return the endpoint to a cache of already opened endpoints.  
  +     * return the endpoint to a cache of already opened endpoints.
        *
        * Note that the first argument is *not* a 'this' pointer, but is
        * rather a pointer to a 'this' pointer.  This is necessary, because
  @@ -313,7 +313,7 @@
   
   /*
    * The worker 'class', which represents something to which the web server
  - * can delegate requests. 
  + * can delegate requests.
    *
    * This can mean communicating with a particular servlet engine instance,
    * using a particular protocol.  A single web server instance may have
  @@ -348,7 +348,7 @@
    * imagine that you are seeing the internal vtables of your favorite OO
    * language.  Whatever works for you.
    *
  - * See jk_ajp14_worker.c, jk_ajp13_worker.c and jk_ajp12_worker.c for examples.  
  + * See jk_ajp14_worker.c, jk_ajp13_worker.c and jk_ajp12_worker.c for examples.
    */
   struct jk_worker
   {
  @@ -358,17 +358,17 @@
        * on this worker.
        */
       int retries;
  -    /* 
  +    /*
        * A 'this' pointer which is used by the subclasses of this class to
  -     * point to data/functions which are specific to a given protocol 
  -     * (e.g. ajp12 or ajp13 or ajp14).  
  +     * point to data/functions which are specific to a given protocol
  +     * (e.g. ajp12 or ajp13 or ajp14).
        */
       void *worker_private;
  -    
  +
       int   type;
       /*
        * For all of the below (except destroy), the first argument is
  -     * essentially a 'this' pointer.  
  +     * essentially a 'this' pointer.
        */
   
       /*
  @@ -376,7 +376,7 @@
        * of configuration options (or 'properties'), check to see if it the
        * options are.  This will always be called before the init() method.
        * The init/validate distinction is a bit hazy to me.
  -     * See jk_ajp13_worker.c/jk_ajp14_worker.c and jk_worker.c->wc_create_worker() 
  +     * See jk_ajp13_worker.c/jk_ajp14_worker.c and jk_worker.c->wc_create_worker()
        */
       int (JK_METHOD * validate) (jk_worker_t *w,
                                   jk_map_t *props,
  @@ -391,7 +391,7 @@
   
       /*
        * Do whatever initialization needs to be done to start this worker up.
  -     * Configuration options are passed in via the props parameter.  
  +     * Configuration options are passed in via the props parameter.
        */
       int (JK_METHOD * init) (jk_worker_t *w,
                               jk_map_t *props,
  @@ -400,7 +400,7 @@
   
       /*
        * Obtain an endpoint to service a particular request.  A pointer to
  -     * the endpoint is stored in pend.  
  +     * the endpoint is stored in pend.
        */
       int (JK_METHOD * get_endpoint) (jk_worker_t *w,
                                       jk_endpoint_t **pend, jk_logger_t *l);
  @@ -408,9 +408,15 @@
       /*
        * Shutdown this worker.  The first argument is not a 'this' pointer,
        * but rather a pointer to 'this', so that the object can be free'd (I
  -     * think -- though that doesn't seem to be happening.  Hmmm).  
  +     * think -- though that doesn't seem to be happening.  Hmmm).
        */
       int (JK_METHOD * destroy) (jk_worker_t **w, jk_logger_t *l);
  +
  +    /*
  +     * Maintain this worker.
  +     */
  +    int (JK_METHOD * maintain) (jk_worker_t *w, jk_logger_t *l);
  +
   };
   
   /*
  @@ -424,7 +430,7 @@
    *
    * This allows new workers to be written without modifing the plugin code
    * for the various web servers (since the only link is through
  - * jk_worker_list.h).  
  + * jk_worker_list.h).
    */
   typedef int (JK_METHOD * worker_factory) (jk_worker_t **w,
                                             const char *name,
  
  
  
  1.32      +49 -3     jakarta-tomcat-connectors/jk/native/common/jk_worker.c
  
  Index: jk_worker.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_worker.c,v
  retrieving revision 1.31
  retrieving revision 1.32
  diff -u -r1.31 -r1.32
  --- jk_worker.c	15 Feb 2005 08:52:54 -0000	1.31
  +++ jk_worker.c	24 Apr 2005 09:54:47 -0000	1.32
  @@ -37,16 +37,25 @@
   
   /* Global worker list */
   static jk_map_t *worker_map;
  +static JK_CRIT_SEC worker_lock;
   
   int wc_open(jk_map_t *init_data, jk_worker_env_t *we, jk_logger_t *l)
   {
  -
  +    int rc;
       JK_TRACE_ENTER(l);
   
       if (!jk_map_alloc(&worker_map)) {
           JK_TRACE_EXIT(l);
           return JK_FALSE;
       }
  +    JK_INIT_CS(&worker_lock, rc);
  +    if (rc == JK_FALSE) {
  +        jk_log(l, JK_LOG_ERROR,
  +                "creating thread lock errno=%d",
  +                errno);
  +        JK_TRACE_EXIT(l);
  +        return JK_FALSE;
  +    }
   
       if (!jk_get_worker_list(init_data, &(we->worker_list),
                               &we->num_of_workers)) {
  @@ -72,7 +81,9 @@
   
   void wc_close(jk_logger_t *l)
   {
  +    int rc;
       JK_TRACE_ENTER(l);
  +    JK_DELETE_CS(&worker_lock, rc);
       close_workers(l);
       JK_TRACE_EXIT(l);
   }
  @@ -97,7 +108,6 @@
       return rc;
   }
   
  -
   int wc_create_worker(const char *name,
                        jk_map_t *init_data,
                        jk_worker_t **rc, jk_worker_env_t *we, jk_logger_t *l)
  @@ -261,3 +271,39 @@
   
       return NULL;
   }
  +
  +void wc_maintain(jk_logger_t *l)
  +{
  +    static time_t last_maintain = 0;
  +    int sz = jk_map_size(worker_map);
  +
  +    JK_TRACE_ENTER(l);
  +
  +    /* TODO: make maintatin time configurable
  +     * For now use 10 seconds.
  +     */
  +    if (sz > 0) {
  +        int i;
  +        time_t now;
  +        JK_ENTER_CS(&worker_lock, i);
  +        now = time(NULL);
  +        if (difftime(now, last_maintain) >= 10) {
  +            last_maintain = now;
  +            JK_LEAVE_CS(&worker_lock, i);
  +            for (i = 0; i < sz; i++) {
  +                jk_worker_t *w = jk_map_value_at(worker_map, i);
  +                if (w && w->maintain) {
  +                    if (JK_IS_DEBUG_LEVEL(l))
  +                        jk_log(l, JK_LOG_DEBUG,
  +                               "Maintaining worker %s",
  +                               jk_map_name_at(worker_map, i));
  +                    w->maintain(w, l);
  +                }
  +            }
  +        }
  +        else {
  +            JK_LEAVE_CS(&worker_lock, i);
  +        }
  +    }
  +    JK_TRACE_EXIT(l);
  +}
  
  
  
  1.12      +3 -2      jakarta-tomcat-connectors/jk/native/common/jk_worker.h
  
  Index: jk_worker.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/native/common/jk_worker.h,v
  retrieving revision 1.11
  retrieving revision 1.12
  diff -u -r1.11 -r1.12
  --- jk_worker.h	8 Nov 2004 13:34:37 -0000	1.11
  +++ jk_worker.h	24 Apr 2005 09:54:47 -0000	1.12
  @@ -16,7 +16,7 @@
   
   /***************************************************************************
    * Description: Workers controller header file                             *
  - * Author:      Gal Shachor <sh...@il.ibm.com>                           * 
  + * Author:      Gal Shachor <sh...@il.ibm.com>                           *
    * Version:     $Revision$                                           *
    ***************************************************************************/
   
  @@ -44,6 +44,7 @@
                        jk_worker_t **rc,
                        jk_worker_env_t *we, jk_logger_t *l);
   
  +void wc_maintain(jk_logger_t *l);
   
   #ifdef __cplusplus
   }
  
  
  

---------------------------------------------------------------------
To unsubscribe, e-mail: tomcat-dev-unsubscribe@jakarta.apache.org
For additional commands, e-mail: tomcat-dev-help@jakarta.apache.org