You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tomcat.apache.org by sh...@locus.apache.org on 2000/05/19 08:59:03 UTC

cvs commit: jakarta-tomcat/src/native/jk jk_ajp13.c jk_ajp13.h jk_ajp13_worker.c jk_ajp13_worker.h jk_mt.h jk_ajp23_worker.c jk_connect.c jk_connect.h jk_global.h jk_msg_buff.c jk_msg_buff.h jk_pool.h jk_uri_worker_map.c jk_util.c jk_util.h

shachor     00/05/18 23:59:02

  Modified:    src/native/jk jk_ajp23_worker.c jk_connect.c jk_connect.h
                        jk_global.h jk_msg_buff.c jk_msg_buff.h jk_pool.h
                        jk_uri_worker_map.c jk_util.c jk_util.h
  Added:       src/native/jk jk_ajp13.c jk_ajp13.h jk_ajp13_worker.c
                        jk_ajp13_worker.h jk_mt.h
  Log:
  Adding support for ajp13, a bi-directional & faster communication
  protocol that reuse socket connections.
  
  This also support multi-thread programs.
  
  Work is not done yet. The protocol does not pass POSTed data yet
  
  Revision  Changes    Path
  1.3       +178 -245  jakarta-tomcat/src/native/jk/jk_ajp23_worker.c
  
  Index: jk_ajp23_worker.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_ajp23_worker.c,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- jk_ajp23_worker.c	2000/04/25 00:30:59	1.2
  +++ jk_ajp23_worker.c	2000/05/19 06:58:59	1.3
  @@ -53,12 +53,12 @@
    *
    */
   
  -/** 
  -    Experimental bi-directionl protocol.
  -    ( updated from src/native/apache/connector )
  -    
  -    @author costin@costin.dnt.ro
  -*/
  +/***************************************************************************
  + * Description: Experimental bi-directionl protocol.                       *
  + * Author:      Costin <co...@costin.dnt.ro>                              *
  + * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  + * Version:     $Revision: 1.3 $                                           *
  + ***************************************************************************/
   
   #include "jk_ajp23_worker.h"
   #include "jk_pool.h"
  @@ -66,58 +66,14 @@
   #include "jk_util.h"
   #include "jk_msg_buff.h"
   #include "jk_sockbuf.h"
  +#include "jk_mt.h"
   
   #define AJP_DEF_HOST            ("localhost")
   #define AJP_DEF_PORT            (8008)
   #define READ_BUF_SIZE           (8*1024)
   #define DEF_RETRY_ATTEMPTS      (1)
   
  -/** "PseudoObjects" in JK: 
  -    struct Base {   
  -       void (*method)();
  -       SubClass *subClass;
  -    }
  -
  -    Base *SubClassFactory();
  -    struct SubClass {
  -       char *propertiesSubcl;
  -    }
  -
   
  -    We pass around pointers to the Base object. 
  -    XXX We could check the type of the worker too to validate.
  -    
  -    All methods are returning int==error code ( 0==success )
  -    The first parameter is a pointer to this.
  -    XXX have second param a pointer to ctx ( include log in ctx)
  -    
  -
  -    Conventions: _t == type
  - */
  -
  -/** Objects:
  -    class Worker {
  -      int validate(Properties props);
  -      int init(Properties props );
  -      int get_endpoint( Endpoint *endp );
  -      int destroy();
  -    }
  -
  -    class Endpoint {
  -      int service( WsService s, int *isRecoverable);
  -      int done();
  -    }
  -
  -    class WsService {
  -      Pool *pool;
  -      ... all fields in request_rec ...
  -      int startResponse( int status, String reason, String headers[], String values[], int hCount );
  -      int read( char buff[], int len, int *read);
  -      int write( char buff[], int len );
  -    }
  -
  -*/
  -
   /**
      Utilities:
      
  @@ -232,23 +188,33 @@
       return JK_FALSE;
   }
   
  -static int jk_b_append_headers( MsgBuffer *msg, jk_ws_service_t *s, jk_logger_t *l) {
  +static int jk_b_append_headers(jk_msg_buf_t *msg, 
  +                               jk_ws_service_t *s, 
  +                               jk_logger_t *l) 
  +{
       /* Send the request headers */
  -    int err=jk_b_append_int( msg, s->num_headers);
  -    if(err<0) return err;
  +    int err = jk_b_append_int(msg, (unsigned short)s->num_headers);
  +    if(err<0) {
  +        return err;
  +    }
   
       if(s->num_headers) {
           unsigned  i;
           for(i = 0 ; i < s->num_headers ; ++i) {
  -	    err=jk_b_append_string( msg, s->headers_names[i] );
  -	    if (err<0)  return err;
  -	    err=jk_b_append_string( msg, s->headers_values[i] );
  -	    if (err<0)  return err;
  -	}
  +	        err = jk_b_append_string( msg, s->headers_names[i]);
  +            if(err < 0)  { 
  +                return err;
  +            }
  +	        err = jk_b_append_string(msg, s->headers_values[i]);
  +            if(err < 0) {
  +                return err;
  +            }
  +	    }
       }
  +
       if(!err) {
  -	jk_log(l, JK_LOG_ERROR, "In ajpv12_handle_request, failed to send headers\n");
  -	return JK_FALSE;
  +	    jk_log(l, JK_LOG_ERROR, "In ajpv12_handle_request, failed to send headers\n");
  +	    return JK_FALSE;
       }
       
       return JK_TRUE;
  @@ -276,8 +242,8 @@
       Small methods inlined
       XXX add support for user-defined methods
    */
  -static void encode_request( MsgBuffer *msg, jk_ws_service_t *r, jk_logger_t *l );
  -static int process_callback(  MsgBuffer *msg, jk_ws_service_t *r, jk_logger_t *l);
  +static void encode_request( jk_msg_buf_t *msg, jk_ws_service_t *r, jk_logger_t *l );
  +static int process_callback(  jk_msg_buf_t *msg, jk_ws_service_t *r, jk_logger_t *l);
   
   /* XXX what's above this line should go to .h XXX */
   
  @@ -287,27 +253,28 @@
      For methods not starting with 0xFF type is a length, and 
      the method _name_ follows 
   */
  -static void encode_env(  MsgBuffer *msg, jk_ws_service_t *r ) {
  +static void encode_env( jk_msg_buf_t *msg, jk_ws_service_t *r) 
  +{
       //    jk_b_append_table( msg, r->subprocess_env );
       /* XXX use r instead of env */
       jk_b_append_int( msg, 6 );
  -    jk_b_append_string( msg, "REQUEST_METHOD" );
  -    jk_b_append_string( msg, r->method );
  +    jk_b_append_string(msg, "REQUEST_METHOD" );
  +    jk_b_append_string(msg, r->method );
   
  -    jk_b_append_string( msg, "SERVER_PROTOCOL");
  -    jk_b_append_string( msg, r->protocol );
  +    jk_b_append_string(msg, "SERVER_PROTOCOL");
  +    jk_b_append_string(msg, r->protocol);
   
  -    jk_b_append_string( msg, "REQUEST_URI" );
  -    jk_b_append_string( msg, r->req_uri );
  +    jk_b_append_string(msg, "REQUEST_URI");
  +    jk_b_append_string(msg, r->req_uri );
   
  -    jk_b_append_string( msg, "QUERY_STRING" );
  -    jk_b_append_string( msg, r->query_string );
  +    jk_b_append_string(msg, "QUERY_STRING");
  +    jk_b_append_string(msg, r->query_string );
   
  -    jk_b_append_string( msg, "SERVER_PORT" );
  -    jk_b_append_string( msg, "8080" );
  +    jk_b_append_string(msg, "SERVER_PORT");
  +    jk_b_append_string(msg, "8080" );
   
  -    jk_b_append_string( msg, "REMOTE_ADDR");
  -    jk_b_append_string( msg, r->remote_addr );
  +    jk_b_append_string(msg, "REMOTE_ADDR");
  +    jk_b_append_string(msg, r->remote_addr);
   
   }
   
  @@ -317,43 +284,22 @@
       XXX integrate with mod_session (notes)
       XXX integrate with special module to find the context (notes)
   */
  -static void encode_request( MsgBuffer *msg, jk_ws_service_t *r, jk_logger_t *l ) {
  -
  -    jk_b_append_int( msg, REQUEST_FORWARD ); 
  -    encode_env( msg, r );
  -    jk_b_append_headers( msg, r, l );
  -
  -/*     // Append first chunk of request body ( up to the buffer size ) */
  -    /*     printf("Encode request \n"); */
  -/*     if ( ! ap_should_client_block(r)) { */
  -/* 	// no body, send 0 */
  -	/* printf("No body\n"); */
  +static void encode_request( jk_msg_buf_t *msg, jk_ws_service_t *r, jk_logger_t *l ) 
  +{
  +    jk_b_append_int(msg, REQUEST_FORWARD); 
  +    encode_env(msg, r);
  +    jk_b_append_headers(msg, r, l);
   	jk_b_append_int( msg, 0 );
  -/*     } else { */
  -/*         int maxsize=jk_b_get_size( msg ); */
  -/* 	char *buffer=jk_b_get_buff(msg); */
  -/* 	int posLen= jk_b_get_len( msg ); */
  -/* 	int pos=posLen +2 ; */
  -/*         long rd; */
  -
  -/* 	/* Read in buff, at pos + 2 ( let space for size ), up to  */
  -/* 	   maxsize - pos. */
  -/*         while ( (pos < maxsize ) &&  (rd=ap_get_client_block(r,buffer+pos, maxsize - pos ))>0) { */
  -	    /*     printf( "Reading %d %d %d \n", posLen, pos, maxsize ); */
  -/* 	    pos=pos + rd; */
  -/*         } */
  -	/* 	printf( "End reading %d %d %d \n", posLen, pos, maxsize ); */
  -/* 	jk_b_set_int( msg, posLen, pos - posLen -2 ); */
  -/* 	jk_b_set_len( msg, pos ); */
  -	/* 	jk_b_dump(msg, "Post ");  */
  -    /*     jk_b_dump(msg, "Encode req"); */
   }
   
   /** 
       SetHeaders callback - all headers are added to headers->out, no 
       more parsing 
   */
  -static int setHeaders( MsgBuffer *msg, jk_ws_service_t *r, jk_logger_t *l) {
  +static int setHeaders(jk_msg_buf_t *msg, 
  +                      jk_ws_service_t *r, 
  +                      jk_logger_t *l) 
  +{
       int i;
       int count;
       char **names=NULL;
  @@ -391,37 +337,12 @@
   /** 
       Get Body Chunk
   */
  -static int getBodyChunk( MsgBuffer *msg, jk_ws_service_t *r) {
  -    int i;
  -    int count;
  -
  +static int getBodyChunk(jk_msg_buf_t *msg, jk_ws_service_t *r) 
  +{
       /* No parameters, send body */
  -    jk_b_reset( msg );
  -    jk_b_append_int( msg, SEND_BODY_CHUNK );
  -    
  -/*     if ( ! ap_should_client_block(r)) { */
  -/* 	// no body, send 0 */
  -/* 	printf("No body\n"); */
  -/* 	jk_b_append_int( msg, 0 ); */
  -/*     } else { */
  -/*         int maxsize=jk_b_get_size( msg ); */
  -/* 	char *buffer=jk_b_get_buff(msg); */
  -/* 	int posLen= jk_b_get_len( msg ); */
  -/* 	int pos=posLen +2 ; */
  -/*         long rd; */
  -	
  -/* 	/* Read in buff, at pos + 2 ( let space for size ), up to  */
  -/* 	   maxsize - pos. */
  -/*         while ( (pos < maxsize ) &&  (rd=ap_get_client_block(r,buffer+pos, maxsize - pos ))>0) { */
  -/* 	    printf( "Reading %d %d %d \n", posLen, pos, maxsize ); */
  -/* 	    pos=pos + rd; */
  -/*         } */
  -/* 	printf( "End reading %d %d %d \n", posLen, pos, maxsize ); */
  -/* 	jk_b_set_int( msg, posLen, pos - posLen -2 ); */
  -/* 	jk_b_set_len( msg, pos ); */
  -/* 	jk_b_dump(msg, "Post additional data");  */
  -/*     } */
  -    
  +    jk_b_reset(msg);
  +    jk_b_append_int(msg, SEND_BODY_CHUNK);
  +
       return HAS_RESPONSE;
   }
   
  @@ -429,29 +350,34 @@
       Small methods inlined
       XXX add support for user-defined methods
    */
  -int process_callback( MsgBuffer *msg, jk_ws_service_t *r, jk_logger_t *l) {
  +int process_callback(jk_msg_buf_t *msg, 
  +                     jk_ws_service_t *r, 
  +                     jk_logger_t *l) 
  +{
       int len;
  -
  -    /*     printf("Callback %x\n", jk_b_getCode(msg)); */
       switch( jk_b_pget_int(msg,0 ) ) {
  -    case SET_HEADERS:
  -	setHeaders( msg , r, l);
  -	break;
  -    case SEND_BODY_CHUNK:
  -	len=jk_b_get_int( msg );
  -	r->write( r, msg->buf + msg->pos, len);
  -	break;
  -    case GET_BODY_CHUNK:
  -	getBodyChunk( msg, r );
  -	return HAS_RESPONSE;
  -	break;
  -    case END_RESPONSE:
  -	break;
  -    default:
  -	jk_b_dump( msg , "Invalid code");
  -	jk_log( l, JK_LOG_ERROR,
  -		"Invalid code: %d\n", jk_b_pget_int(msg,0));
  -	return -1;
  +        case SET_HEADERS:
  +	        setHeaders( msg , r, l);
  +	    break;
  +
  +        case SEND_BODY_CHUNK:
  +	        len=jk_b_get_int( msg );
  +	        r->write( r, jk_b_get_buff(msg) + jk_b_get_pos(msg), len);
  +	    break;
  +
  +        case GET_BODY_CHUNK:
  +	        getBodyChunk( msg, r );
  +	        return HAS_RESPONSE;
  +	    break;
  +
  +        case END_RESPONSE:
  +	    break;
  +
  +        default:
  +	        jk_b_dump( msg , "Invalid code");
  +	        jk_log(l, JK_LOG_ERROR,
  +		           "Invalid code: %d\n", jk_b_pget_int(msg,0));
  +	        return -1;
       }
       
       return NO_RESPONSE;
  @@ -460,26 +386,33 @@
   
   // ---------------------------------------- START TCP ----------------------------------------
   
  -static int connection_tcp_send_message(  ajp23_endpoint_t *con, MsgBuffer *msg, jk_logger_t *l ) {
  +static int connection_tcp_send_message(ajp23_endpoint_t *con, 
  +                                       jk_msg_buf_t *msg, 
  +                                       jk_logger_t *l ) 
  +{
       int sent=0;
       int i;
  -    
  -    jk_b_end( msg );
  +    int len;
  +
  +    jk_b_end(msg);
  +    len = jk_b_get_len(msg);
       /*     printf("Sending %x %x %x %x\n", msg->buf[0],msg->buf[1],msg->buf[2],msg->buf[3]) ;  */
  -    while( sent < msg->len ) {
  -	i=write( con->sd, msg->buf + sent , msg->len - sent );
  -	/* 	printf("i=%d\n", i); */
  -	if( i == 0 ) {
  -	    return -2;
  -	}
  -	if( i < 0 ) {
  -	    return -3;
  -	}
  -	sent += i;
  +    while(sent < len) {
  +	    i = send(con->sd, 
  +                 jk_b_get_buff(msg) + sent , 
  +                 len - sent, 
  +                 0);
  +
  +	    /* 	printf("i=%d\n", i); */
  +	    if(i == 0) {
  +	        return -2;
  +	    }
  +	    if(i < 0) {
  +	        return -3;
  +	    }
  +	    sent += i;
       }
   
  -    /* ... */
  -    /*     flush( con->socket ); */
       return 0;
   }
   
  @@ -494,7 +427,7 @@
       int i;
   
       while( rdlen < msglen ) {
  -	i=read( con->sd, buff + rdlen, msglen - rdlen );
  +	i=recv( con->sd, buff + rdlen, msglen - rdlen, 0 );
   	/* 	printf( "Read: %d %d %x %x %x\n", i, rdlen, i, rdlen, msglen ); */
   	
   	if(i==-1) {
  @@ -520,43 +453,41 @@
   /** 
       Read a callback 
    */
  -static int connection_tcp_get_message( ajp23_endpoint_t *con, MsgBuffer *msg, jk_logger_t *l ) {
  +static int connection_tcp_get_message(ajp23_endpoint_t *con, 
  +                                      jk_msg_buf_t *msg, 
  +                                      jk_logger_t *l ) 
  +{
       char head[6];
  -    int rdlen;
  -    int i;
  -    int pos;
  +    int rc;
       int msglen;
  -    int off;
  -    char *message;
  -    int *imessage;
  -
  -    /*     // mark[2] + len[2]  */
  -    i=read_full( con, head, 4, l );
  -    /*     printf( "XXX %d \n" , i ) ;  */
  -    if(i<0) return i;
  -    
  -    if( (head[0] != 'A') || (head[1] != 'B' )) {
  -	return ERR_BAD_PACKET ;
  -    }
  -
  -    /*    sreq->msglen=get_I( head, &pos ); */
  -    msglen=((head[2]&0xff)<<8);
  -    msglen+= (head[3] & 0xFF);
  +
  +    rc = read_full(con, head, 4, l);
  +
  +    if(rc < 0) {
  +        return rc;
  +    }
  +    
  +    if((head[0] != 'A') || (head[1] != 'B' )) {
  +	    return ERR_BAD_PACKET ;
  +    }
  +
  +    msglen  =((head[2]&0xff)<<8);
  +    msglen += (head[3] & 0xFF);
   
       /* printf( "Packet len %d %x\n", msglen, msglen ); */
   
  -    if(msglen > jk_b_get_size(msg) ) {
  -	printf("Message too long ");
  -	return -5; /* XXX */
  -	/* 	sreq->message=(char *)ap_palloc( p, sreq->msglen ); */
  -	/* 		      "Re-alocating msg buffer, %d %d\n", sreq->buffSize, sreq->msglen);
  -	/* 	sreq->buffSize = sreq->msglen; */
  +    if(msglen > jk_b_get_size(msg)) {
  +    	printf("Message too long ");
  +	    return -5; /* XXX */
       }
       
  -    msg->len=msglen;
  -    msg->pos=2; /* After code */
  -    i=read_full(con, msg->buf, msglen, l );
  -    if( i<0) return i;
  +    jk_b_set_len(msg, msglen);
  +    jk_b_set_pos(msg, 2); /* After code */
  +
  +    rc = read_full(con, jk_b_get_buff(msg), msglen, l);
  +    if(rc < 0) {
  +        return rc;
  +    }
       
       /*     jk_b_dump( msg, " RCV: " ); */
       return 0;
  @@ -590,8 +521,8 @@
               }
           }
           if(p->sd >= 0) {
  -	    MsgBuffer *msg;
  -	    int err;
  +	        jk_msg_buf_t *msg;
  +	        int err;
               /*
                * After we are connected, each error that we are going to
                * have is probably unrecoverable
  @@ -599,44 +530,46 @@
               *is_recoverable_error = JK_FALSE;
               jk_sb_open(&p->sb, p->sd);
   
  -	    msg = jk_b_new( s->pool );
  -	    jk_b_set_buffer_size( msg, 2048); 
  +	        msg = jk_b_new(s->pool);
  +	        jk_b_set_buffer_size( msg, 2048); 
   
  -	    jk_b_reset( msg );
  -	    encode_request( msg , s, l );
  +	        jk_b_reset(msg);
  +	        encode_request(msg , s, l);
       
  -	    err= connection_tcp_send_message( p, msg, l );
  +	        err = connection_tcp_send_message(p, msg, l);
       
  -	    if(err<0) {
  -		jk_log( l, JK_LOG_ERROR,
  -			"Error sending request %d\n", err);
  -		return JK_FALSE;
  -	    }
  +	        if(err < 0) {
  +		        jk_log(l, JK_LOG_ERROR,
  +			           "Error sending request %d\n", err);
  +		        return JK_FALSE;
  +	        }
   	    
   
  -	    while( 1 ) {
  -		int err=connection_tcp_get_message( p, msg, l );
  -		/* 	jk_b_dump(msg, "Get Message: " ); */
  -		if( err < 0 ) {
  -		    jk_log( l, JK_LOG_ERROR,
  -				  "Error reading request %d\n", err);
  -		    // XXX cleanup, close connection if packet error
  -		    return JK_FALSE;
  -		}
  -		if( jk_b_pget_int( msg, 0 ) == END_RESPONSE )
  -		    break;
  -		err=process_callback( msg, s, l );
  -		if( err == HAS_RESPONSE ) {
  -		    err=connection_tcp_send_message( p, msg, l );
  -		    if( err < 0 ) {
  -			jk_log( l, JK_LOG_DEBUG,
  -				      "Error reading response1 %d\n", err);
  -			return JK_FALSE;
  -		    }
  -		}
  -		if( err < 0 ) break; /* XXX error */
  -	    }
  -	    return JK_TRUE;
  +	        while(1) {
  +		        int err = connection_tcp_get_message(p, msg, l);
  +		        /* 	jk_b_dump(msg, "Get Message: " ); */
  +		        if(err < 0) {
  +		            jk_log(l, JK_LOG_ERROR,
  +				           "Error reading request %d\n", err);
  +		            return JK_FALSE;
  +		        }
  +                if(jk_b_pget_int(msg, 0) == END_RESPONSE) {
  +		            break;
  +                }
  +		        err = process_callback(msg, s, l);
  +		        if(err == HAS_RESPONSE) {
  +		            err = connection_tcp_send_message( p, msg, l );
  +		            if(err < 0) {
  +			            jk_log(l, JK_LOG_DEBUG,
  +				               "Error reading response1 %d\n", err);
  +			            return JK_FALSE;
  +		            }
  +                }
  +                if(err < 0) {
  +                    break; /* XXX error */
  +                }
  +	        }
  +	        return JK_TRUE;
   	    
           }
           jk_log(l, JK_LOG_ERROR, "In jk_endpoint_t::service, Error sd = %d\n", p->sd);
  @@ -686,21 +619,21 @@
       jk_log(l, JK_LOG_DEBUG, "Into ajp23_worker_factory\n");
       if(NULL == name || NULL == w) {
           jk_log(l, JK_LOG_ERROR, "In ajp23_worker_factory, NULL parameters\n");
  -	return JK_FALSE;
  +	    return JK_FALSE;
       }
       
       
  -    if(! private_data) {
  +    if(!private_data) {
           jk_log(l, JK_LOG_ERROR, "In ajp23_worker_factory, NULL parameters\n");
  -	return JK_FALSE;
  +	    return JK_FALSE;
       }
   
       private_data->name = strdup(name);          
       
  -    if( ! private_data->name) {
  -	free(private_data);
  -	jk_log(l, JK_LOG_ERROR, "In ajp23_worker_factory, malloc failed\n");
  -	return JK_FALSE;
  +    if(!private_data->name) {
  +	    free(private_data);
  +	    jk_log(l, JK_LOG_ERROR, "In ajp23_worker_factory, malloc failed\n");
  +	    return JK_FALSE;
       } 
   
       private_data->connect_retry_attempts= DEF_RETRY_ATTEMPTS;
  
  
  
  1.2       +58 -1     jakarta-tomcat/src/native/jk/jk_connect.c
  
  Index: jk_connect.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_connect.c,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_connect.c	2000/04/17 09:47:23	1.1
  +++ jk_connect.c	2000/05/19 06:58:59	1.2
  @@ -57,7 +57,7 @@
    * Description: Socket/Naming manipulation functions                       *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
    * Based on:    Various Jserv files                                        *
  - * Version:     $Revision: 1.1 $                                               *
  + * Version:     $Revision: 1.2 $                                               *
    ***************************************************************************/
   
   
  @@ -98,6 +98,7 @@
       return JK_TRUE;
   }
   
  +
   int jk_open_socket(struct sockaddr_in *addr, 
                      int ndelay,
                      jk_logger_t *l)
  @@ -164,4 +165,60 @@
   #endif
   
       return -1;
  +}
  +
  +int jk_tcp_socket_sendfull(int sd, 
  +                           const unsigned char *b,
  +                           int len)
  +{
  +    int sent = 0;
  +
  +    while(sent < len) {
  +        int this_time = send(sd, 
  +                             b + sent , 
  +                             len - sent, 
  +                             0);
  +	    
  +	    if(0 == this_time) {
  +	        return -2;
  +	    }
  +	    if(this_time < 0) {
  +	        return -3;
  +	    }
  +	    sent += this_time;
  +    }
  +
  +    return sent;
  +}
  +
  +int jk_tcp_socket_recvfull(int sd, 
  +                           unsigned char *b, 
  +                           int len) 
  +{
  +    int rdlen = 0;
  +
  +    while(rdlen < len) {
  +	    int this_time = recv(sd, 
  +                             b + rdlen, 
  +                             len - rdlen, 
  +                             0);	
  +	    if(-1 == this_time) {
  +#ifdef WIN32
  +            if(SOCKET_ERROR == this_time) { 
  +                errno = WSAGetLastError() - WSABASEERR;
  +            }
  +#endif /* WIN32 */
  +
  +    	    if(EAGAIN == errno) {
  +                continue;
  +	        } 
  +		    return -1;
  +	    }
  +        if(0 == this_time) {
  +            return -1; 
  +        }
  +	    rdlen += this_time;
  +    }
  +
  +    return rdlen;
   }
  
  
  
  1.2       +9 -1      jakarta-tomcat/src/native/jk/jk_connect.h
  
  Index: jk_connect.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_connect.h,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_connect.h	2000/04/17 09:47:22	1.1
  +++ jk_connect.h	2000/05/19 06:59:00	1.2
  @@ -56,7 +56,7 @@
   /***************************************************************************
    * Description: Socket connections header file                             *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  - * Version:     $Revision: 1.1 $                                               *
  + * Version:     $Revision: 1.2 $                                               *
    ***************************************************************************/
   
   #ifndef JK_CONNECT_H
  @@ -82,6 +82,14 @@
                      jk_logger_t *l);
   
   int jk_close_socket(int s);
  +
  +int jk_tcp_socket_sendfull(int sd, 
  +                           const unsigned char *b,
  +                           int len);
  +
  +int jk_tcp_socket_recvfull(int sd, 
  +                           unsigned char *b, 
  +                           int len);
   
   #ifdef __cplusplus
   }
  
  
  
  1.2       +2 -1      jakarta-tomcat/src/native/jk/jk_global.h
  
  Index: jk_global.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_global.h,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_global.h	2000/04/17 09:47:22	1.1
  +++ jk_global.h	2000/05/19 06:59:00	1.2
  @@ -57,7 +57,7 @@
    * Description: Global definitions and include files that should exist     *
    *              anywhere                                                   *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  - * Version:     $Revision: 1.1 $                                               *
  + * Version:     $Revision: 1.2 $                                               *
    ***************************************************************************/
   
   #ifndef JK_GLOBAL_H
  @@ -83,6 +83,7 @@
   
       #include <netinet/in.h>
       #include <netinet/tcp.h>
  +    #include <arpa/inet.h>
   
       #include <sys/un.h>
       #include <sys/socket.h>
  
  
  
  1.2       +189 -107  jakarta-tomcat/src/native/jk/jk_msg_buff.c
  
  Index: jk_msg_buff.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_msg_buff.c,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_msg_buff.c	2000/04/25 00:31:00	1.1
  +++ jk_msg_buff.c	2000/05/19 06:59:00	1.2
  @@ -53,9 +53,12 @@
    *
    */
   
  -/** 
  -    Simple marshaling code.
  -*/
  +/***************************************************************************
  + * Description: Data marshaling. XDR like                                  *
  + * Author:      Costin <co...@costin.dnt.ro>                              *
  + * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  + * Version:     $Revision: 1.2 $                                           *
  + ***************************************************************************/
   
   #include "jk_pool.h"
   #include "jk_connect.h"
  @@ -63,188 +66,267 @@
   #include "jk_sockbuf.h"
   #include "jk_msg_buff.h"
   
  +struct jk_msg_buf {
  +    jk_pool_t *pool;
   
  +    unsigned char *buf;
  +    int pos; 
  +    int len;
  +    int maxlen;
  +};
  +
  +
  +/*
  + * Simple marshaling code.
  + */
   
   /* XXX what's above this line can go to .h XXX */
  -void jk_b_dump( MsgBuffer *msg, char *err ) {
  -        int i=0;
  +void jk_b_dump(jk_msg_buf_t *msg, 
  +               char *err) 
  +{
  +    int i=0;
   	printf("%s %d/%d/%d %x %x %x %x - %x %x %x %x - %x %x %x %x - %x %x %x %x\n", err, msg->pos, msg->len, msg->maxlen,  
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++],
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++],
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++],
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++]);
   
  -	i=msg->pos - 4;
  -	if( i<0 ) i=0;
  +	i = msg->pos - 4;
  +    if(i < 0) {
  +        i=0;
  +    }
   	
  -        printf("        %x %x %x %x - %x %x %x %x --- %x %x %x %x - %x %x %x %x\n", 
  +    printf("        %x %x %x %x - %x %x %x %x --- %x %x %x %x - %x %x %x %x\n", 
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++],
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++],
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++],
   	       msg->buf[i++],msg->buf[i++],msg->buf[i++],msg->buf[i++]);
   
   }
  +
  +void jk_b_reset(jk_msg_buf_t *msg) 
  +{
  +    msg->len = 4;
  +    msg->pos = 4;
  +}
  +
  +void jk_b_set_int(jk_msg_buf_t *msg, 
  +                  int pos, 
  +                  unsigned short val) 
  +{
  +    msg->buf[pos]       = (unsigned char)((val >> 8) & 0xff);
  +    msg->buf[pos + 1]   = (unsigned char)(val & 0xff);
  +}
   
  -void jk_b_reset( MsgBuffer *msg ) {
  -    msg->len =4;
  -    msg->pos =4;
  -}
  -
  -void jk_b_set_int( MsgBuffer *msg, int pos, unsigned int val ) {
  -    /* XXX optimize - swap if needed or just copyb */
  -    /* #if SWAP */
  -    /*     swap_16( (unsigned char *)&val, msg->buf ) */
  -    /* #else */
  -    /*     ???	 */
  -    /* #endif  */
  -    msg->buf[pos++]=(unsigned char) ( (val >> 8) & 0xff );
  -    msg->buf[pos]= (unsigned char) ( val & 0xff );
  -}
  -
  -int jk_b_append_int( MsgBuffer *msg, unsigned int val ) {
  -    if( msg->len + 2 > msg->maxlen ) 
  -	return -1;
   
  -    jk_b_set_int( msg, msg->len, val );
  -    msg->len +=2;
  +int jk_b_append_int(jk_msg_buf_t *msg, 
  +                    unsigned short val) 
  +{
  +    if(msg->len + 2 > msg->maxlen) {
  +	    return -1;
  +    }
  +
  +    jk_b_set_int(msg, msg->len, val);
  +
  +    msg->len += 2;
  +
       return 0;
   }
   
   
  -void jk_b_end(MsgBuffer *msg) {
  -    /* Ugly way to set the size in the right position */
  -    jk_b_set_int( msg, 2, msg->len - 4 ); /* see protocol */
  -    jk_b_set_int( msg, 0, 0x1234 );
  +void jk_b_set_byte(jk_msg_buf_t *msg, 
  +                   int pos, 
  +                   unsigned char val) 
  +{
  +    msg->buf[pos]= val;
   }
   
  +int jk_b_append_byte(jk_msg_buf_t *msg, 
  +                     unsigned char val)
  +{
  +    if(msg->len + 1 > msg->maxlen) {
  +	    return -1;
  +    }
   
  -/* XXX optimize it ( less function calls, macros )
  -   Ugly pointer arithmetic code
  - */
  -/* XXX io_vec ? XXX just send/map the pool !!! */
  +    jk_b_set_byte(msg, msg->len, val);
  +
  +    msg->len += 1;
   
  -MsgBuffer *jk_b_new(jk_pool_t *p) {
  -    MsgBuffer *msg=(MsgBuffer *)jk_pool_alloc( p, sizeof ( MsgBuffer ));
  -    msg->pool=p;
  -    if(msg==NULL) return NULL;
  +    return 0;
   }
   
  -int jk_b_set_buffer( MsgBuffer *msg, char *data, int buffSize ) {
  -    if(msg==NULL) return -1;
   
  -    msg->len=0;
  -    msg->buf=data;
  -    msg->maxlen=buffSize;
  -    /* XXX error checking !!! */
  +void jk_b_end(jk_msg_buf_t *msg) 
  +{
  +    /* 
  +     * Ugly way to set the size in the right position 
  +     */
  +    jk_b_set_int(msg, 2, (unsigned short )(msg->len - 4)); /* see protocol */
  +    jk_b_set_int(msg, 0, 0x1234);
  +}
  +
  +
  +jk_msg_buf_t *jk_b_new(jk_pool_t *p) 
  +{
  +    jk_msg_buf_t *msg = 
  +            (jk_msg_buf_t *)jk_pool_alloc(p, sizeof(jk_msg_buf_t));
  +
  +    if(!msg) {
  +        return NULL;
  +    }
  +
  +    msg->pool = p;
       
  -    return 0;
  +    return msg;
   }
   
  +int jk_b_set_buffer(jk_msg_buf_t *msg, 
  +                    char *data, 
  +                    int buffSize) 
  +{
  +    if(!msg) {
  +        return -1;
  +    }
   
  -int jk_b_set_buffer_size( MsgBuffer *msg, int buffSize ) {
  +    msg->len = 0;
  +    msg->buf = data;
  +    msg->maxlen = buffSize;
  +    
  +    return 0;
  +}
   
  -    unsigned char *data=(unsigned char *)jk_pool_alloc( msg->pool, buffSize );
  -    if( data==NULL ) {
  -	/* Free - sub-pools */
  -	return -1;
  +
  +int jk_b_set_buffer_size(jk_msg_buf_t *msg, 
  +                         int buffSize) 
  +{
  +    unsigned char *data = (unsigned char *)jk_pool_alloc(msg->pool, buffSize);
  +    
  +    if(!data) {
  +	    return -1;
       }
   
  -    jk_b_set_buffer( msg, data, buffSize );
  +    jk_b_set_buffer(msg, data, buffSize);
  +    return 0;
   }
   
  -unsigned char *jk_b_get_buff( MsgBuffer *msg ) {
  +unsigned char *jk_b_get_buff(jk_msg_buf_t *msg) 
  +{
       return msg->buf;
   }
   
  -unsigned int jk_b_get_pos( MsgBuffer *msg ) {
  +unsigned int jk_b_get_pos(jk_msg_buf_t *msg) 
  +{
       return msg->pos;
   }
   
  -unsigned int jk_b_get_len( MsgBuffer *msg ) {
  +void jk_b_set_pos(jk_msg_buf_t *msg,
  +                          int pos) 
  +{
  +    msg->pos = pos;
  +}
  +
  +unsigned int jk_b_get_len(jk_msg_buf_t *msg) 
  +{
       return msg->len;
   }
   
  -void jk_b_set_len( MsgBuffer *msg, int len ) {
  +void jk_b_set_len(jk_msg_buf_t *msg, 
  +                  int len) 
  +{
       msg->len=len;
   }
   
  -int jk_b_get_size( MsgBuffer *msg ) {
  +int jk_b_get_size(jk_msg_buf_t *msg) 
  +{
       return msg->maxlen;
   }
  -
  -/** Shame-less copy from somewhere.
  -    assert (src != dst)
  - */
  -static void swap_16( unsigned char *src, unsigned char *dst) {
  -    *dst++ = *(src + 1 );
  -    *dst= *src;
  -}
   
  -int jk_b_append_string( MsgBuffer *msg, char *param ) {
  +int jk_b_append_string(jk_msg_buf_t *msg, 
  +                       const char *param) 
  +{
       int len;
   
  -    if( param==NULL ) {
  -	jk_b_append_int( msg, 0xFFFF );
  -	return 0; 
  +    if(!param) {
  +	    jk_b_append_int( msg, 0xFFFF );
  +	    return 0; 
       }
   
  -    len=strlen(param);
  -    if( msg->len + len + 2  > msg->maxlen )
  -	return -1;
  +    len = strlen(param);
  +    if(msg->len + len + 2  > msg->maxlen) {
  +	    return -1;
  +    }
   
  -    // ignore error - we checked once
  -    jk_b_append_int( msg, len );
  +    /* ignore error - we checked once */
  +    jk_b_append_int(msg, (unsigned short )len);
   
  -    // We checked for space !! 
  -    strncpy( msg->buf + msg->len , param, len+1 ); // including \0
  +    /* We checked for space !!  */
  +    strncpy(msg->buf + msg->len , param, len+1); /* including \0 */
       msg->len += len + 1;
  +
       return 0;
   }
   
  -int jk_b_get_int( MsgBuffer *msg) {
  +unsigned short jk_b_get_int(jk_msg_buf_t *msg) 
  +{
       int i;
  -    if( msg->pos + 1 > msg->len ) {
  -	printf( "Read after end \n");
  -	return 0;
  +    if(msg->pos + 1 > msg->len) {
  +	    printf( "Read after end \n");
  +	    return -1;
       }
  -    i= ((msg->buf[msg->pos++]&0xff)<<8);
  -    i+= (msg->buf[(msg->pos++)] & 0xFF);
  +    i  = ((msg->buf[msg->pos++]&0xff)<<8);
  +    i += (msg->buf[(msg->pos++)] & 0xFF);
       return i;
   }
   
  -int jk_b_pget_int( MsgBuffer *msg, int pos) {
  +unsigned short jk_b_pget_int(jk_msg_buf_t *msg, 
  +                             int pos) 
  +{
       int i= ((msg->buf[pos++]&0xff)<<8);
       i+= (msg->buf[pos] & 0xFF);
       return i;
   }
   
  +unsigned char jk_b_get_byte(jk_msg_buf_t *msg) 
  +{
  +    unsigned char rc;
  +    if(msg->pos > msg->len) {
  +	    printf("Read after end \n");
  +	    return -1;
  +    }
  +    rc = msg->buf[msg->pos++];
  +    
  +    return rc;
  +}
  +
  +unsigned char jk_b_pget_byte(jk_msg_buf_t *msg, 
  +                             int pos) 
  +{
  +    return msg->buf[pos];
  +}
  +
   
  -/* int jk_b_getCode( MsgBuffer *msg ) { */
  -/*     return jk_b_pget_int( msg, 0 ); */
  -/* } */
  -
  -unsigned char *jk_b_get_string( MsgBuffer *msg) {
  -    int size, start;
  -    char *str;
  +unsigned char *jk_b_get_string(jk_msg_buf_t *msg) 
  +{
  +    int size = jk_b_get_int(msg);
  +    int start = msg->pos;
   
  -    /*     jk_b_dump(msg, "Before GS: "); */
  -    
  -    size=jk_b_get_int(msg);
  -    start=msg->pos;
  -    if(( size < 0 ) || ( size + start > msg->maxlen ) ) { 
  -	jk_b_dump(msg, "After get int"); 
  -	printf("ERROR\n" );
  -	return "ERROR"; /* XXX */
  +    if((size < 0 ) || (size + start > msg->maxlen)) { 
  +	    jk_b_dump(msg, "After get int"); 
  +	    printf("ERROR\n" );
  +	    return "ERROR"; /* XXX */
       }
   
       msg->pos += size;
  -    msg->pos++; // end 0
  -    str= msg->buf + start;
  -    /*     printf( "Get_string %lx %lx %x\n", msg->buf,  str, size ); */
  -    /*     printf( "Get_string %s \n", str ); */
  +    msg->pos++;  /* terminating NULL */
  +    
       return (unsigned char *)(msg->buf + start); 
   }
  -
  -
   
  -
  +/** Shame-less copy from somewhere.
  +    assert (src != dst)
  + */
  +static void swap_16(unsigned char *src, unsigned char *dst) 
  +{
  +    *dst++ = *(src + 1 );
  +    *dst= *src;
  +}
  
  
  
  1.2       +82 -49    jakarta-tomcat/src/native/jk/jk_msg_buff.h
  
  Index: jk_msg_buff.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_msg_buff.h,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_msg_buff.h	2000/04/25 00:31:00	1.1
  +++ jk_msg_buff.h	2000/05/19 06:59:00	1.2
  @@ -53,15 +53,24 @@
    *
    */
   
  -/** 
  -    Simple marshaling code.
  -*/
  +/***************************************************************************
  + * Description: Data marshaling. XDR like                                  *
  + * Author:      Costin <co...@costin.dnt.ro>                              *
  + * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  + * Version:     $Revision: 1.2 $                                           *
  + ***************************************************************************/
   
  -#include "jk_pool.h"
  -#include "jk_util.h"
  +#ifndef JK_MSG_BUF_H
  +#define JK_MSG_BUF_H
   
   
  -/* // XXX replace all return values with error codes */
  +#ifdef __cplusplus
  +extern "C" {
  +#endif /* __cplusplus */
  +
  +#define DEF_BUFFER_SZ 4 * 1024
  +
  +/* XXX replace all return values with error codes */
   #define ERR_BAD_PACKET -5
   
   /*
  @@ -75,84 +84,108 @@
   
   
    */
  -
  -/* Data marshaling. Like XDR *
  - */
  -struct MsgBuffer_Simple {
  -    jk_pool_t *pool;
  -
  -    unsigned char *buf;
  -    int pos; /* XXX MT */
  -    int len;
  -    int maxlen;
  -};
  -
   
  -typedef struct MsgBuffer_Simple MsgBuffer;
  +struct jk_msg_buf;
  +typedef struct jk_msg_buf jk_msg_buf_t;
   
   /* -------------------- Setup routines -------------------- */
   
   /** Allocate a buffer.
    */
  -MsgBuffer *jk_b_new(jk_pool_t *p); 
  +jk_msg_buf_t *jk_b_new(jk_pool_t *p); 
   
   /** Set up a buffer with an existing buffer
    */
  -int jk_b_set_buffer( MsgBuffer *msg, char *data, int buffSize );
  +int jk_b_set_buffer(jk_msg_buf_t *msg, 
  +                    char *data, 
  +                    int buffSize );
   
  -/** Set up a buffer with a new buffer of buffSize
  +/*
  + * Set up a buffer with a new buffer of buffSize
    */
  -int jk_b_set_buffer_size( MsgBuffer *msg, int buffSize );
  +int jk_b_set_buffer_size(jk_msg_buf_t *msg, 
  +                         int buffSize);
   
  -/** Finalize the buffer before sending - set length fields, etc
  +/*
  + * Finalize the buffer before sending - set length fields, etc
    */
  -void jk_b_end(MsgBuffer *msg);
  +void jk_b_end(jk_msg_buf_t *msg);
   
  -/** Recycle the buffer - prepare for a new invocation 
  +/*
  + * Recycle the buffer - z for a new invocation 
    */
  -void jk_b_reset( MsgBuffer *msg );
  +void jk_b_reset(jk_msg_buf_t *msg );
   
  -/** Return the buffer body 
  +/*
  + * Return the buffer body 
    */ 
  -unsigned char *jk_b_get_buff( MsgBuffer *msg );
  +unsigned char *jk_b_get_buff(jk_msg_buf_t *msg);
   
  -/** Return the current reading position
  +/* 
  + * Return the current reading position
    */
  -unsigned int jk_b_get_pos( MsgBuffer *msg );
  +unsigned int jk_b_get_pos(jk_msg_buf_t *msg);
   
  -/** Buffer size 
  -*/
  -int jk_b_get_size( MsgBuffer *msg );
  +/*
  + * Buffer size 
  + */
  +int jk_b_get_size(jk_msg_buf_t *msg);
   
  -void jk_b_set_len( MsgBuffer *msg, int len );
  +void jk_b_set_len(jk_msg_buf_t *msg, 
  +                  int len);
   
  -/** Get the  message length for incomming buffers
  -    or the current length for outgoing
  -*/
  -unsigned int jk_b_get_len( MsgBuffer *msg );
  +void jk_b_set_pos(jk_msg_buf_t *msg, 
  +                  int pos);
   
  -/** Dump the buffer header
  -    @param err Message text
  -*/
  -void jk_b_dump( MsgBuffer *msg, char *err ); 
  +/*
  + * Get the  message length for incomming buffers
  + *   or the current length for outgoing
  + */
  +unsigned int jk_b_get_len(jk_msg_buf_t *msg);
  +
  +/*
  + * Dump the buffer header
  + *   @param err Message text
  + */
  +void jk_b_dump(jk_msg_buf_t *msg, 
  +               char *err); 
   
   /* -------------------- Real encoding -------------------- */
   
  +void jk_b_set_int(jk_msg_buf_t *msg, 
  +                  int pos, 
  +                  unsigned short val);
   
  -void jk_b_set_int( MsgBuffer *msg, int pos, unsigned int val );
  +int jk_b_append_byte(jk_msg_buf_t *msg, 
  +                     unsigned char val);
   
  -int jk_b_append_int( MsgBuffer *msg, unsigned int val );
  +int jk_b_append_int(jk_msg_buf_t *msg, 
  +                    unsigned short val);
   
  -int jk_b_append_string( MsgBuffer *msg, char *param );
  +int jk_b_append_string(jk_msg_buf_t *msg, 
  +                       const char *param);
   
   
   /* -------------------- Decoding -------------------- */
  +
  +unsigned char *jk_b_get_string(jk_msg_buf_t *msg);
   
  -unsigned char *jk_b_get_string( MsgBuffer *msg);
  +/** Get an int from the current position 
  + */
  +unsigned short jk_b_get_int(jk_msg_buf_t *msg);
   
  +unsigned char jk_b_get_byte(jk_msg_buf_t *msg);
  +
   /** Get an int from an arbitrary position 
    */
  -int jk_b_pget_int( MsgBuffer *msg, int pos);
  +unsigned short jk_b_pget_int(jk_msg_buf_t *msg, 
  +                             int pos);
  +
  +unsigned char jk_b_pget_byte(jk_msg_buf_t *msg, 
  +                             int pos);
   
  -int jk_b_get_int( MsgBuffer *msg);
  +#ifdef __cplusplus
  +}
  +#endif /* __cplusplus */
   
  +#endif /* JK_MSG_BUF_H */
  
  
  
  1.2       +3 -1      jakarta-tomcat/src/native/jk/jk_pool.h
  
  Index: jk_pool.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_pool.h,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_pool.h	2000/04/17 09:47:22	1.1
  +++ jk_pool.h	2000/05/19 06:59:00	1.2
  @@ -56,7 +56,7 @@
   /***************************************************************************
    * Description: Memory Pool object header file                             *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  - * Version:     $Revision: 1.1 $                                               *
  + * Version:     $Revision: 1.2 $                                               *
    ***************************************************************************/
   #ifndef _JK_POOL_H
   #define _JK_POOL_H
  @@ -84,6 +84,8 @@
   #elif defined(SOLARIS)
       typedef long long   jk_pool_atom_t;
   #elif defined(LINUX)
  +    typedef long long   jk_pool_atom_t;
  +#elif defined(OS2)
       typedef long long   jk_pool_atom_t;
   #endif
   
  
  
  
  1.2       +16 -11    jakarta-tomcat/src/native/jk/jk_uri_worker_map.c
  
  Index: jk_uri_worker_map.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_uri_worker_map.c,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- jk_uri_worker_map.c	2000/04/17 09:47:21	1.1
  +++ jk_uri_worker_map.c	2000/05/19 06:59:00	1.2
  @@ -65,7 +65,7 @@
    * servlet container.                                                      *
    *                                                                         *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  - * Version:     $Revision: 1.1 $                                               *
  + * Version:     $Revision: 1.2 $                                               *
    ***************************************************************************/
   
   #include "jk_pool.h"
  @@ -124,16 +124,21 @@
               char *suffix_start;
               for(suffix_start = strstr(uri, uw_map->maps[i].suffix) ;
                   suffix_start ;
  -                suffix_start = strstr(uri, uw_map->maps[i].suffix)) {
  -
  -                char *after_suffix = suffix_start + strlen(uw_map->maps[i].suffix);
  -                if((('.' == *after_suffix) || ('/' == *after_suffix)) && 
  -                   (0 == strncmp(uw_map->maps[i].context, uri, uw_map->maps[i].ctxt_len))) {
  -                    /* 
  -                     * Security violation !!!
  -                     * this is a fraud.
  -                     */
  -                    return i;
  +                suffix_start = strstr(suffix_start + 1, uw_map->maps[i].suffix)) {
  +                
  +                if('.' != *(suffix_start - 1)) {
  +                    continue;
  +                } else {
  +                    char *after_suffix = suffix_start + strlen(uw_map->maps[i].suffix) + 1;
  +                
  +                    if((('.' == *after_suffix) || ('/' == *after_suffix)) && 
  +                       (0 == strncmp(uw_map->maps[i].context, uri, uw_map->maps[i].ctxt_len))) {
  +                        /* 
  +                         * Security violation !!!
  +                         * this is a fraud.
  +                         */
  +                        return i;
  +                    }
                   }
               }
           }
  
  
  
  1.3       +18 -1     jakarta-tomcat/src/native/jk/jk_util.c
  
  Index: jk_util.c
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_util.c,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- jk_util.c	2000/05/01 12:30:32	1.2
  +++ jk_util.c	2000/05/19 06:59:00	1.3
  @@ -56,7 +56,7 @@
   /***************************************************************************
    * Description: Utility functions (mainly configuration)                   *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  - * Version:     $Revision: 1.2 $                                               *
  + * Version:     $Revision: 1.3 $                                               *
    ***************************************************************************/
   
   
  @@ -77,6 +77,7 @@
   #define HOST_OF_WORKER              ("host")
   #define PORT_OF_WORKER              ("port")
   #define TYPE_OF_WORKER              ("type")
  +#define CACHE_OF_WORKER             ("cachesize")
   #define LOAD_FACTOR_OF_WORKER       ("lbfactor")
   #define BALANCED_WORKERS            ("balanced_workers")
   #define WORKER_AJP12                ("ajp12")
  @@ -249,6 +250,22 @@
   
       return map_get_int(m, buf, def);
   }
  +
  +int jk_get_worker_cache_size(jk_map_t *m, 
  +                             const char *wname,
  +                             int def)
  +{
  +    char buf[1024];
  +
  +    if(!m || !wname) {
  +        return -1;
  +    }
  +
  +    sprintf(buf, "%s.%s.%s", PREFIX_OF_WORKER, wname, CACHE_OF_WORKER);
  +
  +    return map_get_int(m, buf, def);
  +}
  +
   
   int jk_get_worker_list(jk_map_t *m,
                          char ***list,
  
  
  
  1.3       +4 -1      jakarta-tomcat/src/native/jk/jk_util.h
  
  Index: jk_util.h
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat/src/native/jk/jk_util.h,v
  retrieving revision 1.2
  retrieving revision 1.3
  diff -u -r1.2 -r1.3
  --- jk_util.h	2000/05/01 12:30:32	1.2
  +++ jk_util.h	2000/05/19 06:59:01	1.3
  @@ -56,7 +56,7 @@
   /***************************************************************************
    * Description: Various utility functions                                  *
    * Author:      Gal Shachor <sh...@il.ibm.com>                           *
  - * Version:     $Revision: 1.2 $                                               *
  + * Version:     $Revision: 1.3 $                                               *
    ***************************************************************************/
   #ifndef _JK_UTIL_H
   #define _JK_UTIL_H
  @@ -92,6 +92,9 @@
                          const char *wname,
                          int def);
   
  +int jk_get_worker_cache_size(jk_map_t *m, 
  +                             const char *wname,
  +                             int def);
   
   int jk_get_worker_list(jk_map_t *m,
                          char ***list,
  
  
  
  1.1                  jakarta-tomcat/src/native/jk/jk_ajp13.c
  
  Index: jk_ajp13.c
  ===================================================================
  /*
   * Copyright (c) 1997-1999 The Java Apache Project.  All rights reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. All advertising materials mentioning features or use of this
   *    software must display the following acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *
   * 4. The names "Apache JServ", "Apache JServ Servlet Engine" and 
   *    "Java Apache Project" must not be used to endorse or promote products 
   *    derived from this software without prior written permission.
   *
   * 5. Products derived from this software may not be called "Apache JServ"
   *    nor may "Apache" nor "Apache JServ" appear in their names without 
   *    prior written permission of the Java Apache Project.
   *
   * 6. Redistributions of any form whatsoever must retain the following
   *    acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *    
   * THIS SOFTWARE IS PROVIDED BY THE JAVA APACHE PROJECT "AS IS" AND ANY
   * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
   * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
   * PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE JAVA APACHE PROJECT OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
   * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
   * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
   * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
   * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
   * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
   * OF THE POSSIBILITY OF SUCH DAMAGE.
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Java Apache Group. For more information
   * on the Java Apache Project and the Apache JServ Servlet Engine project,
   * please see <http://java.apache.org/>.
   *
   */
  
  /***************************************************************************
   * Description: Experimental bi-directionl protocol handler.               *
   * Author:      Gal Shachor <sh...@il.ibm.com>                           *
   * Version:     $Revision: 1.1 $                                           *
   ***************************************************************************/
  
  
  #include "jk_global.h"
  #include "jk_util.h"
  #include "jk_ajp13.h"
  
  /*
   * Conditional request attributes
   * 
   */
  #define SC_A_CONTEXT            (unsigned char)1
  #define SC_A_SERVLET_PATH       (unsigned char)2
  #define SC_A_REMOTE_USER        (unsigned char)3
  #define SC_A_AUTH_TYPE          (unsigned char)4
  #define SC_A_QUERY_STRING       (unsigned char)5
  #define SC_A_JVM_ROUTE          (unsigned char)6
  #define SC_A_SSL_CERT           (unsigned char)7
  #define SC_A_SSL_CIPHER         (unsigned char)8
  #define SC_A_SSL_SESSION        (unsigned char)9
  #define SC_A_ARE_DONE           (unsigned char)0xFF
  
  /*
   * Request methods, coded as numbers instead of strings.
   * The list of methods was taken from Section 5.1.1 of RFC 2616
   *          Method        = "OPTIONS"
   *                        | "GET"    
   *                        | "HEAD"   
   *                        | "POST"   
   *                        | "PUT"    
   *                        | "DELETE" 
   *                        | "TRACE"  
   * 
   */
  #define SC_M_OPTIONS            (unsigned char)1
  #define SC_M_GET                (unsigned char)2
  #define SC_M_HEAD               (unsigned char)3
  #define SC_M_POST               (unsigned char)4
  #define SC_M_PUT                (unsigned char)5
  #define SC_M_DELETE             (unsigned char)6
  #define SC_M_TRACE              (unsigned char)7
  
  
  /*
   * Frequent request headers, these headers are coded as numbers
   * instead of strings.
   * 
   * Accept
   * Accept-Charset
   * Accept-Encoding
   * Accept-Language
   * Authorization
   * Connection
   * Content-Type
   * Content-Length
   * Cookie
   * Cookie2
   * Host
   * Pragma
   * Referer
   * User-Agent
   * 
   */
  
  #define SC_ACCEPT               (unsigned short)0xA001
  #define SC_ACCEPT_CHARSET       (unsigned short)0xA002
  #define SC_ACCEPT_ENCODING      (unsigned short)0xA003
  #define SC_ACCEPT_LANGUAGE      (unsigned short)0xA004
  #define SC_AUTHORIZATION        (unsigned short)0xA005
  #define SC_CONNECTION           (unsigned short)0xA006
  #define SC_CONTENT_TYPE         (unsigned short)0xA007
  #define SC_CONTENT_LENGTH       (unsigned short)0xA008
  #define SC_COOKIE               (unsigned short)0xA009    
  #define SC_COOKIE2              (unsigned short)0xA00A
  #define SC_HOST                 (unsigned short)0xA00B
  #define SC_PRAGMA               (unsigned short)0xA00C
  #define SC_REFERER              (unsigned short)0xA00D
  #define SC_USER_AGENT           (unsigned short)0xA00E
  
  /*
   * Frequent response headers, these headers are coded as numbers
   * instead of strings.
   * 
   * Content-Type
   * Content-Language
   * Content-Length
   * Date
   * Last-Modified
   * Location
   * Set-Cookie
   * Servlet-Engine
   * Status
   * WWW-Authenticate
   * 
   */
  
  #define SC_RESP_CONTENT_TYPE        (unsigned short)0xA001
  #define SC_RESP_CONTENT_LANGUAGE    (unsigned short)0xA002
  #define SC_RESP_CONTENT_LENGTH      (unsigned short)0xA003
  #define SC_RESP_DATE                (unsigned short)0xA004
  #define SC_RESP_LAST_MODIFIED       (unsigned short)0xA005
  #define SC_RESP_LOCATION            (unsigned short)0xA006
  #define SC_RESP_SET_COOKIE          (unsigned short)0xA007
  #define SC_RESP_SET_COOKIE2         (unsigned short)0xA008
  #define SC_RESP_SERVLET_ENGINE      (unsigned short)0xA009
  #define SC_RESP_STATUS              (unsigned short)0xA00A
  #define SC_RESP_WWW_AUTHENTICATE    (unsigned short)0xA00B
  #define SC_RES_HEADERS_NUM          11
  
  const char *response_trans_headers[] = {
      "Content-Type", 
      "Content-Language", 
      "Content-Length", 
      "Date", 
      "Last-Modified", 
      "Location", 
      "Set-Cookie", 
      "Set-Cookie2", 
      "Servlet-Engine", 
      "Status", 
      "WWW-Authenticate"
  };
  
  const char *long_res_header_for_sc(int sc) 
  {
      const char *rc = NULL;
      if(sc <= SC_RES_HEADERS_NUM && sc > 0) {
          rc = response_trans_headers[sc - 1];
      }
  
      return rc;
  }
  
  
  int sc_for_req_method(const char *method,
                        unsigned char *sc) 
  {
      int rc = JK_TRUE;
      if(0 == strcmp(method, "GET")) {
          *sc = SC_M_GET;
      } else if(0 == strcmp(method, "POST")) {
          *sc = SC_M_POST;
      } else if(0 == strcmp(method, "HEAD")) {
          *sc = SC_M_HEAD;
      } else if(0 == strcmp(method, "PUT")) {
          *sc = SC_M_PUT;
      } else if(0 == strcmp(method, "DELETE")) {
          *sc = SC_M_DELETE;
      } else if(0 == strcmp(method, "OPTIONS")) {
          *sc = SC_M_OPTIONS;
      } else if(0 == strcmp(method, "TRACE")) {
          *sc = SC_M_TRACE;
      } else {
          rc = JK_FALSE;
      }
  
      return rc;
  }
  
  int sc_for_req_header(const char *header_name,
                        unsigned short *sc) 
  {
      switch(header_name[0]) {
          case 'a':
              if('c' ==header_name[1] &&
                 'c' ==header_name[2] &&
                 'e' ==header_name[3] &&
                 'p' ==header_name[4] &&
                 't' ==header_name[5]) {
                  if('-' == header_name[6]) {
                      if(!strcmp(header_name + 7, "charset")) {
                          *sc = SC_ACCEPT_CHARSET;
                      } else if(!strcmp(header_name + 7, "encoding")) {
                          *sc = SC_ACCEPT_ENCODING;
                      } else if(!strcmp(header_name + 7, "language")) {
                          *sc = SC_ACCEPT_LANGUAGE;
                      } else {
                          return JK_FALSE;
                      }
                  } else if('\0' == header_name[6]) {
                      *sc = SC_ACCEPT;
                  } else {
                      return JK_FALSE;
                  }
              } else if(!strcmp(header_name, "authorization")) {
                  *sc = SC_ACCEPT_LANGUAGE;
              } else {
                  return JK_FALSE;
              }
          break;
  
          case 'c':
              if(!strcmp(header_name, "cookie")) {
                  *sc = SC_COOKIE;
              } else if(!strcmp(header_name, "connection")) {
                  *sc = SC_CONNECTION;
              } else if(!strcmp(header_name, "content-type")) {
                  *sc = SC_CONTENT_TYPE;
              } else if(!strcmp(header_name, "content-length")) {
                  *sc = SC_CONTENT_LENGTH;
              } else if(!strcmp(header_name, "cookie2")) {
                  *sc = SC_COOKIE2;
              } else {
                  return JK_FALSE;
              }
          break;
  
          case 'h':
              if(!strcmp(header_name, "host")) {
                  *sc = SC_HOST;
              } else {
                  return JK_FALSE;
              }
          break;
  
          case 'p':
              if(!strcmp(header_name, "pragma")) {
                  *sc = SC_PRAGMA;
              } else {
                  return JK_FALSE;
              }
          break;
  
          case 'r':
              if(!strcmp(header_name, "referer")) {
                  *sc = SC_REFERER;
              } else {
                  return JK_FALSE;
              }
          break;
  
          case 'u':
              if(!strcmp(header_name, "user-agent")) {
                  *sc = SC_USER_AGENT;
              } else {
                  return JK_FALSE;
              }
          break;
  
          default:
              return JK_FALSE;
      }
  
      return JK_TRUE;
  }
  
  
  /*
   * Message structure
   *
   *
  AJPV13_REQUEST:=
      request_prefix (1) (byte)
      method         (byte)
      protocol       (string)
      req_uri        (string)
      remote_addr    (string)
      remote_host    (string)
      server_name    (string)
      server_port    (short)
      is_ssl         (boolean)
      num_headers    (short)
      num_headers*(req_header_name header_value)
  
      ?context       (byte)(string)
      ?servlet_path  (byte)(string)
      ?remote_user   (byte)(string)
      ?auth_type     (byte)(string)
      ?query_string  (byte)(string)
      ?jvm_route     (byte)(string)
      ?ssl_cert      (byte)(string)
      ?ssl_cipher    (byte)(string)
      ?ssl_session   (byte)(string)
      request_terminator (byte)
      ?body          content_length*(var binary)
  
   */
  
  int ajp13_marshal_into_msgb(jk_msg_buf_t *msg,
                              jk_ws_service_t *s,
                              jk_logger_t *l)
  {
      unsigned char method;
      unsigned i;
  
      if(!sc_for_req_method(s->method, &method)) { 
          return JK_FALSE;
      }
  
      if(0 != jk_b_append_byte(msg, JK_AJP13_FORWARD_REQUEST)  ||
         0 != jk_b_append_byte(msg, method)               ||
         0 != jk_b_append_string(msg, s->protocol)        ||
         0 != jk_b_append_string(msg, s->req_uri)         ||
         0 != jk_b_append_string(msg, s->remote_addr)     ||
         0 != jk_b_append_string(msg, s->remote_host)     ||
         0 != jk_b_append_string(msg, s->server_name)     ||
         0 != jk_b_append_int(msg, s->server_port)        ||
         0 != jk_b_append_byte(msg, (unsigned char)(s->is_ssl)) ||
         0 != jk_b_append_int(msg, (unsigned short)(s->num_headers))) {
          return JK_FALSE;
      }
  
      for(i = 0 ; i < s->num_headers ; i++) {
          unsigned short sc;
  
          if(sc_for_req_header(s->headers_names[i], &sc) ) {
              if(0 != jk_b_append_int(msg, sc)) {
                  return JK_FALSE;
              }
          } else {
              if(0 != jk_b_append_string(msg, s->headers_names[i])) {
                  return JK_FALSE;
              }
          }
          
          if(0 != jk_b_append_string(msg, s->headers_values[i])) {
              return JK_FALSE;
          }
      }
  
      if(s->remote_user) {
          if(0 != jk_b_append_byte(msg, SC_A_REMOTE_USER) ||
             0 != jk_b_append_string(msg, s->remote_user)) {
              return JK_FALSE;
          }
      }
      if(s->auth_type) {
          if(0 != jk_b_append_byte(msg, SC_A_AUTH_TYPE) ||
             0 != jk_b_append_string(msg, s->auth_type)) {
              return JK_FALSE;
          }
      }
      if(s->query_string) {
          if(0 != jk_b_append_byte(msg, SC_A_QUERY_STRING) ||
             0 != jk_b_append_string(msg, s->query_string)) {
              return JK_FALSE;
          }
      }
      if(s->jvm_route) {
          if(0 != jk_b_append_byte(msg, SC_A_JVM_ROUTE) ||
             0 != jk_b_append_string(msg, s->jvm_route)) {
              return JK_FALSE;
          }
      }
      if(s->ssl_cert_len) {
          if(0 != jk_b_append_byte(msg, SC_A_SSL_CERT) ||
             0 != jk_b_append_string(msg, s->ssl_cert)) {
              return JK_FALSE;
          }
      }
  
      if(s->ssl_cipher) {
          if(0 != jk_b_append_byte(msg, SC_A_SSL_CIPHER) ||
             0 != jk_b_append_string(msg, s->ssl_cipher)) {
              return JK_FALSE;
          }
      }
      if(s->ssl_session) {
          if(0 != jk_b_append_byte(msg, SC_A_SSL_SESSION) ||
             0 != jk_b_append_string(msg, s->ssl_session)) {
              return JK_FALSE;
          }
      }
  
      if(0 != jk_b_append_byte(msg, SC_A_ARE_DONE)) {
          return JK_FALSE;
      }
      
      return JK_TRUE;
  }
  
  /*
  AJPV13_RESPONSE:=
      response_prefix (2)
      status          (short)
      status_msg      (short)
      num_headers     (short)
      num_headers*(res_header_name header_value)
      *body_chunk
      terminator      boolean <! -- recycle connection or not  -->
  
  req_header_name := 
      sc_req_header_name | (string)
  
  res_header_name := 
      sc_res_header_name | (string)
  
  header_value :=
      (string)
  
  body_chunk :=
      length  (short)
      body    length*(var binary)
  
   */
  
  
  int ajp13_unmarshal_response(jk_msg_buf_t *msg,
                               jk_res_data_t *d,
                               jk_pool_t *p,
                               jk_logger_t *l)
  {
      d->status = jk_b_get_int(msg);
  
      if(!d->status) {
          return JK_FALSE;
      }
  
      d->msg = jk_b_get_string(msg);
  
      d->num_headers = jk_b_get_int(msg);
      d->header_names = d->header_values = NULL;
  
      if(d->num_headers) {
          d->header_names = jk_pool_alloc(p, sizeof(char *) * d->num_headers);
          d->header_values = jk_pool_alloc(p, sizeof(char *) * d->num_headers);
  
          if(d->header_names && d->header_values) {
              unsigned i;
              for(i = 0 ; i < d->num_headers ; i++) {
                  unsigned short name = jk_b_pget_int(msg, jk_b_get_pos(msg)) ;
                  
                  if((name & 0XFF00) == 0XA000) {
                      jk_b_get_int(msg);
                      name = name & 0X00FF;
                      if(name < SC_RES_HEADERS_NUM) {
                          d->header_names[i] = (char *)long_res_header_for_sc(name);
                      } else {
                          return JK_FALSE;
                      }
                  } else {
                      d->header_names[i] = jk_b_get_string(msg);
                      if(!d->header_names[i]) {
                          return JK_FALSE;
                      }
                  }
  
                  d->header_values[i] = jk_b_get_string(msg);
                  if(!d->header_values[i]) {
                      return JK_FALSE;
                  }
              }
          }
      }
  
      return JK_TRUE;
  }
  
  
  
  1.1                  jakarta-tomcat/src/native/jk/jk_ajp13.h
  
  Index: jk_ajp13.h
  ===================================================================
  /*
   * Copyright (c) 1997-1999 The Java Apache Project.  All rights reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. All advertising materials mentioning features or use of this
   *    software must display the following acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *
   * 4. The names "Apache JServ", "Apache JServ Servlet Engine" and 
   *    "Java Apache Project" must not be used to endorse or promote products 
   *    derived from this software without prior written permission.
   *
   * 5. Products derived from this software may not be called "Apache JServ"
   *    nor may "Apache" nor "Apache JServ" appear in their names without 
   *    prior written permission of the Java Apache Project.
   *
   * 6. Redistributions of any form whatsoever must retain the following
   *    acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *    
   * THIS SOFTWARE IS PROVIDED BY THE JAVA APACHE PROJECT "AS IS" AND ANY
   * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
   * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
   * PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE JAVA APACHE PROJECT OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
   * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
   * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
   * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
   * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
   * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
   * OF THE POSSIBILITY OF SUCH DAMAGE.
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Java Apache Group. For more information
   * on the Java Apache Project and the Apache JServ Servlet Engine project,
   * please see <http://java.apache.org/>.
   *
   */
  
  /***************************************************************************
   * Description: Experimental bi-directionl protocol handler.               *
   * Author:      Gal Shachor <sh...@il.ibm.com>                           *
   * Version:     $Revision: 1.1 $                                           *
   ***************************************************************************/
  #ifndef JK_AJP13_H
  #define JK_AJP13_H
  
  
  #include "jk_service.h"
  #include "jk_msg_buff.h"
  
  #ifdef __cplusplus
  extern "C" {
  #endif /* __cplusplus */
  
  /*
   * Message does not have a response (for example, JK_AJP13_END_RESPONSE)
   */
  #define JK_AJP13_ERROR              -1
  /*
   * Message does not have a response (for example, JK_AJP13_END_RESPONSE)
   */
  #define JK_AJP13_NO_RESPONSE        0
  /*
   * Message have a response.
   */
  #define JK_AJP13_HAS_RESPONSE       1
  
  /*
   * Forward a request from the web server to the servlet container.
   */
  #define JK_AJP13_FORWARD_REQUEST    (unsigned char)2
  
  /*
   * Write a body chunk from the servlet container to the web server
   */
  #define JK_AJP13_SEND_BODY_CHUNK    (unsigned char)3
  
  /*
   * Send response headers from the servlet container to the web server.
   */
  #define JK_AJP13_SEND_HEADERS       (unsigned char)4
  
  /*
   * Marks the end of response.
   */
  #define JK_AJP13_END_RESPONSE       (unsigned char)5
  
  struct jk_res_data {
      int         status;
      const char *msg;
      unsigned    num_headers;
      char      **header_names;
      char      **header_values;
  };
  typedef struct jk_res_data jk_res_data_t;
  
  int ajp13_marshal_into_msgb(jk_msg_buf_t *msg,
                              jk_ws_service_t *s,
                              jk_logger_t *l);
  
  int ajp13_unmarshal_response(jk_msg_buf_t *msg,
                               jk_res_data_t *d,
                               jk_pool_t *p,
                               jk_logger_t *l);
  #ifdef __cplusplus
  }
  #endif /* __cplusplus */
  
  #endif /* JK_AJP13_H */
  
  
  
  1.1                  jakarta-tomcat/src/native/jk/jk_ajp13_worker.c
  
  Index: jk_ajp13_worker.c
  ===================================================================
  /*
   * Copyright (c) 1997-1999 The Java Apache Project.  All rights reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. All advertising materials mentioning features or use of this
   *    software must display the following acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *
   * 4. The names "Apache JServ", "Apache JServ Servlet Engine" and 
   *    "Java Apache Project" must not be used to endorse or promote products 
   *    derived from this software without prior written permission.
   *
   * 5. Products derived from this software may not be called "Apache JServ"
   *    nor may "Apache" nor "Apache JServ" appear in their names without 
   *    prior written permission of the Java Apache Project.
   *
   * 6. Redistributions of any form whatsoever must retain the following
   *    acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *    
   * THIS SOFTWARE IS PROVIDED BY THE JAVA APACHE PROJECT "AS IS" AND ANY
   * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
   * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
   * PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE JAVA APACHE PROJECT OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
   * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
   * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
   * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
   * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
   * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
   * OF THE POSSIBILITY OF SUCH DAMAGE.
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Java Apache Group. For more information
   * on the Java Apache Project and the Apache JServ Servlet Engine project,
   * please see <http://java.apache.org/>.
   *
   */
  
  /***************************************************************************
   * Description: Experimental bi-directionl protocol.                       *
   * Author:      Costin <co...@costin.dnt.ro>                              *
   * Author:      Gal Shachor <sh...@il.ibm.com>                           *
   * Version:     $Revision: 1.1 $                                           *
   ***************************************************************************/
  
  #include "jk_pool.h"
  #include "jk_connect.h"
  #include "jk_util.h"
  #include "jk_msg_buff.h"
  #include "jk_ajp13.h"
  #include "jk_mt.h"
  
  #define AJP_DEF_HOST            ("localhost")
  #define AJP_DEF_PORT            (8008)
  #define READ_BUF_SIZE           (8*1024)
  #define DEF_RETRY_ATTEMPTS      (1)
  #define DEF_CACHE_SZ            (1)
  #define JK_INTERNAL_ERROR       (-2)
  
  struct ajp13_endpoint;
  typedef struct ajp13_endpoint ajp13_endpoint_t;
  
  struct ajp13_worker {
      struct sockaddr_in worker_inet_addr; /* Contains host and port */
      unsigned connect_retry_attempts;
      char *name; 
  
      /*
       * Open connections cache...
       *
       * 1. Critical section object to protect the cache.
       * 2. Cache size.
       * 3. An array of "open" endpoints.
       */
      JK_CRIT_SEC cs;
      unsigned ep_cache_sz;
      ajp13_endpoint_t **ep_cache;
  
      jk_worker_t worker; 
  };
  typedef struct ajp13_worker ajp13_worker_t;
  
  struct ajp13_endpoint { 
      ajp13_worker_t *worker;
  
      jk_pool_t pool;
      jk_pool_atom_t buf[BIG_POOL_SIZE];
  
      int sd;
      int reuse;
      jk_endpoint_t endpoint;
  };
  
  static void reset_endpoint(ajp13_endpoint_t *ep)
  {
      ep->reuse = JK_FALSE; 
      jk_reset_pool(&(ep->pool));
  }
  
  static void close_endpoint(ajp13_endpoint_t *ep)
  {
      if(ep->sd > 0) {
          jk_close_socket(ep->sd);
      } 
      free(ep);
  }
  
  
  static void connect_to_tomcat(ajp13_endpoint_t *ep,
                                jk_logger_t *l)
  {
      unsigned attempt;
  
      for(attempt = 0 ; attempt < ep->worker->connect_retry_attempts ; attempt++) {
          ep->sd = jk_open_socket(&ep->worker->worker_inet_addr, 
                                 JK_TRUE, 
                                 l);
          if(ep->sd >= 0) {
              jk_log(l, 
                     JK_LOG_DEBUG, 
                     "In jk_endpoint_t::connect_to_tomcat, connected sd = %d\n", ep->sd);
              return;
          }
      }    
  
      jk_log(l, 
             JK_LOG_ERROR, 
             "In jk_endpoint_t::connect_to_tomcat, failed errno = %d\n", errno);
  }
  
  static int connection_tcp_send_message(ajp13_endpoint_t *ep, 
                                         jk_msg_buf_t *msg, 
                                         jk_logger_t *l ) 
  {
      jk_b_end(msg);
      
      if(0 > jk_tcp_socket_sendfull(ep->sd, 
                                    jk_b_get_buff(msg),
                                    jk_b_get_len(msg))) {
          return JK_FALSE;
      }
  
      return JK_TRUE;
  }
  
  static int connection_tcp_get_message(ajp13_endpoint_t *ep, 
                                        jk_msg_buf_t *msg, 
                                        jk_logger_t *l) 
  {
      char head[4];
      int rc;
      int msglen;
  
      rc = jk_tcp_socket_recvfull(ep->sd, head, 4);
  
      if(rc < 0) {
          return JK_FALSE;
      }
      
      if((head[0] != 'A') || (head[1] != 'B' )) {
  	    return JK_FALSE;
      }
  
      msglen  = ((head[2]&0xff)<<8);
      msglen += (head[3] & 0xFF);
  
      if(msglen > jk_b_get_size(msg)) {
  	    return JK_FALSE;
      }
      
      jk_b_set_len(msg, msglen);
      jk_b_set_pos(msg, 0); 
  
      rc = jk_tcp_socket_recvfull(ep->sd, jk_b_get_buff(msg), msglen);
      if(rc < 0) {
          return JK_FALSE;
      }
          
      return JK_TRUE;
  }
  
  static int ajp13_process_callback(jk_msg_buf_t *msg, 
                                    ajp13_endpoint_t *ep,
                                    jk_ws_service_t *r, 
                                    jk_logger_t *l) 
  {
      int code = (int)jk_b_get_byte(msg);
  
      switch(code) {
          case JK_AJP13_SEND_HEADERS:
              {
                  jk_res_data_t res;
                  if(!ajp13_unmarshal_response(msg,
                                               &res,
                                               &ep->pool,
                                               l)) {
                      return JK_AJP13_ERROR;
                  }
                  if(!r->start_response(r, 
                                        res.status, 
                                        res.msg, 
                                        (const char * const *)res.header_names,
                                        (const char * const *)res.header_values,
                                        res.num_headers)) {
                      return JK_INTERNAL_ERROR;
                  }
              }
  	    break;
  
          case JK_AJP13_SEND_BODY_CHUNK:
              {
  	            unsigned len = (unsigned)jk_b_get_int(msg);
                  if(!r->write(r, jk_b_get_buff(msg) + jk_b_get_pos(msg), len)) {
                      return JK_INTERNAL_ERROR;
                  }
              }
  	    break;
  
          case JK_AJP13_END_RESPONSE:
              {
                  ep->reuse = (int)jk_b_get_byte(msg);
                  
                  if((ep->reuse & 0X01) != ep->reuse) {
                      /*
                       * Strange protocol error.
                       */
                      ep->reuse = JK_FALSE;
                  }
              }
              return JK_AJP13_END_RESPONSE;
  	    break;
  
          default:
  	        jk_b_dump(msg , "Invalid code");
  	        jk_log(l, 
                     JK_LOG_ERROR,
  		           "Invalid code: %d\n", code);
  	        return JK_AJP13_ERROR;
      }
      
      return JK_AJP13_NO_RESPONSE;
  }
  
  /* -------------------- Method -------------------- */
  static int JK_METHOD validate(jk_worker_t *pThis,
                                jk_map_t *props,                            
                                jk_logger_t *l)
  {
      jk_log(l, JK_LOG_DEBUG, "Into jk_worker_t::validate\n");
  
      if(pThis && pThis->worker_private) {        
          ajp13_worker_t *p = pThis->worker_private;
          int port = jk_get_worker_port(props, 
                                        p->name,
                                        AJP_DEF_PORT);
  
          char *host = jk_get_worker_host(props, 
                                          p->name,
                                          AJP_DEF_HOST);
  
          jk_log(l, 
                 JK_LOG_DEBUG, 
                 "In jk_worker_t::validate for worker %s contact is %s:%d\n", 
                 p->name, host, port);
  	
          if(port > 1024 && host) {
              if(jk_resolve(host, (short)port, &p->worker_inet_addr)) {
                  return JK_TRUE;
              }
              jk_log(l, JK_LOG_ERROR, "In jk_worker_t::validate, resolve failed\n");
          }
          jk_log(l, JK_LOG_ERROR, "In jk_worker_t::validate, Error %s %d\n", host, port);
      } else {
          jk_log(l, JK_LOG_ERROR, "In jk_worker_t::validate, NULL parameters\n");
      }
      
      return JK_FALSE;
  }
  
  
  static int JK_METHOD init(jk_worker_t *pThis,
                            jk_map_t *props, 
                            jk_logger_t *l)
  {
      /* 
       * start the connection cache
       */
      jk_log(l, JK_LOG_DEBUG, "Into jk_worker_t::init\n");
  
      if(pThis && pThis->worker_private) {        
          ajp13_worker_t *p = pThis->worker_private;
          int cache_sz = jk_get_worker_cache_size(props, 
                                                  p->name,
                                                  DEF_CACHE_SZ);
  
          if(cache_sz > 0) {
              p->ep_cache = 
                  (ajp13_endpoint_t **)malloc(sizeof(ajp13_endpoint_t *) * cache_sz);
              if(p->ep_cache) {
                  int i;
                  p->ep_cache_sz = cache_sz;
                  for(i = 0 ; i < cache_sz ; i++) {
                      p->ep_cache[i] = NULL;
                  }
                  JK_INIT_CS(&(p->cs), i);
                  if(i) {
                      return JK_TRUE;
                  }
              }
          }        
      } else {
          jk_log(l, 
                 JK_LOG_ERROR, 
                 "In jk_worker_t::init, NULL parameters\n");
      }
      
      return JK_FALSE;
  }
  
  
  static int JK_METHOD destroy(jk_worker_t **pThis,
                               jk_logger_t *l)
  {
      jk_log(l, JK_LOG_DEBUG, "Into jk_worker_t::destroy\n");
      if(pThis && *pThis && (*pThis)->worker_private) {
          ajp13_worker_t *private_data = (*pThis)->worker_private;
          
          free(private_data->name);
  
          if(private_data->ep_cache_sz) {
              unsigned i;
              for(i = 0 ; i < private_data->ep_cache_sz ; i++) {
                  if(private_data->ep_cache[i]) {
  
                      reset_endpoint(private_data->ep_cache[i]);
                      close_endpoint(private_data->ep_cache[i]);
                      free(private_data->ep_cache[i]);
                  }                
              }
              free(private_data->ep_cache);
              JK_DELETE_CS(&(private_data->cs), i);
          }
  
          free(private_data);
  
          return JK_TRUE;
      }
  
      jk_log(l, JK_LOG_ERROR, "In jk_worker_t::destroy, NULL parameters\n");
      return JK_FALSE;
  }
  
  
  static int JK_METHOD done(jk_endpoint_t **e,
                            jk_logger_t *l)
  {
      jk_log(l, JK_LOG_DEBUG, "Into jk_endpoint_t::done\n");
      if(e && *e && (*e)->endpoint_private) {
          ajp13_endpoint_t *p = (*e)->endpoint_private;
          int reuse_ep = p->reuse;
  
          reset_endpoint(p);
  
          if(reuse_ep) {
              ajp13_worker_t *w = p->worker;
              if(w->ep_cache_sz) {
                  int rc;
                  JK_ENTER_CS(&w->cs, rc);
                  if(rc) {
                      unsigned i;
                      
                      for(i = 0 ; i < w->ep_cache_sz ; i++) {
                          if(!w->ep_cache[i]) {
                              w->ep_cache[i] = p;
                              break;
                          }
                      }
                      JK_LEAVE_CS(&w->cs, rc);
                      if(i < w->ep_cache_sz) {
                          return JK_TRUE;
                      }
                  }
              }
          }
  
          close_endpoint(p);
          *e = NULL;
  
          return JK_TRUE;
      }
  
      jk_log(l, 
             JK_LOG_ERROR, 
             "In jk_endpoint_t::done, NULL parameters\n");
      return JK_FALSE;
  }
  
  static int JK_METHOD service(jk_endpoint_t *e, 
                               jk_ws_service_t *s,
                               jk_logger_t *l,
                               int *is_recoverable_error)
  {
      jk_log(l, 
             JK_LOG_DEBUG, 
             "Into jk_endpoint_t::service\n");
  
      if(e && e->endpoint_private && s && is_recoverable_error) {
          ajp13_endpoint_t *p = e->endpoint_private;
          *is_recoverable_error = JK_TRUE;
  
          p->reuse = JK_FALSE;
          if(p->sd < 0) {
  
              connect_to_tomcat(p, l);
  
              if(p->sd >= 0) {
                  /*
                   * After we are connected, each error that we are going to
                   * have is probably unrecoverable
                   */            
                  *is_recoverable_error = JK_FALSE;
              }
          }
  
          if(p->sd >= 0) {
  	        jk_msg_buf_t *msg = jk_b_new(&(p->pool));
  
  	        jk_b_set_buffer_size( msg, DEF_BUFFER_SZ); 
  	        jk_b_reset(msg);
  
              if(!ajp13_marshal_into_msgb(msg, s, l)) {
                  *is_recoverable_error = JK_FALSE;                
                  return JK_FALSE;
              }
     
              if(!connection_tcp_send_message(p, msg, l)) {
      	        jk_log(l, JK_LOG_ERROR,
  			           "Error sending request\n");
  		        return JK_FALSE;
              }   
  
  	        while(1) {
                  int rc = 0;
                  
  		        if(!connection_tcp_get_message(p, msg, l)) {
  		            jk_log(l, JK_LOG_ERROR,
  				           "Error reading request\n");
  		            return JK_FALSE;
  		        }
  
                  rc = ajp13_process_callback(msg, p, s, l);
                  if(JK_AJP13_END_RESPONSE == rc) {
                      return JK_TRUE;
                  } else if(JK_AJP13_HAS_RESPONSE == rc) {
  		            rc = connection_tcp_send_message(p, msg, l);
  		            if(rc < 0) {
  			            jk_log(l, JK_LOG_DEBUG,
  				               "Error reading response1 %d\n", rc);
  			            return JK_FALSE;
  		            }
                  } else if(rc < 0) {
                      break; /* XXX error */
                  }
  	        }        
          } else {
              jk_log(l, 
                     JK_LOG_ERROR, 
                     "In jk_endpoint_t::service, Error sd = %d\n", p->sd);
          }
      } else {
          jk_log(l, 
                 JK_LOG_ERROR, 
                 "In jk_endpoint_t::service, NULL parameters\n");
      }
  
      return JK_FALSE;
  }
  
  static int JK_METHOD get_endpoint(jk_worker_t *pThis,
                                    jk_endpoint_t **pend,
                                    jk_logger_t *l)
  {
      jk_log(l, JK_LOG_DEBUG, "Into jk_worker_t::get_endpoint\n");
  
      if(pThis && pThis->worker_private && pend) {        
          ajp13_worker_t *p = pThis->worker_private;
          ajp13_endpoint_t *ep = NULL;
  
          if(p->ep_cache_sz) {
              int rc;
              JK_ENTER_CS(&p->cs, rc);
              if(rc) {
                  unsigned i;
                  
                  for(i = 0 ; i < p->ep_cache_sz ; i++) {
                      if(p->ep_cache[i]) {
                          ep = p->ep_cache[i];
                          p->ep_cache[i] = NULL;
                          break;
                      }
                  }
                  JK_LEAVE_CS(&p->cs, rc);
                  if(ep) {
                      *pend = &ep->endpoint;
                      return JK_TRUE;
                  }
              }
          } 
  
          ep = (ajp13_endpoint_t *)malloc(sizeof(ajp13_endpoint_t));
          if(ep) {
              ep->sd = -1;         
              ep->reuse = JK_FALSE;
              jk_open_pool(&ep->pool, ep->buf, sizeof(ep->buf));
              ep->worker = pThis->worker_private;
              ep->endpoint.endpoint_private = ep;
              ep->endpoint.service = service;
              ep->endpoint.done = done;
              *pend = &ep->endpoint;
              return JK_TRUE;
          }
          jk_log(l, JK_LOG_ERROR, "In jk_worker_t::get_endpoint, malloc failed\n");
      } else {
          jk_log(l, JK_LOG_ERROR, "In jk_worker_t::get_endpoint, NULL parameters\n");
      }
  
      return JK_FALSE;
  }
  
  
  
  int JK_METHOD ajp13_worker_factory(jk_worker_t **w,
                                     const char *name,
                                     jk_logger_t *l)
  {
      ajp13_worker_t *private_data = 
              (ajp13_worker_t *)malloc(sizeof(ajp13_worker_t));
      
      jk_log(l, 
             JK_LOG_DEBUG, 
             "Into ajp23_worker_factory\n");
      if(NULL == name || NULL == w) {
          jk_log(l, 
                 JK_LOG_ERROR, 
                 "In ajp23_worker_factory, NULL parameters\n");
  	    return JK_FALSE;
      }
          
      if(!private_data) {
          jk_log(l, JK_LOG_ERROR, "In ajp23_worker_factory, NULL parameters\n");
  	    return JK_FALSE;
      }
  
      private_data->name = strdup(name);          
      
      if(!private_data->name) {
  	    free(private_data);
  	    jk_log(l, JK_LOG_ERROR, "In ajp23_worker_factory, malloc failed\n");
  	    return JK_FALSE;
      } 
  
  
      private_data->ep_cache_sz            = 0;
      private_data->ep_cache               = NULL;
      private_data->connect_retry_attempts = DEF_RETRY_ATTEMPTS;
      private_data->worker.worker_private  = private_data;
      
      private_data->worker.validate        = validate;
      private_data->worker.init            = init;
      private_data->worker.get_endpoint    = get_endpoint;
      private_data->worker.destroy         = destroy;
      
      *w = &private_data->worker;
      return JK_TRUE;
  }
  
  
  
  1.1                  jakarta-tomcat/src/native/jk/jk_ajp13_worker.h
  
  Index: jk_ajp13_worker.h
  ===================================================================
  /*
   * Copyright (c) 1997-1999 The Java Apache Project.  All rights reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. All advertising materials mentioning features or use of this
   *    software must display the following acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *
   * 4. The names "Apache JServ", "Apache JServ Servlet Engine" and 
   *    "Java Apache Project" must not be used to endorse or promote products 
   *    derived from this software without prior written permission.
   *
   * 5. Products derived from this software may not be called "Apache JServ"
   *    nor may "Apache" nor "Apache JServ" appear in their names without 
   *    prior written permission of the Java Apache Project.
   *
   * 6. Redistributions of any form whatsoever must retain the following
   *    acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *    
   * THIS SOFTWARE IS PROVIDED BY THE JAVA APACHE PROJECT "AS IS" AND ANY
   * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
   * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
   * PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE JAVA APACHE PROJECT OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
   * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
   * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
   * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
   * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
   * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
   * OF THE POSSIBILITY OF SUCH DAMAGE.
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Java Apache Group. For more information
   * on the Java Apache Project and the Apache JServ Servlet Engine project,
   * please see <http://java.apache.org/>.
   *
   */
  
  /***************************************************************************
   * Description: ajpv1.3 worker header file                                 *
   * Author:      Gal Shachor <sh...@il.ibm.com>                           *
   * Version:     $Revision: 1.1 $                                           *
   ***************************************************************************/
  
  #ifndef JK_AJP13_WORKER_H
  #define JK_AJP13_WORKER_H
  
  #include "jk_logger.h"
  #include "jk_service.h"
  
  #ifdef __cplusplus
  extern "C" {
  #endif /* __cplusplus */
  
  #define JK_AJP13_WORKER_NAME ("ajp13")
  
  int JK_METHOD ajp13_worker_factory(jk_worker_t **w,
                                     const char *name,
                                     jk_logger_t *l);
  
  #ifdef __cplusplus
  }
  #endif /* __cplusplus */
  
  #endif /* JK_AJP13_WORKER_H */
  
  
  
  1.1                  jakarta-tomcat/src/native/jk/jk_mt.h
  
  Index: jk_mt.h
  ===================================================================
  /*
   * Copyright (c) 1997-1999 The Java Apache Project.  All rights reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. All advertising materials mentioning features or use of this
   *    software must display the following acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *
   * 4. The names "Apache JServ", "Apache JServ Servlet Engine" and 
   *    "Java Apache Project" must not be used to endorse or promote products 
   *    derived from this software without prior written permission.
   *
   * 5. Products derived from this software may not be called "Apache JServ"
   *    nor may "Apache" nor "Apache JServ" appear in their names without 
   *    prior written permission of the Java Apache Project.
   *
   * 6. Redistributions of any form whatsoever must retain the following
   *    acknowledgment:
   *    "This product includes software developed by the Java Apache 
   *    Project for use in the Apache JServ servlet engine project
   *    <http://java.apache.org/>."
   *    
   * THIS SOFTWARE IS PROVIDED BY THE JAVA APACHE PROJECT "AS IS" AND ANY
   * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
   * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
   * PURPOSE ARE DISCLAIMED.  IN NO EVENT SHALL THE JAVA APACHE PROJECT OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT
   * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
   * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
   * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
   * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
   * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
   * OF THE POSSIBILITY OF SUCH DAMAGE.
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Java Apache Group. For more information
   * on the Java Apache Project and the Apache JServ Servlet Engine project,
   * please see <http://java.apache.org/>.
   *
   */
  
  /***************************************************************************
   * Description: Multi thread portability code for JK                       *
   * Author:      Gal Shachor <sh...@il.ibm.com>                           *
   * Version:     $Revision: 1.1 $                                            *
   ***************************************************************************/
  
  #ifndef _JK_MT_H
  #define _JK_MT_H
  
  #include "jk_global.h"
  
  /*
   * All WIN32 code is MT, UNIX code that uses pthreads is marked by the POSIX 
   * _REENTRANT define.
   */
  #if defined (WIN32) || defined(_REENTRANT)
  
      /*
       * Marks execution under MT compilation
       */
      #define _MT_CODE
  
      #ifdef WIN32
  
          #include <windows.h>
  
          typedef CRITICAL_SECTION JK_CRIT_SEC;
  
          #define JK_INIT_CS(x, rc) InitializeCriticalSection(x); rc = JK_TRUE;
          #define JK_DELETE_CS(x, rc) DeleteCriticalSection(x); rc = JK_TRUE;
          #define JK_ENTER_CS(x, rc) EnterCriticalSection(x); rc = JK_TRUE;
          #define JK_LEAVE_CS(x, rc) LeaveCriticalSection(x); rc = JK_TRUE;
  
      #else /* Unix pthreads */
  
          #include <pthread.h>
  
          typedef pthread_mutex_t	JK_CRIT_SEC;
  
          #define JK_INIT_CS(x, rc)\
              if(pthread_mutex_init(x, NULL)) rc = JK_FALSE; else rc = JK_TRUE; 
  
          #define JK_DELETE_CS(x, rc)\
              if(pthread_mutex_lock(x)) rc = JK_FALSE; else rc = JK_TRUE; 
  
          #define JK_ENTER_CS(x, rc)\
              if(pthread_mutex_unlock(x)) rc = JK_FALSE; else rc = JK_TRUE; 
  
          #define JK_LEAVE_CS(x, rc)\
              if(pthread_mutex_destroy(x)) rc = JK_FALSE; else rc = JK_TRUE; 
      #endif /* Unix pthreads */
  
  #else /* Not an MT code */
  
      typedef void *JK_CRIT_SEC;
  
      #define JK_INIT_CS(x, rc) rc = JK_TRUE;
      #define JK_DELETE_CS(x, rc) rc = JK_TRUE;
      #define JK_ENTER_CS(x, rc) rc = JK_TRUE;
      #define JK_LEAVE_CS(x, rc) rc = JK_TRUE;
  
  #endif /* Not an MT code */
  
  #endif /* _JK_MT_H */