You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@tomcat.apache.org by bi...@apache.org on 2005/05/15 21:14:38 UTC

cvs commit: jakarta-tomcat-connectors/jk/java/org/apache/jk/server JkCoyoteHandler.java

billbarker    2005/05/15 12:14:38

  Modified:    jk/java/org/apache/jk/common ChannelNioSocket.java
                        ChannelSocket.java HandlerRequest.java
                        JkInputStream.java JniHandler.java
               jk/java/org/apache/jk/core MsgContext.java
               jk/java/org/apache/jk/server JkCoyoteHandler.java
  Added:       jk/java/org/apache/jk/common AjpConstants.java
  Log:
  This looks big, but it's mostly a cut-and-paste job.  The effects are to tie this tighter to Coyote and the AJP protocol.
  
  The changes to Channel* are pretty much just cosmetic.
  
  The AJP protocol constants have been factored into their own class, since finding them was too hard before.
  
  The per-thread MsgContext is now the ActionHook, so it is possible to handle Actions that don't pass the Request/Response as the param (such as Mark's Body-Replay for Form auth).  Also added some explicit get/setters for things that were being handled as Notes.
  
  The (now misnamed :) JkInputStream is the Coyote Input/OutputBuffer.  This should make it easier to implement proposed extensions like compression (encryption is better handled by the Channel).
  
  There is almost no functional change, so this should be as stable as it was before.
  
  Revision  Changes    Path
  1.4       +2 -1      jakarta-tomcat-connectors/jk/java/org/apache/jk/common/ChannelNioSocket.java
  
  Index: ChannelNioSocket.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/common/ChannelNioSocket.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- ChannelNioSocket.java	14 May 2005 03:27:19 -0000	1.3
  +++ ChannelNioSocket.java	15 May 2005 19:14:38 -0000	1.4
  @@ -634,7 +634,7 @@
       void acceptConnections() {
           if( running ) {
               try{
  -                MsgContext ep=new MsgContext();
  +                MsgContext ep=createMsgContext();
                   ep.setSource(this);
                   ep.setWorkerEnv( wEnv );
                   this.accept(ep);
  @@ -1050,6 +1050,7 @@
                       nr = -1;
                   } catch(IOException iex) {
                       log.warn("Exception processing read",iex);
  +                    nr = -1; // Can't handle this yet
                   }
                   if(nr < 0) {
                       isClosed = true;
  
  
  
  1.55      +1 -1      jakarta-tomcat-connectors/jk/java/org/apache/jk/common/ChannelSocket.java
  
  Index: ChannelSocket.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/common/ChannelSocket.java,v
  retrieving revision 1.54
  retrieving revision 1.55
  diff -u -r1.54 -r1.55
  --- ChannelSocket.java	17 Apr 2005 03:41:08 -0000	1.54
  +++ ChannelSocket.java	15 May 2005 19:14:38 -0000	1.55
  @@ -637,7 +637,7 @@
               log.debug("Accepting ajp connections on " + port);
           while( running ) {
   	    try{
  -                MsgContext ep=new MsgContext();
  +                MsgContext ep=createMsgContext();
                   ep.setSource(this);
                   ep.setWorkerEnv( wEnv );
                   this.accept(ep);
  
  
  
  1.45      +48 -177   jakarta-tomcat-connectors/jk/java/org/apache/jk/common/HandlerRequest.java
  
  Index: HandlerRequest.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/common/HandlerRequest.java,v
  retrieving revision 1.44
  retrieving revision 1.45
  diff -u -r1.44 -r1.45
  --- HandlerRequest.java	10 Apr 2005 16:23:08 -0000	1.44
  +++ HandlerRequest.java	15 May 2005 19:14:38 -0000	1.45
  @@ -1,5 +1,5 @@
   /*
  - *  Copyright 1999-2004 The Apache Software Foundation
  + *  Copyright 1999-2005 The Apache Software Foundation
    *
    *  Licensed under the Apache License, Version 2.0 (the "License");
    *  you may not use this file except in compliance with the License.
  @@ -65,122 +65,6 @@
       private static org.apache.commons.logging.Log log=
           org.apache.commons.logging.LogFactory.getLog( HandlerRequest.class );
   
  -    // XXX Will move to a registry system.
  -    
  -    // Prefix codes for message types from server to container
  -    public static final byte JK_AJP13_FORWARD_REQUEST   = 2;
  -        public static final byte JK_AJP13_SHUTDOWN          = 7;
  -        public static final byte JK_AJP13_PING_REQUEST      = 8;
  -        public static final byte JK_AJP13_CPING_REQUEST     = 10;
  -
  -    // Prefix codes for message types from container to server
  -    public static final byte JK_AJP13_SEND_BODY_CHUNK   = 3;
  -    public static final byte JK_AJP13_SEND_HEADERS      = 4;
  -    public static final byte JK_AJP13_END_RESPONSE      = 5;
  -        public static final byte JK_AJP13_GET_BODY_CHUNK    = 6;
  -        public static final byte JK_AJP13_CPONG_REPLY       = 9;
  -    
  -    // Integer codes for common response header strings
  -    public static final int SC_RESP_CONTENT_TYPE        = 0xA001;
  -    public static final int SC_RESP_CONTENT_LANGUAGE    = 0xA002;
  -    public static final int SC_RESP_CONTENT_LENGTH      = 0xA003;
  -    public static final int SC_RESP_DATE                = 0xA004;
  -    public static final int SC_RESP_LAST_MODIFIED       = 0xA005;
  -    public static final int SC_RESP_LOCATION            = 0xA006;
  -    public static final int SC_RESP_SET_COOKIE          = 0xA007;
  -    public static final int SC_RESP_SET_COOKIE2         = 0xA008;
  -    public static final int SC_RESP_SERVLET_ENGINE      = 0xA009;
  -    public static final int SC_RESP_STATUS              = 0xA00A;
  -    public static final int SC_RESP_WWW_AUTHENTICATE    = 0xA00B;
  -        
  -    // Integer codes for common (optional) request attribute names
  -    public static final byte SC_A_CONTEXT       = 1;  // XXX Unused
  -    public static final byte SC_A_SERVLET_PATH  = 2;  // XXX Unused
  -    public static final byte SC_A_REMOTE_USER   = 3;
  -    public static final byte SC_A_AUTH_TYPE     = 4;
  -    public static final byte SC_A_QUERY_STRING  = 5;
  -    public static final byte SC_A_JVM_ROUTE     = 6;
  -    public static final byte SC_A_SSL_CERT      = 7;
  -    public static final byte SC_A_SSL_CIPHER    = 8;
  -    public static final byte SC_A_SSL_SESSION   = 9;
  -    public static final byte SC_A_SSL_KEYSIZE   = 11;
  -    public static final byte SC_A_SECRET        = 12;
  -    public static final byte SC_A_STORED_METHOD = 13;
  -
  -    // Used for attributes which are not in the list above
  -    public static final byte SC_A_REQ_ATTRIBUTE = 10; 
  -
  -    // Terminates list of attributes
  -    public static final byte SC_A_ARE_DONE      = (byte)0xFF;
  -    
  -    // Translates integer codes to names of HTTP methods
  -    public static final String []methodTransArray = {
  -        "OPTIONS",
  -        "GET",
  -        "HEAD",
  -        "POST",
  -        "PUT",
  -        "DELETE",
  -        "TRACE",
  -        "PROPFIND",
  -        "PROPPATCH",
  -        "MKCOL",
  -        "COPY",
  -        "MOVE",
  -        "LOCK",
  -        "UNLOCK",
  -        "ACL",
  -        "REPORT",
  -        "VERSION-CONTROL",
  -        "CHECKIN",
  -        "CHECKOUT",
  -        "UNCHECKOUT",
  -        "SEARCH",
  -        "MKWORKSPACE",
  -        "UPDATE",
  -        "LABEL",
  -        "MERGE",
  -        "BASELINE-CONTROL",
  -        "MKACTIVITY"
  -    };
  -    public static final int SC_M_JK_STORED = (byte) 0xFF;
  -    
  -    // id's for common request headers
  -    public static final int SC_REQ_ACCEPT          = 1;
  -    public static final int SC_REQ_ACCEPT_CHARSET  = 2;
  -    public static final int SC_REQ_ACCEPT_ENCODING = 3;
  -    public static final int SC_REQ_ACCEPT_LANGUAGE = 4;
  -    public static final int SC_REQ_AUTHORIZATION   = 5;
  -    public static final int SC_REQ_CONNECTION      = 6;
  -    public static final int SC_REQ_CONTENT_TYPE    = 7;
  -    public static final int SC_REQ_CONTENT_LENGTH  = 8;
  -    public static final int SC_REQ_COOKIE          = 9;
  -    public static final int SC_REQ_COOKIE2         = 10;
  -    public static final int SC_REQ_HOST            = 11;
  -    public static final int SC_REQ_PRAGMA          = 12;
  -    public static final int SC_REQ_REFERER         = 13;
  -    public static final int SC_REQ_USER_AGENT      = 14;
  -    // AJP14 new header
  -    public static final byte SC_A_SSL_KEY_SIZE  = 11; // XXX ??? 
  -
  -    // Translates integer codes to request header names    
  -    public static final String []headerTransArray = {
  -        "accept",
  -        "accept-charset",
  -        "accept-encoding",
  -        "accept-language",
  -        "authorization",
  -        "connection",
  -        "content-type",
  -        "content-length",
  -        "cookie",
  -        "cookie2",
  -        "host",
  -        "pragma",
  -        "referer",
  -        "user-agent"
  -    };
  -
       /*
        * Note for Host parsing.
        */
  @@ -191,42 +75,40 @@
        */
       private static Object lock = new Object();
   
  -    HandlerDispatch dispatch;
  -    String ajpidDir="conf";
  +    private HandlerDispatch dispatch;
  +    private String ajpidDir="conf";
       
   
  -    public HandlerRequest() 
  -    {
  +    public HandlerRequest() {
       }
   
       public void init() {
           dispatch=(HandlerDispatch)wEnv.getHandler( "dispatch" );
           if( dispatch != null ) {
               // register incoming message handlers
  -            dispatch.registerMessageType( JK_AJP13_FORWARD_REQUEST,
  +            dispatch.registerMessageType( AjpConstants.JK_AJP13_FORWARD_REQUEST,
                                             "JK_AJP13_FORWARD_REQUEST",
                                             this, null); // 2
               
  -            dispatch.registerMessageType( JK_AJP13_SHUTDOWN,
  +            dispatch.registerMessageType( AjpConstants.JK_AJP13_SHUTDOWN,
                                             "JK_AJP13_SHUTDOWN",
                                             this, null); // 7
               
  -            dispatch.registerMessageType( JK_AJP13_CPING_REQUEST,
  +            dispatch.registerMessageType( AjpConstants.JK_AJP13_CPING_REQUEST,
                                             "JK_AJP13_CPING_REQUEST",
                                              this, null); // 10
               dispatch.registerMessageType( HANDLE_THREAD_END,
                                            "HANDLE_THREAD_END",
                                            this, null);
               // register outgoing messages handler
  -            dispatch.registerMessageType( JK_AJP13_SEND_BODY_CHUNK, // 3
  +            dispatch.registerMessageType( AjpConstants.JK_AJP13_SEND_BODY_CHUNK, // 3
                                             "JK_AJP13_SEND_BODY_CHUNK",
                                             this,null );
           }
   
  -        bodyNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "jkInputStream" );
           tmpBufNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "tmpBuf" );
           secretNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "secret" );
  -
  +        
           if( next==null )
               next=wEnv.getHandler( "container" );
           if( log.isDebugEnabled() )
  @@ -242,7 +124,9 @@
       }
   
       public void setUseSecret( boolean b ) {
  -        requiredSecret=Double.toString(Math.random());
  +        if(b) {
  +            requiredSecret=Double.toString(Math.random());
  +        }
       }
   
       public void setDecodedUri( boolean b ) {
  @@ -327,9 +211,8 @@
       
       // -------------------- Incoming message --------------------
       String requiredSecret=null;
  -    int bodyNote;
  -    int tmpBufNote;
       int secretNote;
  +    int tmpBufNote;
   
       boolean decoded=true;
       boolean tomcatAuthentication=true;
  @@ -337,25 +220,24 @@
       boolean shutdownEnabled=false;
       
       public int invoke(Msg msg, MsgContext ep ) 
  -        throws IOException
  -    {
  +        throws IOException    {
           int type=msg.getByte();
           ThreadWithAttributes twa = null;
           if (Thread.currentThread() instanceof ThreadWithAttributes) {
               twa = (ThreadWithAttributes) Thread.currentThread();
           }
           Object control=ep.getControl();
  -
           MessageBytes tmpMB=(MessageBytes)ep.getNote( tmpBufNote );
           if( tmpMB==null ) {
               tmpMB= MessageBytes.newInstance();
               ep.setNote( tmpBufNote, tmpMB);
           }
  +
           if( log.isDebugEnabled() )
               log.debug( "Handling " + type );
           
           switch( type ) {
  -        case JK_AJP13_FORWARD_REQUEST:
  +        case AjpConstants.JK_AJP13_FORWARD_REQUEST:
               try {
                   if (twa != null) {
                       twa.setCurrentStage(control, "JkDecode");
  @@ -390,7 +272,7 @@
               if( log.isDebugEnabled() )
                   log.debug( "Invoke returned " + err );
               return err;
  -        case JK_AJP13_SHUTDOWN:
  +        case AjpConstants.JK_AJP13_SHUTDOWN:
               String epSecret=null;
               if( msg.getLen() > 3 ) {
                   // we have a secret
  @@ -407,9 +289,9 @@
   
               // XXX add isSameAddress check
               JkChannel ch=ep.getSource();
  -	    if( !ch.isSameAddress(ep) ) {
  -		log.error("Shutdown request not from 'same address' ");
  -		return ERROR;
  +            if( !ch.isSameAddress(ep) ) {
  +                log.error("Shutdown request not from 'same address' ");
  +                return ERROR;
               }
   
               if( !shutdownEnabled ) {
  @@ -427,12 +309,11 @@
               return OK;
   
               // We got a PING REQUEST, quickly respond with a PONG
  -        case JK_AJP13_CPING_REQUEST:
  +        case AjpConstants.JK_AJP13_CPING_REQUEST:
               msg.reset();
  -            msg.appendByte(JK_AJP13_CPONG_REPLY);
  -            ep.setType( JkHandler.HANDLE_SEND_PACKET );
  +            msg.appendByte(AjpConstants.JK_AJP13_CPONG_REPLY);
               ep.getSource().send( msg, ep );
  -	    return OK;
  +            return OK;
   
           case HANDLE_THREAD_END:
               return OK;
  @@ -464,32 +345,22 @@
       }
   
       private int decodeRequest( Msg msg, MsgContext ep, MessageBytes tmpMB )
  -        throws IOException
  -    {
  +        throws IOException    {
           // FORWARD_REQUEST handler
           Request req = checkRequest(ep);
   
  -	RequestInfo rp = req.getRequestProcessor();
  -	rp.setStage(Constants.STAGE_PARSE);
  +        RequestInfo rp = req.getRequestProcessor();
  +        rp.setStage(Constants.STAGE_PARSE);
           MessageBytes tmpMB2 = (MessageBytes)req.getNote(WorkerEnv.SSL_CERT_NOTE);
           if(tmpMB2 != null) {
               tmpMB2.recycle();
           }
           req.setStartTime(System.currentTimeMillis());
  -        JkInputStream jkBody=(JkInputStream)ep.getNote( bodyNote );
  -        if( jkBody==null ) {
  -            jkBody=new JkInputStream();
  -            jkBody.setMsgContext( ep );
  -
  -            ep.setNote( bodyNote, jkBody );
  -        }
  -
  -        jkBody.recycle();
           
           // Translate the HTTP method code to a String.
           byte methodCode = msg.getByte();
  -        if (methodCode != SC_M_JK_STORED) {
  -            String mName=methodTransArray[(int)methodCode - 1];
  +        if (methodCode != AjpConstants.SC_M_JK_STORED) {
  +            String mName=AjpConstants.methodTransArray[(int)methodCode - 1];
               req.method().setString(mName);
           }
   
  @@ -511,7 +382,7 @@
   
           decodeAttributes( ep, msg, req, tmpMB );
   
  -	rp.setStage(Constants.STAGE_PREPARE);
  +        rp.setStage(Constants.STAGE_PREPARE);
           MessageBytes valueMB = req.getMimeHeaders().getValue("host");
           parseHost(valueMB, req);
           // set cookies on request now that we have all headers
  @@ -521,8 +392,8 @@
           // immediately after
           int cl=req.getContentLength();
           if(cl > 0) {
  -            jkBody.setContentLength( cl );
  -            jkBody.receive();
  +            // This is hidious.  Look to remove it.
  +            ep.getInputStream().receive();
           }
       
           if (log.isTraceEnabled()) {
  @@ -538,19 +409,19 @@
   
           while( moreAttr ) {
               byte attributeCode=msg.getByte();
  -            if( attributeCode == SC_A_ARE_DONE )
  +            if( attributeCode == AjpConstants.SC_A_ARE_DONE )
                   return 200;
   
               /* Special case ( XXX in future API make it separate type !)
                */
  -            if( attributeCode == SC_A_SSL_KEY_SIZE ) {
  +            if( attributeCode == AjpConstants.SC_A_SSL_KEY_SIZE ) {
                   // Bug 1326: it's an Integer.
                   req.setAttribute(SSLSupport.KEY_SIZE_KEY,
                                    new Integer( msg.getInt()));
                  //Integer.toString(msg.getInt()));
               }
   
  -            if( attributeCode == SC_A_REQ_ATTRIBUTE ) {
  +            if( attributeCode == AjpConstants.SC_A_REQ_ATTRIBUTE ) {
                   // 2 strings ???...
                   msg.getBytes( tmpMB );
                   String n=tmpMB.toString();
  @@ -564,17 +435,17 @@
   
               // 1 string attributes
               switch(attributeCode) {
  -            case SC_A_CONTEXT      :
  +            case AjpConstants.SC_A_CONTEXT      :
                   msg.getBytes( tmpMB );
                   // nothing
                   break;
                   
  -            case SC_A_SERVLET_PATH :
  +            case AjpConstants.SC_A_SERVLET_PATH :
                   msg.getBytes( tmpMB );
                   // nothing 
                   break;
                   
  -            case SC_A_REMOTE_USER  :
  +            case AjpConstants.SC_A_REMOTE_USER  :
                   if( tomcatAuthentication ) {
                       // ignore server
                       msg.getBytes( tmpMB );
  @@ -583,7 +454,7 @@
                   }
                   break;
                   
  -            case SC_A_AUTH_TYPE    :
  +            case AjpConstants.SC_A_AUTH_TYPE    :
                   if( tomcatAuthentication ) {
                       // ignore server
                       msg.getBytes( tmpMB );
  @@ -592,15 +463,15 @@
                   }
                   break;
                   
  -            case SC_A_QUERY_STRING :
  +            case AjpConstants.SC_A_QUERY_STRING :
                   msg.getBytes(req.queryString());
                   break;
                   
  -            case SC_A_JVM_ROUTE    :
  +            case AjpConstants.SC_A_JVM_ROUTE    :
                   msg.getBytes(req.instanceId());
                   break;
                   
  -            case SC_A_SSL_CERT     :
  +            case AjpConstants.SC_A_SSL_CERT     :
                   req.scheme().setString( "https" );
                   // Transform the string into certificate.
                   MessageBytes tmpMB2 = (MessageBytes)req.getNote(WorkerEnv.SSL_CERT_NOTE);
  @@ -612,21 +483,21 @@
                   msg.getBytes(tmpMB2);
                   break;
                   
  -            case SC_A_SSL_CIPHER   :
  +            case AjpConstants.SC_A_SSL_CIPHER   :
                   req.scheme().setString( "https" );
                   msg.getBytes(tmpMB);
                   req.setAttribute(SSLSupport.CIPHER_SUITE_KEY,
                                    tmpMB.toString());
                   break;
                   
  -            case SC_A_SSL_SESSION  :
  +            case AjpConstants.SC_A_SSL_SESSION  :
                   req.scheme().setString( "https" );
                   msg.getBytes(tmpMB);
                   req.setAttribute(SSLSupport.SESSION_ID_KEY, 
                                     tmpMB.toString());
                   break;
                   
  -            case SC_A_SECRET  :
  +            case AjpConstants.SC_A_SECRET  :
                   msg.getBytes(tmpMB);
                   String secret=tmpMB.toString();
                   if(log.isInfoEnabled())
  @@ -635,7 +506,7 @@
                   ep.setNote( secretNote, secret );
                   break;
                   
  -            case SC_A_STORED_METHOD:
  +            case AjpConstants.SC_A_STORED_METHOD:
                   msg.getBytes(req.method()); 
                   break;
                   
  @@ -665,7 +536,7 @@
               isc &= 0xFF00;
               if(0xA000 == isc) {
                   msg.getInt(); // To advance the read position
  -                hName = headerTransArray[hId - 1];
  +                hName = AjpConstants.headerTransArray[hId - 1];
                   vMB=headers.addValue( hName );
               } else {
                   // reset hId -- if the header currently being read
  @@ -683,11 +554,11 @@
   
               msg.getBytes(vMB);
   
  -            if (hId == SC_REQ_CONTENT_LENGTH ||
  +            if (hId == AjpConstants.SC_REQ_CONTENT_LENGTH ||
                   (hId == -1 && tmpMB.equalsIgnoreCase("Content-Length"))) {
                   // just read the content-length header, so set it
                   req.setContentLength( vMB.getInt() );
  -            } else if (hId == SC_REQ_CONTENT_TYPE ||
  +            } else if (hId == AjpConstants.SC_REQ_CONTENT_TYPE ||
                   (hId == -1 && tmpMB.equalsIgnoreCase("Content-Type"))) {
                   // just read the content-type header, so set it
                   ByteChunk bchunk = vMB.getByteChunk();
  
  
  
  1.17      +163 -216  jakarta-tomcat-connectors/jk/java/org/apache/jk/common/JkInputStream.java
  
  Index: JkInputStream.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/common/JkInputStream.java,v
  retrieving revision 1.16
  retrieving revision 1.17
  diff -u -r1.16 -r1.17
  --- JkInputStream.java	31 May 2004 04:48:54 -0000	1.16
  +++ JkInputStream.java	15 May 2005 19:14:38 -0000	1.17
  @@ -17,273 +17,171 @@
   package org.apache.jk.common;
   
   import java.io.IOException;
  -import java.io.InputStream;
  -import org.apache.jk.core.JkHandler;
  +
  +import org.apache.coyote.OutputBuffer;
  +import org.apache.coyote.InputBuffer;
  +import org.apache.coyote.Request;
  +import org.apache.coyote.Response;
  +
   import org.apache.jk.core.Msg;
   import org.apache.jk.core.MsgContext;
  -import org.apache.tomcat.util.buf.ByteChunk;
   
  +import org.apache.tomcat.util.buf.ByteChunk;
  +import org.apache.tomcat.util.buf.MessageBytes;
  +import org.apache.tomcat.util.buf.C2BConverter;
  +import org.apache.tomcat.util.http.HttpMessages;
  +import org.apache.tomcat.util.http.MimeHeaders;
   
   /** Generic input stream impl on top of ajp
    */
  -public class JkInputStream extends InputStream {
  +public class JkInputStream implements InputBuffer, OutputBuffer {
       private static org.apache.commons.logging.Log log=
           org.apache.commons.logging.LogFactory.getLog( JkInputStream.class );
   
  -    public JkInputStream() {
  -    }
  -
  -    public int available() throws IOException {
  -        if( log.isDebugEnabled() )
  -            log.debug( "available(): "  + blen + " " + pos );
  -        return blen-pos;
  -    }
  +    private Msg bodyMsg = new MsgAjp();
  +    private Msg outputMsg = new MsgAjp();
  +    private MsgContext mc;
   
  -    public void close() throws IOException {
  -        if( log.isDebugEnabled() )
  -            log.debug( "cloae() " );
  -        this.closed=true;
  -    }
  -
  -    public void mark(int readLimit) {
  -    }
  -
  -    public boolean markSupported() {
  -        return false;
  -    }
  -
  -    public void reset() throws IOException {
  -        throw new IOException("reset() not supported");
  -    }
  -
  -    public int read() throws IOException {
  -        if( contentLength == -1 ) {
  -            return doRead1();
  -	}
  -	if( available <= 0 ) {
  -            if( log.isDebugEnabled() )
  -                log.debug("doRead() nothing available" );
  -            return -1;
  -        }
  -	available--;
  -
  -        return doRead1();
  -    }
  -    
  -    public int read(byte[] b) throws IOException {
  -        int rd=read( b, 0, b.length);
  -        if( log.isDebugEnabled() )
  -            log.debug("read(" + b + ")=" + rd + " / " + b.length);
  -        return rd;
  -    }
       
  -    public int read(byte[] b, int off, int len) throws IOException {
  -      	int rd=-1;
  -	if( contentLength == -1 ) {
  -	    rd=doRead1(b,off,len);
  -	    return rd;
  -	}
  -	if( available <= 0 ) {
  -            if( log.isDebugEnabled() ) log.debug("doRead() nothing available" );
  -	    return -1;
  +    // Holds incoming chunks of request body data
  +    private MessageBytes bodyBuff = MessageBytes.newInstance();
  +    private MessageBytes tempMB = MessageBytes.newInstance();
  +    private boolean end_of_stream=false; 
  +    private boolean isEmpty = true;
  +    private boolean isFirst = true;
  +
  +    static {
  +        // Make certain HttpMessages is loaded for SecurityManager
  +        try {
  +            Class.forName("org.apache.tomcat.util.http.HttpMessages");
  +        } catch(Exception ex) {
  +            // ignore
           }
  -        
  -	rd=doRead1( b,off, len );
  -	available -= rd;
  -	if( log.isDebugEnabled() )
  -            log.debug("Read: " + new String( b,off, len ));
  -	return rd;
       }
   
  -    public long skip(long n) throws IOException {
  -        if (n > Integer.MAX_VALUE) {
  -            throw new IOException("can't skip than many:  " + n);
  -        }
  -        // XXX if n is big, split this in multiple reads
  -        byte[] b = new byte[(int)n];
  -        return read(b, 0, b.length);
  +    public JkInputStream(MsgContext context) {
  +        mc = context;
       }
   
  -
       // -------------------- Jk specific methods --------------------
   
  -    Msg bodyMsg=new MsgAjp();
  -    MsgContext mc;
  -
  -    // Total length of the body - maximum we can read
  -    // If -1, we don't use any limit, and we don't count available
  -    int contentLength;
  -    // How much remains unread.
  -    int available;
  -
  -    boolean closed=false;
  -
  -    // Ajp13 specific -  needs refactoring for the new model
  -    public static final int MAX_PACKET_SIZE=8192;
  -    public static final int H_SIZE=4;  // Size of basic packet header
  -    public static final int  MAX_READ_SIZE = MAX_PACKET_SIZE - H_SIZE - 2;
  -    public static final byte JK_AJP13_GET_BODY_CHUNK    = 6;
  -
       
  -    // Holds incoming chunks of request body data
  -    // XXX We do a copy that could be avoided !
  -    byte []bodyBuff = new byte[9000];
  -    int blen;  // Length of current chunk of body data in buffer
  -    int pos;   // Current read position within that buffer
  -
  -    boolean end_of_stream=false; // true if we've received an empty packet
  -    
  -    private int doRead1() throws IOException {
  -        if(pos >= blen) {
  -            if( ! refillReadBuffer()) {
  -		return -1;
  -	    }
  -        }
  -        int i=bodyBuff[pos++] & 0xFF;
  -        if( log.isDebugEnabled() ) log.debug("doRead1 " + (char)i );
  -        return i;  // prevent sign extension of byte value
  +    /** Must be called before or after each request
  +     */
  +    public void recycle() {
  +        end_of_stream = false;
  +        isEmpty = true;
  +        isFirst = true;
  +        bodyBuff.recycle();
  +        tempMB.recycle();
       }
   
  -    public int doRead1(byte[] b, int off, int len) throws IOException 
  -    {
  -	if(pos >= blen) {
  -	    if( ! refillReadBuffer()) {
  -		return -1;
  -	    }
  -	}
  -
  -	if(pos + len <= blen) { // Fear the off by one error
  -	    // Sanity check b.length > off + len?
  -	    System.arraycopy(bodyBuff, pos, b, off, len);
  -	    if( log.isDebugEnabled() )
  -		log.debug("doRead1: " + pos + " " + len + " " + blen);
  -            if( log.isTraceEnabled() )
  -                log.trace("Data: \n" + new String( b, off, len ));
  -	    pos += len;
  -	    return len;
  -	}
  -
  -	// Not enough data (blen < pos + len) or chunked encoded
  -	int toCopy = len;
  -	while(toCopy > 0) {
  -	    int bytesRemaining = blen - pos;
  -	    if(bytesRemaining < 0) 
  -		bytesRemaining = 0;
  -	    int c = bytesRemaining < toCopy ? bytesRemaining : toCopy;
  -
  -	    System.arraycopy(bodyBuff, pos, b, off, c);
  -	    if( log.isDebugEnabled() )
  -		log.debug("doRead2: " + pos + " " + len + " " +
  -                          blen + " " + c);
  -            if( log.isTraceEnabled() )
  -                log.trace("Data: \n" + new String( b, off, (len<blen-1)?len:blen-1 ));
  -
  -	    toCopy    -= c;
  -
  -	    off       += c;
  -	    pos       += c; // In case we exactly consume the buffer
  -
  -	    if(toCopy > 0) 
  -		if( ! refillReadBuffer()) { // Resets blen and pos
  -		    break;
  -		}
  -	}
   
  -	return len - toCopy;
  +    public void endMessage() throws IOException {
  +        outputMsg.reset();
  +        outputMsg.appendByte(AjpConstants.JK_AJP13_END_RESPONSE);
  +        outputMsg.appendByte(1);
  +        mc.getSource().send(outputMsg, mc);
  +        mc.getSource().flush(outputMsg, mc);
       }
   
  -    /** Must be called after the request is parsed, before
  -     *  any input
  -     */
  -    public void setContentLength( int i ) {
  -        contentLength=i;
  -        available=i;
  -    }
   
  -    /** Must be called when the stream is created
  -     */
  -    public void setMsgContext( MsgContext mc ) {
  -        this.mc=mc;
  -    }
  +    // -------------------- OutputBuffer implementation --------------------
   
  -    /** Must be called before or after each request
  -     */
  -    public void recycle() {
  -        available=0;
  -        blen = 0;
  -        pos = 0;
  -        closed=false;
  -        end_of_stream = false;
  -        contentLength=-1;
  +        
  +    public int doWrite(ByteChunk chunk, Response res) 
  +        throws IOException    {
  +        if (!res.isCommitted()) {
  +            // Send the connector a request for commit. The connector should
  +            // then validate the headers, send them (using sendHeader) and 
  +            // set the filters accordingly.
  +            res.sendHeaders();
  +        }
  +
  +        int len=chunk.getLength();
  +        byte buf[]=outputMsg.getBuffer();
  +        // 4 - hardcoded, byte[] marshalling overhead 
  +        int chunkSize=buf.length - outputMsg.getHeaderLength() - 4;
  +        int off=0;
  +        while( len > 0 ) {
  +            int thisTime=len;
  +            if( thisTime > chunkSize ) {
  +                thisTime=chunkSize;
  +            }
  +            len-=thisTime;
  +            
  +            outputMsg.reset();
  +            outputMsg.appendByte( AjpConstants.JK_AJP13_SEND_BODY_CHUNK);
  +            if( log.isTraceEnabled() ) 
  +                log.trace("doWrite " + off + " " + thisTime + " " + len );
  +            outputMsg.appendBytes( chunk.getBytes(), chunk.getOffset() + off, thisTime );
  +            off+=thisTime;
  +            mc.getSource().send( outputMsg, mc );
  +        }
  +        return 0;
       }
   
  -    /**
  -     */
  -    public int doRead(ByteChunk responseChunk ) throws IOException {
  +    public int doRead(ByteChunk responseChunk, Request req) 
  +        throws IOException {
  +
           if( log.isDebugEnabled())
  -            log.debug( "doRead " + pos + " " + blen + " " + available + " " + end_of_stream+
  +            log.debug( "doRead "  + end_of_stream+
                          " " + responseChunk.getOffset()+ " " + responseChunk.getLength());
           if( end_of_stream ) {
               return -1;
           }
  -        if( blen == pos ) {
  +        if( isFirst ) {
  +            // Handle special first-body-chunk
  +            if( !receive() ) {
  +                return 0;
  +            }
  +        } else if(isEmpty) {
               if ( !refillReadBuffer() ){
                   return -1;
               }
           }
  -        responseChunk.setBytes( bodyBuff, pos, blen );
  -        pos=blen;
  -        return blen;
  +        ByteChunk bc = bodyBuff.getByteChunk();
  +        responseChunk.setBytes( bc.getBuffer(), bc.getStart(), bc.getLength() );
  +        isEmpty = true;
  +        return responseChunk.getLength();
       }
       
       /** Receive a chunk of data. Called to implement the
        *  'special' packet in ajp13 and to receive the data
        *  after we send a GET_BODY packet
        */
  -    public boolean receive() throws IOException
  -    {
  -        mc.setType( JkHandler.HANDLE_RECEIVE_PACKET );
  +    public boolean receive() throws IOException {
  +        isFirst = false;
           bodyMsg.reset();
           int err = mc.getSource().receive(bodyMsg, mc);
           if( log.isDebugEnabled() )
               log.info( "Receiving: getting request body chunk " + err + " " + bodyMsg.getLen() );
           
           if(err < 0) {
  -	    throw new IOException();
  -	}
  -
  -        pos=0;
  -        blen=0;
  +            throw new IOException();
  +        }
   
           // No data received.
  -	if( bodyMsg.getLen() == 0 ) { // just the header
  +        if( bodyMsg.getLen() == 0 ) { // just the header
               // Don't mark 'end of stream' for the first chunk.
               // end_of_stream = true;
  -	    return false;
  -	}
  -    	blen = bodyMsg.peekInt();
  +            return false;
  +        }
  +        int blen = bodyMsg.peekInt();
   
           if( blen == 0 ) {
               return false;
           }
   
  -        if( blen > bodyBuff.length ) {
  -            bodyMsg.dump("Body");
  -        }
  -        
           if( log.isTraceEnabled() ) {
               bodyMsg.dump("Body buffer");
           }
           
  -    	int cpl=bodyMsg.getBytes(bodyBuff);
  -
  -        if( log.isDebugEnabled() )
  -            log.debug( "Copy into body buffer2 " + bodyBuff + " " + cpl + " " + blen );
  -
  +        bodyMsg.getBytes(bodyBuff);
           if( log.isTraceEnabled() )
  -            log.trace( "Data:\n" + new String( bodyBuff, 0, cpl ));
  -
  -	return (blen > 0);
  +            log.trace( "Data:\n" + bodyBuff);
  +        isEmpty = false;
  +        return true;
       }
       
       /**
  @@ -294,29 +192,27 @@
        */
       private boolean refillReadBuffer() throws IOException 
       {
  -	// If the server returns an empty packet, assume that that end of
  -	// the stream has been reached (yuck -- fix protocol??).
  +        // If the server returns an empty packet, assume that that end of
  +        // the stream has been reached (yuck -- fix protocol??).
           if (end_of_stream) {
  -            if( log.isDebugEnabled() ) log.debug("refillReadBuffer: end of stream " );
  +            if( log.isDebugEnabled() ) 
  +                log.debug("refillReadBuffer: end of stream " );
               return false;
           }
   
  -	// Why not use outBuf??
  -	bodyMsg.reset();
  -	bodyMsg.appendByte(JK_AJP13_GET_BODY_CHUNK);
  -	bodyMsg.appendInt(MAX_READ_SIZE);
  +        // Why not use outBuf??
  +        bodyMsg.reset();
  +        bodyMsg.appendByte(AjpConstants.JK_AJP13_GET_BODY_CHUNK);
  +        bodyMsg.appendInt(AjpConstants.MAX_READ_SIZE);
           
  -	if( log.isDebugEnabled() )
  +        if( log.isDebugEnabled() )
               log.debug("refillReadBuffer " + Thread.currentThread());
   
  -        mc.setType( JkHandler.HANDLE_SEND_PACKET );
  -	mc.getSource().send(bodyMsg, mc);
  +        mc.getSource().send(bodyMsg, mc);
   
           // In JNI mode, response will be in bodyMsg. In TCP mode, response need to be
           // read
   
  -        //bodyMsg.dump("refillReadBuffer ");
  -        
           boolean moreData=receive();
           if( !moreData ) {
               end_of_stream=true;
  @@ -324,4 +220,55 @@
           return moreData;
       }
   
  +    public void appendHead(Response res) throws IOException {
  +        if( log.isDebugEnabled() )
  +            log.debug("COMMIT sending headers " + res + " " + res.getMimeHeaders() );
  +        
  +        C2BConverter c2b=mc.getConverter();
  +        
  +        outputMsg.reset();
  +        outputMsg.appendByte(AjpConstants.JK_AJP13_SEND_HEADERS);
  +        outputMsg.appendInt( res.getStatus() );
  +        
  +        String message=res.getMessage();
  +        if( message==null ){
  +            message= HttpMessages.getMessage(res.getStatus());
  +        } else {
  +            message = message.replace('\n', ' ').replace('\r', ' ');
  +        }
  +        tempMB.setString( message );
  +        c2b.convert( tempMB );
  +        outputMsg.appendBytes(tempMB);
  +
  +        // XXX add headers
  +        
  +        MimeHeaders headers=res.getMimeHeaders();
  +        String contentType = res.getContentType();
  +        if( contentType != null ) {
  +            headers.setValue("Content-Type").setString(contentType);
  +        }
  +        String contentLanguage = res.getContentLanguage();
  +        if( contentLanguage != null ) {
  +            headers.setValue("Content-Language").setString(contentLanguage);
  +        }
  +        int contentLength = res.getContentLength();
  +        if( contentLength >= 0 ) {
  +            headers.setValue("Content-Length").setInt(contentLength);
  +        }
  +        int numHeaders = headers.size();
  +        outputMsg.appendInt(numHeaders);
  +        for( int i=0; i<numHeaders; i++ ) {
  +            MessageBytes hN=headers.getName(i);
  +            // no header to sc conversion - there's little benefit
  +            // on this direction
  +            c2b.convert ( hN );
  +            outputMsg.appendBytes( hN );
  +                        
  +            MessageBytes hV=headers.getValue(i);
  +            c2b.convert( hV );
  +            outputMsg.appendBytes( hV );
  +        }
  +        mc.getSource().send( outputMsg, mc );
  +    }
  +
   }
  
  
  
  1.18      +1 -2      jakarta-tomcat-connectors/jk/java/org/apache/jk/common/JniHandler.java
  
  Index: JniHandler.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/common/JniHandler.java,v
  retrieving revision 1.17
  retrieving revision 1.18
  diff -u -r1.17 -r1.18
  --- JniHandler.java	11 Jan 2005 13:37:46 -0000	1.17
  +++ JniHandler.java	15 May 2005 19:14:38 -0000	1.18
  @@ -59,7 +59,6 @@
   
   
       public static final int MSG_NOTE=0;
  -    public static final int C2B_NOTE=1;
       public static final int MB_NOTE=2;
       private boolean paused = false;
   
  @@ -178,7 +177,7 @@
               msgCtx.setMsg( MSG_NOTE, msg); // XXX Use noteId
   
               C2BConverter c2b=new C2BConverter(  "iso-8859-1" );
  -            msgCtx.setNote( C2B_NOTE, c2b );
  +            msgCtx.setConverter( c2b );
   
               MessageBytes tmpMB= MessageBytes.newInstance();
               msgCtx.setNote( MB_NOTE, tmpMB );
  
  
  
  1.1                  jakarta-tomcat-connectors/jk/java/org/apache/jk/common/AjpConstants.java
  
  Index: AjpConstants.java
  ===================================================================
  /*
   *  Copyright 1999-2005 The Apache Software Foundation
   *
   *  Licensed under the Apache License, Version 2.0 (the "License");
   *  you may not use this file except in compliance with the License.
   *  You may obtain a copy of the License at
   *
   *      http://www.apache.org/licenses/LICENSE-2.0
   *
   *  Unless required by applicable law or agreed to in writing, software
   *  distributed under the License is distributed on an "AS IS" BASIS,
   *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
   *  See the License for the specific language governing permissions and
   *  limitations under the License.
   */
  
  package org.apache.jk.common;
  
  
  /**
   * Common class for the AJP Protocol values
   */
  
  public class AjpConstants {
      // Prefix codes for message types from server to container
      /**
       * Message code for initial Request packet
       */
      public static final byte JK_AJP13_FORWARD_REQUEST   = 2;
      /**
       * Message code for a request to shutdown Tomcat
       */
      public static final byte JK_AJP13_SHUTDOWN          = 7;
      /**
       * Message code for a Ping request (obsolete)
       */
      public static final byte JK_AJP13_PING_REQUEST      = 8;
      /**
       * Message code for a CPing request
       */
      public static final byte JK_AJP13_CPING_REQUEST     = 10;
  
      // Prefix codes for message types from container to server
      /**
       * Response code that the package is part of the Response body
       */
      public static final byte JK_AJP13_SEND_BODY_CHUNK   = 3;
      /**
       * Response code that the package is the HTTP headers
       */
      public static final byte JK_AJP13_SEND_HEADERS      = 4;
      /**
       * Response code for EOT
       */
      public static final byte JK_AJP13_END_RESPONSE      = 5;
      /**
       * Response code to request the next Request body chunk
       */
      public static final byte JK_AJP13_GET_BODY_CHUNK    = 6;
      /**
       * Response code to reply to a CPing
       */
      public static final byte JK_AJP13_CPONG_REPLY       = 9;
      
      // Integer codes for common response header strings
      public static final int SC_RESP_CONTENT_TYPE        = 0xA001;
      public static final int SC_RESP_CONTENT_LANGUAGE    = 0xA002;
      public static final int SC_RESP_CONTENT_LENGTH      = 0xA003;
      public static final int SC_RESP_DATE                = 0xA004;
      public static final int SC_RESP_LAST_MODIFIED       = 0xA005;
      public static final int SC_RESP_LOCATION            = 0xA006;
      public static final int SC_RESP_SET_COOKIE          = 0xA007;
      public static final int SC_RESP_SET_COOKIE2         = 0xA008;
      public static final int SC_RESP_SERVLET_ENGINE      = 0xA009;
      public static final int SC_RESP_STATUS              = 0xA00A;
      public static final int SC_RESP_WWW_AUTHENTICATE    = 0xA00B;
          
      // Integer codes for common (optional) request attribute names
      public static final byte SC_A_CONTEXT       = 1;  // XXX Unused
      public static final byte SC_A_SERVLET_PATH  = 2;  // XXX Unused
      public static final byte SC_A_REMOTE_USER   = 3;
      public static final byte SC_A_AUTH_TYPE     = 4;
      public static final byte SC_A_QUERY_STRING  = 5;
      public static final byte SC_A_JVM_ROUTE     = 6;
      public static final byte SC_A_SSL_CERT      = 7;
      public static final byte SC_A_SSL_CIPHER    = 8;
      public static final byte SC_A_SSL_SESSION   = 9;
      public static final byte SC_A_SSL_KEYSIZE   = 11;
      public static final byte SC_A_SECRET        = 12;
      public static final byte SC_A_STORED_METHOD = 13;
  
      // Used for attributes which are not in the list above
      /**
       * Request Attribute is passed as a String
       */
      public static final byte SC_A_REQ_ATTRIBUTE = 10; 
  
      /**
       * Terminates list of attributes
       */
      public static final byte SC_A_ARE_DONE      = (byte)0xFF;
      
      /**
       * Translates integer codes to names of HTTP methods
       */
      public static final String []methodTransArray = {
          "OPTIONS",
          "GET",
          "HEAD",
          "POST",
          "PUT",
          "DELETE",
          "TRACE",
          "PROPFIND",
          "PROPPATCH",
          "MKCOL",
          "COPY",
          "MOVE",
          "LOCK",
          "UNLOCK",
          "ACL",
          "REPORT",
          "VERSION-CONTROL",
          "CHECKIN",
          "CHECKOUT",
          "UNCHECKOUT",
          "SEARCH",
          "MKWORKSPACE",
          "UPDATE",
          "LABEL",
          "MERGE",
          "BASELINE-CONTROL",
          "MKACTIVITY"
      };
  
      /**
       * Request Method is passed as a String
       */
      public static final int SC_M_JK_STORED = (byte) 0xFF;
      
      // id's for common request headers
      public static final int SC_REQ_ACCEPT          = 1;
      public static final int SC_REQ_ACCEPT_CHARSET  = 2;
      public static final int SC_REQ_ACCEPT_ENCODING = 3;
      public static final int SC_REQ_ACCEPT_LANGUAGE = 4;
      public static final int SC_REQ_AUTHORIZATION   = 5;
      public static final int SC_REQ_CONNECTION      = 6;
      public static final int SC_REQ_CONTENT_TYPE    = 7;
      public static final int SC_REQ_CONTENT_LENGTH  = 8;
      public static final int SC_REQ_COOKIE          = 9;
      public static final int SC_REQ_COOKIE2         = 10;
      public static final int SC_REQ_HOST            = 11;
      public static final int SC_REQ_PRAGMA          = 12;
      public static final int SC_REQ_REFERER         = 13;
      public static final int SC_REQ_USER_AGENT      = 14;
      // AJP14 new header
      public static final byte SC_A_SSL_KEY_SIZE  = 11; // XXX ??? 
  
      /**
       *  Translates integer codes to request header names    
       */
      public static final String []headerTransArray = {
          "accept",
          "accept-charset",
          "accept-encoding",
          "accept-language",
          "authorization",
          "connection",
          "content-type",
          "content-length",
          "cookie",
          "cookie2",
          "host",
          "pragma",
          "referer",
          "user-agent"
      };
      // Ajp13 specific -  needs refactoring for the new model
      /**
       * Maximum Total byte size for a AJP packet
       */
      public static final int MAX_PACKET_SIZE=8192;
      /**
       * Size of basic packet header
       */
      public static final int H_SIZE=4;  
      /**
       * Maximum size of data that can be sent in one packet
       */
      public static final int  MAX_READ_SIZE = MAX_PACKET_SIZE - H_SIZE - 2;
  
  }
  
  
  
  1.11      +190 -3    jakarta-tomcat-connectors/jk/java/org/apache/jk/core/MsgContext.java
  
  Index: MsgContext.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/core/MsgContext.java,v
  retrieving revision 1.10
  retrieving revision 1.11
  diff -u -r1.10 -r1.11
  --- MsgContext.java	31 May 2004 04:48:54 -0000	1.10
  +++ MsgContext.java	15 May 2005 19:14:38 -0000	1.11
  @@ -17,6 +17,21 @@
   package org.apache.jk.core;
   
   import java.io.IOException;
  +import java.io.ByteArrayInputStream;
  +import java.net.InetAddress;
  +import java.security.cert.CertificateFactory;
  +import java.security.cert.X509Certificate;
  +
  +import org.apache.coyote.ActionCode;
  +import org.apache.coyote.ActionHook;
  +import org.apache.coyote.Request;
  +import org.apache.coyote.Response;
  +
  +import org.apache.tomcat.util.buf.C2BConverter;
  +import org.apache.tomcat.util.buf.MessageBytes;
  +import org.apache.tomcat.util.buf.ByteChunk;
  +import org.apache.tomcat.util.net.SSLSupport;
  +import org.apache.jk.common.JkInputStream;
   
   
   /**
  @@ -27,12 +42,19 @@
    * @author Kevin Seguin
    * @author Costin Manolache
    */
  -public class MsgContext {
  +public class MsgContext implements ActionHook {
  +    private static org.apache.commons.logging.Log log =
  +        org.apache.commons.logging.LogFactory.getLog(MsgContext.class);
  +    private static org.apache.commons.logging.Log logTime=
  +        org.apache.commons.logging.LogFactory.getLog( "org.apache.jk.REQ_TIME" );
  +
       private int type;
       private Object notes[]=new Object[32];
       private JkHandler next;
       private JkChannel source;
  -    private Object req;
  +    private JkInputStream jkIS = new JkInputStream(this);
  +    private C2BConverter c2b;
  +    private Request req;
       private WorkerEnv wEnv;
       private Msg msgs[]=new Msg[10];
       private int status=0;
  @@ -50,6 +72,20 @@
       public static final int TIMER_RECEIVED=0;
       public static final int TIMER_PRE_REQUEST=1;
       public static final int TIMER_POST_REQUEST=2;
  +
  +    // Status codes
  +    public static final int JK_STATUS_NEW=0;
  +    public static final int JK_STATUS_HEAD=1;
  +    public static final int JK_STATUS_CLOSED=2;
  +    public static final int JK_STATUS_ERROR=3;
  +
  +    public MsgContext() {
  +        try {
  +            c2b = new C2BConverter("iso-8859-1");
  +        } catch(IOException iex) {
  +            log.warn("Can't happen", iex);
  +        }
  +    }
       
       public final Object getNote( int id ) {
           return notes[id];
  @@ -112,8 +148,12 @@
   
       /** The high level request object associated with this context
        */
  -    public final void setRequest( Object req ) {
  +    public final void setRequest( Request req ) {
           this.req=req;
  +        req.setInputBuffer(jkIS);
  +        Response res = req.getResponse();
  +        res.setOutputBuffer(jkIS);
  +        res.setHook(this);
       }
   
       public final  Object getRequest() {
  @@ -129,7 +169,23 @@
       public final void setMsg(int i, Msg msg) {
           this.msgs[i]=msg;
       }
  +
  +    public final C2BConverter getConverter() {
  +        return c2b;
  +    }
  +
  +    public final void setConverter(C2BConverter c2b) {
  +        this.c2b = c2b;
  +    }
       
  +    public final boolean isLogTimeEnabled() {
  +        return logTime.isDebugEnabled();
  +    }
  +
  +    public JkInputStream getInputStream() {
  +        return jkIS;
  +    }
  +
       /** Each context contains a number of byte[] buffers used for communication.
        *  The C side will contain a char * equivalent - both buffers are long-lived
        *  and recycled.
  @@ -186,4 +242,135 @@
       public void setControl(Object control) {
           this.control = control;
       }
  +
  +    // -------------------- Coyote Action implementation --------------------
  +    
  +    public void action(ActionCode actionCode, Object param) {
  +        if( actionCode==ActionCode.ACTION_COMMIT ) {
  +            if( log.isDebugEnabled() ) log.debug("COMMIT " );
  +            Response res=(Response)param;
  +
  +            if(  res.isCommitted() ) {
  +                if( log.isInfoEnabled() )
  +                    log.info("Response already committed " );
  +            } else {
  +                try {
  +                    jkIS.appendHead( res );
  +                } catch(IOException iex) {
  +                    log.warn("Unable to send headers",iex);
  +                    setStatus(JK_STATUS_ERROR);
  +                }
  +            }
  +        } else if( actionCode==ActionCode.ACTION_RESET ) {
  +            if( log.isDebugEnabled() )
  +                log.debug("RESET " );
  +            
  +        } else if( actionCode==ActionCode.ACTION_CLIENT_FLUSH ) {
  +            if( log.isDebugEnabled() ) log.debug("CLIENT_FLUSH " );
  +            try {
  +                source.flush( null, this );
  +            } catch(IOException iex) {
  +                // This is logged elsewhere, so debug only here
  +                log.debug("Error during flush",iex);
  +                Response res = (Response)param;
  +                res.setErrorException(iex);
  +                setStatus(JK_STATUS_ERROR);
  +            }
  +            
  +        } else if( actionCode==ActionCode.ACTION_CLOSE ) {
  +            if( log.isDebugEnabled() ) log.debug("CLOSE " );
  +            
  +            Response res=(Response)param;
  +            if( getStatus()== JK_STATUS_CLOSED ) {
  +                // Double close - it may happen with forward 
  +                if( log.isDebugEnabled() ) log.debug("Double CLOSE - forward ? " + res.getRequest().requestURI() );
  +                return;
  +            }
  +                 
  +            if( !res.isCommitted() )
  +                this.action( ActionCode.ACTION_COMMIT, param );
  +            try {            
  +                jkIS.endMessage();
  +            } catch(IOException iex) {
  +                log.warn("Error sending end packet",iex);
  +                setStatus(JK_STATUS_ERROR);
  +            }
  +            if(getStatus() != JK_STATUS_ERROR) {
  +                setStatus(JK_STATUS_CLOSED );
  +            }
  +
  +            if( logTime.isDebugEnabled() ) 
  +                logTime(res.getRequest(), res);
  +        } else if( actionCode==ActionCode.ACTION_REQ_SSL_ATTRIBUTE ) {
  +            Request req=(Request)param;
  +
  +            // Extract SSL certificate information (if requested)
  +            MessageBytes certString = (MessageBytes)req.getNote(WorkerEnv.SSL_CERT_NOTE);
  +            if( certString != null && !certString.isNull() ) {
  +                ByteChunk certData = certString.getByteChunk();
  +                ByteArrayInputStream bais = 
  +                    new ByteArrayInputStream(certData.getBytes(),
  +                                             certData.getStart(),
  +                                             certData.getLength());
  + 
  +                // Fill the first element.
  +                X509Certificate jsseCerts[] = null;
  +                try {
  +                    CertificateFactory cf =
  +                        CertificateFactory.getInstance("X.509");
  +                    X509Certificate cert = (X509Certificate)
  +                        cf.generateCertificate(bais);
  +                    jsseCerts =  new X509Certificate[1];
  +                    jsseCerts[0] = cert;
  +                } catch(java.security.cert.CertificateException e) {
  +                    log.error("Certificate convertion failed" , e );
  +                    return;
  +                }
  + 
  +                req.setAttribute(SSLSupport.CERTIFICATE_KEY, 
  +                                 jsseCerts);
  +            }
  +                
  +        } else if( actionCode==ActionCode.ACTION_REQ_HOST_ATTRIBUTE ) {
  +            Request req=(Request)param;
  +
  +            // If remoteHost not set by JK, get it's name from it's remoteAddr
  +            if( req.remoteHost().isNull()) {
  +                try {
  +                    req.remoteHost().setString(InetAddress.getByName(
  +                                               req.remoteAddr().toString()).
  +                                               getHostName());
  +                } catch(IOException iex) {
  +                    if(log.isDebugEnabled())
  +                        log.debug("Unable to resolve "+req.remoteAddr());
  +                }
  +            }
  +        } else if( actionCode==ActionCode.ACTION_ACK ) {
  +            if( log.isTraceEnabled() )
  +                log.trace("ACK " );
  +        }
  +    }
  +    
  +
  +    private void logTime(Request req, Response res ) {
  +        // called after the request
  +        //            org.apache.coyote.Request req=(org.apache.coyote.Request)param;
  +        //            Response res=req.getResponse();
  +        String uri=req.requestURI().toString();
  +        if( uri.indexOf( ".gif" ) >0 ) return;
  +        
  +        setLong( MsgContext.TIMER_POST_REQUEST, System.currentTimeMillis());
  +        long t1= getLong( MsgContext.TIMER_PRE_REQUEST ) -
  +            getLong( MsgContext.TIMER_RECEIVED );
  +        long t2= getLong( MsgContext.TIMER_POST_REQUEST ) -
  +            getLong( MsgContext.TIMER_PRE_REQUEST );
  +        
  +        logTime.debug("Time pre=" + t1 + "/ service=" + t2 + " " +
  +                      res.getContentLength() + " " + 
  +                      uri );
  +    }
  +
  +    public void recycle() {
  +        jkIS.recycle();
  +    }
   }
  
  
  
  1.62      +12 -348   jakarta-tomcat-connectors/jk/java/org/apache/jk/server/JkCoyoteHandler.java
  
  Index: JkCoyoteHandler.java
  ===================================================================
  RCS file: /home/cvs/jakarta-tomcat-connectors/jk/java/org/apache/jk/server/JkCoyoteHandler.java,v
  retrieving revision 1.61
  retrieving revision 1.62
  diff -u -r1.61 -r1.62
  --- JkCoyoteHandler.java	25 Mar 2005 04:02:34 -0000	1.61
  +++ JkCoyoteHandler.java	15 May 2005 19:14:38 -0000	1.62
  @@ -16,23 +16,13 @@
   
   package org.apache.jk.server;
   
  -import java.io.ByteArrayInputStream;
   import java.io.IOException;
  -import java.net.InetAddress;
  -import java.security.cert.CertificateFactory;
  -import java.security.cert.X509Certificate;
  -import java.security.PrivilegedExceptionAction;
  -import java.security.AccessController;
  -import java.security.PrivilegedActionException;
  -import java.security.PrivilegedAction;
   import java.util.Iterator;
   
   import javax.management.MBeanServer;
   import javax.management.ObjectName;
   
   import org.apache.commons.modeler.Registry;
  -import org.apache.coyote.ActionCode;
  -import org.apache.coyote.ActionHook;
   import org.apache.coyote.Adapter;
   import org.apache.coyote.ProtocolHandler;
   import org.apache.coyote.Request;
  @@ -58,46 +48,17 @@
    * @jmx:notification-handler name="org.apache.jk.SEND_PACKET
    * @jmx:notification-handler name="org.apache.coyote.ACTION_COMMIT
    */
  -public class JkCoyoteHandler extends JkHandler implements
  -    ProtocolHandler,
  -    ActionHook,
  -    org.apache.coyote.OutputBuffer,
  -    org.apache.coyote.InputBuffer
  -{
  +public class JkCoyoteHandler extends JkHandler implements ProtocolHandler {
       protected static org.apache.commons.logging.Log log 
           = org.apache.commons.logging.LogFactory.getLog(JkCoyoteHandler.class);
       // Set debug on this logger to see the container request time
  -    private static org.apache.commons.logging.Log logTime=
  -        org.apache.commons.logging.LogFactory.getLog( "org.apache.jk.REQ_TIME" );
   
       // ----------------------------------------------------------- DoPrivileged
  -    private final class StatusLinePrivilegedAction implements PrivilegedAction {
  -	int status;
  -	StatusLinePrivilegedAction(int status) {
  -	    this.status = status;
  -	}
  -	public Object run() {
  -	    return HttpMessages.getMessage(status);
  -	}
  -    }
  -
  -    int headersMsgNote;
  -    int c2bConvertersNote;
  -    int tmpMessageBytesNote;
  -    int utfC2bNote;
  -    int obNote;
  -    int epNote;
  -    int inputStreamNote;
       private boolean paused = false;
  -    
  +    int epNote;
       Adapter adapter;
       protected JkMain jkMain=null;
   
  -    public final int JK_STATUS_NEW=0;
  -    public final int JK_STATUS_HEAD=1;
  -    public final int JK_STATUS_CLOSED=2;
  -    public final int JK_STATUS_ERROR=3;
  -
       /** Set a property. Name is a "component.property". JMX should
        * be used instead.
        */
  @@ -172,14 +133,6 @@
               
               getJkMain().init();
   
  -            headersMsgNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "headerMsg" );
  -            tmpMessageBytesNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "tmpMessageBytes" );
  -            utfC2bNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "utfC2B" );
  -            epNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "ep" );
  -            obNote=wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE, "coyoteBuffer" );
  -            inputStreamNote= wEnv.getNoteId( WorkerEnv.ENDPOINT_NOTE,
  -                                             "jkInputStream");
  -
           } catch( Exception ex ) {
               log.error("Error during init",ex);
           }
  @@ -224,335 +177,46 @@
           getJkMain().stop();
       }
   
  -    // -------------------- OutputBuffer implementation --------------------
  -
  -        
  -    public int doWrite(ByteChunk chunk, Response res) 
  -        throws IOException
  -    {
  -        if (!res.isCommitted()) {
  -            // Send the connector a request for commit. The connector should
  -            // then validate the headers, send them (using sendHeader) and 
  -            // set the filters accordingly.
  -            res.sendHeaders();
  -        }
  -        MsgContext ep=(MsgContext)res.getNote( epNote );
  -
  -        MsgAjp msg=(MsgAjp)ep.getNote( headersMsgNote );
  -
  -        int len=chunk.getLength();
  -        byte buf[]=msg.getBuffer();
  -        // 4 - hardcoded, byte[] marshalling overhead 
  -        int chunkSize=buf.length - msg.getHeaderLength() - 4;
  -        int off=0;
  -        while( len > 0 ) {
  -            int thisTime=len;
  -            if( thisTime > chunkSize ) {
  -                thisTime=chunkSize;
  -            }
  -            len-=thisTime;
  -            
  -            msg.reset();
  -            msg.appendByte( HandlerRequest.JK_AJP13_SEND_BODY_CHUNK);
  -            if( log.isDebugEnabled() ) log.debug("doWrite " + off + " " + thisTime + " " + len );
  -            msg.appendBytes( chunk.getBytes(), chunk.getOffset() + off, thisTime );
  -            off+=thisTime;
  -            ep.setType( JkHandler.HANDLE_SEND_PACKET );
  -            ep.getSource().send( msg, ep );
  -        }
  -        return 0;
  -    }
  -    
  -    public int doRead(ByteChunk chunk, Request req) 
  -        throws IOException
  -    {
  -        Response res=req.getResponse();
  -        if( log.isDebugEnabled() )
  -            log.debug("doRead " + chunk.getBytes() + " " +  chunk.getOffset() + " " + chunk.getLength());
  -        MsgContext ep=(MsgContext)res.getNote( epNote );
  -        
  -        JkInputStream jkIS=(JkInputStream)ep.getNote( inputStreamNote );
  -        // return jkIS.read( chunk.getBytes(), chunk.getOffset(), chunk.getLength());
  -        return jkIS.doRead( chunk );
  -    }
       
       // -------------------- Jk handler implementation --------------------
       // Jk Handler mehod
       public int invoke( Msg msg, MsgContext ep ) 
           throws IOException
       {
  -        if( logTime.isDebugEnabled() ) 
  -                ep.setLong( MsgContext.TIMER_PRE_REQUEST, System.currentTimeMillis());
  +        if( ep.isLogTimeEnabled() ) 
  +            ep.setLong( MsgContext.TIMER_PRE_REQUEST, System.currentTimeMillis());
           
           org.apache.coyote.Request req=(org.apache.coyote.Request)ep.getRequest();
           org.apache.coyote.Response res=req.getResponse();
  -        res.setHook( this );
   
           if( log.isDebugEnabled() )
               log.debug( "Invoke " + req + " " + res + " " + req.requestURI().toString());
           
  -        res.setOutputBuffer( this );
  -        req.setInputBuffer( this );
  -        
  -        if( ep.getNote( headersMsgNote ) == null ) {
  -            Msg msg2=new MsgAjp();
  -            ep.setNote( headersMsgNote, msg2 );
  -        }
  -        
           res.setNote( epNote, ep );
  -        ep.setStatus( JK_STATUS_HEAD );
  -	RequestInfo rp = req.getRequestProcessor();
  -	rp.setStage(Constants.STAGE_SERVICE);
  +        ep.setStatus( MsgContext.JK_STATUS_HEAD );
  +        RequestInfo rp = req.getRequestProcessor();
  +        rp.setStage(Constants.STAGE_SERVICE);
           try {
               adapter.service( req, res );
           } catch( Exception ex ) {
               log.info("Error servicing request " + req,ex);
           }
  -        if(ep.getStatus() != JK_STATUS_CLOSED) {
  +        if(ep.getStatus() != MsgContext.JK_STATUS_CLOSED) {
               res.finish();
           }
   
           req.recycle();
           req.updateCounters();
           res.recycle();
  -        if( ep.getStatus() == JK_STATUS_ERROR ) {
  +        ep.recycle();
  +        if( ep.getStatus() == MsgContext.JK_STATUS_ERROR ) {
               return ERROR;
           }
  -        ep.setStatus( JK_STATUS_NEW );
  -	rp.setStage(Constants.STAGE_KEEPALIVE);
  +        ep.setStatus( MsgContext.JK_STATUS_NEW );
  +        rp.setStage(Constants.STAGE_KEEPALIVE);
           return OK;
       }
   
  -    private void appendHead(org.apache.coyote.Response res)
  -        throws IOException
  -    {
  -        if( log.isDebugEnabled() )
  -            log.debug("COMMIT sending headers " + res + " " + res.getMimeHeaders() );
  -        
  -        C2BConverter c2b=(C2BConverter)res.getNote( utfC2bNote );
  -        if( c2b==null ) {
  -            if(System.getSecurityManager() != null) {
  -                try {
  -                    c2b = (C2BConverter)
  -                        AccessController.doPrivileged(
  -                              new PrivilegedExceptionAction () {
  -                                      public Object run() 
  -                                          throws IOException{
  -                                          return new C2BConverter(  "iso-8859-1" );
  -                                      }
  -                                  });
  -                } catch(PrivilegedActionException pae) {
  -                    Exception ex = pae.getException();
  -                    if(ex instanceof IOException)
  -                        throw (IOException)ex;
  -                }
  -            } else {
  -                c2b=new C2BConverter(  "iso-8859-1" );
  -	    }
  -            res.setNote( utfC2bNote, c2b );
  -        }
  -        
  -        MsgContext ep=(MsgContext)res.getNote( epNote );
  -        MsgAjp msg=(MsgAjp)ep.getNote( headersMsgNote );
  -        msg.reset();
  -        msg.appendByte(HandlerRequest.JK_AJP13_SEND_HEADERS);
  -        msg.appendInt( res.getStatus() );
  -        
  -        MessageBytes mb=(MessageBytes)ep.getNote( tmpMessageBytesNote );
  -        if( mb==null ) {
  -            mb=MessageBytes.newInstance();
  -            ep.setNote( tmpMessageBytesNote, mb );
  -        }
  -        String message=res.getMessage();
  -        if( message==null ){
  -	    if( System.getSecurityManager() != null ) {
  -		message = (String)AccessController.doPrivileged(
  -               				new StatusLinePrivilegedAction(res.getStatus()));
  -	    } else {
  -		message= HttpMessages.getMessage(res.getStatus());
  -	    }
  -        } else {
  -            message = message.replace('\n', ' ').replace('\r', ' ');
  -        }
  -        mb.setString( message );
  -        c2b.convert( mb );
  -        msg.appendBytes(mb);
  -
  -        // XXX add headers
  -        
  -        MimeHeaders headers=res.getMimeHeaders();
  -        String contentType = res.getContentType();
  -        if( contentType != null ) {
  -            headers.setValue("Content-Type").setString(contentType);
  -        }
  -        String contentLanguage = res.getContentLanguage();
  -        if( contentLanguage != null ) {
  -            headers.setValue("Content-Language").setString(contentLanguage);
  -        }
  -	int contentLength = res.getContentLength();
  -        if( contentLength >= 0 ) {
  -            headers.setValue("Content-Length").setInt(contentLength);
  -        }
  -        int numHeaders = headers.size();
  -        msg.appendInt(numHeaders);
  -        for( int i=0; i<numHeaders; i++ ) {
  -            MessageBytes hN=headers.getName(i);
  -            // no header to sc conversion - there's little benefit
  -            // on this direction
  -            c2b.convert ( hN );
  -            msg.appendBytes( hN );
  -                        
  -            MessageBytes hV=headers.getValue(i);
  -            c2b.convert( hV );
  -            msg.appendBytes( hV );
  -        }
  -        ep.setType( JkHandler.HANDLE_SEND_PACKET );
  -        ep.getSource().send( msg, ep );
  -    }
  -    
  -    // -------------------- Coyote Action implementation --------------------
  -    
  -    public void action(ActionCode actionCode, Object param) {
  -        if( actionCode==ActionCode.ACTION_COMMIT ) {
  -            if( log.isDebugEnabled() ) log.debug("COMMIT " );
  -            org.apache.coyote.Response res=(org.apache.coyote.Response)param;
  -
  -            if(  res.isCommitted() ) {
  -                if( log.isInfoEnabled() )
  -                    log.info("Response already committed " );
  -            } else {
  -                try {
  -                    appendHead( res );
  -                } catch(IOException iex) {
  -                    log.warn("Unable to send headers",iex);
  -                    MsgContext ep=(MsgContext)res.getNote( epNote );
  -                    ep.setStatus(JK_STATUS_ERROR);
  -                }
  -            }
  -        } else if( actionCode==ActionCode.ACTION_RESET ) {
  -            if( log.isDebugEnabled() )
  -                log.debug("RESET " );
  -            
  -        } else if( actionCode==ActionCode.ACTION_CLIENT_FLUSH ) {
  -            if( log.isDebugEnabled() ) log.debug("CLIENT_FLUSH " );
  -            org.apache.coyote.Response res=(org.apache.coyote.Response)param;
  -            MsgContext ep=(MsgContext)res.getNote( epNote );
  -            ep.setType( JkHandler.HANDLE_FLUSH );
  -            try {
  -                ep.getSource().flush( null, ep );
  -            } catch(IOException iex) {
  -                // This is logged elsewhere, so debug only here
  -                log.debug("Error during flush",iex);
  -                res.setErrorException(iex);
  -                ep.setStatus(JK_STATUS_ERROR);
  -            }
  -            
  -        } else if( actionCode==ActionCode.ACTION_CLOSE ) {
  -            if( log.isDebugEnabled() ) log.debug("CLOSE " );
  -            
  -            org.apache.coyote.Response res=(org.apache.coyote.Response)param;
  -            MsgContext ep=(MsgContext)res.getNote( epNote );
  -            if( ep.getStatus()== JK_STATUS_CLOSED ) {
  -                // Double close - it may happen with forward 
  -                if( log.isDebugEnabled() ) log.debug("Double CLOSE - forward ? " + res.getRequest().requestURI() );
  -                return;
  -            }
  -                 
  -            if( !res.isCommitted() )
  -                this.action( ActionCode.ACTION_COMMIT, param );
  -            
  -            MsgAjp msg=(MsgAjp)ep.getNote( headersMsgNote );
  -            msg.reset();
  -            msg.appendByte( HandlerRequest.JK_AJP13_END_RESPONSE );
  -            msg.appendByte( 1 );
  -
  -            try {                
  -                ep.setType( JkHandler.HANDLE_SEND_PACKET );
  -                ep.getSource().send( msg, ep );
  -                
  -                ep.setType( JkHandler.HANDLE_FLUSH );
  -                ep.getSource().flush( msg, ep );
  -            } catch(IOException iex) {
  -                log.debug("Connection error ending request.",iex);
  -                ep.setStatus(JK_STATUS_ERROR);
  -            }
  -            if(ep.getStatus() != JK_STATUS_ERROR) {
  -                ep.setStatus(JK_STATUS_CLOSED );
  -            }
  -
  -            if( logTime.isDebugEnabled() ) 
  -                logTime(res.getRequest(), res);
  -        } else if( actionCode==ActionCode.ACTION_REQ_SSL_ATTRIBUTE ) {
  -            org.apache.coyote.Request req=(org.apache.coyote.Request)param;
  -
  -            // Extract SSL certificate information (if requested)
  -            MessageBytes certString = (MessageBytes)req.getNote(WorkerEnv.SSL_CERT_NOTE);
  -            if( certString != null && !certString.isNull() ) {
  -                ByteChunk certData = certString.getByteChunk();
  -                ByteArrayInputStream bais = 
  -                    new ByteArrayInputStream(certData.getBytes(),
  -                                             certData.getStart(),
  -                                             certData.getLength());
  - 
  -                // Fill the first element.
  -                X509Certificate jsseCerts[] = null;
  -                try {
  -                    CertificateFactory cf =
  -                        CertificateFactory.getInstance("X.509");
  -                    X509Certificate cert = (X509Certificate)
  -                        cf.generateCertificate(bais);
  -                    jsseCerts =  new X509Certificate[1];
  -                    jsseCerts[0] = cert;
  -                } catch(java.security.cert.CertificateException e) {
  -                    log.error("Certificate convertion failed" , e );
  -                    return;
  -                }
  - 
  -                req.setAttribute(SSLSupport.CERTIFICATE_KEY, 
  -                                 jsseCerts);
  -            }
  -                
  -        } else if( actionCode==ActionCode.ACTION_REQ_HOST_ATTRIBUTE ) {
  -            org.apache.coyote.Request req=(org.apache.coyote.Request)param;
  -
  -            // If remoteHost not set by JK, get it's name from it's remoteAddr
  -            if( req.remoteHost().isNull()) {
  -                try {
  -                    req.remoteHost().setString(InetAddress.getByName(
  -                                               req.remoteAddr().toString()).
  -                                               getHostName());
  -                } catch(IOException iex) {
  -                    if(log.isDebugEnabled())
  -                        log.debug("Unable to resolve "+req.remoteAddr());
  -                }
  -            }
  -        // } else if( actionCode==ActionCode.ACTION_POST_REQUEST ) {
  -
  -        } else if( actionCode==ActionCode.ACTION_ACK ) {
  -            if( log.isDebugEnabled() )
  -                log.debug("ACK " );
  -            // What should we do here ? Who calls it ? 
  -        }
  -    }
  -
  -    private void logTime(Request req, Response res ) {
  -        // called after the request
  -        //            org.apache.coyote.Request req=(org.apache.coyote.Request)param;
  -        //            Response res=req.getResponse();
  -        MsgContext ep=(MsgContext)res.getNote( epNote );
  -        String uri=req.requestURI().toString();
  -        if( uri.indexOf( ".gif" ) >0 ) return;
  -        
  -        ep.setLong( MsgContext.TIMER_POST_REQUEST, System.currentTimeMillis());
  -        long t1= ep.getLong( MsgContext.TIMER_PRE_REQUEST ) -
  -            ep.getLong( MsgContext.TIMER_RECEIVED );
  -        long t2= ep.getLong( MsgContext.TIMER_POST_REQUEST ) -
  -            ep.getLong( MsgContext.TIMER_PRE_REQUEST );
  -        
  -        logTime.debug("Time pre=" + t1 + "/ service=" + t2 + " " +
  -                      res.getContentLength() + " " + 
  -                      uri );
  -    }
   
       public ObjectName preRegister(MBeanServer server,
                                     ObjectName oname) throws Exception
  
  
  

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